From 68d0a25183f8f13ee0e010b0a07e2d3516232cae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Thu, 13 May 2021 11:55:39 +0800 Subject: [PATCH 01/42] test: make TestExtractStartTs stable (#24585) --- store/tikv/extract_start_ts_test.go | 74 +++++++++++++---------------- store/tikv/kv.go | 14 ++++++ 2 files changed, 47 insertions(+), 41 deletions(-) diff --git a/store/tikv/extract_start_ts_test.go b/store/tikv/extract_start_ts_test.go index a108a0f7e41cb..b392ca365cde8 100644 --- a/store/tikv/extract_start_ts_test.go +++ b/store/tikv/extract_start_ts_test.go @@ -14,9 +14,8 @@ package tikv import ( - "context" - . "github.com/pingcap/check" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore/unistore" @@ -28,7 +27,7 @@ type extractStartTsSuite struct { store *KVStore } -var _ = Suite(&extractStartTsSuite{}) +var _ = SerialSuites(&extractStartTsSuite{}) func (s *extractStartTsSuite) SetUpTest(c *C) { client, pdClient, cluster, err := unistore.New("") @@ -63,60 +62,53 @@ func (s *extractStartTsSuite) SetUpTest(c *C) { func (s *extractStartTsSuite) TestExtractStartTs(c *C) { i := uint64(100) - cases := []kv.TransactionOption{ + // to prevent time change during test case execution + // we use failpoint to make it "fixed" + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockStalenessTimestamp", "return(200)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockCurrentTimestamp", `return(300)`), IsNil) + + cases := []struct { + expectedTS uint64 + option kv.TransactionOption + }{ // StartTS setted - {TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}, + {100, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, // PrevSec setted - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}, + {200, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}}, // MinStartTS setted, global - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}, + {101, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, // MinStartTS setted, local - {TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}, + {102, kv.TransactionOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, // MaxPrevSec setted // however we need to add more cases to check the behavior when it fall backs to MinStartTS setted // see `TestMaxPrevSecFallback` - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + {200, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, // nothing setted - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}, - } - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - stalenessTimestamp, _ := s.store.getStalenessTimestamp(bo, oracle.GlobalTxnScope, 100) - expectedTs := []uint64{ - 100, - stalenessTimestamp, - - 101, - 102, - - stalenessTimestamp, - // it's too hard to figure out the value `getTimestampWithRetry` returns - // so we just check whether it is greater than stalenessTimestamp - 0, + {300, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, } - for i, cs := range cases { - expected := expectedTs[i] - result, _ := extractStartTs(s.store, cs) - if expected == 0 { - c.Assert(result, Greater, stalenessTimestamp) - } else { - c.Assert(result, Equals, expected) - } + for _, cs := range cases { + expected := cs.expectedTS + result, _ := extractStartTs(s.store, cs.option) + c.Assert(result, Equals, expected) } + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockStalenessTimestamp"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockCurrentTimestamp"), IsNil) } func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { s.store.setSafeTS(2, 0x8000000000000002) s.store.setSafeTS(3, 0x8000000000000001) - i := uint64(100) - cases := []kv.TransactionOption{ - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, - {TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + cases := []struct { + expectedTS uint64 + option kv.TransactionOption + }{ + {0x8000000000000001, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, + {0x8000000000000002, kv.TransactionOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, } - expectedTs := []uint64{0x8000000000000001, 0x8000000000000002} - for i, cs := range cases { - expected := expectedTs[i] - result, _ := extractStartTs(s.store, cs) - c.Assert(result, Equals, expected) + for _, cs := range cases { + result, _ := extractStartTs(s.store, cs.option) + c.Assert(result, Equals, cs.expectedTS) } } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index a487b0024e3e9..981a1b7bc5cab 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -235,6 +235,13 @@ func (s *KVStore) CurrentTimestamp(txnScope string) (uint64, error) { } func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, error) { + failpoint.Inject("MockCurrentTimestamp", func(val failpoint.Value) { + if v, ok := val.(int); ok { + failpoint.Return(uint64(v), nil) + } else { + panic("MockCurrentTimestamp should be a number, try use this failpoint with \"return(ts)\"") + } + }) if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("TiKVStore.getTimestampWithRetry", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -264,6 +271,13 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, } func (s *KVStore) getStalenessTimestamp(bo *Backoffer, txnScope string, prevSec uint64) (uint64, error) { + failpoint.Inject("MockStalenessTimestamp", func(val failpoint.Value) { + if v, ok := val.(int); ok { + failpoint.Return(uint64(v), nil) + } else { + panic("MockStalenessTimestamp should be a number, try use this failpoint with \"return(ts)\"") + } + }) for { startTS, err := s.oracle.GetStaleTimestamp(bo.ctx, txnScope, prevSec) if err == nil { From 5fd39d1ed0e4e5baeb7b759c24e140f7038fcff4 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Thu, 13 May 2021 12:09:39 +0800 Subject: [PATCH 02/42] ddl: forbid recover/flashback temporary tables (#24518) --- executor/ddl.go | 7 ++++ executor/executor_test.go | 71 ++++++++++++++++++++++----------------- 2 files changed, 48 insertions(+), 30 deletions(-) diff --git a/executor/ddl.go b/executor/ddl.go index 2f10555d21e1e..5058704ee2f9e 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -26,12 +26,14 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" @@ -563,6 +565,11 @@ func (e *DDLExec) getRecoverTableByTableName(tableName *ast.TableName) (*model.J if tableInfo == nil || jobInfo == nil { return nil, nil, errors.Errorf("Can't find dropped/truncated table: %v in DDL history jobs", tableName.Name) } + // Dropping local temporary tables won't appear in DDL jobs. + if tableInfo.TempTableType == model.TempTableGlobal { + msg := mysql.Message("Recover/flashback table is not supported on temporary tables", nil) + return nil, nil, dbterror.ClassDDL.NewStdErr(errno.ErrUnsupportedDDLOperation, msg) + } return jobInfo, tableInfo, nil } diff --git a/executor/executor_test.go b/executor/executor_test.go index e69b956f8d82b..3b168636606ed 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5697,24 +5697,15 @@ func (s *testRecoverTable) TearDownSuite(c *C) { s.dom.Close() } -func (s *testRecoverTable) TestRecoverTable(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange") - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database if not exists test_recover") - tk.MustExec("use test_recover") - tk.MustExec("drop table if exists t_recover") - tk.MustExec("create table t_recover (a int);") - defer func(originGC bool) { +func (s *testRecoverTable) mockGC(tk *testkit.TestKit) (string, string, string, func()) { + originGC := ddl.IsEmulatorGCEnable() + resetGC := func() { if originGC { ddl.EmulatorGCEnable() } else { ddl.EmulatorGCDisable() } - }(ddl.IsEmulatorGCEnable()) + } // disable emulator GC. // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. @@ -5727,6 +5718,23 @@ func (s *testRecoverTable) TestRecoverTable(c *C) { UPDATE variable_value = '%[1]s'` // clear GC variables first. tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )") + return timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC +} + +func (s *testRecoverTable) TestRecoverTable(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) + defer func() { + err := failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange") + c.Assert(err, IsNil) + }() + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test_recover") + tk.MustExec("use test_recover") + tk.MustExec("drop table if exists t_recover") + tk.MustExec("create table t_recover (a int);") + + timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC := s.mockGC(tk) + defer resetGC() tk.MustExec("insert into t_recover values (1),(2),(3)") tk.MustExec("drop table t_recover") @@ -5819,24 +5827,10 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { tk.MustExec("use test_flashback") tk.MustExec("drop table if exists t_flashback") tk.MustExec("create table t_flashback (a int);") - defer func(originGC bool) { - if originGC { - ddl.EmulatorGCEnable() - } else { - ddl.EmulatorGCDisable() - } - }(ddl.IsEmulatorGCEnable()) - // Disable emulator GC. - // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. - ddl.EmulatorGCDisable() - gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) - safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') - ON DUPLICATE KEY - UPDATE variable_value = '%[1]s'` - // Clear GC variables first. - tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )") + timeBeforeDrop, _, safePointSQL, resetGC := s.mockGC(tk) + defer resetGC() + // Set GC safe point tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) // Set GC enable. @@ -5939,6 +5933,23 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { tk.MustQuery("select a from t order by a").Check(testkit.Rows("1", "2", "3")) } +func (s *testRecoverTable) TestRecoverTempTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test_recover") + tk.MustExec("use test_recover") + tk.MustExec("drop table if exists t_recover") + tk.MustExec("create global temporary table t_recover (a int) on commit delete rows;") + + timeBeforeDrop, _, safePointSQL, resetGC := s.mockGC(tk) + defer resetGC() + // Set GC safe point + tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) + + tk.MustExec("drop table t_recover") + tk.MustGetErrCode("recover table t_recover;", errno.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("flashback table t_recover;", errno.ErrUnsupportedDDLOperation) +} + func (s *testSuiteP2) TestPointGetPreparedPlan(c *C) { tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("drop database if exists ps_text") From acf2e82b5227cea429347a3c9493214f209205f8 Mon Sep 17 00:00:00 2001 From: lysu Date: Thu, 13 May 2021 12:45:40 +0800 Subject: [PATCH 03/42] executor: fix point_get result on clustered index when new-row-format disabled but new-collation enabled (#24544) --- executor/point_get.go | 3 +++ session/clustered_index_test.go | 15 +++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/executor/point_get.go b/executor/point_get.go index b0cd700c5c920..c34987b7f0c1d 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -531,6 +531,9 @@ func tryDecodeFromHandle(tblInfo *model.TableInfo, schemaColIdx int, col *expres chk.AppendInt64(schemaColIdx, handle.IntValue()) return true, nil } + if types.NeedRestoredData(col.RetType) { + return false, nil + } // Try to decode common handle. if mysql.HasPriKeyFlag(col.RetType.Flag) { for i, hid := range pkCols { diff --git a/session/clustered_index_test.go b/session/clustered_index_test.go index 0f79b1b13fc2e..b7e529f29fe0e 100644 --- a/session/clustered_index_test.go +++ b/session/clustered_index_test.go @@ -665,3 +665,18 @@ func (s *testClusteredSerialSuite) TestPrefixedClusteredIndexUniqueKeyWithNewCol tk.MustExec("admin check table t;") tk.MustExec("drop table t;") } + +func (s *testClusteredSerialSuite) TestClusteredIndexNewCollationWithOldRowFormat(c *C) { + // This case maybe not useful, because newCollation isn't convenience to run on TiKV(it's required serialSuit) + // but unistore doesn't support old row format. + defer collate.SetNewCollationEnabledForTest(false) + collate.SetNewCollationEnabledForTest(true) + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Se.GetSessionVars().RowEncoder.Enable = false + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2(col_1 varchar(132) CHARACTER SET utf8 COLLATE utf8_unicode_ci, primary key(col_1) clustered)") + tk.MustExec("insert into t2 select 'aBc'") + tk.MustQuery("select col_1 from t2 where col_1 = 'aBc'").Check(testkit.Rows("aBc")) +} From 2105317479e90af9282638b18a919262f220095a Mon Sep 17 00:00:00 2001 From: mmyj Date: Thu, 13 May 2021 12:57:39 +0800 Subject: [PATCH 04/42] executor: Improve the performance of appending not fixed columns (#20969) --- util/chunk/chunk.go | 4 ++- util/chunk/chunk_test.go | 77 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 80 insertions(+), 1 deletion(-) diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index a4350bd9628e3..e91cff2559d79 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -505,8 +505,10 @@ func (c *Chunk) Append(other *Chunk, begin, end int) { } else { beginOffset, endOffset := src.offsets[begin], src.offsets[end] dst.data = append(dst.data, src.data[beginOffset:endOffset]...) + lastOffset := dst.offsets[len(dst.offsets)-1] for i := begin; i < end; i++ { - dst.offsets = append(dst.offsets, dst.offsets[len(dst.offsets)-1]+src.offsets[i+1]-src.offsets[i]) + lastOffset += src.offsets[i+1] - src.offsets[i] + dst.offsets = append(dst.offsets, lastOffset) } } for i := begin; i < end; i++ { diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index 67222328794db..22a6ac5b473dc 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -1179,3 +1179,80 @@ func BenchmarkBatchAppendRows(b *testing.B) { }) } } + +func BenchmarkAppendRows(b *testing.B) { + b.ReportAllocs() + rowChk := newChunk(8, 8, 0, 0) + + for i := 0; i < 4096; i++ { + rowChk.AppendNull(0) + rowChk.AppendInt64(1, 1) + rowChk.AppendString(2, "abcd") + rowChk.AppendBytes(3, []byte("abcd")) + } + + type testCaseConf struct { + batchSize int + } + testCaseConfs := []testCaseConf{ + {batchSize: 2}, + {batchSize: 8}, + {batchSize: 16}, + {batchSize: 100}, + {batchSize: 1000}, + {batchSize: 4000}, + } + + chk := newChunk(8, 8, 0, 0) + for _, conf := range testCaseConfs { + b.ResetTimer() + b.Run(fmt.Sprintf("row-%d", conf.batchSize), func(b *testing.B) { + for i := 0; i < b.N; i++ { + chk.Reset() + for j := 0; j < conf.batchSize; j++ { + chk.AppendRow(rowChk.GetRow(j)) + } + } + }) + b.ResetTimer() + b.Run(fmt.Sprintf("column-%d", conf.batchSize), func(b *testing.B) { + for i := 0; i < b.N; i++ { + chk.Reset() + chk.Append(rowChk, 0, conf.batchSize) + } + }) + } +} + +func BenchmarkAppend(b *testing.B) { + b.ReportAllocs() + rowChk := newChunk(0, 0) + + for i := 0; i < 4096; i++ { + rowChk.AppendString(0, "abcd") + rowChk.AppendBytes(1, []byte("abcd")) + } + + type testCaseConf struct { + batchSize int + } + testCaseConfs := []testCaseConf{ + {batchSize: 2}, + {batchSize: 8}, + {batchSize: 16}, + {batchSize: 100}, + {batchSize: 1000}, + {batchSize: 4000}, + } + + chk := newChunk(0, 0) + for _, conf := range testCaseConfs { + b.ResetTimer() + b.Run(fmt.Sprintf("column-%d", conf.batchSize), func(b *testing.B) { + for i := 0; i < b.N; i++ { + chk.Reset() + chk.Append(rowChk, 0, conf.batchSize) + } + }) + } +} From c90f48d80d70d455e93cc606e3ef5162dbf566d6 Mon Sep 17 00:00:00 2001 From: Howie Date: Thu, 13 May 2021 13:07:39 +0800 Subject: [PATCH 05/42] *: typo fix (#24564) --- config/config.go | 6 +++--- tidb-server/main.go | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/config/config.go b/config/config.go index 7d6560b5d6783..664cb5bceaa7d 100644 --- a/config/config.go +++ b/config/config.go @@ -60,9 +60,9 @@ const ( DefHost = "0.0.0.0" // DefStatusHost is the default status host of TiDB DefStatusHost = "0.0.0.0" - // Def TableColumnCountLimit is limit of the number of columns in a table + // DefTableColumnCountLimit is limit of the number of columns in a table DefTableColumnCountLimit = 1017 - // Def TableColumnCountLimit is maximum limitation of the number of columns in a table + // DefMaxOfTableColumnCountLimit is maximum limitation of the number of columns in a table DefMaxOfTableColumnCountLimit = 4096 ) @@ -73,7 +73,7 @@ var ( "tikv": true, "unistore": true, } - // checkTableBeforeDrop enable to execute `admin check table` before `drop table`. + // CheckTableBeforeDrop enable to execute `admin check table` before `drop table`. CheckTableBeforeDrop = false // checkBeforeDropLDFlag is a go build flag. checkBeforeDropLDFlag = "None" diff --git a/tidb-server/main.go b/tidb-server/main.go index 3e2351bf7c352..6429ba960a0cb 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -629,7 +629,7 @@ func setupMetrics() { metrics.TimeJumpBackCounter.Inc() } callBackCount := 0 - sucessCallBack := func() { + successCallBack := func() { callBackCount++ // It is callback by monitor per second, we increase metrics.KeepAliveCounter per 5s. if callBackCount >= 5 { @@ -637,7 +637,7 @@ func setupMetrics() { metrics.KeepAliveCounter.Inc() } } - go systimemon.StartMonitor(time.Now, systimeErrHandler, sucessCallBack) + go systimemon.StartMonitor(time.Now, systimeErrHandler, successCallBack) pushMetric(cfg.Status.MetricsAddr, time.Duration(cfg.Status.MetricsInterval)*time.Second) } From 956149cdbb8359e618240c7f5568ba5af819eb57 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 13 May 2021 13:35:39 +0800 Subject: [PATCH 06/42] planner/core: refresh stale regions in cache for batch cop response (#24457) --- go.mod | 2 +- go.sum | 4 ++-- store/copr/batch_coprocessor.go | 14 ++++++++++++++ store/mockstore/unistore/tikv/server.go | 5 +++++ store/tikv/region_request_test.go | 4 ++++ 5 files changed, 26 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index b2bd664da13f3..c4b668a81dcf1 100644 --- a/go.mod +++ b/go.mod @@ -45,7 +45,7 @@ require ( github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e + github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 diff --git a/go.sum b/go.sum index c42d129ae5b8b..212917e414404 100644 --- a/go.sum +++ b/go.sum @@ -436,8 +436,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17Xtb github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e h1:oUMZ6X/Kpaoxfejh9/jQ+4UZ5xk9MRYcouWJ0oXRKNE= -github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c h1:cy87vgUJT0U4JuxC7R14PuwBrabI9fDawYhyKTbjOBQ= +github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 4bec370a9a4d5..bf8d1b7c893e0 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -406,6 +406,20 @@ func (b *batchCopIterator) handleBatchCopResponse(bo *backoffer, response *copro return errors.Trace(err) } + if len(response.RetryRegions) > 0 { + logutil.BgLogger().Info("multiple regions are stale and need to be refreshed", zap.Int("region size", len(response.RetryRegions))) + for idx, retry := range response.RetryRegions { + id := tikv.NewRegionVerID(retry.Id, retry.RegionEpoch.ConfVer, retry.RegionEpoch.Version) + logutil.BgLogger().Info("invalid region because tiflash detected stale region", zap.String("region id", id.String())) + b.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) + if idx >= 10 { + logutil.BgLogger().Info("stale regions are too many, so we omit the rest ones") + break + } + } + return + } + resp := batchCopResponse{ pbResp: response, detail: new(CopRuntimeStats), diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index f571ff4fe963f..adf3049330897 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -845,6 +845,11 @@ func (svr *Server) ReadIndex(context.Context, *kvrpcpb.ReadIndexRequest) (*kvrpc return &kvrpcpb.ReadIndexResponse{}, nil } +// GetLockWaitInfo implements implements the tikvpb.TikvServer interface. +func (svr *Server) GetLockWaitInfo(ctx context.Context, _ *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + return &kvrpcpb.GetLockWaitInfoResponse{}, nil +} + // transaction debugger commands. // MvccGetByKey implements implements the tikvpb.TikvServer interface. diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 9c5172e52f372..8d531ee209a78 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -462,6 +462,10 @@ func (s *mockTikvGrpcServer) SplitRegion(context.Context, *kvrpcpb.SplitRegionRe return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + return nil, errors.New("unreachable") +} + func (s *mockTikvGrpcServer) CoprocessorStream(*coprocessor.Request, tikvpb.Tikv_CoprocessorStreamServer) error { return errors.New("unreachable") } From f2c2fbda476f71bd1341c2c35ba727d0256a4914 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Thu, 13 May 2021 14:13:39 +0800 Subject: [PATCH 07/42] binlog: DML on temporary tables do not write binlog (#24570) --- sessionctx/binloginfo/binloginfo_test.go | 33 ++++++++++++++++++++++++ table/tables/tables.go | 3 +++ 2 files changed, 36 insertions(+) diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 2dfca57d73f4c..b0d9ec91888e4 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -698,3 +698,36 @@ func testGetTableByName(c *C, ctx sessionctx.Context, db, table string) table.Ta c.Assert(err, IsNil) return tbl } + +func (s *testBinlogSuite) TestTempTableBinlog(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.Se.GetSessionVars().BinlogClient = s.client + tk.MustExec("begin") + tk.MustExec("drop table if exists temp_table") + ddlQuery := "create global temporary table temp_table(id int) on commit delete rows" + tk.MustExec(ddlQuery) + ok := mustGetDDLBinlog(s, ddlQuery, c) + c.Assert(ok, IsTrue) + + tk.MustExec("insert temp_table value(1)") + tk.MustExec("update temp_table set id=id+1") + tk.MustExec("commit") + prewriteVal := getLatestBinlogPrewriteValue(c, s.pump) + c.Assert(len(prewriteVal.Mutations), Equals, 0) + + tk.MustExec("begin") + tk.MustExec("delete from temp_table") + tk.MustExec("commit") + prewriteVal = getLatestBinlogPrewriteValue(c, s.pump) + c.Assert(len(prewriteVal.Mutations), Equals, 0) + + ddlQuery = "truncate table temp_table" + tk.MustExec(ddlQuery) + ok = mustGetDDLBinlog(s, ddlQuery, c) + c.Assert(ok, IsTrue) + + ddlQuery = "drop table if exists temp_table" + tk.MustExec(ddlQuery) + ok = mustGetDDLBinlog(s, ddlQuery, c) + c.Assert(ok, IsTrue) +} diff --git a/table/tables/tables.go b/table/tables/tables.go index 74fd2d82f3ef9..a6a4180401d6a 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1414,6 +1414,9 @@ func shouldWriteBinlog(ctx sessionctx.Context, tblInfo *model.TableInfo) bool { if ctx.GetSessionVars().BinlogClient == nil { return false } + if tblInfo.TempTableType != model.TempTableNone { + return false + } return !ctx.GetSessionVars().InRestrictedSQL } From cc83cc524f8d3fd661f6e62d129ba043cc74501e Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 14:45:40 +0800 Subject: [PATCH 08/42] store/tikv: remove use of GuaranteeLinearizability option in store/tikv (#24605) --- session/session.go | 2 +- store/driver/txn/txn_driver.go | 4 ++++ store/tikv/2pc.go | 4 +--- store/tikv/tests/async_commit_test.go | 3 +-- store/tikv/tests/snapshot_fail_test.go | 2 +- store/tikv/txn.go | 14 ++++++++++++++ 6 files changed, 22 insertions(+), 7 deletions(-) diff --git a/session/session.go b/session/session.go index 0b4cb309f434b..19312e5fc391e 100644 --- a/session/session.go +++ b/session/session.go @@ -517,7 +517,7 @@ func (s *session) doCommit(ctx context.Context) error { s.txn.SetOption(tikvstore.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) s.txn.SetOption(tikvstore.Enable1PC, s.GetSessionVars().Enable1PC) // priority of the sysvar is lower than `start transaction with causal consistency only` - if s.txn.GetOption(tikvstore.GuaranteeLinearizability) == nil { + if val := s.txn.GetOption(tikvstore.GuaranteeLinearizability); val == nil || val.(bool) { // We needn't ask the TiKV client to guarantee linearizability for auto-commit transactions // because the property is naturally holds: // We guarantee the commitTS of any transaction must not exceed the next timestamp from the TSO. diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 12f2c8233ccb1..8e8d776eea982 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -162,6 +162,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetEnableAsyncCommit(val.(bool)) case tikvstore.Enable1PC: txn.SetEnable1PC(val.(bool)) + case tikvstore.GuaranteeLinearizability: + txn.SetCausalConsistency(!val.(bool)) case tikvstore.TxnScope: txn.SetScope(val.(string)) case tikvstore.IsStalenessReadOnly: @@ -175,6 +177,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { func (txn *tikvTxn) GetOption(opt int) interface{} { switch opt { + case tikvstore.GuaranteeLinearizability: + return !txn.KVTxn.IsCasualConsistency() case tikvstore.TxnScope: return txn.KVTxn.GetScope() default: diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index ee94eceec166a..b0fa2018670e1 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -854,9 +854,7 @@ func (c *twoPhaseCommitter) checkOnePC() bool { } func (c *twoPhaseCommitter) needLinearizability() bool { - GuaranteeLinearizabilityOption := c.txn.us.GetOption(kv.GuaranteeLinearizability) - // by default, guarantee - return GuaranteeLinearizabilityOption == nil || GuaranteeLinearizabilityOption.(bool) + return !c.txn.causalConsistency } func (c *twoPhaseCommitter) isAsyncCommit() bool { diff --git a/store/tikv/tests/async_commit_test.go b/store/tikv/tests/async_commit_test.go index 381771bfa0836..f67482e69a44e 100644 --- a/store/tikv/tests/async_commit_test.go +++ b/store/tikv/tests/async_commit_test.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -127,7 +126,7 @@ func (s *testAsyncCommitCommon) mustGetNoneFromSnapshot(c *C, version uint64, ke func (s *testAsyncCommitCommon) beginAsyncCommitWithLinearizability(c *C) tikv.TxnProbe { txn := s.beginAsyncCommit(c) - txn.SetOption(kv.GuaranteeLinearizability, true) + txn.SetCausalConsistency(false) return txn } diff --git a/store/tikv/tests/snapshot_fail_test.go b/store/tikv/tests/snapshot_fail_test.go index aca3c59099cf7..9892061c44b8d 100644 --- a/store/tikv/tests/snapshot_fail_test.go +++ b/store/tikv/tests/snapshot_fail_test.go @@ -213,7 +213,7 @@ func (s *testSnapshotFailSuite) TestRetryPointGetResolveTS(c *C) { c.Assert(err, IsNil) txn.SetEnableAsyncCommit(false) txn.SetEnable1PC(false) - txn.SetOption(kv.GuaranteeLinearizability, false) + txn.SetCausalConsistency(true) // Prewrite the lock without committing it c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforeCommit", `pause`), IsNil) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index a8c0f70f8da8d..baeaa6b7fa508 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -84,6 +84,7 @@ type KVTxn struct { isPessimistic bool enableAsyncCommit bool enable1PC bool + causalConsistency bool scope string kvFilter KVFilter } @@ -283,6 +284,13 @@ func (txn *KVTxn) SetEnable1PC(b bool) { txn.enable1PC = b } +// SetCausalConsistency indicates if the transaction does not need to +// guarantee linearizability. Default value is false which means +// linearizability is guaranteed. +func (txn *KVTxn) SetCausalConsistency(b bool) { + txn.causalConsistency = b +} + // SetScope sets the geographical scope of the transaction. func (txn *KVTxn) SetScope(scope string) { txn.scope = scope @@ -298,6 +306,12 @@ func (txn *KVTxn) IsPessimistic() bool { return txn.isPessimistic } +// IsCasualConsistency returns if the transaction allows linearizability +// inconsistency. +func (txn *KVTxn) IsCasualConsistency() bool { + return txn.causalConsistency +} + // GetScope returns the geographical scope of the transaction. func (txn *KVTxn) GetScope() string { return txn.scope From 9692c1386bbe2caa1dae8a5a3c9aa29c79c49833 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 15:19:40 +0800 Subject: [PATCH 09/42] store/tikv: remove use of CollectRuntimeStats option in store/tikv (#24604) --- store/driver/txn/snapshot.go | 11 +++++++++-- store/driver/txn/txn_driver.go | 11 +++++++++++ store/tikv/snapshot.go | 29 ++++++++--------------------- store/tikv/tests/snapshot_test.go | 3 +-- store/tikv/txn.go | 1 - 5 files changed, 29 insertions(+), 26 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index ee1d1eeee29d8..a6a7d752a72fd 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -81,12 +81,19 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetSampleStep(val.(uint32)) case tikvstore.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) + case tikvstore.CollectRuntimeStats: + s.KVSnapshot.SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) case tikvstore.IsStalenessReadOnly: s.KVSnapshot.SetIsStatenessReadOnly(val.(bool)) case tikvstore.MatchStoreLabels: s.KVSnapshot.SetMatchStoreLabels(val.([]*metapb.StoreLabel)) - default: - s.KVSnapshot.SetOption(opt, val) + } +} + +func (s *tikvSnapshot) DelOption(opt int) { + switch opt { + case tikvstore.CollectRuntimeStats: + s.KVSnapshot.SetRuntimeStats(nil) } } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 8e8d776eea982..0cd51a4480ee1 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -152,6 +152,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case tikvstore.InfoSchema: txn.SetSchemaVer(val.(tikv.SchemaVer)) + case tikvstore.CollectRuntimeStats: + txn.KVTxn.GetSnapshot().SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) case tikvstore.SchemaAmender: txn.SetSchemaAmender(val.(tikv.SchemaAmender)) case tikvstore.SampleStep: @@ -186,6 +188,15 @@ func (txn *tikvTxn) GetOption(opt int) interface{} { } } +func (txn *tikvTxn) DelOption(opt int) { + switch opt { + case tikvstore.CollectRuntimeStats: + txn.KVTxn.GetSnapshot().SetRuntimeStats(nil) + default: + txn.KVTxn.DelOption(opt) + } +} + // SetVars sets variables to the transaction. func (txn *tikvTxn) SetVars(vars interface{}) { if vs, ok := vars.(*tikv.Variables); ok { diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index ae65f15dc18e6..24149fdae44cb 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -561,27 +561,6 @@ func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) { return scanner, errors.Trace(err) } -// SetOption sets an option with a value, when val is nil, uses the default -// value of this option. Only ReplicaRead is supported for snapshot -func (s *KVSnapshot) SetOption(opt int, val interface{}) { - switch opt { - case kv.CollectRuntimeStats: - s.mu.Lock() - s.mu.stats = val.(*SnapshotRuntimeStats) - s.mu.Unlock() - } -} - -// DelOption deletes an option. -func (s *KVSnapshot) DelOption(opt int) { - switch opt { - case kv.CollectRuntimeStats: - s.mu.Lock() - s.mu.stats = nil - s.mu.Unlock() - } -} - // SetNotFillCache indicates whether tikv should skip filling cache when // loading data. func (s *KVSnapshot) SetNotFillCache(b bool) { @@ -623,6 +602,14 @@ func (s *KVSnapshot) SetTaskID(id uint64) { s.mu.taskID = id } +// SetRuntimeStats sets the stats to collect runtime statistics. +// Set it to nil to clear stored stats. +func (s *KVSnapshot) SetRuntimeStats(stats *SnapshotRuntimeStats) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.stats = stats +} + // SetIsStatenessReadOnly indicates whether the transaction is staleness read only transaction func (s *KVSnapshot) SetIsStatenessReadOnly(b bool) { s.mu.Lock() diff --git a/store/tikv/tests/snapshot_test.go b/store/tikv/tests/snapshot_test.go index fa1ccdd5735bd..a126decfc1c7d 100644 --- a/store/tikv/tests/snapshot_test.go +++ b/store/tikv/tests/snapshot_test.go @@ -26,7 +26,6 @@ import ( pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/tikvrpc" "go.uber.org/zap" @@ -270,7 +269,7 @@ func (s *testSnapshotSuite) TestSnapshotRuntimeStats(c *C) { tikv.RecordRegionRequestRuntimeStats(reqStats.Stats, tikvrpc.CmdGet, time.Second) tikv.RecordRegionRequestRuntimeStats(reqStats.Stats, tikvrpc.CmdGet, time.Millisecond) snapshot := s.store.GetSnapshot(0) - snapshot.SetOption(kv.CollectRuntimeStats, &tikv.SnapshotRuntimeStats{}) + snapshot.SetRuntimeStats(&tikv.SnapshotRuntimeStats{}) snapshot.MergeRegionRequestStats(reqStats.Stats) snapshot.MergeRegionRequestStats(reqStats.Stats) bo := tikv.NewBackofferWithVars(context.Background(), 2000, nil) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index baeaa6b7fa508..76891cfa19bdb 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -224,7 +224,6 @@ func (txn *KVTxn) Delete(k []byte) error { // value of this option. func (txn *KVTxn) SetOption(opt int, val interface{}) { txn.us.SetOption(opt, val) - txn.snapshot.SetOption(opt, val) } // GetOption returns the option From 7c8ddd808aad077e8da436d737860626ea7ffc4b Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 13 May 2021 15:47:40 +0800 Subject: [PATCH 10/42] store/tikv: move Backoffer into a single package (#24525) --- store/copr/batch_coprocessor.go | 4 +- store/copr/coprocessor.go | 11 +- store/copr/mpp.go | 4 +- store/driver/tikv_driver.go | 4 +- store/gcworker/gc_worker.go | 14 +- store/gcworker/gc_worker_test.go | 5 +- store/tikv/2pc.go | 84 ++--- store/tikv/backoff.go | 419 ++--------------------- store/tikv/cleanup.go | 3 +- store/tikv/client_batch.go | 5 +- store/tikv/commit.go | 15 +- store/tikv/delete_range.go | 7 +- store/tikv/kv.go | 15 +- store/tikv/lock_resolver.go | 29 +- store/tikv/pessimistic.go | 9 +- store/tikv/prewrite.go | 23 +- store/tikv/range_task.go | 5 +- store/tikv/rawkv.go | 21 +- store/tikv/region_cache.go | 41 +-- store/tikv/region_cache_test.go | 7 +- store/tikv/region_request.go | 52 +-- store/tikv/region_request_test.go | 5 +- store/tikv/retry/backoff.go | 439 +++++++++++++++++++++++++ store/tikv/{ => retry}/backoff_test.go | 2 +- store/tikv/scan.go | 9 +- store/tikv/snapshot.go | 45 +-- store/tikv/split_region.go | 36 +- store/tikv/test_probe.go | 15 +- store/tikv/tests/lock_test.go | 2 +- store/tikv/txn.go | 15 +- 30 files changed, 731 insertions(+), 614 deletions(-) create mode 100644 store/tikv/retry/backoff.go rename store/tikv/{ => retry}/backoff_test.go (98%) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index bf8d1b7c893e0..c070f25a454da 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -177,7 +177,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V if req.KeepOrder || req.Desc { return copErrorResponse{errors.New("batch coprocessor cannot prove keep order or desc property")} } - ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTs) + ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs) bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildBatchCopTasks(bo, c.store.GetRegionCache(), ranges, req.StoreType) @@ -381,7 +381,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b return nil } - if err1 := bo.Backoff(tikv.BoTiKVRPC, errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil { + if err1 := bo.b.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil { return errors.Trace(err) } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 2c1e2d361af76..dd8474fd75c3a 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -72,7 +72,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa logutil.BgLogger().Debug("send batch requests") return c.sendBatch(ctx, req, vars) } - ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTs) + ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs) bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildCopTasks(bo, c.store.GetRegionCache(), ranges, req) @@ -829,11 +829,14 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *backoffer, rpcCtx *ti return nil, nil } - boRPCType := tikv.BoTiKVRPC + err1 := errors.Errorf("recv stream response error: %v, task: %s", err, task) if task.storeType == kv.TiFlash { - boRPCType = tikv.BoTiFlashRPC + err1 = bo.Backoff(tikv.BoTiFlashRPC, err1) + } else { + err1 = bo.b.BackoffTiKVRPC(err1) } - if err1 := bo.Backoff(boRPCType, errors.Errorf("recv stream response error: %v, task: %s", err, task)); err1 != nil { + + if err1 != nil { return nil, errors.Trace(err) } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 377e439a9392c..0d156de69fb20 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -55,7 +55,7 @@ func (c *MPPClient) selectAllTiFlashStore() []kv.MPPTaskMeta { // ConstructMPPTasks receives ScheduleRequest, which are actually collects of kv ranges. We allocates MPPTaskMeta for them and returns. func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest) ([]kv.MPPTaskMeta, error) { - ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTS) + ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTS) bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) if req.KeyRanges == nil { return c.selectAllTiFlashStore(), nil @@ -343,7 +343,7 @@ func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchReques return } - if err1 := bo.Backoff(tikv.BoTiKVRPC, errors.Errorf("recv stream response error: %v", err)); err1 != nil { + if err1 := bo.b.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v", err)); err1 != nil { if errors.Cause(err) == context.Canceled { logutil.BgLogger().Info("stream recv timeout", zap.Error(err)) } else { diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index 398be99520aa6..cc0f217280f31 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -206,6 +206,8 @@ var ( ldflagGetEtcdAddrsFromConfig = "0" // 1:Yes, otherwise:No ) +const getAllMembersBackoff = 5000 + // EtcdAddrs returns etcd server addresses. func (s *tikvStore) EtcdAddrs() ([]string, error) { if s.etcdAddrs == nil { @@ -220,7 +222,7 @@ func (s *tikvStore) EtcdAddrs() ([]string, error) { } ctx := context.Background() - bo := tikv.NewBackoffer(ctx, tikv.GetAllMembersBackoff) + bo := tikv.NewBackoffer(ctx, getAllMembersBackoff) etcdAddrs := make([]string, 0) pdClient := s.GetPDClient() if pdClient == nil { diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index 038efa30b92f3..e0aa993558b6c 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -1050,7 +1050,7 @@ func (w *GCWorker) resolveLocksForRange(ctx context.Context, safePoint uint64, s var stat tikv.RangeTaskStat key := startKey - bo := tikv.NewBackofferWithVars(ctx, tikv.GcResolveLockMaxBackoff, nil) + bo := tikv.NewGcResolveLockMaxBackoffer(ctx) failpoint.Inject("setGcResolveMaxBackoff", func(v failpoint.Value) { sleep := v.(int) // cooperate with github.com/pingcap/tidb/store/tikv/invalidCacheAndRetry @@ -1147,7 +1147,7 @@ retryScanAndResolve: if len(key) == 0 || (len(endKey) != 0 && bytes.Compare(key, endKey) >= 0) { break } - bo = tikv.NewBackofferWithVars(ctx, tikv.GcResolveLockMaxBackoff, nil) + bo = tikv.NewGcResolveLockMaxBackoffer(ctx) failpoint.Inject("setGcResolveMaxBackoff", func(v failpoint.Value) { sleep := v.(int) bo = tikv.NewBackofferWithVars(ctx, sleep, nil) @@ -1460,7 +1460,7 @@ func (w *GCWorker) resolveLocksAcrossRegions(ctx context.Context, locks []*tikv. failpoint.Return(errors.New("injectedError")) }) - bo := tikv.NewBackofferWithVars(ctx, tikv.GcResolveLockMaxBackoff, nil) + bo := tikv.NewGcResolveLockMaxBackoffer(ctx) for { if len(locks) == 0 { @@ -1496,18 +1496,20 @@ func (w *GCWorker) resolveLocksAcrossRegions(ctx context.Context, locks []*tikv. } // Recreate backoffer for next region - bo = tikv.NewBackofferWithVars(ctx, tikv.GcResolveLockMaxBackoff, nil) + bo = tikv.NewGcResolveLockMaxBackoffer(ctx) locks = locks[len(locksInRegion):] } return nil } +const gcOneRegionMaxBackoff = 20000 + func (w *GCWorker) uploadSafePointToPD(ctx context.Context, safePoint uint64) error { var newSafePoint uint64 var err error - bo := tikv.NewBackofferWithVars(ctx, tikv.GcOneRegionMaxBackoff, nil) + bo := tikv.NewBackofferWithVars(ctx, gcOneRegionMaxBackoff, nil) for { newSafePoint, err = w.pdClient.UpdateGCSafePoint(ctx, safePoint) if err != nil { @@ -1544,7 +1546,7 @@ func (w *GCWorker) doGCForRange(ctx context.Context, startKey []byte, endKey []b }() key := startKey for { - bo := tikv.NewBackofferWithVars(ctx, tikv.GcOneRegionMaxBackoff, nil) + bo := tikv.NewBackofferWithVars(ctx, gcOneRegionMaxBackoff, nil) loc, err := w.tikvStore.GetRegionCache().LocateKey(bo, key) if err != nil { return stat, errors.Trace(err) diff --git a/store/gcworker/gc_worker_test.go b/store/gcworker/gc_worker_test.go index 3bfd616929aec..bc09651e0d379 100644 --- a/store/gcworker/gc_worker_test.go +++ b/store/gcworker/gc_worker_test.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/oracle/oracles" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" pd "github.com/tikv/pd/client" ) @@ -412,7 +413,7 @@ func (s *testGCWorkerSuite) TestStatusVars(c *C) { func (s *testGCWorkerSuite) TestDoGCForOneRegion(c *C) { ctx := context.Background() - bo := tikv.NewBackofferWithVars(ctx, tikv.GcOneRegionMaxBackoff, nil) + bo := tikv.NewBackofferWithVars(ctx, gcOneRegionMaxBackoff, nil) loc, err := s.tikvStore.GetRegionCache().LocateKey(bo, []byte("")) c.Assert(err, IsNil) var regionErr *errorpb.Error @@ -943,7 +944,7 @@ func (s *testGCWorkerSuite) TestResolveLockRangeMeetRegionEnlargeCausedByRegionM mCluster.Merge(s.initRegion.regionID, region2) regionMeta, _ := mCluster.GetRegion(s.initRegion.regionID) err := s.tikvStore.GetRegionCache().OnRegionEpochNotMatch( - tikv.NewNoopBackoff(context.Background()), + retry.NewNoopBackoff(context.Background()), &tikv.RPCContext{Region: regionID, Store: &tikv.Store{}}, []*metapb.Region{regionMeta}) c.Assert(err, IsNil) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index b0fa2018670e1..9e4c28ad6fd34 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" @@ -542,7 +543,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh switch act := action.(type) { case actionPrewrite: // Do not update regionTxnSize on retries. They are not used when building a PrewriteRequest. - if len(bo.errors) == 0 { + if bo.ErrorsNum() == 0 { for _, group := range groups { c.regionTxnSize[group.region.id] = group.mutations.Len() } @@ -572,7 +573,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh valStr, ok := val.(string) if ok && c.sessionID > 0 { if firstIsPrimary && actionIsPessimiticLock { - logutil.Logger(bo.ctx).Warn("pessimisticLock failpoint", zap.String("valStr", valStr)) + logutil.Logger(bo.GetCtx()).Warn("pessimisticLock failpoint", zap.String("valStr", valStr)) switch valStr { case "pessimisticLockSkipPrimary": err = c.doActionOnBatches(bo, action, batchBuilder.allBatches()) @@ -587,7 +588,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh failpoint.Inject("pessimisticRollbackDoNth", func() { _, actionIsPessimisticRollback := action.(actionPessimisticRollback) if actionIsPessimisticRollback && c.sessionID > 0 { - logutil.Logger(bo.ctx).Warn("pessimisticRollbackDoNth failpoint") + logutil.Logger(bo.GetCtx()).Warn("pessimisticRollbackDoNth failpoint") failpoint.Return(nil) } }) @@ -607,16 +608,16 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh } // Already spawned a goroutine for async commit transaction. if actionIsCommit && !actionCommit.retry && !c.isAsyncCommit() { - secondaryBo := NewBackofferWithVars(context.Background(), int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) + secondaryBo := retry.NewBackofferWithVars(context.Background(), int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) go func() { if c.sessionID > 0 { failpoint.Inject("beforeCommitSecondaries", func(v failpoint.Value) { if s, ok := v.(string); !ok { - logutil.Logger(bo.ctx).Info("[failpoint] sleep 2s before commit secondary keys", + logutil.Logger(bo.GetCtx()).Info("[failpoint] sleep 2s before commit secondary keys", zap.Uint64("sessionID", c.sessionID), zap.Uint64("txnStartTS", c.startTS), zap.Uint64("txnCommitTS", c.commitTS)) time.Sleep(2 * time.Second) } else if s == "skip" { - logutil.Logger(bo.ctx).Info("[failpoint] injected skip committing secondaries", + logutil.Logger(bo.GetCtx()).Info("[failpoint] injected skip committing secondaries", zap.Uint64("sessionID", c.sessionID), zap.Uint64("txnStartTS", c.startTS), zap.Uint64("txnCommitTS", c.commitTS)) failpoint.Return() } @@ -722,6 +723,8 @@ func (tm *ttlManager) close() { close(tm.ch) } +const pessimisticLockMaxBackoff = 20000 + func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { // Ticker is set to 1/2 of the ManagedLockTTL. ticker := time.NewTicker(time.Duration(atomic.LoadUint64(&ManagedLockTTL)) * time.Millisecond / 2) @@ -735,12 +738,12 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { if tm.lockCtx != nil && tm.lockCtx.Killed != nil && atomic.LoadUint32(tm.lockCtx.Killed) != 0 { return } - bo := NewBackofferWithVars(context.Background(), pessimisticLockMaxBackoff, c.txn.vars) - now, err := c.store.GetOracle().GetTimestamp(bo.ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + bo := retry.NewBackofferWithVars(context.Background(), pessimisticLockMaxBackoff, c.txn.vars) + now, err := c.store.GetOracle().GetTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) if err != nil { - err1 := bo.Backoff(BoPDRPC, err) + err1 := bo.Backoff(retry.BoPDRPC, err) if err1 != nil { - logutil.Logger(bo.ctx).Warn("keepAlive get tso fail", + logutil.Logger(bo.GetCtx()).Warn("keepAlive get tso fail", zap.Error(err)) return } @@ -751,7 +754,7 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { if uptime > config.GetGlobalConfig().MaxTxnTTL { // Checks maximum lifetime for the ttlManager, so when something goes wrong // the key will not be locked forever. - logutil.Logger(bo.ctx).Info("ttlManager live up to its lifetime", + logutil.Logger(bo.GetCtx()).Info("ttlManager live up to its lifetime", zap.Uint64("txnStartTS", c.startTS), zap.Uint64("uptime", uptime), zap.Uint64("maxTxnTTL", config.GetGlobalConfig().MaxTxnTTL)) @@ -765,13 +768,13 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { } newTTL := uptime + atomic.LoadUint64(&ManagedLockTTL) - logutil.Logger(bo.ctx).Info("send TxnHeartBeat", + logutil.Logger(bo.GetCtx()).Info("send TxnHeartBeat", zap.Uint64("startTS", c.startTS), zap.Uint64("newTTL", newTTL)) startTime := time.Now() _, err = sendTxnHeartBeat(bo, c.store, c.primary(), c.startTS, newTTL) if err != nil { metrics.TxnHeartBeatHistogramError.Observe(time.Since(startTime).Seconds()) - logutil.Logger(bo.ctx).Warn("send TxnHeartBeat failed", + logutil.Logger(bo.GetCtx()).Warn("send TxnHeartBeat failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) return @@ -801,7 +804,7 @@ func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, tt return 0, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return 0, errors.Trace(err) } @@ -889,6 +892,11 @@ func (c *twoPhaseCommitter) checkOnePCFallBack(action twoPhaseCommitAction, batc } } +const ( + cleanupMaxBackoff = 20000 + tsoMaxBackoff = 15000 +) + func (c *twoPhaseCommitter) cleanup(ctx context.Context) { c.cleanWg.Add(1) go func() { @@ -899,12 +907,12 @@ func (c *twoPhaseCommitter) cleanup(ctx context.Context) { failpoint.Return() }) - cleanupKeysCtx := context.WithValue(context.Background(), TxnStartKey, ctx.Value(TxnStartKey)) + cleanupKeysCtx := context.WithValue(context.Background(), retry.TxnStartKey, ctx.Value(retry.TxnStartKey)) var err error if !c.isOnePC() { - err = c.cleanupMutations(NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations) + err = c.cleanupMutations(retry.NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations) } else if c.isPessimistic { - err = c.pessimisticRollbackMutations(NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations) + err = c.pessimisticRollbackMutations(retry.NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations) } if err != nil { @@ -1016,7 +1024,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { if c.shouldWriteBinlog() { binlogChan = c.binlog.Prewrite(ctx, c.primary()) } - prewriteBo := NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars) + prewriteBo := retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars) start := time.Now() err = c.prewriteMutations(prewriteBo, c.mutations) @@ -1035,10 +1043,10 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { commitDetail := c.getDetail() commitDetail.PrewriteTime = time.Since(start) - if prewriteBo.totalSleep > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(prewriteBo.totalSleep)*int64(time.Millisecond)) + if prewriteBo.GetTotalSleep() > 0 { + atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(prewriteBo.GetTotalSleep())*int64(time.Millisecond)) commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, prewriteBo.types...) + commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, prewriteBo.GetTypes()...) commitDetail.Mu.Unlock() } if binlogChan != nil { @@ -1092,7 +1100,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { } else { start = time.Now() logutil.Event(ctx, "start get commit ts") - commitTS, err = c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) + commitTS, err = c.store.getTimestampWithRetry(retry.NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) if err != nil { logutil.Logger(ctx).Warn("2PC get commitTS failed", zap.Error(err), @@ -1175,7 +1183,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { failpoint.Inject("asyncCommitDoNothing", func() { failpoint.Return() }) - commitBo := NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) + commitBo := retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) err := c.commitMutations(commitBo, c.mutations) if err != nil { logutil.Logger(ctx).Warn("2PC async commit failed", zap.Uint64("sessionID", c.sessionID), @@ -1191,13 +1199,13 @@ func (c *twoPhaseCommitter) commitTxn(ctx context.Context, commitDetail *util.Co c.txn.GetMemBuffer().DiscardValues() start := time.Now() - commitBo := NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) + commitBo := retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) err := c.commitMutations(commitBo, c.mutations) commitDetail.CommitTime = time.Since(start) - if commitBo.totalSleep > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(commitBo.totalSleep)*int64(time.Millisecond)) + if commitBo.GetTotalSleep() > 0 { + atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(commitBo.GetTotalSleep())*int64(time.Millisecond)) commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, commitBo.types...) + commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, commitBo.GetTypes()...) commitDetail.Mu.Unlock() } if err != nil { @@ -1282,7 +1290,7 @@ func (c *twoPhaseCommitter) amendPessimisticLock(ctx context.Context, addMutatio retryLimit := config.GetGlobalConfig().PessimisticTxn.MaxRetryCount var err error for tryTimes < retryLimit { - pessimisticLockBo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, c.txn.vars) + pessimisticLockBo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, c.txn.vars) err = c.pessimisticLockMutations(pessimisticLockBo, lCtx, &keysNeedToLock) if err != nil { // KeysNeedToLock won't change, so don't async rollback pessimistic locks here for write conflict. @@ -1328,7 +1336,7 @@ func (c *twoPhaseCommitter) tryAmendTxn(ctx context.Context, startInfoSchema Sch return false, err } if c.prewriteStarted { - prewriteBo := NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars) + prewriteBo := retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars) err = c.prewriteMutations(prewriteBo, addMutations) if err != nil { logutil.Logger(ctx).Warn("amend prewrite has failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) @@ -1361,7 +1369,7 @@ func (c *twoPhaseCommitter) tryAmendTxn(ctx context.Context, startInfoSchema Sch func (c *twoPhaseCommitter) getCommitTS(ctx context.Context, commitDetail *util.CommitDetails) (uint64, error) { start := time.Now() logutil.Event(ctx, "start get commit ts") - commitTS, err := c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) + commitTS, err := c.store.getTimestampWithRetry(retry.NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) if err != nil { logutil.Logger(ctx).Warn("2PC get commitTS failed", zap.Error(err), @@ -1575,20 +1583,20 @@ func (batchExe *batchExecutor) startWorker(exitCh chan struct{}, ch chan error, singleBatchBackoffer, singleBatchCancel = batchExe.backoffer.Fork() defer singleBatchCancel() } - beforeSleep := singleBatchBackoffer.totalSleep + beforeSleep := singleBatchBackoffer.GetTotalSleep() ch <- batchExe.action.handleSingleBatch(batchExe.committer, singleBatchBackoffer, batch) commitDetail := batchExe.committer.getDetail() if commitDetail != nil { // lock operations of pessimistic-txn will let commitDetail be nil - if delta := singleBatchBackoffer.totalSleep - beforeSleep; delta > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(singleBatchBackoffer.totalSleep-beforeSleep)*int64(time.Millisecond)) + if delta := singleBatchBackoffer.GetTotalSleep() - beforeSleep; delta > 0 { + atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(singleBatchBackoffer.GetTotalSleep()-beforeSleep)*int64(time.Millisecond)) commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, singleBatchBackoffer.types...) + commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, singleBatchBackoffer.GetTypes()...) commitDetail.Mu.Unlock() } } }() } else { - logutil.Logger(batchExe.backoffer.ctx).Info("break startWorker", + logutil.Logger(batchExe.backoffer.GetCtx()).Info("break startWorker", zap.Stringer("action", batchExe.action), zap.Int("batch size", len(batches)), zap.Int("index", idx)) break @@ -1601,7 +1609,7 @@ func (batchExe *batchExecutor) process(batches []batchMutations) error { var err error err = batchExe.initUtils() if err != nil { - logutil.Logger(batchExe.backoffer.ctx).Error("batchExecutor initUtils failed", zap.Error(err)) + logutil.Logger(batchExe.backoffer.GetCtx()).Error("batchExecutor initUtils failed", zap.Error(err)) return err } @@ -1618,14 +1626,14 @@ func (batchExe *batchExecutor) process(batches []batchMutations) error { // check results for i := 0; i < len(batches); i++ { if e := <-ch; e != nil { - logutil.Logger(batchExe.backoffer.ctx).Debug("2PC doActionOnBatch failed", + logutil.Logger(batchExe.backoffer.GetCtx()).Debug("2PC doActionOnBatch failed", zap.Uint64("session", batchExe.committer.sessionID), zap.Stringer("action type", batchExe.action), zap.Error(e), zap.Uint64("txnStartTS", batchExe.committer.startTS)) // Cancel other requests and return the first error. if cancel != nil { - logutil.Logger(batchExe.backoffer.ctx).Debug("2PC doActionOnBatch to cancel other actions", + logutil.Logger(batchExe.backoffer.GetCtx()).Debug("2PC doActionOnBatch to cancel other actions", zap.Uint64("session", batchExe.committer.sessionID), zap.Stringer("action type", batchExe.action), zap.Uint64("txnStartTS", batchExe.committer.startTS)) diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index e0115c9e3904a..c622e21d2ee5d 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -1,4 +1,4 @@ -// Copyright 2016 PingCAP, Inc. +// Copyright 2021 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,431 +15,54 @@ package tikv import ( "context" - "fmt" - "math" - "math/rand" - "strings" - "sync/atomic" - "time" - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/log" - tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/util" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" + "github.com/pingcap/tidb/store/tikv/retry" ) -const ( - // NoJitter makes the backoff sequence strict exponential. - NoJitter = 1 + iota - // FullJitter applies random factors to strict exponential. - FullJitter - // EqualJitter is also randomized, but prevents very short sleeps. - EqualJitter - // DecorrJitter increases the maximum jitter based on the last random value. - DecorrJitter -) - -func (t BackoffType) metric() prometheus.Observer { - switch t { - // TODO: distinguish tikv and tiflash in metrics - case BoTiKVRPC, BoTiFlashRPC: - return metrics.BackoffHistogramRPC - case BoTxnLock: - return metrics.BackoffHistogramLock - case BoTxnLockFast: - return metrics.BackoffHistogramLockFast - case BoPDRPC: - return metrics.BackoffHistogramPD - case BoRegionMiss: - return metrics.BackoffHistogramRegionMiss - case boTiKVServerBusy, boTiFlashServerBusy: - return metrics.BackoffHistogramServerBusy - case boStaleCmd: - return metrics.BackoffHistogramStaleCmd - } - return metrics.BackoffHistogramEmpty -} - -// NewBackoffFn creates a backoff func which implements exponential backoff with -// optional jitters. -// See http://www.awsarchitectureblog.com/2015/03/backoff.html -func NewBackoffFn(base, cap, jitter int) func(ctx context.Context, maxSleepMs int) int { - if base < 2 { - // Top prevent panic in 'rand.Intn'. - base = 2 - } - attempts := 0 - lastSleep := base - return func(ctx context.Context, maxSleepMs int) int { - var sleep int - switch jitter { - case NoJitter: - sleep = expo(base, cap, attempts) - case FullJitter: - v := expo(base, cap, attempts) - sleep = rand.Intn(v) - case EqualJitter: - v := expo(base, cap, attempts) - sleep = v/2 + rand.Intn(v/2) - case DecorrJitter: - sleep = int(math.Min(float64(cap), float64(base+rand.Intn(lastSleep*3-base)))) - } - logutil.BgLogger().Debug("backoff", - zap.Int("base", base), - zap.Int("sleep", sleep), - zap.Int("attempts", attempts)) - - realSleep := sleep - // when set maxSleepMs >= 0 in `tikv.BackoffWithMaxSleep` will force sleep maxSleepMs milliseconds. - if maxSleepMs >= 0 && realSleep > maxSleepMs { - realSleep = maxSleepMs - } - select { - case <-time.After(time.Duration(realSleep) * time.Millisecond): - attempts++ - lastSleep = sleep - return realSleep - case <-ctx.Done(): - return 0 - } - } -} - -func expo(base, cap, n int) int { - return int(math.Min(float64(cap), float64(base)*math.Pow(2.0, float64(n)))) -} +// Backoffer is a utility for retrying queries. +type Backoffer = retry.Backoffer // BackoffType defines the backoff type. -type BackoffType int +type BackoffType = retry.BackoffType // Back off types. const ( - BoTiKVRPC BackoffType = iota - BoTiFlashRPC - BoTxnLock - BoTxnLockFast - BoPDRPC - BoRegionMiss - boTiKVServerBusy - boTiFlashServerBusy - boTxnNotFound - boStaleCmd - boMaxTsNotSynced + BoRegionMiss = retry.BoRegionMiss + BoTiFlashRPC = retry.BoTiFlashRPC + BoTxnLockFast = retry.BoTxnLockFast + BoTxnLock = retry.BoTxnLock + BoPDRPC = retry.BoPDRPC ) -func (t BackoffType) createFn(vars *kv.Variables) func(context.Context, int) int { - if vars.Hook != nil { - vars.Hook(t.String(), vars) - } - switch t { - case BoTiKVRPC, BoTiFlashRPC: - return NewBackoffFn(100, 2000, EqualJitter) - case BoTxnLock: - return NewBackoffFn(200, 3000, EqualJitter) - case BoTxnLockFast: - return NewBackoffFn(vars.BackoffLockFast, 3000, EqualJitter) - case BoPDRPC: - return NewBackoffFn(500, 3000, EqualJitter) - case BoRegionMiss: - // change base time to 2ms, because it may recover soon. - return NewBackoffFn(2, 500, NoJitter) - case boTxnNotFound: - return NewBackoffFn(2, 500, NoJitter) - case boTiKVServerBusy, boTiFlashServerBusy: - return NewBackoffFn(2000, 10000, EqualJitter) - case boStaleCmd: - return NewBackoffFn(2, 1000, NoJitter) - case boMaxTsNotSynced: - return NewBackoffFn(2, 500, NoJitter) - } - return nil -} - -func (t BackoffType) String() string { - switch t { - case BoTiKVRPC: - return "tikvRPC" - case BoTiFlashRPC: - return "tiflashRPC" - case BoTxnLock: - return "txnLock" - case BoTxnLockFast: - return "txnLockFast" - case BoPDRPC: - return "pdRPC" - case BoRegionMiss: - return "regionMiss" - case boTiKVServerBusy: - return "tikvServerBusy" - case boTiFlashServerBusy: - return "tiflashServerBusy" - case boStaleCmd: - return "staleCommand" - case boTxnNotFound: - return "txnNotFound" - case boMaxTsNotSynced: - return "maxTsNotSynced" - } - return "" -} - -// TError returns pingcap/error of the backoff type. -func (t BackoffType) TError() error { - switch t { - case BoTiKVRPC: - return tikverr.ErrTiKVServerTimeout - case BoTiFlashRPC: - return tikverr.ErrTiFlashServerTimeout - case BoTxnLock, BoTxnLockFast, boTxnNotFound: - return tikverr.ErrResolveLockTimeout - case BoPDRPC: - return tikverr.NewErrPDServerTimeout("") - case BoRegionMiss: - return tikverr.ErrRegionUnavailable - case boTiKVServerBusy: - return tikverr.ErrTiKVServerBusy - case boTiFlashServerBusy: - return tikverr.ErrTiFlashServerBusy - case boStaleCmd: - return tikverr.ErrTiKVStaleCommand - case boMaxTsNotSynced: - return tikverr.ErrTiKVMaxTimestampNotSynced - } - return tikverr.ErrUnknown -} - // Maximum total sleep time(in ms) for kv/cop commands. const ( - GetAllMembersBackoff = 5000 - tsoMaxBackoff = 15000 - scannerNextMaxBackoff = 20000 - batchGetMaxBackoff = 20000 - getMaxBackoff = 20000 - cleanupMaxBackoff = 20000 - GcOneRegionMaxBackoff = 20000 - GcResolveLockMaxBackoff = 100000 - deleteRangeOneRegionMaxBackoff = 100000 - rawkvMaxBackoff = 20000 - splitRegionBackoff = 20000 - maxSplitRegionsBackoff = 120000 - waitScatterRegionFinishBackoff = 120000 - locateRegionMaxBackoff = 20000 - pessimisticLockMaxBackoff = 20000 - pessimisticRollbackMaxBackoff = 20000 + gcResolveLockMaxBackoff = 100000 ) var ( // CommitMaxBackoff is max sleep time of the 'commit' command CommitMaxBackoff = uint64(41000) - // PrewriteMaxBackoff is max sleep time of the `pre-write` command. PrewriteMaxBackoff = 20000 ) -// Backoffer is a utility for retrying queries. -type Backoffer struct { - ctx context.Context - - fn map[BackoffType]func(context.Context, int) int - maxSleep int - totalSleep int - errors []error - types []fmt.Stringer - vars *kv.Variables - noop bool - - backoffSleepMS map[BackoffType]int - backoffTimes map[BackoffType]int -} - -type txnStartCtxKeyType struct{} - -// TxnStartKey is a key for transaction start_ts info in context.Context. -var TxnStartKey interface{} = txnStartCtxKeyType{} - -// NewBackoffer (Deprecated) creates a Backoffer with maximum sleep time(in ms). -func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { - return &Backoffer{ - ctx: ctx, - maxSleep: maxSleep, - vars: kv.DefaultVars, - } -} - // NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *Backoffer { - return NewBackoffer(ctx, maxSleep).withVars(vars) + return retry.NewBackofferWithVars(ctx, maxSleep, vars) } -// NewNoopBackoff create a Backoffer do nothing just return error directly -func NewNoopBackoff(ctx context.Context) *Backoffer { - return &Backoffer{ctx: ctx, noop: true} -} - -// withVars sets the kv.Variables to the Backoffer and return it. -func (b *Backoffer) withVars(vars *kv.Variables) *Backoffer { - if vars != nil { - b.vars = vars - } - // maxSleep is the max sleep time in millisecond. - // When it is multiplied by BackOffWeight, it should not be greater than MaxInt32. - if math.MaxInt32/b.vars.BackOffWeight >= b.maxSleep { - b.maxSleep *= b.vars.BackOffWeight - } - return b -} - -// Backoff sleeps a while base on the backoffType and records the error message. -// It returns a retryable error if total sleep time exceeds maxSleep. -func (b *Backoffer) Backoff(typ BackoffType, err error) error { - if span := opentracing.SpanFromContext(b.ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan(fmt.Sprintf("tikv.backoff.%s", typ), opentracing.ChildOf(span.Context())) - defer span1.Finish() - opentracing.ContextWithSpan(b.ctx, span1) - } - return b.BackoffWithMaxSleep(typ, -1, err) -} - -// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message -// and never sleep more than maxSleepMs for each sleep. -func (b *Backoffer) BackoffWithMaxSleep(typ BackoffType, maxSleepMs int, err error) error { - if strings.Contains(err.Error(), tikverr.MismatchClusterID) { - logutil.BgLogger().Fatal("critical error", zap.Error(err)) - } - select { - case <-b.ctx.Done(): - return errors.Trace(err) - default: - } - - b.errors = append(b.errors, errors.Errorf("%s at %s", err.Error(), time.Now().Format(time.RFC3339Nano))) - b.types = append(b.types, typ) - if b.noop || (b.maxSleep > 0 && b.totalSleep >= b.maxSleep) { - errMsg := fmt.Sprintf("%s backoffer.maxSleep %dms is exceeded, errors:", typ.String(), b.maxSleep) - for i, err := range b.errors { - // Print only last 3 errors for non-DEBUG log levels. - if log.GetLevel() == zapcore.DebugLevel || i >= len(b.errors)-3 { - errMsg += "\n" + err.Error() - } - } - logutil.BgLogger().Warn(errMsg) - // Use the first backoff type to generate a MySQL error. - return b.types[0].(BackoffType).TError() - } - - // Lazy initialize. - if b.fn == nil { - b.fn = make(map[BackoffType]func(context.Context, int) int) - } - f, ok := b.fn[typ] - if !ok { - f = typ.createFn(b.vars) - b.fn[typ] = f - } - - realSleep := f(b.ctx, maxSleepMs) - typ.metric().Observe(float64(realSleep) / 1000) - b.totalSleep += realSleep - if b.backoffSleepMS == nil { - b.backoffSleepMS = make(map[BackoffType]int) - } - b.backoffSleepMS[typ] += realSleep - if b.backoffTimes == nil { - b.backoffTimes = make(map[BackoffType]int) - } - b.backoffTimes[typ]++ - - stmtExec := b.ctx.Value(util.ExecDetailsKey) - if stmtExec != nil { - detail := stmtExec.(*util.ExecDetails) - atomic.AddInt64(&detail.BackoffDuration, int64(realSleep)*int64(time.Millisecond)) - atomic.AddInt64(&detail.BackoffCount, 1) - } - - if b.vars != nil && b.vars.Killed != nil { - if atomic.LoadUint32(b.vars.Killed) == 1 { - return tikverr.ErrQueryInterrupted - } - } - - var startTs interface{} - if ts := b.ctx.Value(TxnStartKey); ts != nil { - startTs = ts - } - logutil.Logger(b.ctx).Debug("retry later", - zap.Error(err), - zap.Int("totalSleep", b.totalSleep), - zap.Int("maxSleep", b.maxSleep), - zap.Stringer("type", typ), - zap.Reflect("txnStartTS", startTs)) - return nil -} - -func (b *Backoffer) String() string { - if b.totalSleep == 0 { - return "" - } - return fmt.Sprintf(" backoff(%dms %v)", b.totalSleep, b.types) -} - -// Clone creates a new Backoffer which keeps current Backoffer's sleep time and errors, and shares -// current Backoffer's context. -func (b *Backoffer) Clone() *Backoffer { - return &Backoffer{ - ctx: b.ctx, - maxSleep: b.maxSleep, - totalSleep: b.totalSleep, - errors: b.errors, - vars: b.vars, - } -} - -// Fork creates a new Backoffer which keeps current Backoffer's sleep time and errors, and holds -// a child context of current Backoffer's context. -func (b *Backoffer) Fork() (*Backoffer, context.CancelFunc) { - ctx, cancel := context.WithCancel(b.ctx) - return &Backoffer{ - ctx: ctx, - maxSleep: b.maxSleep, - totalSleep: b.totalSleep, - errors: b.errors, - vars: b.vars, - }, cancel -} - -// GetVars returns the binded vars. -func (b *Backoffer) GetVars() *kv.Variables { - return b.vars -} - -// GetTotalSleep returns total sleep time. -func (b *Backoffer) GetTotalSleep() int { - return b.totalSleep -} - -// GetTypes returns type list. -func (b *Backoffer) GetTypes() []fmt.Stringer { - return b.types -} - -// GetCtx returns the binded context. -func (b *Backoffer) GetCtx() context.Context { - return b.ctx +// NewBackoffer creates a Backoffer with maximum sleep time(in ms). +func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { + return retry.NewBackoffer(ctx, maxSleep) } -// GetBackoffTimes returns a map contains backoff time count by type. -func (b *Backoffer) GetBackoffTimes() map[BackoffType]int { - return b.backoffTimes +// TxnStartKey is a key for transaction start_ts info in context.Context. +func TxnStartKey() interface{} { + return retry.TxnStartKey } -// GetBackoffSleepMS returns a map contains backoff sleep time by type. -func (b *Backoffer) GetBackoffSleepMS() map[BackoffType]int { - return b.backoffSleepMS +// NewGcResolveLockMaxBackoffer creates a Backoffer for Gc to resolve lock. +func NewGcResolveLockMaxBackoffer(ctx context.Context) *Backoffer { + return retry.NewBackofferWithVars(ctx, gcResolveLockMaxBackoff, nil) } diff --git a/store/tikv/cleanup.go b/store/tikv/cleanup.go index dc96ed32ab54c..0260d770cdd44 100644 --- a/store/tikv/cleanup.go +++ b/store/tikv/cleanup.go @@ -18,6 +18,7 @@ import ( pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" @@ -49,7 +50,7 @@ func (actionCleanup) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batc return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/client_batch.go b/store/tikv/client_batch.go index 886d20abf0e46..e5ec039fc6911 100644 --- a/store/tikv/client_batch.go +++ b/store/tikv/client_batch.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" @@ -662,7 +663,7 @@ func (c *batchCommandsClient) recreateStreamingClient(err error, streamClient *b *epoch++ c.failPendingRequests(err) // fail all pending requests. - b := NewBackofferWithVars(context.Background(), math.MaxInt32, nil) + b := retry.NewBackofferWithVars(context.Background(), math.MaxInt32, nil) for { // try to re-create the streaming in the loop. if c.isStopped() { return true @@ -672,7 +673,7 @@ func (c *batchCommandsClient) recreateStreamingClient(err error, streamClient *b break } - err2 := b.Backoff(BoTiKVRPC, err1) + err2 := b.BackoffTiKVRPC(err1) // As timeout is set to math.MaxUint32, err2 should always be nil. // This line is added to make the 'make errcheck' pass. terror.Log(err2) diff --git a/store/tikv/commit.go b/store/tikv/commit.go index ce9df6a927355..449081860c029 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -22,6 +22,7 @@ import ( tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" @@ -67,7 +68,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -86,7 +87,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch } if keyErr := commitResp.GetError(); keyErr != nil { if rejected := keyErr.GetCommitTsExpired(); rejected != nil { - logutil.Logger(bo.ctx).Info("2PC commitTS rejected by TiKV, retry with a newer commitTS", + logutil.Logger(bo.GetCtx()).Info("2PC commitTS rejected by TiKV, retry with a newer commitTS", zap.Uint64("txnStartTS", c.startTS), zap.Stringer("info", logutil.Hex(rejected))) @@ -101,7 +102,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch // Update commit ts and retry. commitTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) if err != nil { - logutil.Logger(bo.ctx).Warn("2PC get commitTS failed", + logutil.Logger(bo.GetCtx()).Warn("2PC get commitTS failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) return errors.Trace(err) @@ -126,7 +127,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch } return res } - logutil.Logger(bo.ctx).Error("2PC failed commit key after primary key committed", + logutil.Logger(bo.GetCtx()).Error("2PC failed commit key after primary key committed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS), zap.Uint64("commitTS", c.commitTS), @@ -134,7 +135,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch return errors.Trace(err) } // The transaction maybe rolled back by concurrent transactions. - logutil.Logger(bo.ctx).Debug("2PC failed commit primary key", + logutil.Logger(bo.GetCtx()).Debug("2PC failed commit primary key", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) return err @@ -149,10 +150,10 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch } func (c *twoPhaseCommitter) commitMutations(bo *Backoffer, mutations CommitterMutations) error { - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("twoPhaseCommitter.commitMutations", opentracing.ChildOf(span.Context())) defer span1.Finish() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } return c.doActionOnMutations(bo, actionCommit{}, mutations) diff --git a/store/tikv/delete_range.go b/store/tikv/delete_range.go index 4cbe9fc039749..6a826d718cba1 100644 --- a/store/tikv/delete_range.go +++ b/store/tikv/delete_range.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) @@ -78,6 +79,8 @@ func (t *DeleteRangeTask) Execute(ctx context.Context) error { return err } +const deleteRangeOneRegionMaxBackoff = 100000 + // Execute performs the delete range operation. func (t *DeleteRangeTask) sendReqOnRange(ctx context.Context, r kv.KeyRange) (RangeTaskStat, error) { startKey, rangeEndKey := r.StartKey, r.EndKey @@ -93,7 +96,7 @@ func (t *DeleteRangeTask) sendReqOnRange(ctx context.Context, r kv.KeyRange) (Ra break } - bo := NewBackofferWithVars(ctx, deleteRangeOneRegionMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, deleteRangeOneRegionMaxBackoff, nil) loc, err := t.store.GetRegionCache().LocateKey(bo, startKey) if err != nil { return stat, errors.Trace(err) @@ -121,7 +124,7 @@ func (t *DeleteRangeTask) sendReqOnRange(ctx context.Context, r kv.KeyRange) (Ra return stat, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return stat, errors.Trace(err) } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 981a1b7bc5cab..f61db4168ef7d 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/oracle/oracles" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" @@ -226,7 +227,7 @@ func (s *KVStore) UUID() string { // CurrentTimestamp returns current timestamp with the given txnScope (local or global). func (s *KVStore) CurrentTimestamp(txnScope string) (uint64, error) { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) startTS, err := s.getTimestampWithRetry(bo, txnScope) if err != nil { return 0, errors.Trace(err) @@ -242,14 +243,14 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, panic("MockCurrentTimestamp should be a number, try use this failpoint with \"return(ts)\"") } }) - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("TiKVStore.getTimestampWithRetry", opentracing.ChildOf(span.Context())) defer span1.Finish() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } for { - startTS, err := s.oracle.GetTimestamp(bo.ctx, &oracle.Option{TxnScope: txnScope}) + startTS, err := s.oracle.GetTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: txnScope}) // mockGetTSErrorInRetry should wait MockCommitErrorOnce first, then will run into retry() logic. // 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 @@ -263,7 +264,7 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, if err == nil { return startTS, nil } - err = bo.Backoff(BoPDRPC, errors.Errorf("get timestamp failed: %v", err)) + err = bo.Backoff(retry.BoPDRPC, errors.Errorf("get timestamp failed: %v", err)) if err != nil { return 0, errors.Trace(err) } @@ -279,11 +280,11 @@ func (s *KVStore) getStalenessTimestamp(bo *Backoffer, txnScope string, prevSec } }) for { - startTS, err := s.oracle.GetStaleTimestamp(bo.ctx, txnScope, prevSec) + startTS, err := s.oracle.GetStaleTimestamp(bo.GetCtx(), txnScope, prevSec) if err == nil { return startTS, nil } - err = bo.Backoff(BoPDRPC, errors.Errorf("get staleness timestamp failed: %v", err)) + err = bo.Backoff(retry.BoPDRPC, errors.Errorf("get staleness timestamp failed: %v", err)) if err != nil { return 0, errors.Trace(err) } diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index 05feee6d31adb..fe50910a896e6 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/util" pd "github.com/tikv/pd/client" @@ -228,7 +229,7 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc Regi // locks have been cleaned before GC. expiredLocks := locks - callerStartTS, err := lr.store.GetOracle().GetTimestamp(bo.ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + callerStartTS, err := lr.store.GetOracle().GetTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) if err != nil { return false, errors.Trace(err) } @@ -297,7 +298,7 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc Regi } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return false, errors.Trace(err) } @@ -473,8 +474,8 @@ func (t *txnExpireTime) value() int64 { // seconds before calling it after Prewrite. func (lr *LockResolver) GetTxnStatus(txnID uint64, callerStartTS uint64, primary []byte) (TxnStatus, error) { var status TxnStatus - bo := NewBackoffer(context.Background(), cleanupMaxBackoff) - currentTS, err := lr.store.GetOracle().GetLowResolutionTimestamp(bo.ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + bo := retry.NewBackoffer(context.Background(), cleanupMaxBackoff) + currentTS, err := lr.store.GetOracle().GetLowResolutionTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) if err != nil { return status, err } @@ -493,7 +494,7 @@ func (lr *LockResolver) getTxnStatusFromLock(bo *Backoffer, l *Lock, callerStart // Set currentTS to max uint64 to make the lock expired. currentTS = math.MaxUint64 } else { - currentTS, err = lr.store.GetOracle().GetLowResolutionTimestamp(bo.ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + currentTS, err = lr.store.GetOracle().GetLowResolutionTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) if err != nil { return TxnStatus{}, err } @@ -522,12 +523,12 @@ func (lr *LockResolver) getTxnStatusFromLock(bo *Backoffer, l *Lock, callerStart // getTxnStatus() returns it when the secondary locks exist while the primary lock doesn't. // This is likely to happen in the concurrently prewrite when secondary regions // success before the primary region. - if err := bo.Backoff(boTxnNotFound, err); err != nil { - logutil.Logger(bo.ctx).Warn("getTxnStatusFromLock backoff fail", zap.Error(err)) + if err := bo.Backoff(retry.BoTxnNotFound, err); err != nil { + logutil.Logger(bo.GetCtx()).Warn("getTxnStatusFromLock backoff fail", zap.Error(err)) } if lr.store.GetOracle().UntilExpired(l.TxnID, l.TTL, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) <= 0 { - logutil.Logger(bo.ctx).Warn("lock txn not found, lock has expired", + logutil.Logger(bo.GetCtx()).Warn("lock txn not found, lock has expired", zap.Uint64("CallerStartTs", callerStartTS), zap.Stringer("lock str", l)) if l.LockType == kvrpcpb.Op_PessimisticLock { @@ -599,7 +600,7 @@ func (lr *LockResolver) getTxnStatus(bo *Backoffer, txnID uint64, primary []byte return status, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return status, errors.Trace(err) } @@ -735,7 +736,7 @@ func (lr *LockResolver) checkSecondaries(bo *Backoffer, txnID uint64, curKeys [] return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -866,7 +867,7 @@ func (lr *LockResolver) resolveRegionLocks(bo *Backoffer, l *Lock, region Region return errors.Trace(err) } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -934,7 +935,7 @@ func (lr *LockResolver) resolveLock(bo *Backoffer, l *Lock, status TxnStatus, li return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -985,7 +986,7 @@ func (lr *LockResolver) resolvePessimisticLock(bo *Backoffer, l *Lock, cleanRegi return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -997,7 +998,7 @@ func (lr *LockResolver) resolvePessimisticLock(bo *Backoffer, l *Lock, cleanRegi cmdResp := resp.Resp.(*kvrpcpb.PessimisticRollbackResponse) if keyErr := cmdResp.GetErrors(); len(keyErr) > 0 { err = errors.Errorf("unexpected resolve pessimistic lock err: %s, lock: %v", keyErr[0], l) - logutil.Logger(bo.ctx).Error("resolveLock error", zap.Error(err)) + logutil.Logger(bo.GetCtx()).Error("resolveLock error", zap.Error(err)) return err } return nil diff --git a/store/tikv/pessimistic.go b/store/tikv/pessimistic.go index 475efc7ad2a8b..445ced93ff904 100644 --- a/store/tikv/pessimistic.go +++ b/store/tikv/pessimistic.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" @@ -130,7 +131,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -228,7 +229,7 @@ func (actionPessimisticRollback) handleSingleBatch(c *twoPhaseCommitter, bo *Bac return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -247,11 +248,11 @@ func (c *twoPhaseCommitter) pessimisticLockMutations(bo *Backoffer, lockCtx *kv. for _, action := range strings.Split(v, ",") { if action == "delay" { duration := time.Duration(rand.Int63n(int64(time.Second) * 5)) - logutil.Logger(bo.ctx).Info("[failpoint] injected delay at pessimistic lock", + logutil.Logger(bo.GetCtx()).Info("[failpoint] injected delay at pessimistic lock", zap.Uint64("txnStartTS", c.startTS), zap.Duration("duration", duration)) time.Sleep(duration) } else if action == "fail" { - logutil.Logger(bo.ctx).Info("[failpoint] injected failure at pessimistic lock", + logutil.Logger(bo.GetCtx()).Info("[failpoint] injected failure at pessimistic lock", zap.Uint64("txnStartTS", c.startTS)) failpoint.Return(errors.New("injected failure at pessimistic lock")) } diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index 1fea4ba467341..7097ba5dbcd3e 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -27,6 +27,7 @@ import ( tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" @@ -130,7 +131,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff failpoint.Inject("prewritePrimaryFail", func() { // Delay to avoid cancelling other normally ongoing prewrite requests. time.Sleep(time.Millisecond * 50) - logutil.Logger(bo.ctx).Info("[failpoint] injected error on prewriting primary batch", + logutil.Logger(bo.GetCtx()).Info("[failpoint] injected error on prewriting primary batch", zap.Uint64("txnStartTS", c.startTS)) failpoint.Return(errors.New("injected error on prewriting primary batch")) }) @@ -139,7 +140,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff failpoint.Inject("prewriteSecondaryFail", func() { // Delay to avoid cancelling other normally ongoing prewrite requests. time.Sleep(time.Millisecond * 50) - logutil.Logger(bo.ctx).Info("[failpoint] injected error on prewriting secondary batch", + logutil.Logger(bo.GetCtx()).Info("[failpoint] injected error on prewriting secondary batch", zap.Uint64("txnStartTS", c.startTS)) failpoint.Return(errors.New("injected error on prewriting secondary batch")) }) @@ -150,7 +151,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff txnSize := uint64(c.regionTxnSize[batch.region.id]) // When we retry because of a region miss, we don't know the transaction size. We set the transaction size here // to MaxUint64 to avoid unexpected "resolve lock lite". - if len(bo.errors) > 0 { + if bo.ErrorsNum() > 0 { txnSize = math.MaxUint64 } @@ -175,7 +176,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -203,7 +204,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff if prewriteResp.MinCommitTs != 0 { return errors.Trace(errors.New("MinCommitTs must be 0 when 1pc falls back to 2pc")) } - logutil.Logger(bo.ctx).Warn("1pc failed and fallbacks to normal commit procedure", + logutil.Logger(bo.GetCtx()).Warn("1pc failed and fallbacks to normal commit procedure", zap.Uint64("startTS", c.startTS)) metrics.OnePCTxnCounterFallback.Inc() c.setOnePC(false) @@ -212,14 +213,14 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff // For 1PC, there's no racing to access to access `onePCCommmitTS` so it's safe // not to lock the mutex. if c.onePCCommitTS != 0 { - logutil.Logger(bo.ctx).Fatal("one pc happened multiple times", + logutil.Logger(bo.GetCtx()).Fatal("one pc happened multiple times", zap.Uint64("startTS", c.startTS)) } c.onePCCommitTS = prewriteResp.OnePcCommitTs } return nil } else if prewriteResp.OnePcCommitTs != 0 { - logutil.Logger(bo.ctx).Fatal("tikv committed a non-1pc transaction with 1pc protocol", + logutil.Logger(bo.GetCtx()).Fatal("tikv committed a non-1pc transaction with 1pc protocol", zap.Uint64("startTS", c.startTS)) } if c.isAsyncCommit() { @@ -230,7 +231,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff if c.testingKnobs.noFallBack { return nil } - logutil.Logger(bo.ctx).Warn("async commit cannot proceed since the returned minCommitTS is zero, "+ + logutil.Logger(bo.GetCtx()).Warn("async commit cannot proceed since the returned minCommitTS is zero, "+ "fallback to normal path", zap.Uint64("startTS", c.startTS)) c.setAsyncCommit(false) } else { @@ -268,7 +269,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff } atomic.AddInt64(&c.getDetail().ResolveLockTime, int64(time.Since(start))) if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(BoTxnLock, int(msBeforeExpired), errors.Errorf("2PC prewrite lockedKeys: %d", len(locks))) + err = bo.BackoffWithMaxSleep(retry.BoTxnLock, int(msBeforeExpired), errors.Errorf("2PC prewrite lockedKeys: %d", len(locks))) if err != nil { return errors.Trace(err) } @@ -277,10 +278,10 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff } func (c *twoPhaseCommitter) prewriteMutations(bo *Backoffer, mutations CommitterMutations) error { - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("twoPhaseCommitter.prewriteMutations", opentracing.ChildOf(span.Context())) defer span1.Finish() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } // `doActionOnMutations` will unset `useOnePC` if the mutations is splitted into multiple batches. diff --git a/store/tikv/range_task.go b/store/tikv/range_task.go index 1395fac0609a2..bc9b8fa9999c1 100644 --- a/store/tikv/range_task.go +++ b/store/tikv/range_task.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "go.uber.org/zap" ) @@ -91,6 +92,8 @@ func (s *RangeTaskRunner) SetRegionsPerTask(regionsPerTask int) { s.regionsPerTask = regionsPerTask } +const locateRegionMaxBackoff = 20000 + // RunOnRange runs the task on the given range. // Empty startKey or endKey means unbounded. func (s *RangeTaskRunner) RunOnRange(ctx context.Context, startKey, endKey []byte) error { @@ -157,7 +160,7 @@ Loop: default: } - bo := NewBackofferWithVars(ctx, locateRegionMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, locateRegionMaxBackoff, nil) rangeEndKey, err := s.store.GetRegionCache().BatchLoadRegionsFromKey(bo, key, s.regionsPerTask) if err != nil { diff --git a/store/tikv/rawkv.go b/store/tikv/rawkv.go index 2a80d26917a20..b96828df187fd 100644 --- a/store/tikv/rawkv.go +++ b/store/tikv/rawkv.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" pd "github.com/tikv/pd/client" ) @@ -110,6 +111,8 @@ func (c *RawKVClient) Get(key []byte) ([]byte, error) { return cmdResp.Value, nil } +const rawkvMaxBackoff = 20000 + // BatchGet queries values with the keys. func (c *RawKVClient) BatchGet(keys [][]byte) ([][]byte, error) { start := time.Now() @@ -117,7 +120,7 @@ func (c *RawKVClient) BatchGet(keys [][]byte) ([][]byte, error) { metrics.RawkvCmdHistogramWithBatchGet.Observe(time.Since(start).Seconds()) }() - bo := NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) resp, err := c.sendBatchReq(bo, keys, tikvrpc.CmdRawBatchGet) if err != nil { return nil, errors.Trace(err) @@ -184,7 +187,7 @@ func (c *RawKVClient) BatchPut(keys, values [][]byte) error { return errors.New("empty value is not supported") } } - bo := NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) err := c.sendBatchPut(bo, keys, values) return errors.Trace(err) } @@ -218,7 +221,7 @@ func (c *RawKVClient) BatchDelete(keys [][]byte) error { metrics.RawkvCmdHistogramWithBatchDelete.Observe(time.Since(start).Seconds()) }() - bo := NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) resp, err := c.sendBatchReq(bo, keys, tikvrpc.CmdRawBatchDelete) if err != nil { return errors.Trace(err) @@ -350,7 +353,7 @@ func (c *RawKVClient) ReverseScan(startKey, endKey []byte, limit int) (keys [][] } func (c *RawKVClient) sendReq(key []byte, req *tikvrpc.Request, reverse bool) (*tikvrpc.Response, *KeyLocation, error) { - bo := NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) sender := NewRegionRequestSender(c.regionCache, c.rpcClient) for { var loc *KeyLocation @@ -372,7 +375,7 @@ func (c *RawKVClient) sendReq(key []byte, req *tikvrpc.Request, reverse bool) (* return nil, nil, errors.Trace(err) } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return nil, nil, errors.Trace(err) } @@ -456,7 +459,7 @@ func (c *RawKVClient) doBatchReq(bo *Backoffer, batch batch, cmdType tikvrpc.Cmd return batchResp } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { batchResp.err = errors.Trace(err) return batchResp @@ -490,7 +493,7 @@ func (c *RawKVClient) doBatchReq(bo *Backoffer, batch batch, cmdType tikvrpc.Cmd // We can't use sendReq directly, because we need to know the end of the region before we send the request // TODO: Is there any better way to avoid duplicating code with func `sendReq` ? func (c *RawKVClient) sendDeleteRangeReq(startKey []byte, endKey []byte) (*tikvrpc.Response, []byte, error) { - bo := NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) sender := NewRegionRequestSender(c.regionCache, c.rpcClient) for { loc, err := c.regionCache.LocateKey(bo, startKey) @@ -517,7 +520,7 @@ func (c *RawKVClient) sendDeleteRangeReq(startKey []byte, endKey []byte) (*tikvr return nil, nil, errors.Trace(err) } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return nil, nil, errors.Trace(err) } @@ -622,7 +625,7 @@ func (c *RawKVClient) doBatchPut(bo *Backoffer, batch batch) error { return errors.Trace(err) } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index a73684fdf49c5..f6225a2724f8e 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" pd "github.com/tikv/pd/client" atomic2 "go.uber.org/atomic" @@ -194,7 +195,7 @@ func (r *Region) init(c *RegionCache) error { if !exists { store = c.getStoreByStoreID(p.StoreId) } - _, err := store.initResolve(NewNoopBackoff(context.Background()), c) + _, err := store.initResolve(retry.NewNoopBackoff(context.Background()), c) if err != nil { return err } @@ -644,7 +645,7 @@ func (c *RegionCache) findRegionByKey(bo *Backoffer, key []byte, isEndKey bool) // no region data, return error if failure. return nil, err } - logutil.Eventf(bo.ctx, "load region %d from pd, due to cache-miss", lr.GetID()) + logutil.Eventf(bo.GetCtx(), "load region %d from pd, due to cache-miss", lr.GetID()) r = lr c.mu.Lock() c.insertRegionToCache(r) @@ -654,10 +655,10 @@ func (c *RegionCache) findRegionByKey(bo *Backoffer, key []byte, isEndKey bool) lr, err := c.loadRegion(bo, key, isEndKey) if err != nil { // ignore error and use old region info. - logutil.Logger(bo.ctx).Error("load region failure", + logutil.Logger(bo.GetCtx()).Error("load region failure", zap.ByteString("key", key), zap.Error(err)) } else { - logutil.Eventf(bo.ctx, "load region %d from pd, due to need-reload", lr.GetID()) + logutil.Eventf(bo.GetCtx(), "load region %d from pd, due to need-reload", lr.GetID()) r = lr c.mu.Lock() c.insertRegionToCache(r) @@ -674,7 +675,7 @@ func (c *RegionCache) OnSendFail(bo *Backoffer, ctx *RPCContext, scheduleReload if r != nil { peersNum := len(r.meta.Peers) if len(ctx.Meta.Peers) != peersNum { - logutil.Logger(bo.ctx).Info("retry and refresh current ctx after send request fail and up/down stores length changed", + logutil.Logger(bo.GetCtx()).Info("retry and refresh current ctx after send request fail and up/down stores length changed", zap.Stringer("current", ctx), zap.Bool("needReload", scheduleReload), zap.Reflect("oldPeers", ctx.Meta.Peers), @@ -727,20 +728,20 @@ func (c *RegionCache) OnSendFail(bo *Backoffer, ctx *RPCContext, scheduleReload // In case the epoch of the store is increased, try to avoid reloading the current region by also // increasing the epoch stored in `rs`. rs.switchNextProxyStore(r, currentProxyIdx, incEpochStoreIdx) - logutil.Logger(bo.ctx).Info("switch region proxy peer to next due to send request fail", + logutil.Logger(bo.GetCtx()).Info("switch region proxy peer to next due to send request fail", zap.Stringer("current", ctx), zap.Bool("needReload", scheduleReload), zap.Error(err)) } else { rs.switchNextTiKVPeer(r, ctx.AccessIdx) - logutil.Logger(bo.ctx).Info("switch region peer to next due to send request fail", + logutil.Logger(bo.GetCtx()).Info("switch region peer to next due to send request fail", zap.Stringer("current", ctx), zap.Bool("needReload", scheduleReload), zap.Error(err)) } } else { rs.switchNextFlashPeer(r, ctx.AccessIdx) - logutil.Logger(bo.ctx).Info("switch region tiflash peer to next due to send request fail", + logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", zap.Stringer("current", ctx), zap.Bool("needReload", scheduleReload), zap.Error(err)) @@ -763,7 +764,7 @@ func (c *RegionCache) LocateRegionByID(bo *Backoffer, regionID uint64) (*KeyLoca lr, err := c.loadRegionByID(bo, regionID) if err != nil { // ignore error and use old region info. - logutil.Logger(bo.ctx).Error("load region failure", + logutil.Logger(bo.GetCtx()).Error("load region failure", zap.Uint64("regionID", regionID), zap.Error(err)) } else { r = lr @@ -1125,7 +1126,7 @@ func filterUnavailablePeers(region *pd.Region) { // If the given key is the end key of the region that you want, you may set the second argument to true. This is useful // when processing in reverse order. func (c *RegionCache) loadRegion(bo *Backoffer, key []byte, isEndKey bool) (*Region, error) { - ctx := bo.ctx + ctx := bo.GetCtx() if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("loadRegion", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -1136,7 +1137,7 @@ func (c *RegionCache) loadRegion(bo *Backoffer, key []byte, isEndKey bool) (*Reg searchPrev := false for { if backoffErr != nil { - err := bo.Backoff(BoPDRPC, backoffErr) + err := bo.Backoff(retry.BoPDRPC, backoffErr) if err != nil { return nil, errors.Trace(err) } @@ -1183,7 +1184,7 @@ func (c *RegionCache) loadRegion(bo *Backoffer, key []byte, isEndKey bool) (*Reg // loadRegionByID loads region from pd client, and picks the first peer as leader. func (c *RegionCache) loadRegionByID(bo *Backoffer, regionID uint64) (*Region, error) { - ctx := bo.ctx + ctx := bo.GetCtx() if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("loadRegionByID", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -1192,7 +1193,7 @@ func (c *RegionCache) loadRegionByID(bo *Backoffer, regionID uint64) (*Region, e var backoffErr error for { if backoffErr != nil { - err := bo.Backoff(BoPDRPC, backoffErr) + err := bo.Backoff(retry.BoPDRPC, backoffErr) if err != nil { return nil, errors.Trace(err) } @@ -1232,7 +1233,7 @@ func (c *RegionCache) scanRegions(bo *Backoffer, startKey, endKey []byte, limit if limit == 0 { return nil, nil } - ctx := bo.ctx + ctx := bo.GetCtx() if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("scanRegions", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -1242,7 +1243,7 @@ func (c *RegionCache) scanRegions(bo *Backoffer, startKey, endKey []byte, limit var backoffErr error for { if backoffErr != nil { - err := bo.Backoff(BoPDRPC, backoffErr) + err := bo.Backoff(retry.BoPDRPC, backoffErr) if err != nil { return nil, errors.Trace(err) } @@ -1412,7 +1413,7 @@ func (c *RegionCache) OnRegionEpochNotMatch(bo *Backoffer, ctx *RPCContext, curr meta.GetRegionEpoch().GetVersion() < ctx.Region.ver) { err := errors.Errorf("region epoch is ahead of tikv. rpc ctx: %+v, currentRegions: %+v", ctx, currentRegions) logutil.BgLogger().Info("region epoch is ahead of tikv", zap.Error(err)) - return bo.Backoff(BoRegionMiss, err) + return bo.Backoff(retry.BoRegionMiss, err) } } @@ -1775,7 +1776,7 @@ func (s *Store) initResolve(bo *Backoffer, c *RegionCache) (addr string, err err } var store *metapb.Store for { - store, err = c.pdClient.GetStore(bo.ctx, s.storeID) + store, err = c.pdClient.GetStore(bo.GetCtx(), s.storeID) if err != nil { metrics.RegionCacheCounterWithGetStoreError.Inc() } else { @@ -1787,7 +1788,7 @@ func (s *Store) initResolve(bo *Backoffer, c *RegionCache) (addr string, err err return } err = errors.Errorf("loadStore from PD failed, id: %d, err: %v", s.storeID, err) - if err = bo.Backoff(BoPDRPC, err); err != nil { + if err = bo.Backoff(retry.BoPDRPC, err); err != nil { return } continue @@ -1990,7 +1991,7 @@ func (s *Store) checkUntilHealth(c *RegionCache) { } } - bo := NewNoopBackoff(ctx) + bo := retry.NewNoopBackoff(ctx) l := s.requestLiveness(bo, c) if l == reachable { logutil.BgLogger().Info("[health check] store became reachable", zap.Uint64("storeID", s.storeID)) @@ -2020,7 +2021,7 @@ func (s *Store) requestLiveness(bo *Backoffer, c *RegionCache) (l livenessState) }) var ctx context.Context if bo != nil { - ctx = bo.ctx + ctx = bo.GetCtx() } else { ctx = context.Background() } diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index efb2ae9df73ab..7b0e5884abb2a 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" + "github.com/pingcap/tidb/store/tikv/retry" pd "github.com/tikv/pd/client" ) @@ -310,7 +311,7 @@ func (s *testRegionCacheSuite) TestUpdateLeader3(c *C) { ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, seed) c.Assert(err, IsNil) c.Assert(ctx.Addr, Equals, "store2") - s.cache.OnSendFail(NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) + s.cache.OnSendFail(retry.NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) s.cache.checkAndResolve(nil) s.cache.UpdateLeader(loc.Region, s.store2, 0) addr := s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0) @@ -870,7 +871,7 @@ func (s *testRegionCacheSuite) TestRegionEpochAheadOfTiKV(c *C) { c.Assert(err, IsNil) err = cache.OnRegionEpochNotMatch(bo, &RPCContext{Region: region.VerID()}, []*metapb.Region{&r2}) c.Assert(err, IsNil) - c.Assert(len(bo.errors), Equals, 2) + c.Assert(bo.ErrorsNum(), Equals, 2) } func (s *testRegionCacheSuite) TestRegionEpochOnTiFlash(c *C) { @@ -1329,7 +1330,7 @@ func (s *testRegionCacheSuite) TestPeersLenChange(c *C) { s.cache.insertRegionToCache(region) // OnSendFail should not panic - s.cache.OnSendFail(NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) + s.cache.OnSendFail(retry.NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) } func createSampleRegion(startKey, endKey []byte) *Region { diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 74ecdd7ce72b8..ec5b92f81c5ad 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/util" ) @@ -223,12 +224,12 @@ func (s *RegionRequestSender) SendReqCtx( rpcCtx *RPCContext, err error, ) { - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("regionRequest.SendReqCtx", opentracing.ChildOf(span.Context())) defer span1.Finish() // TODO(MyonKeminta): Make sure trace works without cloning the backoffer. // bo = bo.Clone() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } failpoint.Inject("tikvStoreSendReqResult", func(val failpoint.Value) { @@ -248,9 +249,7 @@ func (s *RegionRequestSender) SendReqCtx( }, nil, nil) } case "callBackofferHook": - if bo.vars != nil && bo.vars.Hook != nil { - bo.vars.Hook("callBackofferHook", bo.vars) - } + bo.SetVarsHook("callBackofferHook", bo.GetVars()) case "requestTiDBStoreError": if et == tikvrpc.TiDB { failpoint.Return(nil, nil, tikverr.ErrTiKVServerTimeout) @@ -265,7 +264,7 @@ func (s *RegionRequestSender) SendReqCtx( tryTimes := 0 for { if (tryTimes > 0) && (tryTimes%1000 == 0) { - logutil.Logger(bo.ctx).Warn("retry get ", zap.Uint64("region = ", regionID.GetID()), zap.Int("times = ", tryTimes)) + logutil.Logger(bo.GetCtx()).Warn("retry get ", zap.Uint64("region = ", regionID.GetID()), zap.Int("times = ", tryTimes)) } rpcCtx, err = s.getRPCContext(bo, req, regionID, et, opts...) @@ -275,7 +274,7 @@ func (s *RegionRequestSender) SendReqCtx( failpoint.Inject("invalidCacheAndRetry", func() { // cooperate with github.com/pingcap/tidb/store/gcworker/setGcResolveMaxBackoff - if c := bo.ctx.Value("injectedBackoff"); c != nil { + if c := bo.GetCtx().Value("injectedBackoff"); c != nil { resp, err = tikvrpc.GenRegionErrorResp(req, &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}}) failpoint.Return(resp, nil, err) } @@ -287,12 +286,12 @@ func (s *RegionRequestSender) SendReqCtx( // TODO: Change the returned error to something like "region missing in cache", // and handle this error like EpochNotMatch, which means to re-split the request and retry. - logutil.Logger(bo.ctx).Debug("throwing pseudo region error due to region not found in cache", zap.Stringer("region", ®ionID)) + logutil.Logger(bo.GetCtx()).Debug("throwing pseudo region error due to region not found in cache", zap.Stringer("region", ®ionID)) resp, err = tikvrpc.GenRegionErrorResp(req, &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}}) return resp, nil, err } - logutil.Eventf(bo.ctx, "send %s request to region %d at %s", req.Type, regionID.id, rpcCtx.Addr) + logutil.Eventf(bo.GetCtx(), "send %s request to region %d at %s", req.Type, regionID.id, rpcCtx.Addr) s.storeAddr = rpcCtx.Addr var retry bool resp, retry, err = s.sendReqToRegion(bo, rpcCtx, req, timeout) @@ -301,7 +300,8 @@ func (s *RegionRequestSender) SendReqCtx( } // recheck whether the session/query is killed during the Next() - if bo.vars != nil && bo.vars.Killed != nil && atomic.LoadUint32(bo.vars.Killed) == 1 { + boVars := bo.GetVars() + if boVars != nil && boVars.Killed != nil && atomic.LoadUint32(boVars.Killed) == 1 { return nil, nil, tikverr.ErrQueryInterrupted } failpoint.Inject("mockRetrySendReqToRegion", func(val failpoint.Value) { @@ -392,7 +392,7 @@ func (s *RegionRequestSender) sendReqToRegion(bo *Backoffer, rpcCtx *RPCContext, defer s.releaseStoreToken(rpcCtx.Store) } - ctx := bo.ctx + ctx := bo.GetCtx() if rawHook := ctx.Value(RPCCancellerCtxKey{}); rawHook != nil { var cancel context.CancelFunc ctx, cancel = rawHook.(*RPCCanceller).WithCancel(ctx) @@ -410,7 +410,7 @@ func (s *RegionRequestSender) sendReqToRegion(bo *Backoffer, rpcCtx *RPCContext, } var sessionID uint64 - if v := bo.ctx.Value(util.SessionID); v != nil { + if v := bo.GetCtx().Value(util.SessionID); v != nil { sessionID = v.(uint64) } @@ -443,7 +443,7 @@ func (s *RegionRequestSender) sendReqToRegion(bo *Backoffer, rpcCtx *RPCContext, RecordRegionRequestRuntimeStats(s.Stats, req.Type, time.Since(start)) failpoint.Inject("tikvStoreRespResult", func(val failpoint.Value) { if val.(bool) { - if req.Type == tikvrpc.CmdCop && bo.totalSleep == 0 { + if req.Type == tikvrpc.CmdCop && bo.GetTotalSleep() == 0 { failpoint.Return(&tikvrpc.Response{ Resp: &coprocessor.Response{RegionError: &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}}}, }, false, nil) @@ -544,12 +544,12 @@ func (s *RegionRequestSender) releaseStoreToken(st *Store) { } func (s *RegionRequestSender) onSendFail(bo *Backoffer, ctx *RPCContext, err error) error { - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("regionRequest.onSendFail", opentracing.ChildOf(span.Context())) defer span1.Finish() // TODO(MyonKeminta): Make sure trace works without cloning the backoffer. // bo = bo.Clone() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled { @@ -559,7 +559,7 @@ func (s *RegionRequestSender) onSendFail(bo *Backoffer, ctx *RPCContext, err err } if status.Code(errors.Cause(err)) == codes.Canceled { select { - case <-bo.ctx.Done(): + case <-bo.GetCtx().Done(): return errors.Trace(err) default: // If we don't cancel, but the error code is Canceled, it must be from grpc remote. @@ -578,9 +578,9 @@ func (s *RegionRequestSender) onSendFail(bo *Backoffer, ctx *RPCContext, err err // TODO: the number of retry time should be limited:since region may be unavailable // when some unrecoverable disaster happened. if ctx.Store != nil && ctx.Store.storeType == tikvrpc.TiFlash { - err = bo.Backoff(BoTiFlashRPC, errors.Errorf("send tiflash request error: %v, ctx: %v, try next peer later", err, ctx)) + err = bo.Backoff(retry.BoTiFlashRPC, errors.Errorf("send tiflash request error: %v, ctx: %v, try next peer later", err, ctx)) } else { - err = bo.Backoff(BoTiKVRPC, errors.Errorf("send tikv request error: %v, ctx: %v, try next peer later", err, ctx)) + err = bo.BackoffTiKVRPC(errors.Errorf("send tikv request error: %v, ctx: %v, try next peer later", err, ctx)) } return errors.Trace(err) } @@ -632,13 +632,13 @@ func regionErrorToLabel(e *errorpb.Error) string { return "unknown" } -func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed *uint32, regionErr *errorpb.Error) (retry bool, err error) { - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { +func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed *uint32, regionErr *errorpb.Error) (shouldRetry bool, err error) { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("tikv.onRegionError", opentracing.ChildOf(span.Context())) defer span1.Finish() // TODO(MyonKeminta): Make sure trace works without cloning the backoffer. // bo = bo.Clone() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } metrics.TiKVRegionErrorCounter.WithLabelValues(regionErrorToLabel(regionErr)).Inc() @@ -654,7 +654,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed // isolated and removed from the Raft group. So it's necessary to reload // the region from PD. s.regionCache.InvalidateCachedRegionWithReason(ctx.Region, NoLeader) - if err = bo.Backoff(BoRegionMiss, errors.Errorf("not leader: %v, ctx: %v", notLeader, ctx)); err != nil { + if err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("not leader: %v, ctx: %v", notLeader, ctx)); err != nil { return false, errors.Trace(err) } } else { @@ -690,9 +690,9 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed zap.String("reason", regionErr.GetServerIsBusy().GetReason()), zap.Stringer("ctx", ctx)) if ctx != nil && ctx.Store != nil && ctx.Store.storeType == tikvrpc.TiFlash { - err = bo.Backoff(boTiFlashServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) + err = bo.Backoff(retry.BoTiFlashServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) } else { - err = bo.Backoff(boTiKVServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) + err = bo.Backoff(retry.BoTiKVServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) } if err != nil { return false, errors.Trace(err) @@ -701,7 +701,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed } if regionErr.GetStaleCommand() != nil { logutil.BgLogger().Debug("tikv reports `StaleCommand`", zap.Stringer("ctx", ctx)) - err = bo.Backoff(boStaleCmd, errors.Errorf("stale command, ctx: %v", ctx)) + err = bo.Backoff(retry.BoStaleCmd, errors.Errorf("stale command, ctx: %v", ctx)) if err != nil { return false, errors.Trace(err) } @@ -718,7 +718,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed } if regionErr.GetMaxTimestampNotSynced() != nil { logutil.BgLogger().Warn("tikv reports `MaxTimestampNotSynced`", zap.Stringer("ctx", ctx)) - err = bo.Backoff(boMaxTsNotSynced, errors.Errorf("max timestamp not synced, ctx: %v", ctx)) + err = bo.Backoff(retry.BoMaxTsNotSynced, errors.Errorf("max timestamp not synced, ctx: %v", ctx)) if err != nil { return false, errors.Trace(err) } diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 8d531ee209a78..f7fd2a149060d 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/kvproto/pkg/tikvpb" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" @@ -72,7 +73,7 @@ func (s *testRegionRequestToSingleStoreSuite) SetUpTest(c *C) { s.store, s.peer, s.region = mocktikv.BootstrapWithSingleStore(s.cluster) pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster)} s.cache = NewRegionCache(pdCli) - s.bo = NewNoopBackoff(context.Background()) + s.bo = retry.NewNoopBackoff(context.Background()) s.mvccStore = mocktikv.MustNewMVCCStore() client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) s.regionRequestSender = NewRegionRequestSender(s.cache, client) @@ -83,7 +84,7 @@ func (s *testRegionRequestToThreeStoresSuite) SetUpTest(c *C) { s.storeIDs, s.peerIDs, s.regionID, s.leaderPeer = mocktikv.BootstrapWithMultiStores(s.cluster, 3) pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster)} s.cache = NewRegionCache(pdCli) - s.bo = NewNoopBackoff(context.Background()) + s.bo = retry.NewNoopBackoff(context.Background()) s.mvccStore = mocktikv.MustNewMVCCStore() client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) s.regionRequestSender = NewRegionRequestSender(s.cache, client) diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go new file mode 100644 index 0000000000000..24dc9174f3fec --- /dev/null +++ b/store/tikv/retry/backoff.go @@ -0,0 +1,439 @@ +// Copyright 2016 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package retry + +import ( + "context" + "fmt" + "math" + "math/rand" + "strings" + "sync/atomic" + "time" + + "github.com/opentracing/opentracing-go" + "github.com/pingcap/errors" + "github.com/pingcap/log" + tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/util" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" +) + +const ( + // NoJitter makes the backoff sequence strict exponential. + NoJitter = 1 + iota + // FullJitter applies random factors to strict exponential. + FullJitter + // EqualJitter is also randomized, but prevents very short sleeps. + EqualJitter + // DecorrJitter increases the maximum jitter based on the last random value. + DecorrJitter +) + +func (t BackoffType) metric() prometheus.Observer { + switch t { + // TODO: distinguish tikv and tiflash in metrics + case boTiKVRPC, BoTiFlashRPC: + return metrics.BackoffHistogramRPC + case BoTxnLock: + return metrics.BackoffHistogramLock + case BoTxnLockFast: + return metrics.BackoffHistogramLockFast + case BoPDRPC: + return metrics.BackoffHistogramPD + case BoRegionMiss: + return metrics.BackoffHistogramRegionMiss + case BoTiKVServerBusy, BoTiFlashServerBusy: + return metrics.BackoffHistogramServerBusy + case BoStaleCmd: + return metrics.BackoffHistogramStaleCmd + } + return metrics.BackoffHistogramEmpty +} + +// NewBackoffFn creates a backoff func which implements exponential backoff with +// optional jitters. +// See http://www.awsarchitectureblog.com/2015/03/backoff.html +func NewBackoffFn(base, cap, jitter int) func(ctx context.Context, maxSleepMs int) int { + if base < 2 { + // Top prevent panic in 'rand.Intn'. + base = 2 + } + attempts := 0 + lastSleep := base + return func(ctx context.Context, maxSleepMs int) int { + var sleep int + switch jitter { + case NoJitter: + sleep = expo(base, cap, attempts) + case FullJitter: + v := expo(base, cap, attempts) + sleep = rand.Intn(v) + case EqualJitter: + v := expo(base, cap, attempts) + sleep = v/2 + rand.Intn(v/2) + case DecorrJitter: + sleep = int(math.Min(float64(cap), float64(base+rand.Intn(lastSleep*3-base)))) + } + logutil.BgLogger().Debug("backoff", + zap.Int("base", base), + zap.Int("sleep", sleep), + zap.Int("attempts", attempts)) + + realSleep := sleep + // when set maxSleepMs >= 0 in `tikv.BackoffWithMaxSleep` will force sleep maxSleepMs milliseconds. + if maxSleepMs >= 0 && realSleep > maxSleepMs { + realSleep = maxSleepMs + } + select { + case <-time.After(time.Duration(realSleep) * time.Millisecond): + attempts++ + lastSleep = sleep + return realSleep + case <-ctx.Done(): + return 0 + } + } +} + +func expo(base, cap, n int) int { + return int(math.Min(float64(cap), float64(base)*math.Pow(2.0, float64(n)))) +} + +// BackoffType defines the backoff type. +type BackoffType int + +// Back off types. +const ( + boTiKVRPC BackoffType = iota + BoTiFlashRPC + BoTxnLock + BoTxnLockFast + BoPDRPC + BoRegionMiss + BoTiKVServerBusy + BoTiFlashServerBusy + BoTxnNotFound + BoStaleCmd + BoMaxTsNotSynced +) + +func (t BackoffType) createFn(vars *kv.Variables) func(context.Context, int) int { + if vars.Hook != nil { + vars.Hook(t.String(), vars) + } + switch t { + case boTiKVRPC, BoTiFlashRPC: + return NewBackoffFn(100, 2000, EqualJitter) + case BoTxnLock: + return NewBackoffFn(200, 3000, EqualJitter) + case BoTxnLockFast: + return NewBackoffFn(vars.BackoffLockFast, 3000, EqualJitter) + case BoPDRPC: + return NewBackoffFn(500, 3000, EqualJitter) + case BoRegionMiss: + // change base time to 2ms, because it may recover soon. + return NewBackoffFn(2, 500, NoJitter) + case BoTxnNotFound: + return NewBackoffFn(2, 500, NoJitter) + case BoTiKVServerBusy, BoTiFlashServerBusy: + return NewBackoffFn(2000, 10000, EqualJitter) + case BoStaleCmd: + return NewBackoffFn(2, 1000, NoJitter) + case BoMaxTsNotSynced: + return NewBackoffFn(2, 500, NoJitter) + } + return nil +} + +func (t BackoffType) String() string { + switch t { + case boTiKVRPC: + return "tikvRPC" + case BoTiFlashRPC: + return "tiflashRPC" + case BoTxnLock: + return "txnLock" + case BoTxnLockFast: + return "txnLockFast" + case BoPDRPC: + return "pdRPC" + case BoRegionMiss: + return "regionMiss" + case BoTiKVServerBusy: + return "tikvServerBusy" + case BoTiFlashServerBusy: + return "tiflashServerBusy" + case BoStaleCmd: + return "staleCommand" + case BoTxnNotFound: + return "txnNotFound" + case BoMaxTsNotSynced: + return "maxTsNotSynced" + } + return "" +} + +// TError returns pingcap/error of the backoff type. +func (t BackoffType) TError() error { + switch t { + case boTiKVRPC: + return tikverr.ErrTiKVServerTimeout + case BoTiFlashRPC: + return tikverr.ErrTiFlashServerTimeout + case BoTxnLock, BoTxnLockFast, BoTxnNotFound: + return tikverr.ErrResolveLockTimeout + case BoPDRPC: + return tikverr.NewErrPDServerTimeout("") + case BoRegionMiss: + return tikverr.ErrRegionUnavailable + case BoTiKVServerBusy: + return tikverr.ErrTiKVServerBusy + case BoTiFlashServerBusy: + return tikverr.ErrTiFlashServerBusy + case BoStaleCmd: + return tikverr.ErrTiKVStaleCommand + case BoMaxTsNotSynced: + return tikverr.ErrTiKVMaxTimestampNotSynced + } + return tikverr.ErrUnknown +} + +// Backoffer is a utility for retrying queries. +type Backoffer struct { + ctx context.Context + + fn map[BackoffType]func(context.Context, int) int + maxSleep int + totalSleep int + errors []error + types []fmt.Stringer + vars *kv.Variables + noop bool + + backoffSleepMS map[BackoffType]int + backoffTimes map[BackoffType]int +} + +type txnStartCtxKeyType struct{} + +// TxnStartKey is a key for transaction start_ts info in context.Context. +var TxnStartKey interface{} = txnStartCtxKeyType{} + +// NewBackoffer (Deprecated) creates a Backoffer with maximum sleep time(in ms). +func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { + return &Backoffer{ + ctx: ctx, + maxSleep: maxSleep, + vars: kv.DefaultVars, + } +} + +// NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. +func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *Backoffer { + return NewBackoffer(ctx, maxSleep).withVars(vars) +} + +// NewNoopBackoff create a Backoffer do nothing just return error directly +func NewNoopBackoff(ctx context.Context) *Backoffer { + return &Backoffer{ctx: ctx, noop: true} +} + +// withVars sets the kv.Variables to the Backoffer and return it. +func (b *Backoffer) withVars(vars *kv.Variables) *Backoffer { + if vars != nil { + b.vars = vars + } + // maxSleep is the max sleep time in millisecond. + // When it is multiplied by BackOffWeight, it should not be greater than MaxInt32. + if math.MaxInt32/b.vars.BackOffWeight >= b.maxSleep { + b.maxSleep *= b.vars.BackOffWeight + } + return b +} + +// Backoff sleeps a while base on the backoffType and records the error message. +// It returns a retryable error if total sleep time exceeds maxSleep. +func (b *Backoffer) Backoff(typ BackoffType, err error) error { + if span := opentracing.SpanFromContext(b.ctx); span != nil && span.Tracer() != nil { + span1 := span.Tracer().StartSpan(fmt.Sprintf("tikv.backoff.%s", typ), opentracing.ChildOf(span.Context())) + defer span1.Finish() + opentracing.ContextWithSpan(b.ctx, span1) + } + return b.BackoffWithMaxSleep(typ, -1, err) +} + +// BackoffTiKVRPC calls Backoff with boTiKVRPC. +func (b *Backoffer) BackoffTiKVRPC(err error) error { + return b.Backoff(boTiKVRPC, err) +} + +// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message +// and never sleep more than maxSleepMs for each sleep. +func (b *Backoffer) BackoffWithMaxSleep(typ BackoffType, maxSleepMs int, err error) error { + if strings.Contains(err.Error(), tikverr.MismatchClusterID) { + logutil.BgLogger().Fatal("critical error", zap.Error(err)) + } + select { + case <-b.ctx.Done(): + return errors.Trace(err) + default: + } + + b.errors = append(b.errors, errors.Errorf("%s at %s", err.Error(), time.Now().Format(time.RFC3339Nano))) + b.types = append(b.types, typ) + if b.noop || (b.maxSleep > 0 && b.totalSleep >= b.maxSleep) { + errMsg := fmt.Sprintf("%s backoffer.maxSleep %dms is exceeded, errors:", typ.String(), b.maxSleep) + for i, err := range b.errors { + // Print only last 3 errors for non-DEBUG log levels. + if log.GetLevel() == zapcore.DebugLevel || i >= len(b.errors)-3 { + errMsg += "\n" + err.Error() + } + } + logutil.BgLogger().Warn(errMsg) + // Use the first backoff type to generate a MySQL error. + return b.types[0].(BackoffType).TError() + } + + // Lazy initialize. + if b.fn == nil { + b.fn = make(map[BackoffType]func(context.Context, int) int) + } + f, ok := b.fn[typ] + if !ok { + f = typ.createFn(b.vars) + b.fn[typ] = f + } + + realSleep := f(b.ctx, maxSleepMs) + typ.metric().Observe(float64(realSleep) / 1000) + b.totalSleep += realSleep + if b.backoffSleepMS == nil { + b.backoffSleepMS = make(map[BackoffType]int) + } + b.backoffSleepMS[typ] += realSleep + if b.backoffTimes == nil { + b.backoffTimes = make(map[BackoffType]int) + } + b.backoffTimes[typ]++ + + stmtExec := b.ctx.Value(util.ExecDetailsKey) + if stmtExec != nil { + detail := stmtExec.(*util.ExecDetails) + atomic.AddInt64(&detail.BackoffDuration, int64(realSleep)*int64(time.Millisecond)) + atomic.AddInt64(&detail.BackoffCount, 1) + } + + if b.vars != nil && b.vars.Killed != nil { + if atomic.LoadUint32(b.vars.Killed) == 1 { + return tikverr.ErrQueryInterrupted + } + } + + var startTs interface{} + if ts := b.ctx.Value(TxnStartKey); ts != nil { + startTs = ts + } + logutil.Logger(b.ctx).Debug("retry later", + zap.Error(err), + zap.Int("totalSleep", b.totalSleep), + zap.Int("maxSleep", b.maxSleep), + zap.Stringer("type", typ), + zap.Reflect("txnStartTS", startTs)) + return nil +} + +func (b *Backoffer) String() string { + if b.totalSleep == 0 { + return "" + } + return fmt.Sprintf(" backoff(%dms %v)", b.totalSleep, b.types) +} + +// Clone creates a new Backoffer which keeps current Backoffer's sleep time and errors, and shares +// current Backoffer's context. +func (b *Backoffer) Clone() *Backoffer { + return &Backoffer{ + ctx: b.ctx, + maxSleep: b.maxSleep, + totalSleep: b.totalSleep, + errors: b.errors, + vars: b.vars, + } +} + +// Fork creates a new Backoffer which keeps current Backoffer's sleep time and errors, and holds +// a child context of current Backoffer's context. +func (b *Backoffer) Fork() (*Backoffer, context.CancelFunc) { + ctx, cancel := context.WithCancel(b.ctx) + return &Backoffer{ + ctx: ctx, + maxSleep: b.maxSleep, + totalSleep: b.totalSleep, + errors: b.errors, + vars: b.vars, + }, cancel +} + +// GetVars returns the binded vars. +func (b *Backoffer) GetVars() *kv.Variables { + return b.vars +} + +// GetTotalSleep returns total sleep time. +func (b *Backoffer) GetTotalSleep() int { + return b.totalSleep +} + +// GetTypes returns type list. +func (b *Backoffer) GetTypes() []fmt.Stringer { + return b.types +} + +// GetCtx returns the binded context. +func (b *Backoffer) GetCtx() context.Context { + return b.ctx +} + +// SetCtx sets the binded context to ctx. +func (b *Backoffer) SetCtx(ctx context.Context) { + b.ctx = ctx +} + +// GetBackoffTimes returns a map contains backoff time count by type. +func (b *Backoffer) GetBackoffTimes() map[BackoffType]int { + return b.backoffTimes +} + +// GetBackoffSleepMS returns a map contains backoff sleep time by type. +func (b *Backoffer) GetBackoffSleepMS() map[BackoffType]int { + return b.backoffSleepMS +} + +// ErrorsNum returns the number of errors. +func (b *Backoffer) ErrorsNum() int { + return len(b.errors) +} + +// SetVarsHook sets the vars.Hook is used for test to verify the variable take effect. +func (b *Backoffer) SetVarsHook(name string, vars *kv.Variables) { + if b.vars != nil && b.vars.Hook != nil { + b.vars.Hook(name, vars) + } +} diff --git a/store/tikv/backoff_test.go b/store/tikv/retry/backoff_test.go similarity index 98% rename from store/tikv/backoff_test.go rename to store/tikv/retry/backoff_test.go index 11254937abd72..f8dfb9ed120f3 100644 --- a/store/tikv/backoff_test.go +++ b/store/tikv/retry/backoff_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tikv +package retry import ( "context" diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 19a14b3f73819..6c43b7bdee7cd 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -22,6 +22,7 @@ import ( tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "go.uber.org/zap" ) @@ -85,9 +86,11 @@ func (s *Scanner) Value() []byte { return nil } +const scannerNextMaxBackoff = 20000 + // Next return next element. func (s *Scanner) Next() error { - bo := NewBackofferWithVars(context.WithValue(context.Background(), TxnStartKey, s.snapshot.version), scannerNextMaxBackoff, s.snapshot.vars) + bo := retry.NewBackofferWithVars(context.WithValue(context.Background(), retry.TxnStartKey, s.snapshot.version), scannerNextMaxBackoff, s.snapshot.vars) if !s.valid { return errors.New("scanner iterator is invalid") } @@ -223,7 +226,7 @@ func (s *Scanner) getData(bo *Backoffer) error { if regionErr != nil { logutil.BgLogger().Debug("scanner getData failed", zap.Stringer("regionErr", regionErr)) - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -251,7 +254,7 @@ func (s *Scanner) getData(bo *Backoffer) error { return errors.Trace(err) } if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(BoTxnLockFast, int(msBeforeExpired), errors.Errorf("key is locked during scanning")) + err = bo.BackoffWithMaxSleep(retry.BoTxnLockFast, int(msBeforeExpired), errors.Errorf("key is locked during scanning")) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 24149fdae44cb..ab3862fe4bf0b 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" @@ -126,6 +127,8 @@ func newTiKVSnapshot(store *KVStore, ts uint64, replicaReadSeed uint32) *KVSnaps } } +const batchGetMaxBackoff = 20000 + // SetSnapshotTS resets the timestamp for reads. func (s *KVSnapshot) SetSnapshotTS(ts uint64) { // Sanity check for snapshot version. @@ -170,8 +173,8 @@ func (s *KVSnapshot) BatchGet(ctx context.Context, keys [][]byte) (map[string][] // We want [][]byte instead of []kv.Key, use some magic to save memory. bytesKeys := *(*[][]byte)(unsafe.Pointer(&keys)) - ctx = context.WithValue(ctx, TxnStartKey, s.version) - bo := NewBackofferWithVars(ctx, batchGetMaxBackoff, s.vars) + ctx = context.WithValue(ctx, retry.TxnStartKey, s.version) + bo := retry.NewBackofferWithVars(ctx, batchGetMaxBackoff, s.vars) // Create a map to collect key-values from region servers. var mu sync.Mutex @@ -331,7 +334,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -378,7 +381,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec return errors.Trace(err) } if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(BoTxnLockFast, int(msBeforeExpired), errors.Errorf("batchGet lockedKeys: %d", len(lockedKeys))) + err = bo.BackoffWithMaxSleep(retry.BoTxnLockFast, int(msBeforeExpired), errors.Errorf("batchGet lockedKeys: %d", len(lockedKeys))) if err != nil { return errors.Trace(err) } @@ -394,6 +397,8 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec } } +const getMaxBackoff = 20000 + // Get gets the value for key k from snapshot. func (s *KVSnapshot) Get(ctx context.Context, k []byte) ([]byte, error) { @@ -401,8 +406,8 @@ func (s *KVSnapshot) Get(ctx context.Context, k []byte) ([]byte, error) { metrics.TxnCmdHistogramWithGet.Observe(time.Since(start).Seconds()) }(time.Now()) - ctx = context.WithValue(ctx, TxnStartKey, s.version) - bo := NewBackofferWithVars(ctx, getMaxBackoff, s.vars) + ctx = context.WithValue(ctx, retry.TxnStartKey, s.version) + bo := retry.NewBackofferWithVars(ctx, getMaxBackoff, s.vars) val, err := s.get(ctx, bo, k) s.recordBackoffInfo(bo) if err != nil { @@ -436,7 +441,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, opentracing.ContextWithSpan(ctx, span1) } failpoint.Inject("snapshot-get-cache-fail", func(_ failpoint.Value) { - if bo.ctx.Value("TestSnapshotCache") != nil { + if bo.GetCtx().Value("TestSnapshotCache") != nil { panic("cache miss") } }) @@ -488,7 +493,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, return nil, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return nil, errors.Trace(err) } @@ -522,7 +527,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, return nil, errors.Trace(err) } if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(BoTxnLockFast, int(msBeforeExpired), errors.New(keyErr.String())) + err = bo.BackoffWithMaxSleep(retry.BoTxnLockFast, int(msBeforeExpired), errors.New(keyErr.String())) if err != nil { return nil, errors.Trace(err) } @@ -678,7 +683,7 @@ func extractKeyErr(keyErr *pb.KeyError) error { func (s *KVSnapshot) recordBackoffInfo(bo *Backoffer) { s.mu.RLock() - if s.mu.stats == nil || bo.totalSleep == 0 { + if s.mu.stats == nil || bo.GetTotalSleep() == 0 { s.mu.RUnlock() return } @@ -689,14 +694,14 @@ func (s *KVSnapshot) recordBackoffInfo(bo *Backoffer) { return } if s.mu.stats.backoffSleepMS == nil { - s.mu.stats.backoffSleepMS = bo.backoffSleepMS - s.mu.stats.backoffTimes = bo.backoffTimes + s.mu.stats.backoffSleepMS = bo.GetBackoffSleepMS() + s.mu.stats.backoffTimes = bo.GetBackoffTimes() return } - for k, v := range bo.backoffSleepMS { + for k, v := range bo.GetBackoffSleepMS() { s.mu.stats.backoffSleepMS[k] += v } - for k, v := range bo.backoffTimes { + for k, v := range bo.GetBackoffTimes() { s.mu.stats.backoffTimes[k] += v } } @@ -725,8 +730,8 @@ func (s *KVSnapshot) mergeRegionRequestStats(stats map[tikvrpc.CmdType]*RPCRunti // SnapshotRuntimeStats records the runtime stats of snapshot. type SnapshotRuntimeStats struct { rpcStats RegionRequestRuntimeStats - backoffSleepMS map[BackoffType]int - backoffTimes map[BackoffType]int + backoffSleepMS map[retry.BackoffType]int + backoffTimes map[retry.BackoffType]int scanDetail *util.ScanDetail timeDetail *util.TimeDetail } @@ -740,8 +745,8 @@ func (rs *SnapshotRuntimeStats) Clone() *SnapshotRuntimeStats { } } if len(rs.backoffSleepMS) > 0 { - newRs.backoffSleepMS = make(map[BackoffType]int) - newRs.backoffTimes = make(map[BackoffType]int) + newRs.backoffSleepMS = make(map[retry.BackoffType]int) + newRs.backoffTimes = make(map[retry.BackoffType]int) for k, v := range rs.backoffSleepMS { newRs.backoffSleepMS[k] += v } @@ -762,10 +767,10 @@ func (rs *SnapshotRuntimeStats) Merge(other *SnapshotRuntimeStats) { } if len(other.backoffSleepMS) > 0 { if rs.backoffSleepMS == nil { - rs.backoffSleepMS = make(map[BackoffType]int) + rs.backoffSleepMS = make(map[retry.BackoffType]int) } if rs.backoffTimes == nil { - rs.backoffTimes = make(map[BackoffType]int) + rs.backoffTimes = make(map[retry.BackoffType]int) } for k, v := range other.backoffSleepMS { rs.backoffSleepMS[k] += v diff --git a/store/tikv/split_region.go b/store/tikv/split_region.go index 5839aa4d73c96..38ce24917d1cf 100644 --- a/store/tikv/split_region.go +++ b/store/tikv/split_region.go @@ -27,6 +27,7 @@ import ( tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/util" pd "github.com/tikv/pd/client" @@ -56,7 +57,7 @@ func (s *KVStore) splitBatchRegionsReq(bo *Backoffer, keys [][]byte, scatter boo return nil, nil } // The first time it enters this function. - if bo.totalSleep == 0 { + if bo.GetTotalSleep() == 0 { logutil.BgLogger().Info("split batch regions request", zap.Int("split key count", len(keys)), zap.Int("batch count", len(batches)), @@ -76,8 +77,8 @@ func (s *KVStore) splitBatchRegionsReq(bo *Backoffer, keys [][]byte, scatter boo util.WithRecovery(func() { select { case ch <- s.batchSendSingleRegion(backoffer, b, scatter, tableID): - case <-bo.ctx.Done(): - ch <- singleBatchResp{err: bo.ctx.Err()} + case <-bo.GetCtx().Done(): + ch <- singleBatchResp{err: bo.GetCtx().Err()} } }, func(r interface{}) { if r != nil { @@ -110,8 +111,8 @@ func (s *KVStore) splitBatchRegionsReq(bo *Backoffer, keys [][]byte, scatter boo func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool, tableID *int64) singleBatchResp { if val, err := util.MockSplitRegionTimeout.Eval(); err == nil { if val.(bool) { - if _, ok := bo.ctx.Deadline(); ok { - <-bo.ctx.Done() + if _, ok := bo.GetCtx().Deadline(); ok { + <-bo.GetCtx().Done() } } } @@ -136,7 +137,7 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool return batchResp } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { batchResp.err = errors.Trace(err) return batchResp @@ -192,9 +193,14 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool return batchResp } +const ( + splitRegionBackoff = 20000 + maxSplitRegionsBackoff = 120000 +) + // SplitRegions splits regions by splitKeys. func (s *KVStore) SplitRegions(ctx context.Context, splitKeys [][]byte, scatter bool, tableID *int64) (regionIDs []uint64, err error) { - bo := NewBackofferWithVars(ctx, int(math.Min(float64(len(splitKeys))*splitRegionBackoff, maxSplitRegionsBackoff)), nil) + bo := retry.NewBackofferWithVars(ctx, int(math.Min(float64(len(splitKeys))*splitRegionBackoff, maxSplitRegionsBackoff)), nil) resp, err := s.splitBatchRegionsReq(bo, splitKeys, scatter, tableID) regionIDs = make([]uint64, 0, len(splitKeys)) if resp != nil && resp.Resp != nil { @@ -215,7 +221,7 @@ func (s *KVStore) scatterRegion(bo *Backoffer, regionID uint64, tableID *int64) if tableID != nil { opts = append(opts, pd.WithGroup(fmt.Sprintf("%v", *tableID))) } - _, err := s.pdClient.ScatterRegions(bo.ctx, []uint64{regionID}, opts...) + _, err := s.pdClient.ScatterRegions(bo.GetCtx(), []uint64{regionID}, opts...) if val, err2 := util.MockScatterRegionTimeout.Eval(); err2 == nil { if val.(bool) { @@ -226,7 +232,7 @@ func (s *KVStore) scatterRegion(bo *Backoffer, regionID uint64, tableID *int64) if err == nil { break } - err = bo.Backoff(BoPDRPC, errors.New(err.Error())) + err = bo.Backoff(retry.BoPDRPC, errors.New(err.Error())) if err != nil { return errors.Trace(err) } @@ -273,6 +279,8 @@ func (s *KVStore) preSplitRegion(ctx context.Context, group groupedMutations) bo return true } +const waitScatterRegionFinishBackoff = 120000 + // WaitScatterRegionFinish implements SplittableStore interface. // backOff is the back off time of the wait scatter region.(Milliseconds) // if backOff <= 0, the default wait scatter back off time will be used. @@ -283,7 +291,7 @@ func (s *KVStore) WaitScatterRegionFinish(ctx context.Context, regionID uint64, logutil.BgLogger().Info("wait scatter region", zap.Uint64("regionID", regionID), zap.Int("backoff(ms)", backOff)) - bo := NewBackofferWithVars(ctx, backOff, nil) + bo := retry.NewBackofferWithVars(ctx, backOff, nil) logFreq := 0 for { resp, err := s.pdClient.GetOperator(ctx, regionID) @@ -310,9 +318,9 @@ func (s *KVStore) WaitScatterRegionFinish(ctx context.Context, regionID uint64, logFreq++ } if err != nil { - err = bo.Backoff(BoRegionMiss, errors.New(err.Error())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(err.Error())) } else { - err = bo.Backoff(BoRegionMiss, errors.New("wait scatter region timeout")) + err = bo.Backoff(retry.BoRegionMiss, errors.New("wait scatter region timeout")) } if err != nil { return errors.Trace(err) @@ -322,7 +330,7 @@ func (s *KVStore) WaitScatterRegionFinish(ctx context.Context, regionID uint64, // CheckRegionInScattering uses to check whether scatter region finished. func (s *KVStore) CheckRegionInScattering(regionID uint64) (bool, error) { - bo := NewBackofferWithVars(context.Background(), locateRegionMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), locateRegionMaxBackoff, nil) for { resp, err := s.pdClient.GetOperator(context.Background(), regionID) if err == nil && resp != nil { @@ -331,7 +339,7 @@ func (s *KVStore) CheckRegionInScattering(regionID uint64) (bool, error) { } } if err != nil { - err = bo.Backoff(BoRegionMiss, errors.New(err.Error())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(err.Error())) } else { return true, nil } diff --git a/store/tikv/test_probe.go b/store/tikv/test_probe.go index a6a4f8d826655..1a8dc5062218d 100644 --- a/store/tikv/test_probe.go +++ b/store/tikv/test_probe.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" pd "github.com/tikv/pd/client" @@ -66,7 +67,7 @@ func (s StoreProbe) ClearTxnLatches() { // SendTxnHeartbeat renews a txn's ttl. func (s StoreProbe) SendTxnHeartbeat(ctx context.Context, key []byte, startTS uint64, ttl uint64) (uint64, error) { - bo := NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil) return sendTxnHeartBeat(bo, s.KVStore, key, startTS, ttl) } @@ -266,12 +267,12 @@ func (c CommitterProbe) PrewriteAllMutations(ctx context.Context) error { // PrewriteMutations performs the first phase of commit for given keys. func (c CommitterProbe) PrewriteMutations(ctx context.Context, mutations CommitterMutations) error { - return c.prewriteMutations(NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), mutations) + return c.prewriteMutations(retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), mutations) } // CommitMutations performs the second phase of commit. func (c CommitterProbe) CommitMutations(ctx context.Context) error { - return c.commitMutations(NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), nil), c.mutationsOfKeys([][]byte{c.primaryKey})) + return c.commitMutations(retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), nil), c.mutationsOfKeys([][]byte{c.primaryKey})) } // MutationsOfKeys returns mutations match the keys. @@ -281,7 +282,7 @@ func (c CommitterProbe) MutationsOfKeys(keys [][]byte) CommitterMutations { // PessimisticRollbackMutations rolls mutations back. func (c CommitterProbe) PessimisticRollbackMutations(ctx context.Context, muts CommitterMutations) error { - return c.pessimisticRollbackMutations(NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, nil), muts) + return c.pessimisticRollbackMutations(retry.NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, nil), muts) } // Cleanup cleans dirty data of a committer. @@ -366,7 +367,7 @@ func (c CommitterProbe) SetPrimaryKeyBlocker(ac, bk chan struct{}) { // CleanupMutations performs the clean up phase. func (c CommitterProbe) CleanupMutations(ctx context.Context) error { - bo := NewBackofferWithVars(ctx, cleanupMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, cleanupMaxBackoff, nil) return c.cleanupMutations(bo, c.mutations) } @@ -434,13 +435,13 @@ func (l LockResolverProbe) ResolveLockAsync(bo *Backoffer, lock *Lock, status Tx // ResolveLock resolves single lock. func (l LockResolverProbe) ResolveLock(ctx context.Context, lock *Lock) error { - bo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil) return l.resolveLock(bo, lock, TxnStatus{}, false, make(map[RegionVerID]struct{})) } // ResolvePessimisticLock resolves single pessimistic lock. func (l LockResolverProbe) ResolvePessimisticLock(ctx context.Context, lock *Lock) error { - bo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil) return l.resolvePessimisticLock(bo, lock, make(map[RegionVerID]struct{})) } diff --git a/store/tikv/tests/lock_test.go b/store/tikv/tests/lock_test.go index 3c6c652d96041..f32991877fefd 100644 --- a/store/tikv/tests/lock_test.go +++ b/store/tikv/tests/lock_test.go @@ -493,7 +493,7 @@ func (s *testLockSuite) TestBatchResolveLocks(c *C) { c.Assert(msBeforeLockExpired, Greater, int64(0)) lr := s.store.NewLockResolver() - bo := tikv.NewBackofferWithVars(context.Background(), tikv.GcResolveLockMaxBackoff, nil) + bo := tikv.NewGcResolveLockMaxBackoffer(context.Background()) loc, err := s.store.GetRegionCache().LocateKey(bo, locks[0].Primary) c.Assert(err, IsNil) // Check BatchResolveLocks resolve the lock even the ttl is not expired. diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 76891cfa19bdb..cba091cbdc8da 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" @@ -443,7 +444,7 @@ func (txn *KVTxn) rollbackPessimisticLocks() error { if txn.lockedCnt == 0 { return nil } - bo := NewBackofferWithVars(context.Background(), cleanupMaxBackoff, txn.vars) + bo := retry.NewBackofferWithVars(context.Background(), cleanupMaxBackoff, txn.vars) keys := txn.collectLockedKeys() return txn.committer.pessimisticRollbackMutations(bo, &PlainMutations{keys: keys}) } @@ -584,16 +585,16 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput lockCtx.Stats = &util.LockKeysDetails{ LockKeys: int32(len(keys)), } - bo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, txn.vars) + bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, txn.vars) txn.committer.forUpdateTS = lockCtx.ForUpdateTS // If the number of keys greater than 1, it can be on different region, // concurrently execute on multiple regions may lead to deadlock. txn.committer.isFirstLock = txn.lockedCnt == 0 && len(keys) == 1 err = txn.committer.pessimisticLockMutations(bo, lockCtx, &PlainMutations{keys: keys}) - if bo.totalSleep > 0 { - atomic.AddInt64(&lockCtx.Stats.BackoffTime, int64(bo.totalSleep)*int64(time.Millisecond)) + if bo.GetTotalSleep() > 0 { + atomic.AddInt64(&lockCtx.Stats.BackoffTime, int64(bo.GetTotalSleep())*int64(time.Millisecond)) lockCtx.Stats.Mu.Lock() - lockCtx.Stats.Mu.BackoffTypes = append(lockCtx.Stats.Mu.BackoffTypes, bo.types...) + lockCtx.Stats.Mu.BackoffTypes = append(lockCtx.Stats.Mu.BackoffTypes, bo.GetTypes()...) lockCtx.Stats.Mu.Unlock() } if lockCtx.Killed != nil { @@ -663,6 +664,8 @@ func deduplicateKeys(keys [][]byte) [][]byte { return deduped } +const pessimisticRollbackMaxBackoff = 20000 + func (txn *KVTxn) asyncPessimisticRollback(ctx context.Context, keys [][]byte) *sync.WaitGroup { // Clone a new committer for execute in background. committer := &twoPhaseCommitter{ @@ -691,7 +694,7 @@ func (txn *KVTxn) asyncPessimisticRollback(ctx context.Context, keys [][]byte) * } }) - err := committer.pessimisticRollbackMutations(NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, txn.vars), &PlainMutations{keys: keys}) + err := committer.pessimisticRollbackMutations(retry.NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, txn.vars), &PlainMutations{keys: keys}) if err != nil { logutil.Logger(ctx).Warn("[kv] pessimisticRollback failed.", zap.Error(err)) } From 9c3f1b70c250871db9212d8eb510ae984c02c5ff Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Thu, 13 May 2021 17:11:40 +0800 Subject: [PATCH 11/42] variables: init cte max recursive deeps in a new session (#24609) --- sessionctx/variable/session.go | 1 + sessionctx/variable/varsutil_test.go | 1 + 2 files changed, 2 insertions(+) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 815f42cc03ef3..97597997b36f5 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1063,6 +1063,7 @@ func NewSessionVars() *SessionVars { AnalyzeVersion: DefTiDBAnalyzeVersion, EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, AllowFallbackToTiKV: make(map[kv.StoreType]struct{}), + CTEMaxRecursionDepth: DefCTEMaxRecursionDepth, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index a96897140b9b4..90507f36539d6 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -105,6 +105,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.ShardAllocateStep, Equals, int64(DefTiDBShardAllocateStep)) c.Assert(vars.EnableChangeColumnType, Equals, DefTiDBChangeColumnType) c.Assert(vars.AnalyzeVersion, Equals, DefTiDBAnalyzeVersion) + c.Assert(vars.CTEMaxRecursionDepth, Equals, DefCTEMaxRecursionDepth) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.MemQuota)) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.BatchSize)) From d2bdfd51b71646dd7c78e437ee40458bef6649cd Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 17:49:39 +0800 Subject: [PATCH 12/42] store/tikv: move transaction options out to /kv (#24619) --- ddl/backfilling.go | 2 +- ddl/column.go | 3 +- ddl/index.go | 5 +-- executor/adapter.go | 3 +- executor/analyze.go | 18 ++++---- executor/batch_point_get.go | 12 ++--- executor/insert.go | 5 +-- executor/insert_common.go | 5 +-- executor/point_get.go | 14 +++--- executor/replace.go | 5 +-- executor/simple.go | 5 +-- executor/update.go | 5 +-- kv/mock_test.go | 3 +- kv/option.go | 62 ++++++++++++++++++++++++++ meta/meta.go | 5 +-- session/session.go | 30 ++++++------- sessionctx/binloginfo/binloginfo.go | 3 +- store/driver/txn/snapshot.go | 22 ++++----- store/driver/txn/txn_driver.go | 48 ++++++++++---------- store/tikv/kv/option.go | 48 -------------------- store/tikv/tests/snapshot_fail_test.go | 2 - 21 files changed, 153 insertions(+), 152 deletions(-) create mode 100644 kv/option.go diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 56512eec6ab65..ed279c68675cf 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -677,7 +677,7 @@ func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version ver := kv.Version{Ver: version} snap := store.GetSnapshot(ver) - snap.SetOption(tikvstore.Priority, priority) + snap.SetOption(kv.Priority, priority) it, err := snap.Iter(firstKey, upperBound) if err != nil { diff --git a/ddl/column.go b/ddl/column.go index 18c23b4d9c45a..e18c0c2d37e7b 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -1346,7 +1345,7 @@ func (w *updateColumnWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t errInTxn = kv.RunInNewTxn(context.Background(), w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error { taskCtx.addedCount = 0 taskCtx.scanCount = 0 - txn.SetOption(tikvstore.Priority, w.priority) + txn.SetOption(kv.Priority, w.priority) rowRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange) if err != nil { diff --git a/ddl/index.go b/ddl/index.go index f11a595aa8fb3..b1b4303d7a0f1 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -1117,7 +1116,7 @@ func (w *addIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (taskC errInTxn = kv.RunInNewTxn(context.Background(), w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error { taskCtx.addedCount = 0 taskCtx.scanCount = 0 - txn.SetOption(tikvstore.Priority, w.priority) + txn.SetOption(kv.Priority, w.priority) idxRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange) if err != nil { @@ -1329,7 +1328,7 @@ func (w *cleanUpIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t errInTxn = kv.RunInNewTxn(context.Background(), w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error { taskCtx.addedCount = 0 taskCtx.scanCount = 0 - txn.SetOption(tikvstore.Priority, w.priority) + txn.SetOption(kv.Priority, w.priority) idxRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange) if err != nil { diff --git a/executor/adapter.go b/executor/adapter.go index 44d00cd1efa1e..784696996cb94 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -43,7 +43,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" tikverr "github.com/pingcap/tidb/store/tikv/error" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -629,7 +628,7 @@ func UpdateForUpdateTS(seCtx sessionctx.Context, newForUpdateTS uint64) error { newForUpdateTS = version.Ver } seCtx.GetSessionVars().TxnCtx.SetForUpdateTS(newForUpdateTS) - txn.SetOption(tikvstore.SnapshotTS, seCtx.GetSessionVars().TxnCtx.GetForUpdateTS()) + txn.SetOption(kv.SnapshotTS, seCtx.GetSessionVars().TxnCtx.GetForUpdateTS()) return nil } diff --git a/executor/analyze.go b/executor/analyze.go index 9cf9c75b1261c..fec55d870bf95 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -1121,9 +1121,9 @@ func (e *AnalyzeFastExec) activateTxnForRowCount() (rollbackFn func() error, err return nil, errors.Trace(err) } } - txn.SetOption(tikvstore.Priority, kv.PriorityLow) - txn.SetOption(tikvstore.IsolationLevel, kv.RC) - txn.SetOption(tikvstore.NotFillCache, true) + txn.SetOption(kv.Priority, kv.PriorityLow) + txn.SetOption(kv.IsolationLevel, kv.RC) + txn.SetOption(kv.NotFillCache, true) return rollbackFn, nil } @@ -1322,7 +1322,7 @@ func (e *AnalyzeFastExec) handleScanIter(iter kv.Iterator) (scanKeysSize int, er func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) (keysSize int, err error) { snapshot := e.ctx.GetStore().GetSnapshot(kv.MaxVersion) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } for _, t := range e.scanTasks { iter, err := snapshot.Iter(kv.Key(t.StartKey), kv.Key(t.EndKey)) @@ -1341,11 +1341,11 @@ func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) (keysSize int, err func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { defer e.wg.Done() snapshot := e.ctx.GetStore().GetSnapshot(kv.MaxVersion) - snapshot.SetOption(tikvstore.NotFillCache, true) - snapshot.SetOption(tikvstore.IsolationLevel, kv.RC) - snapshot.SetOption(tikvstore.Priority, kv.PriorityLow) + snapshot.SetOption(kv.NotFillCache, true) + snapshot.SetOption(kv.IsolationLevel, kv.RC) + snapshot.SetOption(kv.Priority, kv.PriorityLow) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } rander := rand.New(rand.NewSource(e.randSeed)) @@ -1356,7 +1356,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { lower, upper := step-uint32(2*math.Sqrt(float64(step))), step step = uint32(rander.Intn(int(upper-lower))) + lower } - snapshot.SetOption(tikvstore.SampleStep, step) + snapshot.SetOption(kv.SampleStep, step) kvMap := make(map[string][]byte) var iter kv.Iterator iter, *err = snapshot.Iter(kv.Key(task.StartKey), kv.Key(task.EndKey)) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index f3ec18106fb21..23debe37404ee 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -113,17 +113,17 @@ func (e *BatchPointGetExec) Open(context.Context) error { e.stats = &runtimeStatsWithSnapshot{ SnapshotRuntimeStats: snapshotStats, } - snapshot.SetOption(tikvstore.CollectRuntimeStats, snapshotStats) + snapshot.SetOption(kv.CollectRuntimeStats, snapshotStats) e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } - snapshot.SetOption(tikvstore.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) + snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness - snapshot.SetOption(tikvstore.IsStalenessReadOnly, isStaleness) + snapshot.SetOption(kv.IsStalenessReadOnly, isStaleness) if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != oracle.GlobalTxnScope { - snapshot.SetOption(tikvstore.MatchStoreLabels, []*metapb.StoreLabel{ + snapshot.SetOption(kv.MatchStoreLabels, []*metapb.StoreLabel{ { Key: placement.DCLabelKey, Value: e.ctx.GetSessionVars().TxnCtx.TxnScope, @@ -149,7 +149,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { // Close implements the Executor interface. func (e *BatchPointGetExec) Close() error { if e.runtimeStats != nil && e.snapshot != nil { - e.snapshot.DelOption(tikvstore.CollectRuntimeStats) + e.snapshot.DelOption(kv.CollectRuntimeStats) } e.inited = 0 e.index = 0 diff --git a/executor/insert.go b/executor/insert.go index e8fdb9da3444e..178aefed5fb8b 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -215,8 +214,8 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D if e.collectRuntimeStatsEnabled() { if snapshot := txn.GetSnapshot(); snapshot != nil { - snapshot.SetOption(tikvstore.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) - defer snapshot.DelOption(tikvstore.CollectRuntimeStats) + snapshot.SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) + defer snapshot.DelOption(kv.CollectRuntimeStats) } } prefetchStart := time.Now() diff --git a/executor/insert_common.go b/executor/insert_common.go index 10fc6cb9edc59..258e873db89db 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" @@ -1049,8 +1048,8 @@ func (e *InsertValues) batchCheckAndInsert(ctx context.Context, rows [][]types.D } if e.collectRuntimeStatsEnabled() { if snapshot := txn.GetSnapshot(); snapshot != nil { - snapshot.SetOption(tikvstore.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) - defer snapshot.DelOption(tikvstore.CollectRuntimeStats) + snapshot.SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) + defer snapshot.DelOption(kv.CollectRuntimeStats) } } prefetchStart := time.Now() diff --git a/executor/point_get.go b/executor/point_get.go index c34987b7f0c1d..8857a4d253fd0 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -144,17 +144,17 @@ func (e *PointGetExecutor) Open(context.Context) error { e.stats = &runtimeStatsWithSnapshot{ SnapshotRuntimeStats: snapshotStats, } - e.snapshot.SetOption(tikvstore.CollectRuntimeStats, snapshotStats) + e.snapshot.SetOption(kv.CollectRuntimeStats, snapshotStats) e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - e.snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + e.snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } - e.snapshot.SetOption(tikvstore.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) + e.snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness - e.snapshot.SetOption(tikvstore.IsStalenessReadOnly, isStaleness) + e.snapshot.SetOption(kv.IsStalenessReadOnly, isStaleness) if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != oracle.GlobalTxnScope { - e.snapshot.SetOption(tikvstore.MatchStoreLabels, []*metapb.StoreLabel{ + e.snapshot.SetOption(kv.MatchStoreLabels, []*metapb.StoreLabel{ { Key: placement.DCLabelKey, Value: e.ctx.GetSessionVars().TxnCtx.TxnScope, @@ -167,7 +167,7 @@ func (e *PointGetExecutor) Open(context.Context) error { // Close implements the Executor interface. func (e *PointGetExecutor) Close() error { if e.runtimeStats != nil && e.snapshot != nil { - e.snapshot.DelOption(tikvstore.CollectRuntimeStats) + e.snapshot.DelOption(kv.CollectRuntimeStats) } if e.idxInfo != nil && e.tblInfo != nil { actRows := int64(0) @@ -391,7 +391,7 @@ func (e *PointGetExecutor) get(ctx context.Context, key kv.Key) ([]byte, error) } func (e *PointGetExecutor) verifyTxnScope() error { - txnScope := e.txn.GetOption(tikvstore.TxnScope).(string) + txnScope := e.txn.GetOption(kv.TxnScope).(string) if txnScope == "" || txnScope == oracle.GlobalTxnScope { return nil } diff --git a/executor/replace.go b/executor/replace.go index 20af75fe4a0ae..8f35be4d05dbd 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -221,8 +220,8 @@ func (e *ReplaceExec) exec(ctx context.Context, newRows [][]types.Datum) error { if e.collectRuntimeStatsEnabled() { if snapshot := txn.GetSnapshot(); snapshot != nil { - snapshot.SetOption(tikvstore.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) - defer snapshot.DelOption(tikvstore.CollectRuntimeStats) + snapshot.SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) + defer snapshot.DelOption(kv.CollectRuntimeStats) } } prefetchStart := time.Now() diff --git a/executor/simple.go b/executor/simple.go index 24cb857aec3d5..74063b2429c06 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -40,7 +40,6 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" @@ -606,10 +605,10 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { return err } if e.ctx.GetSessionVars().TxnCtx.IsPessimistic { - txn.SetOption(tikvstore.Pessimistic, true) + txn.SetOption(kv.Pessimistic, true) } if s.CausalConsistencyOnly { - txn.SetOption(tikvstore.GuaranteeLinearizability, false) + txn.SetOption(kv.GuaranteeLinearizability, false) } return nil } diff --git a/executor/update.go b/executor/update.go index b8c7e2a985142..7c4b07ab8e6f6 100644 --- a/executor/update.go +++ b/executor/update.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -261,7 +260,7 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { if e.collectRuntimeStatsEnabled() { txn, err := e.ctx.Txn(false) if err == nil && txn.GetSnapshot() != nil { - txn.GetSnapshot().SetOption(tikvstore.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) + txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) } } for rowIdx := 0; rowIdx < chk.NumRows(); rowIdx++ { @@ -408,7 +407,7 @@ func (e *UpdateExec) Close() error { if e.runtimeStats != nil && e.stats != nil { txn, err := e.ctx.Txn(false) if err == nil && txn.GetSnapshot() != nil { - txn.GetSnapshot().DelOption(tikvstore.CollectRuntimeStats) + txn.GetSnapshot().DelOption(kv.CollectRuntimeStats) } } return e.children[0].Close() diff --git a/kv/mock_test.go b/kv/mock_test.go index 45e45d5941251..eba059e763f82 100644 --- a/kv/mock_test.go +++ b/kv/mock_test.go @@ -17,7 +17,6 @@ import ( "context" . "github.com/pingcap/check" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" ) @@ -35,7 +34,7 @@ func (s testMockSuite) TestInterface(c *C) { snapshot := storage.GetSnapshot(version) _, err = snapshot.BatchGet(context.Background(), []Key{Key("abc"), Key("def")}) c.Check(err, IsNil) - snapshot.SetOption(tikvstore.Priority, PriorityNormal) + snapshot.SetOption(Priority, PriorityNormal) transaction, err := storage.Begin() c.Check(err, IsNil) diff --git a/kv/option.go b/kv/option.go new file mode 100644 index 0000000000000..5b04dfba06c95 --- /dev/null +++ b/kv/option.go @@ -0,0 +1,62 @@ +// Copyright 2021 PingCAP, Inc. + +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package kv + +// Transaction options +const ( + // BinlogInfo contains the binlog data and client. + BinlogInfo int = iota + 1 + // SchemaChecker is used for checking schema-validity. + SchemaChecker + // IsolationLevel sets isolation level for current transaction. The default level is SI. + IsolationLevel + // Priority marks the priority of this transaction. + Priority + // NotFillCache makes this request do not touch the LRU cache of the underlying storage. + NotFillCache + // SyncLog decides whether the WAL(write-ahead log) of this request should be synchronized. + SyncLog + // KeyOnly retrieve only keys, it can be used in scan now. + KeyOnly + // Pessimistic is defined for pessimistic lock + Pessimistic + // SnapshotTS is defined to set snapshot ts. + SnapshotTS + // Set replica read + ReplicaRead + // Set task ID + TaskID + // InfoSchema is schema version used by txn startTS. + InfoSchema + // CollectRuntimeStats is used to enable collect runtime stats. + CollectRuntimeStats + // SchemaAmender is used to amend mutations for pessimistic transactions + SchemaAmender + // SampleStep skips 'SampleStep - 1' number of keys after each returned key. + SampleStep + // CommitHook is a callback function called right after the transaction gets committed + CommitHook + // EnableAsyncCommit indicates whether async commit is enabled + EnableAsyncCommit + // Enable1PC indicates whether one-phase commit is enabled + Enable1PC + // GuaranteeLinearizability indicates whether to guarantee linearizability at the cost of an extra tso request before prewrite + GuaranteeLinearizability + // TxnScope indicates which @@txn_scope this transaction will work with. + TxnScope + // StalenessReadOnly indicates whether the transaction is staleness read only transaction + IsStalenessReadOnly + // MatchStoreLabels indicates the labels the store should be matched + MatchStoreLabels +) diff --git a/meta/meta.go b/meta/meta.go index 2682ed5b47d1e..3f76d2948e9b1 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/structure" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/logutil" @@ -94,8 +93,8 @@ type Meta struct { // NewMeta creates a Meta in transaction txn. // If the current Meta needs to handle a job, jobListKey is the type of the job's list. func NewMeta(txn kv.Transaction, jobListKeys ...JobListKeyType) *Meta { - txn.SetOption(tikvstore.Priority, kv.PriorityHigh) - txn.SetOption(tikvstore.SyncLog, struct{}{}) + txn.SetOption(kv.Priority, kv.PriorityHigh) + txn.SetOption(kv.SyncLog, struct{}{}) t := structure.NewStructure(txn, txn, mMetaPrefix) listKey := DefaultJobListKey if len(jobListKeys) != 0 { diff --git a/session/session.go b/session/session.go index 19312e5fc391e..8fc7c4c37eac8 100644 --- a/session/session.go +++ b/session/session.go @@ -497,7 +497,7 @@ func (s *session) doCommit(ctx context.Context) error { }, Client: s.sessionVars.BinlogClient, } - s.txn.SetOption(tikvstore.BinlogInfo, info) + s.txn.SetOption(kv.BinlogInfo, info) } } @@ -508,22 +508,22 @@ func (s *session) doCommit(ctx context.Context) error { physicalTableIDs = append(physicalTableIDs, id) } // Set this option for 2 phase commit to validate schema lease. - s.txn.SetOption(tikvstore.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.sessionVars.GetInfoSchema().SchemaMetaVersion(), physicalTableIDs)) - s.txn.SetOption(tikvstore.InfoSchema, s.sessionVars.TxnCtx.InfoSchema) - s.txn.SetOption(tikvstore.CommitHook, func(info string, _ error) { s.sessionVars.LastTxnInfo = info }) + s.txn.SetOption(kv.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.sessionVars.GetInfoSchema().SchemaMetaVersion(), physicalTableIDs)) + s.txn.SetOption(kv.InfoSchema, s.sessionVars.TxnCtx.InfoSchema) + s.txn.SetOption(kv.CommitHook, func(info string, _ error) { s.sessionVars.LastTxnInfo = info }) if s.GetSessionVars().EnableAmendPessimisticTxn { - s.txn.SetOption(tikvstore.SchemaAmender, NewSchemaAmenderForTikvTxn(s)) + s.txn.SetOption(kv.SchemaAmender, NewSchemaAmenderForTikvTxn(s)) } - s.txn.SetOption(tikvstore.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) - s.txn.SetOption(tikvstore.Enable1PC, s.GetSessionVars().Enable1PC) + s.txn.SetOption(kv.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) + s.txn.SetOption(kv.Enable1PC, s.GetSessionVars().Enable1PC) // priority of the sysvar is lower than `start transaction with causal consistency only` - if val := s.txn.GetOption(tikvstore.GuaranteeLinearizability); val == nil || val.(bool) { + if val := s.txn.GetOption(kv.GuaranteeLinearizability); val == nil || val.(bool) { // We needn't ask the TiKV client to guarantee linearizability for auto-commit transactions // because the property is naturally holds: // We guarantee the commitTS of any transaction must not exceed the next timestamp from the TSO. // An auto-commit transaction fetches its startTS from the TSO so its commitTS > its startTS > the commitTS // of any previously committed transactions. - s.txn.SetOption(tikvstore.GuaranteeLinearizability, + s.txn.SetOption(kv.GuaranteeLinearizability, s.GetSessionVars().TxnCtx.IsExplicit && s.GetSessionVars().GuaranteeLinearizability) } @@ -1883,7 +1883,7 @@ func (s *session) Txn(active bool) (kv.Transaction, error) { } s.sessionVars.TxnCtx.StartTS = s.txn.StartTS() if s.sessionVars.TxnCtx.IsPessimistic { - s.txn.SetOption(tikvstore.Pessimistic, true) + s.txn.SetOption(kv.Pessimistic, true) } if !s.sessionVars.IsAutocommit() { s.sessionVars.SetInTxn(true) @@ -1891,7 +1891,7 @@ func (s *session) Txn(active bool) (kv.Transaction, error) { s.sessionVars.TxnCtx.CouldRetry = s.isTxnRetryable() s.txn.SetVars(s.sessionVars.KVVars) if s.sessionVars.GetReplicaRead().IsFollowerRead() { - s.txn.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + s.txn.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } } return &s.txn, nil @@ -1955,7 +1955,7 @@ func (s *session) NewTxn(ctx context.Context) error { } txn.SetVars(s.sessionVars.KVVars) if s.GetSessionVars().GetReplicaRead().IsFollowerRead() { - txn.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + txn.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() @@ -2763,7 +2763,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionctx.StalenessTxnOption) error { if s.txn.Valid() { txnID := s.txn.StartTS() - txnScope := s.txn.GetOption(tikvstore.TxnScope).(string) + txnScope := s.txn.GetOption(kv.TxnScope).(string) err := s.CommitTxn(ctx) if err != nil { return err @@ -2803,8 +2803,8 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc return s.NewTxn(ctx) } txn.SetVars(s.sessionVars.KVVars) - txn.SetOption(tikvstore.IsStalenessReadOnly, true) - txn.SetOption(tikvstore.TxnScope, txnScope) + txn.SetOption(kv.IsStalenessReadOnly, true) + txn.SetOption(kv.TxnScope, txnScope) s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() s.sessionVars.TxnCtx = &variable.TransactionContext{ diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 58313505e1c8e..163c22e4a6fb7 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-binlog" @@ -295,7 +294,7 @@ func SetDDLBinlog(client *pumpcli.PumpsClient, txn kv.Transaction, jobID int64, }, Client: client, } - txn.SetOption(tikvstore.BinlogInfo, info) + txn.SetOption(kv.BinlogInfo, info) } const specialPrefix = `/*T! ` diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index a6a7d752a72fd..405067f5e082b 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -66,33 +66,33 @@ func (s *tikvSnapshot) IterReverse(k kv.Key) (kv.Iterator, error) { func (s *tikvSnapshot) SetOption(opt int, val interface{}) { switch opt { - case tikvstore.IsolationLevel: + case kv.IsolationLevel: level := getTiKVIsolationLevel(val.(kv.IsoLevel)) s.KVSnapshot.SetIsolationLevel(level) - case tikvstore.Priority: + case kv.Priority: s.KVSnapshot.SetPriority(getTiKVPriority(val.(int))) - case tikvstore.NotFillCache: + case kv.NotFillCache: s.KVSnapshot.SetNotFillCache(val.(bool)) - case tikvstore.SnapshotTS: + case kv.SnapshotTS: s.KVSnapshot.SetSnapshotTS(val.(uint64)) - case tikvstore.ReplicaRead: + case kv.ReplicaRead: s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) - case tikvstore.SampleStep: + case kv.SampleStep: s.KVSnapshot.SetSampleStep(val.(uint32)) - case tikvstore.TaskID: + case kv.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) - case tikvstore.CollectRuntimeStats: + case kv.CollectRuntimeStats: s.KVSnapshot.SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) - case tikvstore.IsStalenessReadOnly: + case kv.IsStalenessReadOnly: s.KVSnapshot.SetIsStatenessReadOnly(val.(bool)) - case tikvstore.MatchStoreLabels: + case kv.MatchStoreLabels: s.KVSnapshot.SetMatchStoreLabels(val.([]*metapb.StoreLabel)) } } func (s *tikvSnapshot) DelOption(opt int) { switch opt { - case tikvstore.CollectRuntimeStats: + case kv.CollectRuntimeStats: s.KVSnapshot.SetRuntimeStats(nil) } } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 0cd51a4480ee1..4d5ce77034312 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -126,51 +126,51 @@ func (txn *tikvTxn) GetUnionStore() kv.UnionStore { func (txn *tikvTxn) SetOption(opt int, val interface{}) { switch opt { - case tikvstore.BinlogInfo: + case kv.BinlogInfo: txn.SetBinlogExecutor(&binlogExecutor{ txn: txn.KVTxn, binInfo: val.(*binloginfo.BinlogInfo), // val cannot be other type. }) - case tikvstore.SchemaChecker: + case kv.SchemaChecker: txn.SetSchemaLeaseChecker(val.(tikv.SchemaLeaseChecker)) - case tikvstore.IsolationLevel: + case kv.IsolationLevel: level := getTiKVIsolationLevel(val.(kv.IsoLevel)) txn.KVTxn.GetSnapshot().SetIsolationLevel(level) - case tikvstore.Priority: + case kv.Priority: txn.KVTxn.SetPriority(getTiKVPriority(val.(int))) - case tikvstore.NotFillCache: + case kv.NotFillCache: txn.KVTxn.GetSnapshot().SetNotFillCache(val.(bool)) - case tikvstore.SyncLog: + case kv.SyncLog: txn.EnableForceSyncLog() - case tikvstore.Pessimistic: + case kv.Pessimistic: txn.SetPessimistic(val.(bool)) - case tikvstore.SnapshotTS: + case kv.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) - case tikvstore.ReplicaRead: + case kv.ReplicaRead: txn.KVTxn.GetSnapshot().SetReplicaRead(val.(tikvstore.ReplicaReadType)) - case tikvstore.TaskID: + case kv.TaskID: txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) - case tikvstore.InfoSchema: + case kv.InfoSchema: txn.SetSchemaVer(val.(tikv.SchemaVer)) - case tikvstore.CollectRuntimeStats: + case kv.CollectRuntimeStats: txn.KVTxn.GetSnapshot().SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) - case tikvstore.SchemaAmender: + case kv.SchemaAmender: txn.SetSchemaAmender(val.(tikv.SchemaAmender)) - case tikvstore.SampleStep: + case kv.SampleStep: txn.KVTxn.GetSnapshot().SetSampleStep(val.(uint32)) - case tikvstore.CommitHook: + case kv.CommitHook: txn.SetCommitCallback(val.(func(string, error))) - case tikvstore.EnableAsyncCommit: + case kv.EnableAsyncCommit: txn.SetEnableAsyncCommit(val.(bool)) - case tikvstore.Enable1PC: + case kv.Enable1PC: txn.SetEnable1PC(val.(bool)) - case tikvstore.GuaranteeLinearizability: + case kv.GuaranteeLinearizability: txn.SetCausalConsistency(!val.(bool)) - case tikvstore.TxnScope: + case kv.TxnScope: txn.SetScope(val.(string)) - case tikvstore.IsStalenessReadOnly: + case kv.IsStalenessReadOnly: txn.KVTxn.GetSnapshot().SetIsStatenessReadOnly(val.(bool)) - case tikvstore.MatchStoreLabels: + case kv.MatchStoreLabels: txn.KVTxn.GetSnapshot().SetMatchStoreLabels(val.([]*metapb.StoreLabel)) default: txn.KVTxn.SetOption(opt, val) @@ -179,9 +179,9 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { func (txn *tikvTxn) GetOption(opt int) interface{} { switch opt { - case tikvstore.GuaranteeLinearizability: + case kv.GuaranteeLinearizability: return !txn.KVTxn.IsCasualConsistency() - case tikvstore.TxnScope: + case kv.TxnScope: return txn.KVTxn.GetScope() default: return txn.KVTxn.GetOption(opt) @@ -190,7 +190,7 @@ func (txn *tikvTxn) GetOption(opt int) interface{} { func (txn *tikvTxn) DelOption(opt int) { switch opt { - case tikvstore.CollectRuntimeStats: + case kv.CollectRuntimeStats: txn.KVTxn.GetSnapshot().SetRuntimeStats(nil) default: txn.KVTxn.DelOption(opt) diff --git a/store/tikv/kv/option.go b/store/tikv/kv/option.go index bac9316d41773..7bd36733a568d 100644 --- a/store/tikv/kv/option.go +++ b/store/tikv/kv/option.go @@ -13,54 +13,6 @@ package kv -// Transaction options -const ( - // BinlogInfo contains the binlog data and client. - BinlogInfo int = iota + 1 - // SchemaChecker is used for checking schema-validity. - SchemaChecker - // IsolationLevel sets isolation level for current transaction. The default level is SI. - IsolationLevel - // Priority marks the priority of this transaction. - Priority - // NotFillCache makes this request do not touch the LRU cache of the underlying storage. - NotFillCache - // SyncLog decides whether the WAL(write-ahead log) of this request should be synchronized. - SyncLog - // KeyOnly retrieve only keys, it can be used in scan now. - KeyOnly - // Pessimistic is defined for pessimistic lock - Pessimistic - // SnapshotTS is defined to set snapshot ts. - SnapshotTS - // Set replica read - ReplicaRead - // Set task ID - TaskID - // InfoSchema is schema version used by txn startTS. - InfoSchema - // CollectRuntimeStats is used to enable collect runtime stats. - CollectRuntimeStats - // SchemaAmender is used to amend mutations for pessimistic transactions - SchemaAmender - // SampleStep skips 'SampleStep - 1' number of keys after each returned key. - SampleStep - // CommitHook is a callback function called right after the transaction gets committed - CommitHook - // EnableAsyncCommit indicates whether async commit is enabled - EnableAsyncCommit - // Enable1PC indicates whether one-phase commit is enabled - Enable1PC - // GuaranteeLinearizability indicates whether to guarantee linearizability at the cost of an extra tso request before prewrite - GuaranteeLinearizability - // TxnScope indicates which @@txn_scope this transaction will work with. - TxnScope - // StalenessReadOnly indicates whether the transaction is staleness read only transaction - IsStalenessReadOnly - // MatchStoreLabels indicates the labels the store should be matched - MatchStoreLabels -) - // Priority value for transaction priority. // TODO: remove after BR update. const ( diff --git a/store/tikv/tests/snapshot_fail_test.go b/store/tikv/tests/snapshot_fail_test.go index 9892061c44b8d..ed812b4f46e00 100644 --- a/store/tikv/tests/snapshot_fail_test.go +++ b/store/tikv/tests/snapshot_fail_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" ) type testSnapshotFailSuite struct { @@ -151,7 +150,6 @@ func (s *testSnapshotFailSuite) TestRetryMaxTsPointGetSkipLock(c *C) { c.Assert(err, IsNil) err = txn.Set([]byte("k2"), []byte("v2")) c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, true) txn.SetEnableAsyncCommit(true) c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing", "return"), IsNil) From 769062b3188d5b51240c2d0cda88d3718ea82634 Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 13 May 2021 18:35:39 +0800 Subject: [PATCH 13/42] =?UTF-8?q?store/driver:=20move=20backoff=20driver?= =?UTF-8?q?=20into=20single=20package=20so=20we=20can=20use=20i=E2=80=A6?= =?UTF-8?q?=20(#24624)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- store/copr/batch_coprocessor.go | 19 ++++--- store/copr/batch_request_sender.go | 4 +- store/copr/coprocessor.go | 23 ++++---- store/copr/coprocessor_test.go | 5 +- store/copr/mpp.go | 13 +++-- store/copr/store.go | 62 +------------------- store/driver/backoff/backoff.go | 90 ++++++++++++++++++++++++++++++ store/tikv/retry/backoff.go | 3 +- 8 files changed, 129 insertions(+), 90 deletions(-) create mode 100644 store/driver/backoff/backoff.go diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index c070f25a454da..b0c0ad5c9ea7b 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -97,7 +98,7 @@ type copTaskAndRPCContext struct { ctx *tikv.RPCContext } -func buildBatchCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { +func buildBatchCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { start := time.Now() const cmdType = tikvrpc.CmdBatchCop rangesLen := ranges.Len() @@ -178,7 +179,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V return copErrorResponse{errors.New("batch coprocessor cannot prove keep order or desc property")} } ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs) - bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) + bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildBatchCopTasks(bo, c.store.GetRegionCache(), ranges, req.StoreType) if err != nil { @@ -223,7 +224,7 @@ func (b *batchCopIterator) run(ctx context.Context) { // We run workers for every batch cop. for _, task := range b.tasks { b.wg.Add(1) - bo := newBackofferWithVars(ctx, copNextMaxBackoff, b.vars) + bo := backoff.NewBackofferWithVars(ctx, copNextMaxBackoff, b.vars) go b.handleTask(ctx, bo, task) } b.wg.Wait() @@ -293,7 +294,7 @@ func (b *batchCopIterator) Close() error { return nil } -func (b *batchCopIterator) handleTask(ctx context.Context, bo *backoffer, task *batchCopTask) { +func (b *batchCopIterator) handleTask(ctx context.Context, bo *Backoffer, task *batchCopTask) { tasks := []*batchCopTask{task} for idx := 0; idx < len(tasks); idx++ { ret, err := b.handleTaskOnce(ctx, bo, tasks[idx]) @@ -308,7 +309,7 @@ func (b *batchCopIterator) handleTask(ctx context.Context, bo *backoffer, task * } // Merge all ranges and request again. -func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { +func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *Backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { var ranges []tikvstore.KeyRange for _, taskCtx := range batchTask.copTasks { taskCtx.task.ranges.Do(func(ran *tikvstore.KeyRange) { @@ -318,7 +319,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoffer, return buildBatchCopTasks(bo, b.store.GetRegionCache(), tikv.NewKeyRanges(ranges), b.req.StoreType) } -func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *backoffer, task *batchCopTask) ([]*batchCopTask, error) { +func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *Backoffer, task *batchCopTask) ([]*batchCopTask, error) { sender := NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) var regionInfos = make([]*coprocessor.RegionInfo, 0, len(task.copTasks)) for _, task := range task.copTasks { @@ -363,7 +364,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *backoffer, ta return nil, b.handleStreamedBatchCopResponse(ctx, bo, resp.Resp.(*tikvrpc.BatchCopStreamResponse), task) } -func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (err error) { +func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *Backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (err error) { defer response.Close() resp := response.BatchResponse if resp == nil { @@ -381,7 +382,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b return nil } - if err1 := bo.b.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil { + if err1 := bo.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil { return errors.Trace(err) } @@ -396,7 +397,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b } } -func (b *batchCopIterator) handleBatchCopResponse(bo *backoffer, response *coprocessor.BatchResponse, task *batchCopTask) (err error) { +func (b *batchCopIterator) handleBatchCopResponse(bo *Backoffer, response *coprocessor.BatchResponse, task *batchCopTask) (err error) { if otherErr := response.GetOtherError(); otherErr != "" { err = errors.Errorf("other error: %s", otherErr) logutil.BgLogger().Warn("other error", diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index 139ee087ec290..422306382337d 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -38,7 +38,7 @@ func NewRegionBatchRequestSender(cache *tikv.RegionCache, client tikv.Client) *R } } -func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { +func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *Backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { // use the first ctx to send request, because every ctx has same address. cancel = func() {} rpcCtx := ctxs[0].ctx @@ -67,7 +67,7 @@ func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *backoffer, ctxs []co return } -func (ss *RegionBatchRequestSender) onSendFail(bo *backoffer, ctxs []copTaskAndRPCContext, err error) error { +func (ss *RegionBatchRequestSender) onSendFail(bo *Backoffer, ctxs []copTaskAndRPCContext, err error) error { // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { return errors.Trace(err) diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index dd8474fd75c3a..e9d9e6b8f1ebb 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" tidbmetrics "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -73,7 +74,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa return c.sendBatch(ctx, req, vars) } ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs) - bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) + bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildCopTasks(bo, c.store.GetRegionCache(), ranges, req) if err != nil { @@ -144,7 +145,7 @@ func (r *copTask) String() string { // rangesPerTask limits the length of the ranges slice sent in one copTask. const rangesPerTask = 25000 -func buildCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, req *kv.Request) ([]*copTask, error) { +func buildCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, req *kv.Request) ([]*copTask, error) { start := time.Now() cmdType := tikvrpc.CmdCop if req.Streaming { @@ -605,12 +606,12 @@ func (it *copIterator) Next(ctx context.Context) (kv.ResultSubset, error) { // Associate each region with an independent backoffer. In this way, when multiple regions are // unavailable, TiDB can execute very quickly without blocking -func chooseBackoffer(ctx context.Context, backoffermap map[uint64]*backoffer, task *copTask, worker *copIteratorWorker) *backoffer { +func chooseBackoffer(ctx context.Context, backoffermap map[uint64]*Backoffer, task *copTask, worker *copIteratorWorker) *Backoffer { bo, ok := backoffermap[task.region.GetID()] if ok { return bo } - newbo := newBackofferWithVars(ctx, copNextMaxBackoff, worker.vars) + newbo := backoff.NewBackofferWithVars(ctx, copNextMaxBackoff, worker.vars) backoffermap[task.region.GetID()] = newbo return newbo } @@ -629,7 +630,7 @@ func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, } }() remainTasks := []*copTask{task} - backoffermap := make(map[uint64]*backoffer) + backoffermap := make(map[uint64]*Backoffer) for len(remainTasks) > 0 { curTask := remainTasks[0] bo := chooseBackoffer(ctx, backoffermap, curTask, worker) @@ -657,7 +658,7 @@ func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, // 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) { +func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch chan<- *copResponse) ([]*copTask, error) { failpoint.Inject("handleTaskOnceError", func(val failpoint.Value) { if val.(bool) { failpoint.Return(nil, errors.New("mock handleTaskOnce error")) @@ -747,7 +748,7 @@ const ( minLogKVProcessTime = 100 ) -func (worker *copIteratorWorker) logTimeCopTask(costTime time.Duration, task *copTask, bo *backoffer, resp *tikvrpc.Response) { +func (worker *copIteratorWorker) logTimeCopTask(costTime time.Duration, task *copTask, bo *Backoffer, resp *tikvrpc.Response) { logStr := fmt.Sprintf("[TIME_COP_PROCESS] resp_time:%s txnStartTS:%d region_id:%d store_addr:%s", costTime, worker.req.StartTs, task.region.GetID(), task.storeAddr) if bo.GetTotalSleep() > minLogBackoffTime { backoffTypes := strings.Replace(fmt.Sprintf("%v", bo.TiKVBackoffer().GetTypes()), " ", ",", -1) @@ -809,7 +810,7 @@ func appendScanDetail(logStr string, columnFamily string, scanInfo *kvrpcpb.Scan return logStr } -func (worker *copIteratorWorker) handleCopStreamResult(bo *backoffer, rpcCtx *tikv.RPCContext, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- *copResponse, costTime time.Duration) ([]*copTask, error) { +func (worker *copIteratorWorker) handleCopStreamResult(bo *Backoffer, rpcCtx *tikv.RPCContext, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- *copResponse, costTime time.Duration) ([]*copTask, error) { defer stream.Close() var resp *coprocessor.Response var lastRange *coprocessor.KeyRange @@ -833,7 +834,7 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *backoffer, rpcCtx *ti if task.storeType == kv.TiFlash { err1 = bo.Backoff(tikv.BoTiFlashRPC, err1) } else { - err1 = bo.b.BackoffTiKVRPC(err1) + err1 = bo.BackoffTiKVRPC(err1) } if err1 != nil { @@ -858,7 +859,7 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *backoffer, rpcCtx *ti // returns more tasks when that happens, or handles the response if no error. // if we're handling streaming coprocessor response, lastRange is the range of last // successful response, otherwise it's nil. -func (worker *copIteratorWorker) handleCopResponse(bo *backoffer, rpcCtx *tikv.RPCContext, resp *copResponse, cacheKey []byte, cacheValue *coprCacheValue, task *copTask, ch chan<- *copResponse, lastRange *coprocessor.KeyRange, costTime time.Duration) ([]*copTask, error) { +func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.RPCContext, resp *copResponse, cacheKey []byte, cacheValue *coprCacheValue, task *copTask, ch chan<- *copResponse, lastRange *coprocessor.KeyRange, costTime time.Duration) ([]*copTask, error) { if regionErr := resp.pbResp.GetRegionError(); regionErr != nil { if rpcCtx != nil && task.storeType == kv.TiDB { resp.err = errors.Errorf("error: %v", regionErr) @@ -1015,7 +1016,7 @@ func (worker *copIteratorWorker) handleTiDBSendReqErr(err error, task *copTask, return nil } -func (worker *copIteratorWorker) buildCopTasksFromRemain(bo *backoffer, lastRange *coprocessor.KeyRange, task *copTask) ([]*copTask, error) { +func (worker *copIteratorWorker) buildCopTasksFromRemain(bo *Backoffer, lastRange *coprocessor.KeyRange, task *copTask) ([]*copTask, error) { remainedRanges := task.ranges if worker.req.Streaming && lastRange != nil { remainedRanges = worker.calculateRemain(task.ranges, lastRange, worker.req.Desc) diff --git a/store/copr/coprocessor_test.go b/store/copr/coprocessor_test.go index d7a6d52c5b4bb..3bd34f05d95f9 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -19,6 +19,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/driver/backoff" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" @@ -43,7 +44,7 @@ func (s *testCoprocessorSuite) TestBuildTasks(c *C) { cache := tikv.NewRegionCache(pdCli) defer cache.Close() - bo := newBackofferWithVars(context.Background(), 3000, nil) + bo := backoff.NewBackofferWithVars(context.Background(), 3000, nil) req := &kv.Request{} flashReq := &kv.Request{} @@ -212,7 +213,7 @@ func (s *testCoprocessorSuite) TestRebuild(c *C) { pdCli := &tikv.CodecPDClient{Client: mocktikv.NewPDClient(cluster)} cache := tikv.NewRegionCache(pdCli) defer cache.Close() - bo := newBackofferWithVars(context.Background(), 3000, nil) + bo := backoff.NewBackofferWithVars(context.Background(), 3000, nil) req := &kv.Request{} tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "z"), req) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 0d156de69fb20..9869fa501d430 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -56,7 +57,7 @@ func (c *MPPClient) selectAllTiFlashStore() []kv.MPPTaskMeta { // ConstructMPPTasks receives ScheduleRequest, which are actually collects of kv ranges. We allocates MPPTaskMeta for them and returns. func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest) ([]kv.MPPTaskMeta, error) { ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTS) - bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) + bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) if req.KeyRanges == nil { return c.selectAllTiFlashStore(), nil } @@ -152,7 +153,7 @@ func (m *mppIterator) run(ctx context.Context) { break } m.wg.Add(1) - bo := newBackoffer(ctx, copNextMaxBackoff) + bo := backoff.NewBackoffer(ctx, copNextMaxBackoff) go m.handleDispatchReq(ctx, bo, task) } m.wg.Wait() @@ -176,7 +177,7 @@ func (m *mppIterator) sendToRespCh(resp *mppResponse) (exit bool) { // TODO:: Consider that which way is better: // - dispatch all tasks at once, and connect tasks at second. // - dispatch tasks and establish connection at the same time. -func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req *kv.MPPDispatchRequest) { +func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req *kv.MPPDispatchRequest) { defer func() { m.wg.Done() }() @@ -299,7 +300,7 @@ func (m *mppIterator) cancelMppTasks() { } } -func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchRequest, taskMeta *mpp.TaskMeta) { +func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchRequest, taskMeta *mpp.TaskMeta) { connReq := &mpp.EstablishMPPConnectionRequest{ SenderMeta: taskMeta, ReceiverMeta: &mpp.TaskMeta{ @@ -343,7 +344,7 @@ func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchReques return } - if err1 := bo.b.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v", err)); err1 != nil { + if err1 := bo.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v", err)); err1 != nil { if errors.Cause(err) == context.Canceled { logutil.BgLogger().Info("stream recv timeout", zap.Error(err)) } else { @@ -366,7 +367,7 @@ func (m *mppIterator) Close() error { return nil } -func (m *mppIterator) handleMPPStreamResponse(bo *backoffer, response *mpp.MPPDataPacket, req *kv.MPPDispatchRequest) (err error) { +func (m *mppIterator) handleMPPStreamResponse(bo *Backoffer, response *mpp.MPPDataPacket, req *kv.MPPDispatchRequest) (err error) { if response.Error != nil { err = errors.Errorf("other error for mpp stream: %s", response.Error.Msg) logutil.BgLogger().Warn("other error", diff --git a/store/copr/store.go b/store/copr/store.go index d3f132f85238f..7fa4aeafb5135 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" @@ -122,62 +123,5 @@ func getEndPointType(t kv.StoreType) tikvrpc.EndpointType { } } -// backoffer wraps tikv.Backoffer and converts the error which returns by the functions of tikv.Backoffer to tidb error. -type backoffer struct { - b *tikv.Backoffer -} - -// newBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. -func newBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *backoffer { - b := tikv.NewBackofferWithVars(ctx, maxSleep, vars) - return &backoffer{b: b} -} - -func newBackoffer(ctx context.Context, maxSleep int) *backoffer { - b := tikv.NewBackoffer(ctx, maxSleep) - return &backoffer{b: b} -} - -// TiKVBackoffer returns tikv.Backoffer. -func (b *backoffer) TiKVBackoffer() *tikv.Backoffer { - return b.b -} - -// Backoff sleeps a while base on the backoffType and records the error message. -// It returns a retryable error if total sleep time exceeds maxSleep. -func (b *backoffer) Backoff(typ tikv.BackoffType, err error) error { - e := b.b.Backoff(typ, err) - return derr.ToTiDBErr(e) -} - -// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message -// and never sleep more than maxSleepMs for each sleep. -func (b *backoffer) BackoffWithMaxSleep(typ tikv.BackoffType, maxSleepMs int, err error) error { - e := b.b.BackoffWithMaxSleep(typ, maxSleepMs, err) - return derr.ToTiDBErr(e) -} - -// GetBackoffTimes returns a map contains backoff time count by type. -func (b *backoffer) GetBackoffTimes() map[tikv.BackoffType]int { - return b.b.GetBackoffTimes() -} - -// GetCtx returns the binded context. -func (b *backoffer) GetCtx() context.Context { - return b.b.GetCtx() -} - -// GetVars returns the binded vars. -func (b *backoffer) GetVars() *tikv.Variables { - return b.b.GetVars() -} - -// GetBackoffSleepMS returns a map contains backoff sleep time by type. -func (b *backoffer) GetBackoffSleepMS() map[tikv.BackoffType]int { - return b.b.GetBackoffSleepMS() -} - -// GetTotalSleep returns total sleep time. -func (b *backoffer) GetTotalSleep() int { - return b.b.GetTotalSleep() -} +// Backoffer wraps tikv.Backoffer and converts the error which returns by the functions of tikv.Backoffer to tidb error. +type Backoffer = backoff.Backoffer diff --git a/store/driver/backoff/backoff.go b/store/driver/backoff/backoff.go new file mode 100644 index 0000000000000..f634366381d06 --- /dev/null +++ b/store/driver/backoff/backoff.go @@ -0,0 +1,90 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package backoff + +import ( + "context" + + "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/tikv" +) + +// Backoffer wraps tikv.Backoffer and converts the error which returns by the functions of tikv.Backoffer to tidb error. +type Backoffer struct { + b *tikv.Backoffer +} + +// NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. +func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *Backoffer { + b := tikv.NewBackofferWithVars(ctx, maxSleep, vars) + return &Backoffer{b: b} +} + +// NewBackoffer creates a Backoffer with maximum sleep time(in ms). +func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { + b := tikv.NewBackoffer(ctx, maxSleep) + return &Backoffer{b: b} +} + +// TiKVBackoffer returns tikv.Backoffer. +func (b *Backoffer) TiKVBackoffer() *tikv.Backoffer { + return b.b +} + +// Backoff sleeps a while base on the backoffType and records the error message. +// It returns a retryable error if total sleep time exceeds maxSleep. +func (b *Backoffer) Backoff(typ tikv.BackoffType, err error) error { + e := b.b.Backoff(typ, err) + return derr.ToTiDBErr(e) +} + +// BackoffTiKVRPC sleeps a while base on the TiKVRPC and records the error message. +// It returns a retryable error if total sleep time exceeds maxSleep. +func (b *Backoffer) BackoffTiKVRPC(err error) error { + e := b.b.BackoffTiKVRPC(err) + return derr.ToTiDBErr(e) +} + +// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message +// and never sleep more than maxSleepMs for each sleep. +func (b *Backoffer) BackoffWithMaxSleep(typ tikv.BackoffType, maxSleepMs int, err error) error { + e := b.b.BackoffWithMaxSleep(typ, maxSleepMs, err) + return derr.ToTiDBErr(e) +} + +// GetBackoffTimes returns a map contains backoff time count by type. +func (b *Backoffer) GetBackoffTimes() map[tikv.BackoffType]int { + return b.b.GetBackoffTimes() +} + +// GetCtx returns the binded context. +func (b *Backoffer) GetCtx() context.Context { + return b.b.GetCtx() +} + +// GetVars returns the binded vars. +func (b *Backoffer) GetVars() *tikv.Variables { + return b.b.GetVars() +} + +// GetBackoffSleepMS returns a map contains backoff sleep time by type. +func (b *Backoffer) GetBackoffSleepMS() map[tikv.BackoffType]int { + return b.b.GetBackoffSleepMS() +} + +// GetTotalSleep returns total sleep time. +func (b *Backoffer) GetTotalSleep() int { + return b.b.GetTotalSleep() +} diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go index 24dc9174f3fec..a563ec7359d22 100644 --- a/store/tikv/retry/backoff.go +++ b/store/tikv/retry/backoff.go @@ -279,7 +279,8 @@ func (b *Backoffer) Backoff(typ BackoffType, err error) error { return b.BackoffWithMaxSleep(typ, -1, err) } -// BackoffTiKVRPC calls Backoff with boTiKVRPC. +// BackoffTiKVRPC sleeps a while base on the TiKVRPC and records the error message. +// It returns a retryable error if total sleep time exceeds maxSleep. func (b *Backoffer) BackoffTiKVRPC(err error) error { return b.Backoff(boTiKVRPC, err) } From 1c4fbfce501d9f7061a5c6fee648eae9f2281892 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 13 May 2021 18:47:39 +0800 Subject: [PATCH 14/42] server: close the temporary session in HTTP API to avoid memory leak (#24339) --- server/http_handler.go | 53 ++++++++++++++---------------------- server/sql_info_fetcher.go | 1 + server/statistics_handler.go | 2 ++ 3 files changed, 23 insertions(+), 33 deletions(-) diff --git a/server/http_handler.go b/server/http_handler.go index 67babd1f05e8d..093a57e45c1ab 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -45,7 +45,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -355,11 +354,11 @@ func (t *tikvHandlerTool) getPartition(tableVal table.Table, partitionName strin } func (t *tikvHandlerTool) schema() (infoschema.InfoSchema, error) { - session, err := session.CreateSession(t.Store) + dom, err := session.GetDomain(t.Store) if err != nil { - return nil, errors.Trace(err) + return nil, err } - return domain.GetDomain(session.(sessionctx.Context)).InfoSchema(), nil + return dom.InfoSchema(), nil } func (t *tikvHandlerTool) handleMvccGetByHex(params map[string]string) (*mvccKV, error) { @@ -712,14 +711,13 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } } if asyncCommit := req.Form.Get("tidb_enable_async_commit"); asyncCommit != "" { - s, err := session.CreateSession(h.Store.(kv.Storage)) + s, err := session.CreateSession(h.Store) if err != nil { writeError(w, err) return } - if s != nil { - defer s.Close() - } + defer s.Close() + switch asyncCommit { case "0": err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnableAsyncCommit, variable.Off) @@ -735,14 +733,13 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } } if onePC := req.Form.Get("tidb_enable_1pc"); onePC != "" { - s, err := session.CreateSession(h.Store.(kv.Storage)) + s, err := session.CreateSession(h.Store) if err != nil { writeError(w, err) return } - if s != nil { - defer s.Close() - } + defer s.Close() + switch onePC { case "0": err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnable1PC, variable.Off) @@ -878,14 +875,11 @@ func (h flashReplicaHandler) getTiFlashReplicaInfo(tblInfo *model.TableInfo, rep } func (h flashReplicaHandler) getDropOrTruncateTableTiflash(currentSchema infoschema.InfoSchema) ([]*tableFlashReplicaInfo, error) { - s, err := session.CreateSession(h.Store.(kv.Storage)) + s, err := session.CreateSession(h.Store) if err != nil { return nil, errors.Trace(err) } - - if s != nil { - defer s.Close() - } + defer s.Close() store := domain.GetDomain(s).Store() txn, err := store.Begin() @@ -948,16 +942,18 @@ func (h flashReplicaHandler) handleStatusReport(w http.ResponseWriter, req *http writeError(w, err) return } - do, err := session.GetDomain(h.Store.(kv.Storage)) + do, err := session.GetDomain(h.Store) if err != nil { writeError(w, err) return } - s, err := session.CreateSession(h.Store.(kv.Storage)) + s, err := session.CreateSession(h.Store) if err != nil { writeError(w, err) return } + defer s.Close() + available := status.checkTableFlashReplicaAvailable() err = do.DDL().UpdateTableReplicaInfo(s, status.ID, available) if err != nil { @@ -1123,18 +1119,7 @@ func (h ddlHistoryJobHandler) ServeHTTP(w http.ResponseWriter, req *http.Request } func (h ddlHistoryJobHandler) getAllHistoryDDL() ([]*model.Job, error) { - s, err := session.CreateSession(h.Store.(kv.Storage)) - if err != nil { - return nil, errors.Trace(err) - } - - if s != nil { - defer s.Close() - } - - store := domain.GetDomain(s.(sessionctx.Context)).Store() - txn, err := store.Begin() - + txn, err := h.Store.Begin() if err != nil { return nil, errors.Trace(err) } @@ -1741,7 +1726,7 @@ type serverInfo struct { // ServeHTTP handles request of ddl server info. func (h serverInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { - do, err := session.GetDomain(h.Store.(kv.Storage)) + do, err := session.GetDomain(h.Store) if err != nil { writeError(w, errors.New("create session error")) log.Error(err) @@ -1771,7 +1756,7 @@ type clusterServerInfo struct { // ServeHTTP handles request of all ddl servers info. func (h allServerInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { - do, err := session.GetDomain(h.Store.(kv.Storage)) + do, err := session.GetDomain(h.Store) if err != nil { writeError(w, errors.New("create session error")) log.Error(err) @@ -1872,6 +1857,8 @@ func (h profileHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { writeError(w, err) return } + defer sctx.Close() + var start, end time.Time if req.FormValue("end") != "" { end, err = time.ParseInLocation(time.RFC3339, req.FormValue("end"), sctx.GetSessionVars().Location()) diff --git a/server/sql_info_fetcher.go b/server/sql_info_fetcher.go index a7be33ea00154..6fc80daf506d6 100644 --- a/server/sql_info_fetcher.go +++ b/server/sql_info_fetcher.go @@ -81,6 +81,7 @@ func (sh *sqlInfoFetcher) zipInfoForSQL(w http.ResponseWriter, r *http.Request) return } defer sh.s.Close() + sh.do = domain.GetDomain(sh.s) reqCtx := r.Context() sql := r.FormValue("sql") diff --git a/server/statistics_handler.go b/server/statistics_handler.go index 733a0559f4943..55e9e4f16df18 100644 --- a/server/statistics_handler.go +++ b/server/statistics_handler.go @@ -92,6 +92,8 @@ func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request writeError(w, err) return } + defer se.Close() + se.GetSessionVars().StmtCtx.TimeZone = time.Local t, err := types.ParseTime(se.GetSessionVars().StmtCtx, params[pSnapshot], mysql.TypeTimestamp, 6) if err != nil { From aa3e64d08bf19c7230cc3b53c3a57f68a50ae3c3 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Thu, 13 May 2021 19:31:39 +0800 Subject: [PATCH 15/42] store/tikv: use latest PD TS plus one as min commit ts (#24579) --- store/tikv/2pc.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 9e4c28ad6fd34..19f3e4faf40e3 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -996,17 +996,18 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { // If we want to use async commit or 1PC and also want linearizability across // all nodes, we have to make sure the commit TS of this transaction is greater // than the snapshot TS of all existent readers. So we get a new timestamp - // from PD as our MinCommitTS. + // from PD and plus one as our MinCommitTS. if commitTSMayBeCalculated && c.needLinearizability() { failpoint.Inject("getMinCommitTSFromTSO", nil) - minCommitTS, err := c.store.oracle.GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + latestTS, err := c.store.oracle.GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) // If we fail to get a timestamp from PD, we just propagate the failure // instead of falling back to the normal 2PC because a normal 2PC will // also be likely to fail due to the same timestamp issue. if err != nil { return errors.Trace(err) } - c.minCommitTS = minCommitTS + // Plus 1 to avoid producing the same commit TS with previously committed transactions + c.minCommitTS = latestTS + 1 } // Calculate maxCommitTS if necessary if commitTSMayBeCalculated { From 1df03a6808b3d69add58a1c55d61146bca13b8f2 Mon Sep 17 00:00:00 2001 From: sylzd Date: Thu, 13 May 2021 19:51:38 +0800 Subject: [PATCH 16/42] planner: fix incorrect TableDual plan built from nulleq (#24596) --- util/ranger/points.go | 3 ++- util/ranger/ranger_test.go | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/util/ranger/points.go b/util/ranger/points.go index d98b548dcbb7a..9c33ccef7feb3 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -246,7 +246,8 @@ func (r *builder) buildFormBinOp(expr *expression.ScalarFunction) []*point { if col.RetType.EvalType() == types.ETString && (value.Kind() == types.KindString || value.Kind() == types.KindBinaryLiteral) { value.SetString(value.GetString(), col.RetType.Collate) } - if col.GetType().Tp == mysql.TypeYear { + // If nulleq with null value, values.ToInt64 will return err + if col.GetType().Tp == mysql.TypeYear && !value.IsNull() { // If the original value is adjusted, we need to change the condition. // For example, col < 2156. Since the max year is 2155, 2156 is changed to 2155. // col < 2155 is wrong. It should be col <= 2155. diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 194ba1e779dc3..575d7f196fbf9 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -1504,12 +1504,13 @@ func (s *testRangerSuite) TestIndexRangeForYear(c *C) { // test index range testKit.MustExec("DROP TABLE IF EXISTS t") testKit.MustExec("CREATE TABLE t (a year(4), key(a))") - testKit.MustExec("INSERT INTO t VALUES (1), (70), (99), (0), ('0')") + testKit.MustExec("INSERT INTO t VALUES (1), (70), (99), (0), ('0'), (NULL)") testKit.MustQuery("SELECT * FROM t WHERE a < 15698").Check(testkit.Rows("0", "1970", "1999", "2000", "2001")) testKit.MustQuery("SELECT * FROM t WHERE a <= 0").Check(testkit.Rows("0")) testKit.MustQuery("SELECT * FROM t WHERE a <= 1").Check(testkit.Rows("0", "1970", "1999", "2000", "2001")) testKit.MustQuery("SELECT * FROM t WHERE a < 2000").Check(testkit.Rows("0", "1970", "1999")) testKit.MustQuery("SELECT * FROM t WHERE a > -1").Check(testkit.Rows("0", "1970", "1999", "2000", "2001")) + testKit.MustQuery("SELECT * FROM t WHERE a <=> NULL").Check(testkit.Rows("")) tests := []struct { indexPos int From aecff1c42e3a05b911311f695db66fce228db369 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 13 May 2021 20:03:38 +0800 Subject: [PATCH 17/42] ranger: fix the case which could have duplicate ranges (#24590) --- util/ranger/ranger.go | 6 +++--- util/ranger/ranger_test.go | 10 +++++++++- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 4f1efef7a7d90..d69c3dbc02392 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -474,7 +474,7 @@ func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bo for _, ran := range ranges { lowTail := len(ran.LowVal) - 1 for i := 0; i < lowTail; i++ { - CutDatumByPrefixLen(&ran.LowVal[i], lengths[i], tp[i]) + hasCut = CutDatumByPrefixLen(&ran.LowVal[i], lengths[i], tp[i]) || hasCut } lowCut := CutDatumByPrefixLen(&ran.LowVal[lowTail], lengths[lowTail], tp[lowTail]) // If the length of the last column of LowVal is equal to the prefix length, LowExclude should be set false. @@ -485,13 +485,13 @@ func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bo } highTail := len(ran.HighVal) - 1 for i := 0; i < highTail; i++ { - CutDatumByPrefixLen(&ran.HighVal[i], lengths[i], tp[i]) + hasCut = CutDatumByPrefixLen(&ran.HighVal[i], lengths[i], tp[i]) || hasCut } highCut := CutDatumByPrefixLen(&ran.HighVal[highTail], lengths[highTail], tp[highTail]) if highCut { ran.HighExclude = false } - hasCut = lowCut || highCut + hasCut = hasCut || lowCut || highCut } return hasCut } diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 575d7f196fbf9..ed4722566033a 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -351,7 +351,8 @@ create table t( index idx_cb(c, a), index idx_d(d(2)), index idx_e(e(2)), - index idx_f(f) + index idx_f(f), + index idx_de(d(2), e) )`) tests := []struct { @@ -620,6 +621,13 @@ create table t( filterConds: "[like(test.t.f, @%, 92)]", resultStr: "[[NULL,+inf]]", }, + { + indexPos: 5, + exprStr: "d in ('aab', 'aac') and e = 'a'", + accessConds: "[in(test.t.d, aab, aac) eq(test.t.e, a)]", + filterConds: "[in(test.t.d, aab, aac)]", + resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", + }, } collate.SetNewCollationEnabledForTest(true) From 5d40ea459a4bf3fc862d193c246dda96da976fc4 Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Thu, 13 May 2021 20:43:38 +0800 Subject: [PATCH 18/42] executor, store: Pass the SQL digest down to pessimistic lock request (#24380) --- executor/executor.go | 3 + store/mockstore/unistore/tikv/deadlock.go | 32 ++++- store/mockstore/unistore/tikv/detector.go | 48 ++++++- .../mockstore/unistore/tikv/detector_test.go | 37 +++-- store/mockstore/unistore/tikv/errors.go | 2 + store/mockstore/unistore/tikv/mvcc.go | 12 +- store/mockstore/unistore/tikv/server.go | 12 +- store/tikv/kv/kv.go | 1 + store/tikv/pessimistic.go | 2 +- store/tikv/region_request_test.go | 8 +- store/tikv/tests/lock_test.go | 132 ++++++++++++++++++ store/tikv/txn.go | 21 +-- util/resourcegrouptag/resource_group_tag.go | 85 +++++++++++ .../resource_group_tag_test.go | 111 +++++++++++++++ 14 files changed, 465 insertions(+), 41 deletions(-) create mode 100644 util/resourcegrouptag/resource_group_tag.go create mode 100644 util/resourcegrouptag/resource_group_tag_test.go diff --git a/executor/executor.go b/executor/executor.go index e5d5d44efefe3..1666f6955bba9 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -62,6 +62,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/resourcegrouptag" "go.uber.org/zap" ) @@ -971,6 +972,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { } func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.LockCtx { + _, sqlDigest := seVars.StmtCtx.SQLDigest() return &tikvstore.LockCtx{ Killed: &seVars.Killed, ForUpdateTS: seVars.TxnCtx.GetForUpdateTS(), @@ -980,6 +982,7 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.Loc LockKeysDuration: &seVars.StmtCtx.LockKeysDuration, LockKeysCount: &seVars.StmtCtx.LockKeysCount, LockExpired: &seVars.TxnCtx.LockExpire, + ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(sqlDigest), } } diff --git a/store/mockstore/unistore/tikv/deadlock.go b/store/mockstore/unistore/tikv/deadlock.go index 6641a500e2cc1..de2eaf8fa61d9 100644 --- a/store/mockstore/unistore/tikv/deadlock.go +++ b/store/mockstore/unistore/tikv/deadlock.go @@ -44,7 +44,10 @@ type DetectorServer struct { func (ds *DetectorServer) Detect(req *deadlockPb.DeadlockRequest) *deadlockPb.DeadlockResponse { switch req.Tp { case deadlockPb.DeadlockRequestType_Detect: - err := ds.Detector.Detect(req.Entry.Txn, req.Entry.WaitForTxn, req.Entry.KeyHash) + err := ds.Detector.Detect(req.Entry.Txn, req.Entry.WaitForTxn, req.Entry.KeyHash, diagnosticContext{ + key: req.Entry.Key, + resourceGroupTag: req.Entry.ResourceGroupTag, + }) if err != nil { resp := convertErrToResp(err, req.Entry.Txn, req.Entry.WaitForTxn, req.Entry.KeyHash) return resp @@ -178,30 +181,35 @@ func (dt *DetectorClient) recvLoop(streamCli deadlockPb.Deadlock_DetectClient) { } func (dt *DetectorClient) handleRemoteTask(requestType deadlockPb.DeadlockRequestType, - txnTs uint64, waitForTxnTs uint64, keyHash uint64) { + txnTs uint64, waitForTxnTs uint64, keyHash uint64, diagCtx diagnosticContext) { detectReq := &deadlockPb.DeadlockRequest{} detectReq.Tp = requestType detectReq.Entry.Txn = txnTs detectReq.Entry.WaitForTxn = waitForTxnTs detectReq.Entry.KeyHash = keyHash + detectReq.Entry.Key = diagCtx.key + detectReq.Entry.ResourceGroupTag = diagCtx.resourceGroupTag dt.sendCh <- detectReq } // CleanUp processes cleaup task on local detector // user interfaces func (dt *DetectorClient) CleanUp(startTs uint64) { - dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUp, startTs, 0, 0) + dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUp, startTs, 0, 0, diagnosticContext{}) } // CleanUpWaitFor cleans up the specific wait edge in detector's wait map func (dt *DetectorClient) CleanUpWaitFor(txnTs, waitForTxn, keyHash uint64) { - dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUpWaitFor, txnTs, waitForTxn, keyHash) + dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUpWaitFor, txnTs, waitForTxn, keyHash, diagnosticContext{}) } // Detect post the detection request to local deadlock detector or remote first region leader, // the caller should use `waiter.ch` to receive possible deadlock response -func (dt *DetectorClient) Detect(txnTs uint64, waitForTxnTs uint64, keyHash uint64) { - dt.handleRemoteTask(deadlockPb.DeadlockRequestType_Detect, txnTs, waitForTxnTs, keyHash) +func (dt *DetectorClient) Detect(txnTs uint64, waitForTxnTs uint64, keyHash uint64, key []byte, resourceGroupTag []byte) { + dt.handleRemoteTask(deadlockPb.DeadlockRequestType_Detect, txnTs, waitForTxnTs, keyHash, diagnosticContext{ + key: key, + resourceGroupTag: resourceGroupTag, + }) } // convertErrToResp converts `ErrDeadlock` to `DeadlockResponse` proto type @@ -213,6 +221,18 @@ func convertErrToResp(errDeadlock *ErrDeadlock, txnTs, waitForTxnTs, keyHash uin resp := &deadlockPb.DeadlockResponse{} resp.Entry = entry resp.DeadlockKeyHash = errDeadlock.DeadlockKeyHash + + resp.WaitChain = make([]*deadlockPb.WaitForEntry, 0, len(errDeadlock.WaitChain)) + for _, item := range errDeadlock.WaitChain { + resp.WaitChain = append(resp.WaitChain, &deadlockPb.WaitForEntry{ + Txn: item.Txn, + WaitForTxn: item.WaitForTxn, + KeyHash: item.KeyHash, + Key: item.Key, + ResourceGroupTag: item.ResourceGroupTag, + }) + } + return resp } diff --git a/store/mockstore/unistore/tikv/detector.go b/store/mockstore/unistore/tikv/detector.go index 0273bed5fe6a8..a27adb3f35b6c 100644 --- a/store/mockstore/unistore/tikv/detector.go +++ b/store/mockstore/unistore/tikv/detector.go @@ -30,6 +30,7 @@ import ( "sync" "time" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/log" "go.uber.org/zap" ) @@ -54,6 +55,12 @@ type txnKeyHashPair struct { txn uint64 keyHash uint64 registerTime time.Time + diagCtx diagnosticContext +} + +type diagnosticContext struct { + key []byte + resourceGroupTag []byte } func (p *txnKeyHashPair) isExpired(ttl time.Duration, nowTime time.Time) bool { @@ -75,13 +82,27 @@ func NewDetector(ttl time.Duration, urgentSize uint64, expireInterval time.Durat } // Detect detects deadlock for the sourceTxn on a locked key. -func (d *Detector) Detect(sourceTxn, waitForTxn, keyHash uint64) *ErrDeadlock { +func (d *Detector) Detect(sourceTxn, waitForTxn, keyHash uint64, diagCtx diagnosticContext) *ErrDeadlock { d.lock.Lock() nowTime := time.Now() d.activeExpire(nowTime) err := d.doDetect(nowTime, sourceTxn, waitForTxn) if err == nil { - d.register(sourceTxn, waitForTxn, keyHash) + d.register(sourceTxn, waitForTxn, keyHash, diagCtx) + } else { + // Reverse the wait chain so that the order will be each one waiting for the next one, and append the current + // entry that finally caused the deadlock. + for i := 0; i < len(err.WaitChain)/2; i++ { + j := len(err.WaitChain) - i - 1 + err.WaitChain[i], err.WaitChain[j] = err.WaitChain[j], err.WaitChain[i] + } + err.WaitChain = append(err.WaitChain, &deadlockPB.WaitForEntry{ + Txn: sourceTxn, + Key: diagCtx.key, + KeyHash: keyHash, + ResourceGroupTag: diagCtx.resourceGroupTag, + WaitForTxn: waitForTxn, + }) } d.lock.Unlock() return err @@ -103,9 +124,26 @@ func (d *Detector) doDetect(nowTime time.Time, sourceTxn, waitForTxn uint64) *Er continue } if keyHashPair.txn == sourceTxn { - return &ErrDeadlock{DeadlockKeyHash: keyHashPair.keyHash} + return &ErrDeadlock{DeadlockKeyHash: keyHashPair.keyHash, + WaitChain: []*deadlockPB.WaitForEntry{ + { + Txn: waitForTxn, + Key: keyHashPair.diagCtx.key, + KeyHash: keyHashPair.keyHash, + ResourceGroupTag: keyHashPair.diagCtx.resourceGroupTag, + WaitForTxn: keyHashPair.txn, + }, + }, + } } if err := d.doDetect(nowTime, sourceTxn, keyHashPair.txn); err != nil { + err.WaitChain = append(err.WaitChain, &deadlockPB.WaitForEntry{ + Txn: waitForTxn, + Key: keyHashPair.diagCtx.key, + KeyHash: keyHashPair.keyHash, + ResourceGroupTag: keyHashPair.diagCtx.resourceGroupTag, + WaitForTxn: keyHashPair.txn, + }) return err } } @@ -115,9 +153,9 @@ func (d *Detector) doDetect(nowTime time.Time, sourceTxn, waitForTxn uint64) *Er return nil } -func (d *Detector) register(sourceTxn, waitForTxn, keyHash uint64) { +func (d *Detector) register(sourceTxn, waitForTxn, keyHash uint64, diagCtx diagnosticContext) { val := d.waitForMap[sourceTxn] - pair := txnKeyHashPair{txn: waitForTxn, keyHash: keyHash, registerTime: time.Now()} + pair := txnKeyHashPair{txn: waitForTxn, keyHash: keyHash, registerTime: time.Now(), diagCtx: diagCtx} if val == nil { newList := &txnList{txns: list.New()} newList.txns.PushBack(&pair) diff --git a/store/mockstore/unistore/tikv/detector_test.go b/store/mockstore/unistore/tikv/detector_test.go index 1768cc377ec7c..b0d3a074ff840 100644 --- a/store/mockstore/unistore/tikv/detector_test.go +++ b/store/mockstore/unistore/tikv/detector_test.go @@ -31,6 +31,7 @@ import ( "time" . "github.com/pingcap/check" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" ) func TestT(t *testing.T) { @@ -42,19 +43,38 @@ var _ = Suite(&testDeadlockSuite{}) type testDeadlockSuite struct{} func (s *testDeadlockSuite) TestDeadlock(c *C) { + makeDiagCtx := func(key string, resourceGroupTag string) diagnosticContext { + return diagnosticContext{ + key: []byte(key), + resourceGroupTag: []byte(resourceGroupTag), + } + } + checkWaitChainEntry := func(entry *deadlockPB.WaitForEntry, txn, waitForTxn uint64, key, resourceGroupTag string) { + c.Assert(entry.Txn, Equals, txn) + c.Assert(entry.WaitForTxn, Equals, waitForTxn) + c.Assert(string(entry.Key), Equals, key) + c.Assert(string(entry.ResourceGroupTag), Equals, resourceGroupTag) + } + ttl := 50 * time.Millisecond expireInterval := 100 * time.Millisecond urgentSize := uint64(1) detector := NewDetector(ttl, urgentSize, expireInterval) - err := detector.Detect(1, 2, 100) + err := detector.Detect(1, 2, 100, makeDiagCtx("k1", "tag1")) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(1)) - err = detector.Detect(2, 3, 200) + err = detector.Detect(2, 3, 200, makeDiagCtx("k2", "tag2")) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(2)) - err = detector.Detect(3, 1, 300) + err = detector.Detect(3, 1, 300, makeDiagCtx("k3", "tag3")) c.Assert(err, NotNil) c.Assert(err.Error(), Equals, fmt.Sprintf("deadlock")) + c.Assert(len(err.WaitChain), Equals, 3) + // The order of entries in the wait chain is specific: each item is waiting for the next one. + checkWaitChainEntry(err.WaitChain[0], 1, 2, "k1", "tag1") + checkWaitChainEntry(err.WaitChain[1], 2, 3, "k2", "tag2") + checkWaitChainEntry(err.WaitChain[2], 3, 1, "k3", "tag3") + c.Assert(detector.totalSize, Equals, uint64(2)) detector.CleanUp(2) list2 := detector.waitForMap[2] @@ -62,20 +82,21 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { c.Assert(detector.totalSize, Equals, uint64(1)) // After cycle is broken, no deadlock now. - err = detector.Detect(3, 1, 300) + diagCtx := diagnosticContext{} + err = detector.Detect(3, 1, 300, diagCtx) c.Assert(err, IsNil) list3 := detector.waitForMap[3] c.Assert(list3.txns.Len(), Equals, 1) c.Assert(detector.totalSize, Equals, uint64(2)) // Different keyHash grows the list. - err = detector.Detect(3, 1, 400) + err = detector.Detect(3, 1, 400, diagCtx) c.Assert(err, IsNil) c.Assert(list3.txns.Len(), Equals, 2) c.Assert(detector.totalSize, Equals, uint64(3)) // Same waitFor and key hash doesn't grow the list. - err = detector.Detect(3, 1, 400) + err = detector.Detect(3, 1, 400, diagCtx) c.Assert(err, IsNil) c.Assert(list3.txns.Len(), Equals, 2) c.Assert(detector.totalSize, Equals, uint64(3)) @@ -90,7 +111,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { // after 100ms, all entries expired, detect non exist edges time.Sleep(100 * time.Millisecond) - err = detector.Detect(100, 200, 100) + err = detector.Detect(100, 200, 100, diagCtx) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(1)) c.Assert(len(detector.waitForMap), Equals, 1) @@ -98,7 +119,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { // expired entry should not report deadlock, detect will remove this entry // not dependent on expire check interval time.Sleep(60 * time.Millisecond) - err = detector.Detect(200, 100, 200) + err = detector.Detect(200, 100, 200, diagCtx) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(1)) c.Assert(len(detector.waitForMap), Equals, 1) diff --git a/store/mockstore/unistore/tikv/errors.go b/store/mockstore/unistore/tikv/errors.go index 01d28fb73c896..98a70951871d5 100644 --- a/store/mockstore/unistore/tikv/errors.go +++ b/store/mockstore/unistore/tikv/errors.go @@ -16,6 +16,7 @@ package tikv import ( "fmt" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/mockstore/unistore/tikv/mvcc" ) @@ -90,6 +91,7 @@ type ErrDeadlock struct { LockKey []byte LockTS uint64 DeadlockKeyHash uint64 + WaitChain []*deadlockPB.WaitForEntry } func (e ErrDeadlock) Error() string { diff --git a/store/mockstore/unistore/tikv/mvcc.go b/store/mockstore/unistore/tikv/mvcc.go index 4e3eb4f7d7df8..fe5a75b549945 100644 --- a/store/mockstore/unistore/tikv/mvcc.go +++ b/store/mockstore/unistore/tikv/mvcc.go @@ -239,7 +239,11 @@ func (store *MVCCStore) PessimisticLock(reqCtx *requestCtx, req *kvrpcpb.Pessimi for _, m := range mutations { lock, err := store.checkConflictInLockStore(reqCtx, m, startTS) if err != nil { - return store.handleCheckPessimisticErr(startTS, err, req.IsFirstLock, req.WaitTimeout) + var resourceGroupTag []byte = nil + if req.Context != nil { + resourceGroupTag = req.Context.ResourceGroupTag + } + return store.handleCheckPessimisticErr(startTS, err, req.IsFirstLock, req.WaitTimeout, m.Key, resourceGroupTag) } if lock != nil { if lock.Op != uint8(kvrpcpb.Op_PessimisticLock) { @@ -533,11 +537,13 @@ func (store *MVCCStore) CheckSecondaryLocks(reqCtx *requestCtx, keys [][]byte, s func (store *MVCCStore) normalizeWaitTime(lockWaitTime int64) time.Duration { if lockWaitTime > store.conf.PessimisticTxn.WaitForLockTimeout { lockWaitTime = store.conf.PessimisticTxn.WaitForLockTimeout + } else if lockWaitTime == 0 { + lockWaitTime = store.conf.PessimisticTxn.WaitForLockTimeout } return time.Duration(lockWaitTime) * time.Millisecond } -func (store *MVCCStore) handleCheckPessimisticErr(startTS uint64, err error, isFirstLock bool, lockWaitTime int64) (*lockwaiter.Waiter, error) { +func (store *MVCCStore) handleCheckPessimisticErr(startTS uint64, err error, isFirstLock bool, lockWaitTime int64, key []byte, resourceGroupTag []byte) (*lockwaiter.Waiter, error) { if locked, ok := err.(*ErrLocked); ok { if lockWaitTime != lockwaiter.LockNoWait { keyHash := farm.Fingerprint64(locked.Key) @@ -546,7 +552,7 @@ func (store *MVCCStore) handleCheckPessimisticErr(startTS uint64, err error, isF log.S().Debugf("%d blocked by %d on key %d", startTS, lock.StartTS, keyHash) waiter := store.lockWaiterManager.NewWaiter(startTS, lock.StartTS, keyHash, waitTimeDuration) if !isFirstLock { - store.DeadlockDetectCli.Detect(startTS, lock.StartTS, keyHash) + store.DeadlockDetectCli.Detect(startTS, lock.StartTS, keyHash, key, resourceGroupTag) } return waiter, err } diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index adf3049330897..036d824a39ff9 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -217,6 +217,7 @@ func (svr *Server) KvPessimisticLock(ctx context.Context, req *kvrpcpb.Pessimist LockKey: errLocked.Key, LockTS: errLocked.Lock.StartTS, DeadlockKeyHash: result.DeadlockResp.DeadlockKeyHash, + WaitChain: result.DeadlockResp.WaitChain, } resp.Errors, resp.RegionError = convertToPBErrors(deadlockErr) return resp, nil @@ -845,11 +846,6 @@ func (svr *Server) ReadIndex(context.Context, *kvrpcpb.ReadIndexRequest) (*kvrpc return &kvrpcpb.ReadIndexResponse{}, nil } -// GetLockWaitInfo implements implements the tikvpb.TikvServer interface. -func (svr *Server) GetLockWaitInfo(ctx context.Context, _ *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { - return &kvrpcpb.GetLockWaitInfoResponse{}, nil -} - // transaction debugger commands. // MvccGetByKey implements implements the tikvpb.TikvServer interface. @@ -976,6 +972,11 @@ func (svr *Server) GetStoreSafeTS(context.Context, *kvrpcpb.StoreSafeTSRequest) return &kvrpcpb.StoreSafeTSResponse{}, nil } +// GetLockWaitInfo implements the tikvpb.TikvServer interface. +func (svr *Server) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + panic("unimplemented") +} + func convertToKeyError(err error) *kvrpcpb.KeyError { if err == nil { return nil @@ -1011,6 +1012,7 @@ func convertToKeyError(err error) *kvrpcpb.KeyError { LockKey: x.LockKey, LockTs: x.LockTS, DeadlockKeyHash: x.DeadlockKeyHash, + WaitChain: x.WaitChain, }, } case *ErrCommitExpire: diff --git a/store/tikv/kv/kv.go b/store/tikv/kv/kv.go index 2b7e87ecd2e47..8ba36a749db4f 100644 --- a/store/tikv/kv/kv.go +++ b/store/tikv/kv/kv.go @@ -27,4 +27,5 @@ type LockCtx struct { ValuesLock sync.Mutex LockExpired *uint32 Stats *util.LockKeysDetails + ResourceGroupTag []byte } diff --git a/store/tikv/pessimistic.go b/store/tikv/pessimistic.go index 445ced93ff904..2da8e93dad946 100644 --- a/store/tikv/pessimistic.go +++ b/store/tikv/pessimistic.go @@ -101,7 +101,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * WaitTimeout: action.LockWaitTime, ReturnValues: action.ReturnValues, MinCommitTs: c.forUpdateTS + 1, - }, pb.Context{Priority: c.priority, SyncLog: c.syncLog}) + }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: action.LockCtx.ResourceGroupTag}) lockWaitStartTime := action.WaitStartTime for { // if lockWaitTime set, refine the request `WaitTimeout` field based on timeout limit diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index f7fd2a149060d..bbe7ff8d19479 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -463,10 +463,6 @@ func (s *mockTikvGrpcServer) SplitRegion(context.Context, *kvrpcpb.SplitRegionRe return nil, errors.New("unreachable") } -func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { - return nil, errors.New("unreachable") -} - func (s *mockTikvGrpcServer) CoprocessorStream(*coprocessor.Request, tikvpb.Tikv_CoprocessorStreamServer) error { return errors.New("unreachable") } @@ -495,6 +491,10 @@ func (s *mockTikvGrpcServer) CoprocessorV2(context.Context, *coprocessor_v2.RawC return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + return nil, errors.New("unreachable") +} + func (s *testRegionRequestToSingleStoreSuite) TestNoReloadRegionForGrpcWhenCtxCanceled(c *C) { // prepare a mock tikv grpc server addr := "localhost:56341" diff --git a/store/tikv/tests/lock_test.go b/store/tikv/tests/lock_test.go index f32991877fefd..d64c1d102e6d1 100644 --- a/store/tikv/tests/lock_test.go +++ b/store/tikv/tests/lock_test.go @@ -19,13 +19,17 @@ import ( "fmt" "math" "runtime" + "sync" "time" . "github.com/pingcap/check" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) @@ -640,3 +644,131 @@ func (s *testLockSuite) TestBatchResolveTxnFallenBackFromAsyncCommit(c *C) { _, err = t3.Get(context.Background(), []byte("fb2")) c.Assert(tikverr.IsErrNotFound(err), IsTrue) } + +func (s *testLockSuite) TestDeadlockReportWaitChain(c *C) { + // Utilities to make the test logic clear and simple. + type txnWrapper struct { + tikv.TxnProbe + wg sync.WaitGroup + } + + makeLockCtx := func(txn *txnWrapper, resourceGroupTag string) *kv.LockCtx { + return &kv.LockCtx{ + ForUpdateTS: txn.StartTS(), + WaitStartTime: time.Now(), + LockWaitTime: 1000, + ResourceGroupTag: []byte(resourceGroupTag), + } + } + + // Prepares several transactions and each locks a key. + prepareTxns := func(num int) []*txnWrapper { + res := make([]*txnWrapper, 0, num) + for i := 0; i < num; i++ { + txnProbe, err := s.store.Begin() + c.Assert(err, IsNil) + txn := &txnWrapper{TxnProbe: txnProbe} + txn.SetPessimistic(true) + tag := fmt.Sprintf("tag-init%v", i) + key := []byte{'k', byte(i)} + err = txn.LockKeys(context.Background(), makeLockCtx(txn, tag), key) + c.Assert(err, IsNil) + + res = append(res, txn) + } + return res + } + + // Let the i-th trnasaction lock the key that has been locked by j-th transaction + tryLock := func(txns []*txnWrapper, i int, j int) error { + c.Logf("txn %v try locking %v", i, j) + txn := txns[i] + tag := fmt.Sprintf("tag-%v-%v", i, j) + key := []byte{'k', byte(j)} + return txn.LockKeys(context.Background(), makeLockCtx(txn, tag), key) + } + + // Asserts the i-th transaction waits for the j-th transaction. + makeWaitFor := func(txns []*txnWrapper, i int, j int) { + txns[i].wg.Add(1) + go func() { + defer txns[i].wg.Done() + err := tryLock(txns, i, j) + // After the lock being waited for is released, the transaction returns a WriteConflict error + // unconditionally, which is by design. + c.Assert(err, NotNil) + c.Logf("txn %v wait for %v finished, err: %s", i, j, err.Error()) + _, ok := errors.Cause(err).(*tikverr.ErrWriteConflict) + c.Assert(ok, IsTrue) + }() + } + + waitAndRollback := func(txns []*txnWrapper, i int) { + // It's expected that each transaction should be rolled back after its blocker, so that `Rollback` will not + // run when there's concurrent `LockKeys` running. + // If it's blocked on the `Wait` forever, it means the transaction's blocker is not rolled back. + c.Logf("rollback txn %v", i) + txns[i].wg.Wait() + err := txns[i].Rollback() + c.Assert(err, IsNil) + } + + // Check the given WaitForEntry is caused by txn[i] waiting for txn[j]. + checkWaitChainEntry := func(txns []*txnWrapper, entry *deadlockPB.WaitForEntry, i, j int) { + c.Assert(entry.Txn, Equals, txns[i].StartTS()) + c.Assert(entry.WaitForTxn, Equals, txns[j].StartTS()) + c.Assert(entry.Key, BytesEquals, []byte{'k', byte(j)}) + c.Assert(string(entry.ResourceGroupTag), Equals, fmt.Sprintf("tag-%v-%v", i, j)) + } + + c.Log("test case 1: 1->0->1") + + txns := prepareTxns(2) + + makeWaitFor(txns, 0, 1) + // Sleep for a while to make sure it has been blocked. + time.Sleep(time.Millisecond * 100) + + // txn2 tries locking k1 and encounters deadlock error. + err := tryLock(txns, 1, 0) + c.Assert(err, NotNil) + dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock) + c.Assert(ok, IsTrue) + + waitChain := dl.GetWaitChain() + c.Assert(len(waitChain), Equals, 2) + checkWaitChainEntry(txns, waitChain[0], 0, 1) + checkWaitChainEntry(txns, waitChain[1], 1, 0) + + // Each transaction should be rolled back after its blocker being rolled back + waitAndRollback(txns, 1) + waitAndRollback(txns, 0) + + c.Log("test case 2: 3->2->0->1->3") + txns = prepareTxns(4) + + makeWaitFor(txns, 0, 1) + makeWaitFor(txns, 2, 0) + makeWaitFor(txns, 1, 3) + // Sleep for a while to make sure it has been blocked. + time.Sleep(time.Millisecond * 100) + + err = tryLock(txns, 3, 2) + c.Assert(err, NotNil) + dl, ok = errors.Cause(err).(*tikverr.ErrDeadlock) + c.Assert(ok, IsTrue) + + waitChain = dl.GetWaitChain() + c.Assert(len(waitChain), Equals, 4) + c.Logf("wait chain: \n** %v\n**%v\n**%v\n**%v\n", waitChain[0], waitChain[1], waitChain[2], waitChain[3]) + checkWaitChainEntry(txns, waitChain[0], 2, 0) + checkWaitChainEntry(txns, waitChain[1], 0, 1) + checkWaitChainEntry(txns, waitChain[2], 1, 3) + checkWaitChainEntry(txns, waitChain[3], 3, 2) + + // Each transaction should be rolled back after its blocker being rolled back + waitAndRollback(txns, 3) + waitAndRollback(txns, 1) + waitAndRollback(txns, 0) + waitAndRollback(txns, 2) +} diff --git a/store/tikv/txn.go b/store/tikv/txn.go index cba091cbdc8da..20bf0491ed294 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -613,15 +613,18 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput // If there is only 1 key and lock fails, no need to do pessimistic rollback. if len(keys) > 1 || keyMayBeLocked { wg := txn.asyncPessimisticRollback(ctx, keys) - if dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok && hashInKeys(dl.DeadlockKeyHash, keys) { - dl.IsRetryable = true - // Wait for the pessimistic rollback to finish before we retry the statement. - wg.Wait() - // Sleep a little, wait for the other transaction that blocked by this transaction to acquire the lock. - time.Sleep(time.Millisecond * 5) - failpoint.Inject("SingleStmtDeadLockRetrySleep", func() { - time.Sleep(300 * time.Millisecond) - }) + if dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok { + logutil.Logger(ctx).Debug("deadlock error received", zap.Uint64("startTS", txn.startTS), zap.Stringer("deadlockInfo", dl)) + if hashInKeys(dl.DeadlockKeyHash, keys) { + dl.IsRetryable = true + // Wait for the pessimistic rollback to finish before we retry the statement. + wg.Wait() + // Sleep a little, wait for the other transaction that blocked by this transaction to acquire the lock. + time.Sleep(time.Millisecond * 5) + failpoint.Inject("SingleStmtDeadLockRetrySleep", func() { + time.Sleep(300 * time.Millisecond) + }) + } } } if assignedPrimaryKey { diff --git a/util/resourcegrouptag/resource_group_tag.go b/util/resourcegrouptag/resource_group_tag.go new file mode 100644 index 0000000000000..cacbf574b91fb --- /dev/null +++ b/util/resourcegrouptag/resource_group_tag.go @@ -0,0 +1,85 @@ +package resourcegrouptag + +import ( + "encoding/hex" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + +const ( + resourceGroupTagPrefixSQLDigest = byte(1) +) + +// EncodeResourceGroupTag encodes sqlDigest into resource group tag. +// A resource group tag can be carried in the Context field of TiKV requests, which is a byte array, and sent to TiKV as +// diagnostic information. Currently it contains only the SQL Digest, and the codec method is naive but extendable. +// This function doesn't return error. When there's some error, which can only be caused by unexpected format of the +// arguments, it simply returns an empty result. +// The format: +// +-----------+-----------------------+----------------------------+---------------+----------------+---- +// | version=1 | field1 prefix (1byte) | field1 content (var bytes) | field2 prefix | field2 content | ... +// +-----------+-----------------------+----------------------------+---------------+----------------+---- +// The `version` section marks the codec version, which makes it easier for changing the format in the future. +// Each field starts with a byte to mark what field it is, and the length of the content depends on the field's +// definition. +// Currently there's only one field (SQL Digest), and its content starts with a byte `B` describing it's length, and +// then follows by exactly `B` bytes. +func EncodeResourceGroupTag(sqlDigest string) []byte { + if len(sqlDigest) == 0 { + return nil + } + if len(sqlDigest) >= 512 { + logutil.BgLogger().Warn("failed to encode sql digest to resource group tag: length too long", zap.String("sqlDigest", sqlDigest)) + return nil + } + + res := make([]byte, 3+len(sqlDigest)/2) + + const encodingVersion = 1 + res[0] = encodingVersion + + res[1] = resourceGroupTagPrefixSQLDigest + // The SQL Digest is expected to be a hex string. Convert it back to bytes to save half of the memory. + res[2] = byte(len(sqlDigest) / 2) + _, err := hex.Decode(res[3:], []byte(sqlDigest)) + if err != nil { + logutil.BgLogger().Warn("failed to encode sql digest to resource group tag: invalid hex string", zap.String("sqlDigest", sqlDigest)) + return nil + } + + return res +} + +// DecodeResourceGroupTag decodes a resource group tag into various information contained in it. Currently it contains +// only the SQL Digest. +func DecodeResourceGroupTag(data []byte) (sqlDigest string, err error) { + if len(data) == 0 { + return "", nil + } + + encodingVersion := data[0] + if encodingVersion != 1 { + return "", errors.Errorf("unsupported resource group tag version %v", data[0]) + } + rem := data[1:] + + for len(rem) > 0 { + switch rem[0] { + case resourceGroupTagPrefixSQLDigest: + // There must be one more byte at rem[1] to represent the content's length, and the remaining bytes should + // not be shorter than the length specified by rem[1]. + if len(rem) < 2 || len(rem)-2 < int(rem[1]) { + return "", errors.Errorf("cannot parse resource group tag: field length mismatch, tag: %v", hex.EncodeToString(data)) + } + fieldLen := int(rem[1]) + sqlDigest = hex.EncodeToString(rem[2 : 2+fieldLen]) + rem = rem[2+fieldLen:] + default: + return "", errors.Errorf("resource group tag field not recognized, prefix: %v, tag: %v", rem[0], hex.EncodeToString(data)) + } + } + + return +} diff --git a/util/resourcegrouptag/resource_group_tag_test.go b/util/resourcegrouptag/resource_group_tag_test.go new file mode 100644 index 0000000000000..a979b92fce315 --- /dev/null +++ b/util/resourcegrouptag/resource_group_tag_test.go @@ -0,0 +1,111 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package resourcegrouptag + +import ( + "math/rand" + "testing" + + . "github.com/pingcap/check" +) + +type testUtilsSuite struct{} + +var _ = Suite(&testUtilsSuite{}) + +func TestT(t *testing.T) { + TestingT(t) +} + +func (s *testUtilsSuite) TestResourceGroupTagEncoding(c *C) { + sqlDigest := "" + tag := EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + decodedSQLDigest, err := DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(len(decodedSQLDigest), Equals, 0) + + sqlDigest = "aa" + tag = EncodeResourceGroupTag(sqlDigest) + // version(1) + prefix(1) + length(1) + content(2hex -> 1byte) + c.Assert(len(tag), Equals, 4) + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, sqlDigest) + + sqlDigest = genRandHex(64) + tag = EncodeResourceGroupTag(sqlDigest) + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, sqlDigest) + + sqlDigest = genRandHex(510) + tag = EncodeResourceGroupTag(sqlDigest) + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, sqlDigest) + + // The max supported length is 255 bytes (510 hex digits). + sqlDigest = genRandHex(512) + tag = EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + + // A hex string can't have odd length. + sqlDigest = genRandHex(15) + tag = EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + + // Non-hexadecimal character is invalid + sqlDigest = "aabbccddgg" + tag = EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + + // A tag should start with a supported version + tag = []byte("\x00") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) + + // The fields should have format like `[prefix, length, content...]`, otherwise decoding it should returns error. + tag = []byte("\x01\x01") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) + + tag = []byte("\x01\x01\x02") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) + + tag = []byte("\x01\x01\x02AB") + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, "4142") + + tag = []byte("\x01\x01\x00") + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(len(decodedSQLDigest), Equals, 0) + + // Unsupported field + tag = []byte("\x01\x99") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) +} + +func genRandHex(length int) string { + const chars = "0123456789abcdef" + res := make([]byte, length) + for i := 0; i < length; i++ { + res[i] = chars[rand.Intn(len(chars))] + } + return string(res) +} From b7c22aa31d062b77a96493519a6b50ca9b627971 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 21:29:39 +0800 Subject: [PATCH 19/42] kv: remove UnionStore interface (#24625) --- ddl/index.go | 2 +- ddl/index_change_test.go | 2 +- executor/admin.go | 2 +- executor/admin_test.go | 44 +++++++++++++-------------- executor/distsql_test.go | 2 +- executor/executor_test.go | 4 +-- kv/interface_mock_test.go | 4 --- kv/kv.go | 2 -- kv/union_store.go | 17 +---------- store/driver/txn/txn_driver.go | 4 --- store/driver/txn/unionstore_driver.go | 36 ---------------------- table/index.go | 6 ++-- table/tables/index.go | 27 ++++++++-------- table/tables/index_test.go | 12 ++++---- table/tables/tables.go | 4 +-- util/admin/admin.go | 2 +- util/mock/context.go | 8 ----- 17 files changed, 54 insertions(+), 124 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index b1b4303d7a0f1..fdba6c65008f6 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1343,7 +1343,7 @@ func (w *cleanUpIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t // we fetch records row by row, so records will belong to // index[0], index[1] ... index[n-1], index[0], index[1] ... // respectively. So indexes[i%n] is the index of idxRecords[i]. - err := w.indexes[i%n].Delete(w.sessCtx.GetSessionVars().StmtCtx, txn.GetUnionStore(), idxRecord.vals, idxRecord.handle) + err := w.indexes[i%n].Delete(w.sessCtx.GetSessionVars().StmtCtx, txn, idxRecord.vals, idxRecord.handle) if err != nil { return errors.Trace(err) } diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index 0a54b6b25e694..dfdfc7111c372 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -198,7 +198,7 @@ func checkIndexExists(ctx sessionctx.Context, tbl table.Table, indexValue interf if err != nil { return errors.Trace(err) } - doesExist, _, err := idx.Exist(ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), types.MakeDatums(indexValue), kv.IntHandle(handle)) + doesExist, _, err := idx.Exist(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(indexValue), kv.IntHandle(handle)) if err != nil { return errors.Trace(err) } diff --git a/executor/admin.go b/executor/admin.go index 7e15a24e667ca..be46e39a1d2a3 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -575,7 +575,7 @@ func (e *CleanupIndexExec) deleteDanglingIdx(txn kv.Transaction, values map[stri return errors.Trace(errors.Errorf("batch keys are inconsistent with handles")) } for _, handleIdxVals := range handleIdxValsGroup.([][]types.Datum) { - if err := e.index.Delete(e.ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), handleIdxVals, handle); err != nil { + if err := e.index.Delete(e.ctx.GetSessionVars().StmtCtx, txn, handleIdxVals, handle); err != nil { return err } e.removeCnt++ diff --git a/executor/admin_test.go b/executor/admin_test.go index 20095eb59a0ba..35e3d08345d63 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -135,7 +135,7 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { sc := s.ctx.GetSessionVars().StmtCtx txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(1), kv.IntHandle(1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(1), kv.IntHandle(1)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -158,7 +158,7 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(10), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(10), kv.IntHandle(10)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -172,15 +172,15 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(1), kv.IntHandle(1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(1), kv.IntHandle(1)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(2), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(2), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(3), kv.IntHandle(3)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(3), kv.IntHandle(3)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(10), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(10), kv.IntHandle(10)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(20), kv.IntHandle(20)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(20)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -236,7 +236,7 @@ func (s *testSuite5) TestClusteredIndexAdminRecoverIndex(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) cHandle := testutil.MustNewCommonHandle(c, "1", "3") - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(2), cHandle) + err = indexOpr.Delete(sc, txn, types.MakeDatums(2), cHandle) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -269,7 +269,7 @@ func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { sc := s.ctx.GetSessionVars().StmtCtx txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(idxValue), kv.IntHandle(idxValue)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(idxValue), kv.IntHandle(idxValue)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -345,13 +345,13 @@ func (s *testSuite5) TestAdminRecoverIndex1(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("1"), kv.IntHandle(1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("1"), kv.IntHandle(1)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("2"), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("2"), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("3"), kv.IntHandle(3)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("3"), kv.IntHandle(3)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("10"), kv.IntHandle(4)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("10"), kv.IntHandle(4)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -745,7 +745,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo) txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(i), kv.IntHandle(i)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i), kv.IntHandle(i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -784,7 +784,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(i+8), kv.IntHandle(i+8)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), kv.IntHandle(i+8)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -807,7 +807,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(i+8), kv.IntHandle(i)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), kv.IntHandle(i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -842,7 +842,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Index c2 is missing 11. txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(-10), kv.IntHandle(-1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(-10), kv.IntHandle(-1)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -873,7 +873,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Index c2 has two more values than table data: 10, 13, and these handles have correlative record. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(0), kv.IntHandle(0)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(0), kv.IntHandle(0)) c.Assert(err, IsNil) // Make sure the index value "19" is smaller "21". Then we scan to "19" before "21". _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(19), kv.IntHandle(10), nil) @@ -890,9 +890,9 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Two indices have the same handle. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(13), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(13), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(12), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(12), kv.IntHandle(2)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -906,7 +906,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(12), kv.IntHandle(2), nil) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(20), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(10)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -917,7 +917,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Recover records. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(19), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(19), kv.IntHandle(10)) c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(20), kv.IntHandle(10), nil) c.Assert(err, IsNil) diff --git a/executor/distsql_test.go b/executor/distsql_test.go index eca6e1d016b40..6f06fd550f0b4 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -241,7 +241,7 @@ func (s *testSuite3) TestInconsistentIndex(c *C) { for i := 0; i < 10; i++ { txn, err := s.store.Begin() c.Assert(err, IsNil) - err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), types.MakeDatums(i+10), kv.IntHandle(100+i)) + err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(i+10), kv.IntHandle(100+i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/executor/executor_test.go b/executor/executor_test.go index 3b168636606ed..7b433e71e60ac 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3852,9 +3852,9 @@ func (s *testSuite) TestCheckIndex(c *C) { // table data (handle, data): (1, 10), (2, 20), (4, 40) txn, err = s.store.Begin() c.Assert(err, IsNil) - err = idx.Delete(sc, txn.GetUnionStore(), types.MakeDatums(int64(30)), kv.IntHandle(3)) + err = idx.Delete(sc, txn, types.MakeDatums(int64(30)), kv.IntHandle(3)) c.Assert(err, IsNil) - err = idx.Delete(sc, txn.GetUnionStore(), types.MakeDatums(int64(20)), kv.IntHandle(2)) + err = idx.Delete(sc, txn, types.MakeDatums(int64(20)), kv.IntHandle(2)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index 2388c4f48b9f3..e1d41f1693088 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -106,10 +106,6 @@ func (t *mockTxn) GetSnapshot() Snapshot { return nil } -func (t *mockTxn) GetUnionStore() UnionStore { - return nil -} - func (t *mockTxn) NewStagingBuffer() MemBuffer { return nil } diff --git a/kv/kv.go b/kv/kv.go index 1fad79d641009..e5ab4eed6f812 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -174,8 +174,6 @@ type Transaction interface { GetMemBuffer() MemBuffer // GetSnapshot returns the Snapshot binding to this transaction. GetSnapshot() Snapshot - // GetUnionStore returns the UnionStore binding to this transaction. - GetUnionStore() UnionStore // SetVars sets variables to the transaction. SetVars(vars interface{}) // GetVars gets variables from the transaction. diff --git a/kv/union_store.go b/kv/union_store.go index 0e9a6768c5ebc..1d12e0a7fd37b 100644 --- a/kv/union_store.go +++ b/kv/union_store.go @@ -15,21 +15,6 @@ package kv // UnionStore is a store that wraps a snapshot for read and a MemBuffer for buffered write. // Also, it provides some transaction related utilities. +// TODO: Remove after upgrading BR. type UnionStore interface { - Retriever - - // HasPresumeKeyNotExists returns whether the key presumed key not exists error for the lazy check. - HasPresumeKeyNotExists(k Key) bool - // UnmarkPresumeKeyNotExists deletes the key presume key not exists error flag for the lazy check. - UnmarkPresumeKeyNotExists(k Key) - - // SetOption sets an option with a value, when val is nil, uses the default - // value of this option. - SetOption(opt int, val interface{}) - // DelOption deletes an option. - DelOption(opt int) - // GetOption gets an option. - GetOption(opt int) interface{} - // GetMemBuffer return the MemBuffer binding to this unionStore. - GetMemBuffer() MemBuffer } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 4d5ce77034312..72c1aac8c8e71 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -120,10 +120,6 @@ func (txn *tikvTxn) GetMemBuffer() kv.MemBuffer { return newMemBuffer(txn.KVTxn.GetMemBuffer()) } -func (txn *tikvTxn) GetUnionStore() kv.UnionStore { - return &tikvUnionStore{txn.KVTxn.GetUnionStore()} -} - func (txn *tikvTxn) SetOption(opt int, val interface{}) { switch opt { case kv.BinlogInfo: diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index 5a2f56bfe4233..b54bffa139700 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -111,42 +111,6 @@ func (m *memBuffer) SnapshotGetter() kv.Getter { return newKVGetter(m.MemDB.SnapshotGetter()) } -//tikvUnionStore implements kv.UnionStore -type tikvUnionStore struct { - *unionstore.KVUnionStore -} - -func (u *tikvUnionStore) GetMemBuffer() kv.MemBuffer { - return newMemBuffer(u.KVUnionStore.GetMemBuffer()) -} - -func (u *tikvUnionStore) Get(ctx context.Context, k kv.Key) ([]byte, error) { - data, err := u.KVUnionStore.Get(ctx, k) - return data, derr.ToTiDBErr(err) -} - -func (u *tikvUnionStore) HasPresumeKeyNotExists(k kv.Key) bool { - return u.KVUnionStore.HasPresumeKeyNotExists(k) -} - -func (u *tikvUnionStore) UnmarkPresumeKeyNotExists(k kv.Key) { - u.KVUnionStore.UnmarkPresumeKeyNotExists(k) -} - -func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { - it, err := u.KVUnionStore.Iter(k, upperBound) - return newKVIterator(it), derr.ToTiDBErr(err) -} - -// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. -// The returned iterator will iterate from greater key to smaller key. -// If k is nil, the returned iterator will be positioned at the last key. -// TODO: Add lower bound limit -func (u *tikvUnionStore) IterReverse(k kv.Key) (kv.Iterator, error) { - it, err := u.KVUnionStore.IterReverse(k) - return newKVIterator(it), derr.ToTiDBErr(err) -} - type tikvGetter struct { unionstore.Getter } diff --git a/table/index.go b/table/index.go index 5a9f32fbbfd3f..336efb7f574c2 100644 --- a/table/index.go +++ b/table/index.go @@ -66,11 +66,11 @@ type Index interface { // Create supports insert into statement. Create(ctx sessionctx.Context, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle, handleRestoreData []types.Datum, opts ...CreateIdxOptFunc) (kv.Handle, error) // Delete supports delete from statement. - Delete(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) error + Delete(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) error // Drop supports drop table, drop index statements. - Drop(us kv.UnionStore) error + Drop(txn kv.Transaction) error // Exist supports check index exists or not. - Exist(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) + Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) // GenIndexKey generates an index key. GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h kv.Handle, buf []byte) (key []byte, distinct bool, err error) // Seek supports where clause. diff --git a/table/tables/index.go b/table/tables/index.go index 8b4630d47f70d..ab6296390fbfd 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -184,9 +184,8 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue return nil, err } - us := txn.GetUnionStore() if !distinct || skipCheck || opt.Untouched { - err = us.GetMemBuffer().Set(key, idxVal) + err = txn.GetMemBuffer().Set(key, idxVal) return nil, err } @@ -202,18 +201,18 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue var value []byte if sctx.GetSessionVars().LazyCheckKeyNotExists() { - value, err = us.GetMemBuffer().Get(ctx, key) + value, err = txn.GetMemBuffer().Get(ctx, key) } else { - value, err = us.Get(ctx, key) + value, err = txn.Get(ctx, key) } if err != nil && !kv.IsErrNotFound(err) { return nil, err } if err != nil || len(value) == 0 { if sctx.GetSessionVars().LazyCheckKeyNotExists() && err != nil { - err = us.GetMemBuffer().SetWithFlags(key, idxVal, tikvstore.SetPresumeKeyNotExists) + err = txn.GetMemBuffer().SetWithFlags(key, idxVal, tikvstore.SetPresumeKeyNotExists) } else { - err = us.GetMemBuffer().Set(key, idxVal) + err = txn.GetMemBuffer().Set(key, idxVal) } return nil, err } @@ -226,22 +225,22 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue } // Delete removes the entry for handle h and indexedValues from KV index. -func (c *index) Delete(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) error { +func (c *index) Delete(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) error { key, distinct, err := c.GenIndexKey(sc, indexedValues, h, nil) if err != nil { return err } if distinct { - err = us.GetMemBuffer().DeleteWithFlags(key, tikvstore.SetNeedLocked) + err = txn.GetMemBuffer().DeleteWithFlags(key, tikvstore.SetNeedLocked) } else { - err = us.GetMemBuffer().Delete(key) + err = txn.GetMemBuffer().Delete(key) } return err } // Drop removes the KV index from store. -func (c *index) Drop(us kv.UnionStore) error { - it, err := us.Iter(c.prefix, c.prefix.PrefixNext()) +func (c *index) Drop(txn kv.Transaction) error { + it, err := txn.Iter(c.prefix, c.prefix.PrefixNext()) if err != nil { return err } @@ -252,7 +251,7 @@ func (c *index) Drop(us kv.UnionStore) error { if !it.Key().HasPrefix(c.prefix) { break } - err := us.GetMemBuffer().Delete(it.Key()) + err := txn.GetMemBuffer().Delete(it.Key()) if err != nil { return err } @@ -298,13 +297,13 @@ func (c *index) SeekFirst(r kv.Retriever) (iter table.IndexIterator, err error) return &indexIter{it: it, idx: c, prefix: c.prefix, colInfos: colInfos, tps: tps}, nil } -func (c *index) Exist(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) { +func (c *index) Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) { key, distinct, err := c.GenIndexKey(sc, indexedValues, h, nil) if err != nil { return false, nil, err } - value, err := us.Get(context.TODO(), key) + value, err := txn.Get(context.TODO(), key) if kv.IsErrNotFound(err) { return false, nil, nil } diff --git a/table/tables/index_test.go b/table/tables/index_test.go index 9345e86bab185..2c0a417746d42 100644 --- a/table/tables/index_test.go +++ b/table/tables/index_test.go @@ -104,15 +104,15 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(h.IntValue(), Equals, int64(1)) it.Close() sc := &stmtctx.StatementContext{TimeZone: time.Local} - exist, _, err := index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(100)) + exist, _, err := index.Exist(sc, txn, values, kv.IntHandle(100)) c.Assert(err, IsNil) c.Assert(exist, IsFalse) - exist, _, err = index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(1)) + exist, _, err = index.Exist(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) c.Assert(exist, IsTrue) - err = index.Delete(sc, txn.GetUnionStore(), values, kv.IntHandle(1)) + err = index.Delete(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) it, err = index.SeekFirst(txn) @@ -132,7 +132,7 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(err, IsNil) c.Assert(hit, IsFalse) - err = index.Drop(txn.GetUnionStore()) + err = index.Drop(txn) c.Assert(err, IsNil) it, hit, err = index.Seek(sc, txn, values) @@ -194,12 +194,12 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(h.IntValue(), Equals, int64(1)) it.Close() - exist, h, err = index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(1)) + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) c.Assert(h.IntValue(), Equals, int64(1)) c.Assert(exist, IsTrue) - exist, h, err = index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(2)) + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(2)) c.Assert(err, NotNil) c.Assert(h.IntValue(), Equals, int64(1)) c.Assert(exist, IsTrue) diff --git a/table/tables/tables.go b/table/tables/tables.go index a6a4180401d6a..37e6ab1103a89 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1152,7 +1152,7 @@ func (t *TableCommon) removeRowIndices(ctx sessionctx.Context, h kv.Handle, rec logutil.BgLogger().Info("remove row index failed", zap.Any("index", v.Meta()), zap.Uint64("txnStartTS", txn.StartTS()), zap.String("handle", h.String()), zap.Any("record", rec), zap.Error(err)) return err } - if err = v.Delete(ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), vals, h); err != nil { + if err = v.Delete(ctx.GetSessionVars().StmtCtx, txn, vals, h); err != nil { if v.Meta().State != model.StatePublic && kv.ErrNotExist.Equal(err) { // If the index is not in public state, we may have not created the index, // or already deleted the index, so skip ErrNotExist error. @@ -1167,7 +1167,7 @@ func (t *TableCommon) removeRowIndices(ctx sessionctx.Context, h kv.Handle, rec // removeRowIndex implements table.Table RemoveRowIndex interface. func (t *TableCommon) removeRowIndex(sc *stmtctx.StatementContext, h kv.Handle, vals []types.Datum, idx table.Index, txn kv.Transaction) error { - return idx.Delete(sc, txn.GetUnionStore(), vals, h) + return idx.Delete(sc, txn, vals, h) } // buildIndexForRow implements table.Table BuildIndexForRow interface. diff --git a/util/admin/admin.go b/util/admin/admin.go index 20217a53c1b6d..608040a89d6b0 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -389,7 +389,7 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table vals1[i] = colDefVal } } - isExist, h2, err := idx.Exist(sc, txn.GetUnionStore(), vals1, h1) + isExist, h2, err := idx.Exist(sc, txn, vals1, h1) if kv.ErrKeyExists.Equal(err) { record1 := &RecordData{Handle: h1, Values: vals1} record2 := &RecordData{Handle: h2, Values: vals1} diff --git a/util/mock/context.go b/util/mock/context.go index 4b329e0ff1f55..d6a5f1d913902 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -59,14 +59,6 @@ func (txn *wrapTxn) Valid() bool { return txn.Transaction != nil && txn.Transaction.Valid() } -// GetUnionStore implements GetUnionStore -func (txn *wrapTxn) GetUnionStore() kv.UnionStore { - if txn.Transaction == nil { - return nil - } - return txn.Transaction.GetUnionStore() -} - func (txn *wrapTxn) CacheTableInfo(id int64, info *model.TableInfo) { if txn.Transaction == nil { return From 18cbfaac15f8478902726e1e64971aa96b862462 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 14 May 2021 09:15:38 +0800 Subject: [PATCH 20/42] *: enable gosimple linter (#24617) --- Makefile | 1 + ddl/db_test.go | 8 +++---- ddl/ddl_api.go | 2 +- ddl/serial_test.go | 10 ++++---- executor/aggfuncs/func_percentile.go | 4 +--- executor/concurrent_map.go | 1 - executor/delete.go | 5 +--- executor/executor_test.go | 4 ++-- executor/explainfor_test.go | 24 +++++++++---------- executor/grant_test.go | 4 ++-- executor/infoschema_reader.go | 1 - executor/insert_test.go | 2 +- executor/join.go | 2 +- executor/memtable_reader.go | 4 ++-- executor/merge_join_test.go | 4 ++-- executor/parallel_apply_test.go | 1 - executor/partition_table_test.go | 2 +- executor/show.go | 8 +++---- expression/integration_test.go | 2 +- go.mod | 2 +- go.sum | 5 ++-- infoschema/metrics_schema.go | 14 +++-------- planner/cascades/implementation_rules.go | 10 ++------ planner/cascades/transformation_rules.go | 5 +--- planner/core/exhaust_physical_plans.go | 1 + planner/core/partition_pruner_test.go | 8 +++---- planner/core/pb_to_plan.go | 22 ++++++++--------- planner/core/physical_plans.go | 4 +--- planner/core/rule_partition_processor.go | 8 +++---- planner/core/stringer.go | 12 +++++----- planner/core/util.go | 2 +- plugin/conn_ip_example/conn_ip_example.go | 5 ---- server/http_handler_test.go | 6 ++--- server/server_test.go | 6 ++--- sessionctx/binloginfo/binloginfo.go | 20 +++++++--------- statistics/cmsketch.go | 9 +++---- statistics/feedback.go | 2 -- statistics/handle/handle.go | 1 - statistics/handle/handle_test.go | 2 +- statistics/handle/update_test.go | 4 ++-- store/copr/mpp.go | 8 ++----- .../unistore/cophandler/closure_exec.go | 4 +--- .../unistore/tikv/dbreader/db_reader.go | 1 - store/mockstore/unistore/tikv/detector.go | 5 +--- .../mockstore/unistore/tikv/detector_test.go | 3 +-- store/tikv/region_request.go | 5 +--- store/tikv/txn.go | 2 +- util/chunk/row_container.go | 2 -- util/execdetails/execdetails.go | 4 +--- util/expensivequery/memory_usage_alarm.go | 1 - util/profile/trackerRecorder.go | 10 ++++---- util/rowcodec/decoder.go | 5 +--- 52 files changed, 111 insertions(+), 176 deletions(-) diff --git a/Makefile b/Makefile index dd0e5f521acab..d6681083dc284 100644 --- a/Makefile +++ b/Makefile @@ -57,6 +57,7 @@ check-static: tools/bin/golangci-lint --enable=unused \ --enable=structcheck \ --enable=deadcode \ + --enable=gosimple \ $$($(PACKAGE_DIRECTORIES)) check-slow:tools/bin/gometalinter tools/bin/gosec diff --git a/ddl/db_test.go b/ddl/db_test.go index eddad6d0d635d..ecef0d0144215 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2280,10 +2280,8 @@ func (s *testDBSuite6) TestDropColumn(c *C) { testddlutil.ExecMultiSQLInGoroutine(c, s.store, "drop_col_db", []string{"insert into t2 set c1 = 1, c2 = 1, c3 = 1, c4 = 1"}, dmlDone) } for i := 0; i < num; i++ { - select { - case err := <-ddlDone: - c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) - } + err := <-ddlDone + c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) } // Test for drop partition table column. @@ -6575,7 +6573,7 @@ func (s *testSerialDBSuite) TestModifyColumnTypeWhenInterception(c *C) { count := defaultBatchSize * 4 // Add some rows. - dml := fmt.Sprintf("insert into t values") + dml := "insert into t values" for i := 1; i <= count; i++ { dml += fmt.Sprintf("(%d, %f)", i, 11.22) if i != count { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e6d77c9e674e9..528a6087638fc 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4040,7 +4040,7 @@ func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNe autoid.MaxAutoRandomBits, newRandBits, specNewColumn.Name.Name.O) return 0, ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) } - break // Increasing auto_random shard bits is allowed. + // increasing auto_random shard bits is allowed. case oldRandBits > newRandBits: if newRandBits == 0 { return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg) diff --git a/ddl/serial_test.go b/ddl/serial_test.go index e532bfc2352af..a8d245007f39c 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -503,15 +503,15 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) { // for failure table cases tk.MustExec("use ctwl_db") - failSQL := fmt.Sprintf("create table t1 like test_not_exist.t") + failSQL := "create table t1 like test_not_exist.t" tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable) - failSQL = fmt.Sprintf("create table t1 like test.t_not_exist") + failSQL = "create table t1 like test.t_not_exist" tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable) - failSQL = fmt.Sprintf("create table t1 (like test_not_exist.t)") + failSQL = "create table t1 (like test_not_exist.t)" tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable) - failSQL = fmt.Sprintf("create table test_not_exis.t1 like ctwl_db.t") + failSQL = "create table test_not_exis.t1 like ctwl_db.t" tk.MustGetErrCode(failSQL, mysql.ErrBadDB) - failSQL = fmt.Sprintf("create table t1 like ctwl_db.t") + failSQL = "create table t1 like ctwl_db.t" tk.MustGetErrCode(failSQL, mysql.ErrTableExists) // test failure for wrong object cases diff --git a/executor/aggfuncs/func_percentile.go b/executor/aggfuncs/func_percentile.go index 31855f791fb0f..ff13392276c61 100644 --- a/executor/aggfuncs/func_percentile.go +++ b/executor/aggfuncs/func_percentile.go @@ -53,9 +53,7 @@ func (e *basePercentile) AllocPartialResult() (pr PartialResult, memDelta int64) return } -func (e *basePercentile) ResetPartialResult(pr PartialResult) { - return -} +func (e *basePercentile) ResetPartialResult(pr PartialResult) {} func (e *basePercentile) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) (memDelta int64, err error) { return diff --git a/executor/concurrent_map.go b/executor/concurrent_map.go index 27f13a4f21dcb..3d6ef1082f605 100644 --- a/executor/concurrent_map.go +++ b/executor/concurrent_map.go @@ -56,7 +56,6 @@ func (m concurrentMap) Insert(key uint64, value *entry) { shard.items[key] = value } shard.Unlock() - return } // UpsertCb : Callback to return new element to be inserted into the map diff --git a/executor/delete.go b/executor/delete.go index 1fe9c26b8ac82..16f0e9c421b19 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -188,10 +188,7 @@ func (e *DeleteExec) removeRowsInTblRowMap(tblRowMap tableRowMapType) error { var err error rowMap.Range(func(h kv.Handle, val interface{}) bool { err = e.removeRow(e.ctx, e.tblID2Table[id], h, val.([]types.Datum)) - if err != nil { - return false - } - return true + return err == nil }) if err != nil { return err diff --git a/executor/executor_test.go b/executor/executor_test.go index 7b433e71e60ac..65af164174e6f 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5809,7 +5809,7 @@ func (s *testRecoverTable) TestRecoverTable(c *C) { // Test for recover one table multiple time. tk.MustExec("drop table t_recover") tk.MustExec("flashback table t_recover to t_recover_tmp") - _, err = tk.Exec(fmt.Sprintf("recover table t_recover")) + _, err = tk.Exec("recover table t_recover") c.Assert(infoschema.ErrTableExists.Equal(err), IsTrue) gcEnable, err := gcutil.CheckGCEnable(tk.Se) @@ -5876,7 +5876,7 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { tk.MustQuery("select a,_tidb_rowid from t_flashback2;").Check(testkit.Rows("1 1", "2 2", "3 3", "4 5001", "5 5002", "6 5003", "7 10001", "8 10002", "9 10003")) // Test for flashback one table multiple time. - _, err = tk.Exec(fmt.Sprintf("flashback table t_flashback to t_flashback4")) + _, err = tk.Exec("flashback table t_flashback to t_flashback4") c.Assert(infoschema.ErrTableExists.Equal(err), IsTrue) // Test for flashback truncated table to new table. diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index e29a7a3e24cee..46df545b1ff47 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -181,11 +181,11 @@ func (s *testSuite) TestExplainMemTablePredicate(c *C) { func (s *testSuite) TestExplainClusterTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where type in ('tikv', 'tidb')")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where type in ('tikv', 'tidb')").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG node_types:["tidb","tikv"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where instance='192.168.1.7:2379'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where instance='192.168.1.7:2379'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG instances:["192.168.1.7:2379"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where type='tidb' and instance='192.168.1.7:2379'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where type='tidb' and instance='192.168.1.7:2379'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG node_types:["tidb"], instances:["192.168.1.7:2379"]`)) } @@ -203,11 +203,11 @@ func (s *testSuite) TestInspectionResultTable(c *C) { func (s *testSuite) TestInspectionRuleTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES node_types:["inspection"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection' or type='summary'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection' or type='summary'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES node_types:["inspection","summary"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection' and type='summary'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection' and type='summary'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES skip_request: true`)) } @@ -355,12 +355,12 @@ func (s *testPrepareSerialSuite) TestExplainDotForQuery(c *C) { func (s *testSuite) TestExplainTableStorage(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"]"))) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_NAME = 'schemata'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]"))) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]"))) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"]")) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_NAME = 'schemata'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]")) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]")) } func (s *testSuite) TestInspectionSummaryTable(c *C) { diff --git a/executor/grant_test.go b/executor/grant_test.go index 13686494feddc..0e41c75e44353 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -68,7 +68,7 @@ func (s *testSuite3) TestGrantDBScope(c *C) { createUserSQL := `CREATE USER 'testDB'@'localhost' IDENTIFIED BY '123';` tk.MustExec(createUserSQL) // Make sure all the db privs for new user is empty. - sql := fmt.Sprintf("SELECT * FROM mysql.db WHERE User=\"testDB\" and host=\"localhost\"") + sql := `SELECT * FROM mysql.db WHERE User="testDB" and host="localhost"` tk.MustQuery(sql).Check(testkit.Rows()) // Grant each priv to the user. @@ -101,7 +101,7 @@ func (s *testSuite3) TestWithGrantOption(c *C) { createUserSQL := `CREATE USER 'testWithGrant'@'localhost' IDENTIFIED BY '123';` tk.MustExec(createUserSQL) // Make sure all the db privs for new user is empty. - sql := fmt.Sprintf("SELECT * FROM mysql.db WHERE User=\"testWithGrant\" and host=\"localhost\"") + sql := `SELECT * FROM mysql.db WHERE User="testWithGrant" and host="localhost"` tk.MustQuery(sql).Check(testkit.Rows()) // Grant select priv to the user, with grant option. diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index ae338bdd644d2..4d56cc55accac 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1856,7 +1856,6 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, s } } e.rows = rows - return } func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, tableName string) error { diff --git a/executor/insert_test.go b/executor/insert_test.go index ffcfdc214bdb9..351b337bd84ae 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -426,7 +426,7 @@ func (s *testSuite3) TestInsertDateTimeWithTimeZone(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (ts timestamp)") tk.MustExec("insert into t values ('2020-10-22T12:00:00Z'), ('2020-10-22T13:00:00Z'), ('2020-10-22T14:00:00Z')") - tk.MustQuery(fmt.Sprintf("select count(*) from t where ts > '2020-10-22T12:00:00Z'")).Check(testkit.Rows("2")) + tk.MustQuery("select count(*) from t where ts > '2020-10-22T12:00:00Z'").Check(testkit.Rows("2")) // test for datetime with fsp fspCases := []struct { diff --git a/executor/join.go b/executor/join.go index c1a8045aba9a3..1a3f62de47ac1 100644 --- a/executor/join.go +++ b/executor/join.go @@ -1078,7 +1078,7 @@ func (e *joinRuntimeStats) String() string { if e.cache.useCache { buf.WriteString(fmt.Sprintf(", cache:ON, cacheHitRatio:%.3f%%", e.cache.hitRatio*100)) } else { - buf.WriteString(fmt.Sprintf(", cache:OFF")) + buf.WriteString(", cache:OFF") } } if e.hasHashStat { diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index 24e2001131580..76ea478d91e71 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -230,9 +230,9 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String continue } var str string - switch val.(type) { + switch val := val.(type) { case string: // remove quotes - str = val.(string) + str = val default: tmp, err := json.Marshal(val) if err != nil { diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 94b9c33a5a9d7..cf5328fc93263 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -277,7 +277,6 @@ func (s *testSerialSuite1) TestShuffleMergeJoinInDisk(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), Greater, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.BytesConsumed(), Equals, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), Greater, int64(0)) - return } func (s *testSerialSuite1) TestMergeJoinInDisk(c *C) { defer config.RestoreFunc()() @@ -313,7 +312,6 @@ func (s *testSerialSuite1) TestMergeJoinInDisk(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), Greater, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.BytesConsumed(), Equals, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), Greater, int64(0)) - return } func (s *testSuite2) TestMergeJoin(c *C) { @@ -726,6 +724,7 @@ func (s *testSuite2) TestMergeJoinDifferentTypes(c *C) { } // TestVectorizedMergeJoin is used to test vectorized merge join with some corner cases. +//nolint:gosimple // generates false positive fmt.Sprintf warnings which keep aligned func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -841,6 +840,7 @@ func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) { } // TestVectorizedShuffleMergeJoin is used to test vectorized shuffle merge join with some corner cases. +//nolint:gosimple // generates false positive fmt.Sprintf warnings which keep aligned func (s *testSuiteJoin3) TestVectorizedShuffleMergeJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@session.tidb_merge_join_concurrency = 4;") diff --git a/executor/parallel_apply_test.go b/executor/parallel_apply_test.go index b849d3d961043..c0ecb19783273 100644 --- a/executor/parallel_apply_test.go +++ b/executor/parallel_apply_test.go @@ -43,7 +43,6 @@ func checkApplyPlan(c *C, tk *testkit.TestKit, sql string, parallel int) { } } c.Assert(containApply, IsTrue) - return } func (s *testSuite) TestParallelApply(c *C) { diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 5be39c3a04d54..ab5a19ca823e9 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -263,7 +263,7 @@ func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { } // test empty PointGet - queryHash := fmt.Sprintf("select a from thash where a=200") + queryHash := "select a from thash where a=200" c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used tk.MustQuery(queryHash).Check(testkit.Rows()) diff --git a/executor/show.go b/executor/show.go index c5df3bb290268..2bd9b786fcffb 100644 --- a/executor/show.go +++ b/executor/show.go @@ -862,10 +862,10 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } if pkCol != nil { - // If PKIsHanle, pk info is not in tb.Indices(). We should handle it here. + // If PKIsHandle, pk info is not in tb.Indices(). We should handle it here. buf.WriteString(",\n") fmt.Fprintf(buf, " PRIMARY KEY (%s)", stringutil.Escape(pkCol.Name.O, sqlMode)) - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] CLUSTERED */")) + buf.WriteString(" /*T![clustered_index] CLUSTERED */") } publicIndices := make([]*model.IndexInfo, 0, len(tableInfo.Indices)) @@ -906,9 +906,9 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } if idxInfo.Primary { if tableInfo.PKIsHandle || tableInfo.IsCommonHandle { - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] CLUSTERED */")) + buf.WriteString(" /*T![clustered_index] CLUSTERED */") } else { - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] NONCLUSTERED */")) + buf.WriteString(" /*T![clustered_index] NONCLUSTERED */") } } if i != len(publicIndices)-1 { diff --git a/expression/integration_test.go b/expression/integration_test.go index a3d983069cce9..8e0f70823f327 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -5833,7 +5833,7 @@ func (s *testIntegrationSuite) TestDecodetoChunkReuse(c *C) { tk.MustExec("create table chk (a int,b varchar(20))") for i := 0; i < 200; i++ { if i%5 == 0 { - tk.MustExec(fmt.Sprintf("insert chk values (NULL,NULL)")) + tk.MustExec("insert chk values (NULL,NULL)") continue } tk.MustExec(fmt.Sprintf("insert chk values (%d,'%s')", i, strconv.Itoa(i))) diff --git a/go.mod b/go.mod index c4b668a81dcf1..fe8e08ae42e47 100644 --- a/go.mod +++ b/go.mod @@ -82,7 +82,7 @@ require ( gopkg.in/natefinch/lumberjack.v2 v2.0.0 gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 // indirect - honnef.co/go/tools v0.1.3 // indirect + honnef.co/go/tools v0.1.4 // indirect modernc.org/mathutil v1.2.2 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 diff --git a/go.sum b/go.sum index 212917e414404..3ee71da011a54 100644 --- a/go.sum +++ b/go.sum @@ -500,7 +500,6 @@ github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQD github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= -github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A= @@ -933,8 +932,8 @@ honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.1.3 h1:qTakTkI6ni6LFD5sBwwsdSO+AQqbSIxOauHTTQKZ/7o= -honnef.co/go/tools v0.1.3/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= +honnef.co/go/tools v0.1.4 h1:SadWOkti5uVN1FAMgxn165+Mw00fuQKyk4Gyn/inxNQ= +honnef.co/go/tools v0.1.4/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= modernc.org/mathutil v1.2.2 h1:+yFk8hBprV+4c0U9GjFtL+dV3N8hOJ8JCituQcMShFY= modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= diff --git a/infoschema/metrics_schema.go b/infoschema/metrics_schema.go index 49a57e4ac9eeb..3b4654f90f7f2 100644 --- a/infoschema/metrics_schema.go +++ b/infoschema/metrics_schema.go @@ -100,17 +100,9 @@ func (def *MetricTableDef) genColumnInfos() []columnInfo { // GenPromQL generates the promQL. func (def *MetricTableDef) GenPromQL(sctx sessionctx.Context, labels map[string]set.StringSet, quantile float64) string { promQL := def.PromQL - if strings.Contains(promQL, promQLQuantileKey) { - promQL = strings.Replace(promQL, promQLQuantileKey, strconv.FormatFloat(quantile, 'f', -1, 64), -1) - } - - if strings.Contains(promQL, promQLLabelConditionKey) { - promQL = strings.Replace(promQL, promQLLabelConditionKey, def.genLabelCondition(labels), -1) - } - - if strings.Contains(promQL, promQRangeDurationKey) { - promQL = strings.Replace(promQL, promQRangeDurationKey, strconv.FormatInt(sctx.GetSessionVars().MetricSchemaRangeDuration, 10)+"s", -1) - } + promQL = strings.Replace(promQL, promQLQuantileKey, strconv.FormatFloat(quantile, 'f', -1, 64), -1) + promQL = strings.Replace(promQL, promQLLabelConditionKey, def.genLabelCondition(labels), -1) + promQL = strings.Replace(promQL, promQRangeDurationKey, strconv.FormatInt(sctx.GetSessionVars().MetricSchemaRangeDuration, 10)+"s", -1) return promQL } diff --git a/planner/cascades/implementation_rules.go b/planner/cascades/implementation_rules.go index d7a08b4fabaab..56c2141cfb213 100644 --- a/planner/cascades/implementation_rules.go +++ b/planner/cascades/implementation_rules.go @@ -95,10 +95,7 @@ type ImplTableDual struct { // Match implements ImplementationRule Match interface. func (r *ImplTableDual) Match(expr *memo.GroupExpr, prop *property.PhysicalProperty) (matched bool) { - if !prop.IsEmpty() { - return false - } - return true + return prop.IsEmpty() } // OnImplement implements ImplementationRule OnImplement interface. @@ -116,10 +113,7 @@ type ImplMemTableScan struct { // Match implements ImplementationRule Match interface. func (r *ImplMemTableScan) Match(expr *memo.GroupExpr, prop *property.PhysicalProperty) (matched bool) { - if !prop.IsEmpty() { - return false - } - return true + return prop.IsEmpty() } // OnImplement implements ImplementationRule OnImplement interface. diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 6d23e063f5877..9961509299a52 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -1503,10 +1503,7 @@ func NewRuleMergeAggregationProjection() Transformation { // Match implements Transformation interface. func (r *MergeAggregationProjection) Match(old *memo.ExprIter) bool { proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - if plannercore.ExprsHasSideEffects(proj.Exprs) { - return false - } - return true + return !plannercore.ExprsHasSideEffects(proj.Exprs) } // OnTransform implements Transformation interface. diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index cd227657a75d9..7c7493eedd5c5 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1416,6 +1416,7 @@ func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges []*ranger.Range, p func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool) (ranges []*ranger.Range, emptyRange bool, err error) { pointLength := matchedKeyCnt + len(eqAndInFuncs) + //nolint:gosimple // false positive unnecessary nil check if nextColRange != nil { for _, colRan := range nextColRange { // The range's exclude status is the same with last col's. diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index dd614239de55d..e151efdecce03 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -327,7 +327,7 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPrunerRandom(c *C) { tk1.MustExec(insert) // Test query without condition - query := fmt.Sprintf("select * from t1 order by id,a,b") + query := "select * from t1 order by id,a,b" tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) } @@ -467,9 +467,9 @@ func (s *testPartitionPruneSuit) TestRangePartitionPredicatePruner(c *C) { tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec(`create table t (a int(11) default null) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin partition by range(a) ( - partition p0 values less than (1), - partition p1 values less than (2), - partition p2 values less than (3), + partition p0 values less than (1), + partition p1 values less than (2), + partition p2 values less than (3), partition p_max values less than (maxvalue));`) var input []string diff --git a/planner/core/pb_to_plan.go b/planner/core/pb_to_plan.go index 3cfaf6708affe..a453596a0f288 100644 --- a/planner/core/pb_to_plan.go +++ b/planner/core/pb_to_plan.go @@ -255,15 +255,15 @@ func (b *PBPlanBuilder) pbToKill(e *tipb.Executor) (PhysicalPlan, error) { return &PhysicalSimpleWrapper{Inner: simple}, nil } -func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { - if p == nil { - return predicates, p +func (b *PBPlanBuilder) predicatePushDown(physicalPlan PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { + if physicalPlan == nil { + return predicates, physicalPlan } - switch p.(type) { + switch plan := physicalPlan.(type) { case *PhysicalMemTable: - memTable := p.(*PhysicalMemTable) + memTable := plan if memTable.Extractor == nil { - return predicates, p + return predicates, plan } names := make([]*types.FieldName, 0, len(memTable.Columns)) for _, col := range memTable.Columns { @@ -284,8 +284,8 @@ func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expressio predicates = memTable.Extractor.Extract(b.sctx, memTable.schema, names, predicates) return predicates, memTable case *PhysicalSelection: - selection := p.(*PhysicalSelection) - conditions, child := b.predicatePushDown(p.Children()[0], selection.Conditions) + selection := plan + conditions, child := b.predicatePushDown(plan.Children()[0], selection.Conditions) if len(conditions) > 0 { selection.Conditions = conditions selection.SetChildren(child) @@ -293,10 +293,10 @@ func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expressio } return predicates, child default: - if children := p.Children(); len(children) > 0 { + if children := plan.Children(); len(children) > 0 { _, child := b.predicatePushDown(children[0], nil) - p.SetChildren(child) + plan.SetChildren(child) } - return predicates, p + return predicates, plan } } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 43304971b4680..f4e6769ee8d70 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -911,9 +911,7 @@ func (p *PhysicalMergeJoin) Clone() (PhysicalPlan, error) { return nil, err } cloned.basePhysicalJoin = *base - for _, cf := range p.CompareFuncs { - cloned.CompareFuncs = append(cloned.CompareFuncs, cf) - } + cloned.CompareFuncs = append(cloned.CompareFuncs, p.CompareFuncs...) cloned.Desc = p.Desc return cloned, nil } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 57858679b7795..17f8c5b2a88b1 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -297,7 +297,7 @@ func (s *partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types. }) continue } - return nil, errors.New(fmt.Sprintf("information of column %v is not found", colExpr.String())) + return nil, fmt.Errorf("information of column %v is not found", colExpr.String()) } return names, nil } @@ -1345,9 +1345,9 @@ func appendWarnForUnknownPartitions(ctx sessionctx.Context, hintName string, unk if len(unknownPartitions) == 0 { return } - ctx.GetSessionVars().StmtCtx.AppendWarning( - errors.New(fmt.Sprintf("Unknown partitions (%s) in optimizer hint %s", - strings.Join(unknownPartitions, ","), hintName))) + + warning := fmt.Errorf("Unknown partitions (%s) in optimizer hint %s", strings.Join(unknownPartitions, ","), hintName) + ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } func (s *partitionProcessor) checkHintsApplicable(ds *DataSource, partitionSet set.StringSet) { diff --git a/planner/core/stringer.go b/planner/core/stringer.go index 4c63f6ff244b9..346b5b50e5742 100644 --- a/planner/core/stringer.go +++ b/planner/core/stringer.go @@ -270,31 +270,31 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { case *PhysicalShuffleReceiverStub: str = fmt.Sprintf("PartitionReceiverStub(%s)", x.ExplainInfo()) case *PointGetPlan: - str = fmt.Sprintf("PointGet(") + str = "PointGet(" if x.IndexInfo != nil { str += fmt.Sprintf("Index(%s.%s)%v)", x.TblInfo.Name.L, x.IndexInfo.Name.L, x.IndexValues) } else { str += fmt.Sprintf("Handle(%s.%s)%v)", x.TblInfo.Name.L, x.TblInfo.GetPkName().L, x.Handle) } case *BatchPointGetPlan: - str = fmt.Sprintf("BatchPointGet(") + str = "BatchPointGet(" if x.IndexInfo != nil { str += fmt.Sprintf("Index(%s.%s)%v)", x.TblInfo.Name.L, x.IndexInfo.Name.L, x.IndexValues) } else { str += fmt.Sprintf("Handle(%s.%s)%v)", x.TblInfo.Name.L, x.TblInfo.GetPkName().L, x.Handles) } case *PhysicalExchangeReceiver: - str = fmt.Sprintf("Recv(") + str = "Recv(" for _, task := range x.Tasks { str += fmt.Sprintf("%d, ", task.ID) } - str += fmt.Sprintf(")") + str += ")" case *PhysicalExchangeSender: - str = fmt.Sprintf("Send(") + str = "Send(" for _, task := range x.TargetTasks { str += fmt.Sprintf("%d, ", task.ID) } - str += fmt.Sprintf(")") + str += ")" default: str = fmt.Sprintf("%T", in) } diff --git a/planner/core/util.go b/planner/core/util.go index 753445f07ee42..19ce0a47673a2 100644 --- a/planner/core/util.go +++ b/planner/core/util.go @@ -291,7 +291,7 @@ func extractStringFromStringSet(set set.StringSet) string { l = append(l, fmt.Sprintf(`"%s"`, k)) } sort.Strings(l) - return fmt.Sprintf("%s", strings.Join(l, ",")) + return strings.Join(l, ",") } func tableHasDirtyContent(ctx sessionctx.Context, tableInfo *model.TableInfo) bool { diff --git a/plugin/conn_ip_example/conn_ip_example.go b/plugin/conn_ip_example/conn_ip_example.go index bae1b3ff37497..24d0bf04b0309 100644 --- a/plugin/conn_ip_example/conn_ip_example.go +++ b/plugin/conn_ip_example/conn_ip_example.go @@ -62,19 +62,14 @@ func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugi switch event { case plugin.Log: fmt.Println("---- event: Log") - break case plugin.Error: fmt.Println("---- event: Error") - break case plugin.Result: fmt.Println("---- event: Result") - break case plugin.Status: fmt.Println("---- event: Status") - break default: fmt.Println("---- event: unrecognized") - break } fmt.Printf("---- cmd: %s\n", cmd) } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index a7495b987f24a..7cdd7137138bd 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -578,7 +578,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { ts.prepareData(c) defer ts.stopServer(c) - resp, err := ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1")) + resp, err := ts.fetchStatus("/mvcc/key/tidb/test/1") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) var data mvccKV @@ -621,7 +621,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { c.Assert(err, IsNil) c.Assert(data2, DeepEquals, data) - resp, err = ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1?decode=true")) + resp, err = ts.fetchStatus("/mvcc/key/tidb/test/1?decode=true") c.Assert(err, IsNil) decoder = json.NewDecoder(resp.Body) var data3 map[string]interface{} @@ -667,7 +667,7 @@ func (ts *HTTPHandlerTestSuite) TestGetMVCCNotFound(c *C) { ts.startServer(c) ts.prepareData(c) defer ts.stopServer(c) - resp, err := ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1234")) + resp, err := ts.fetchStatus("/mvcc/key/tidb/test/1234") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) var data mvccKV diff --git a/server/server_test.go b/server/server_test.go index 20c0c2b508213..a342dab77d79a 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -499,14 +499,14 @@ func (cli *testServerClient) runTestLoadDataForSlowLog(c *C, server *Server) { } // Test for record slow log for load data statement. - rows := dbt.mustQuery(fmt.Sprintf("select plan from information_schema.slow_query where query like 'load data local infile %% into table t_slow;' order by time desc limit 1")) + rows := dbt.mustQuery("select plan from information_schema.slow_query where query like 'load data local infile % into table t_slow;' order by time desc limit 1") expectedPlan := ".*LoadData.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.* commit_txn.*" checkPlan(rows, expectedPlan) // Test for record statements_summary for load data statement. - rows = dbt.mustQuery(fmt.Sprintf("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %%' limit 1")) + rows = dbt.mustQuery("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %' limit 1") checkPlan(rows, expectedPlan) // Test log normal statement after executing load date. - rows = dbt.mustQuery(fmt.Sprintf("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1")) + rows = dbt.mustQuery("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1") expectedPlan = ".*Insert.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.*" checkPlan(rows, expectedPlan) }) diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 163c22e4a6fb7..5645b7496ebcc 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -154,17 +154,15 @@ func WaitBinlogRecover(timeout time.Duration) error { defer ticker.Stop() start := time.Now() for { - select { - case <-ticker.C: - if atomic.LoadInt32(&skippedCommitterCounter) == 0 { - logutil.BgLogger().Warn("[binloginfo] binlog recovered") - return nil - } - if time.Since(start) > timeout { - logutil.BgLogger().Warn("[binloginfo] waiting for binlog recovering timed out", - zap.Duration("duration", timeout)) - return errors.New("timeout") - } + <-ticker.C + if atomic.LoadInt32(&skippedCommitterCounter) == 0 { + logutil.BgLogger().Warn("[binloginfo] binlog recovered") + return nil + } + if time.Since(start) > timeout { + logutil.BgLogger().Warn("[binloginfo] waiting for binlog recovering timed out", + zap.Duration("duration", timeout)) + return errors.New("timeout") } } } diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index f682a1507a4bf..68791a7a0787f 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -818,14 +818,11 @@ func MergeTopN(topNs []*TopN, n uint32) (*TopN, []TopNMeta) { } func checkEmptyTopNs(topNs []*TopN) bool { - totCnt := uint64(0) + count := uint64(0) for _, topN := range topNs { - totCnt += topN.TotalCount() + count += topN.TotalCount() } - if totCnt == 0 { - return true - } - return false + return count == 0 } func getMergedTopNFromSortedSlice(sorted []TopNMeta, n uint32) (*TopN, []TopNMeta) { diff --git a/statistics/feedback.go b/statistics/feedback.go index 89aeab32152b3..5b69163e21b2b 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -107,7 +107,6 @@ func (m *QueryFeedbackMap) Append(q *QueryFeedback) { Tp: q.Tp, } m.append(k, []*QueryFeedback{q}) - return } // MaxQueryFeedbackCount is the max number of feedbacks that are cached in memory. @@ -136,7 +135,6 @@ func (m *QueryFeedbackMap) Merge(r *QueryFeedbackMap) { break } } - return } var ( diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index a8f51f8924e81..fe3644739c1b8 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -554,7 +554,6 @@ func (sc statsCache) initMemoryUsage() { sum += tb.MemoryUsage() } sc.memUsage = sum - return } // update updates the statistics table cache using copy on write. diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 2bf1d93ff4246..6d39e42b076e0 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2377,7 +2377,7 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { // test NDV checkNDV := func(rows, ndv int) { tk.MustExec("analyze table t") - rs := tk.MustQuery(fmt.Sprintf("select value from mysql.stats_fm_sketch")).Rows() + rs := tk.MustQuery("select value from mysql.stats_fm_sketch").Rows() c.Assert(len(rs), Equals, rows) for i := range rs { fm, err := statistics.DecodeFMSketch([]byte(rs[i][0].(string))) diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index b105738098f4b..a1de28e78eeef 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -577,8 +577,8 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeOnEmptyTable(c *C) { // test if it will be limited by the time range c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsFalse) - tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='00:00 +0000'")) - tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='23:59 +0000'")) + tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'") + tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'") c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsTrue) } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 9869fa501d430..2aaf4223ed8e5 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -144,14 +144,10 @@ func (m *mppIterator) run(ctx context.Context) { break } m.mu.Lock() - switch task.State { - case kv.MppTaskReady: + if task.State == kv.MppTaskReady { task.State = kv.MppTaskRunning - m.mu.Unlock() - default: - m.mu.Unlock() - break } + m.mu.Unlock() m.wg.Add(1) bo := backoff.NewBackoffer(ctx, copNextMaxBackoff) go m.handleDispatchReq(ctx, bo, task) diff --git a/store/mockstore/unistore/cophandler/closure_exec.go b/store/mockstore/unistore/cophandler/closure_exec.go index 152362c13d5e3..696a1497b068e 100644 --- a/store/mockstore/unistore/cophandler/closure_exec.go +++ b/store/mockstore/unistore/cophandler/closure_exec.go @@ -124,9 +124,7 @@ func buildClosureExecutorFromExecutorList(dagCtx *dagContext, executors []*tipb. outputFieldTypes = append(outputFieldTypes, originalOutputFieldTypes[idx]) } } else { - for _, tp := range originalOutputFieldTypes { - outputFieldTypes = append(outputFieldTypes, tp) - } + outputFieldTypes = append(outputFieldTypes, originalOutputFieldTypes...) } if len(executors) == 1 { ce.resultFieldType = outputFieldTypes diff --git a/store/mockstore/unistore/tikv/dbreader/db_reader.go b/store/mockstore/unistore/tikv/dbreader/db_reader.go index 158f295cc1bdf..6e3909ada740f 100644 --- a/store/mockstore/unistore/tikv/dbreader/db_reader.go +++ b/store/mockstore/unistore/tikv/dbreader/db_reader.go @@ -165,7 +165,6 @@ func (r *DBReader) BatchGet(keys [][]byte, startTS uint64, f BatchGetFunc) { } f(key, val, err) } - return } // ErrScanBreak is returned by ScanFunc to break the scan loop. diff --git a/store/mockstore/unistore/tikv/detector.go b/store/mockstore/unistore/tikv/detector.go index a27adb3f35b6c..6a56a0d9fd73c 100644 --- a/store/mockstore/unistore/tikv/detector.go +++ b/store/mockstore/unistore/tikv/detector.go @@ -64,10 +64,7 @@ type diagnosticContext struct { } func (p *txnKeyHashPair) isExpired(ttl time.Duration, nowTime time.Time) bool { - if p.registerTime.Add(ttl).Before(nowTime) { - return true - } - return false + return p.registerTime.Add(ttl).Before(nowTime) } // NewDetector creates a new Detector. diff --git a/store/mockstore/unistore/tikv/detector_test.go b/store/mockstore/unistore/tikv/detector_test.go index b0d3a074ff840..189a8b00b8217 100644 --- a/store/mockstore/unistore/tikv/detector_test.go +++ b/store/mockstore/unistore/tikv/detector_test.go @@ -26,7 +26,6 @@ package tikv import ( - "fmt" "testing" "time" @@ -68,7 +67,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { c.Assert(detector.totalSize, Equals, uint64(2)) err = detector.Detect(3, 1, 300, makeDiagCtx("k3", "tag3")) c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, fmt.Sprintf("deadlock")) + c.Assert(err.Error(), Equals, "deadlock") c.Assert(len(err.WaitChain), Equals, 3) // The order of entries in the wait chain is specific: each item is waiting for the next one. checkWaitChainEntry(err.WaitChain[0], 1, 2, "k1", "tag1") diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index ec5b92f81c5ad..b8b61aac05fc8 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -477,10 +477,7 @@ func (s *RegionRequestSender) sendReqToRegion(bo *Backoffer, rpcCtx *RPCContext, if val.(bool) { ctx1, cancel := context.WithCancel(context.Background()) cancel() - select { - case <-ctx1.Done(): - } - + <-ctx1.Done() ctx = ctx1 err = ctx.Err() resp = nil diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 20bf0491ed294..e444f5adda7f6 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -642,7 +642,7 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput // PointGet and BatchPointGet will return value in pessimistic lock response, the value may not exist. // For other lock modes, the locked key values always exist. if lockCtx.ReturnValues { - val, _ := lockCtx.Values[string(key)] + val := lockCtx.Values[string(key)] if len(val.Value) == 0 { valExists = tikv.SetKeyLockedValueNotExists } diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index f67cbb36b76da..0ef0b573e1bb2 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -90,7 +90,6 @@ func (c *RowContainer) SpillToDisk() { } } c.m.records.Clear() - return } // Reset resets RowContainer. @@ -447,7 +446,6 @@ func (c *SortedRowContainer) Sort() { func (c *SortedRowContainer) sortAndSpillToDisk() { c.Sort() c.RowContainer.SpillToDisk() - return } // Add appends a chunk into the SortedRowContainer. diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index ca045352dbd33..676f0a241d489 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -703,9 +703,7 @@ func (e *RuntimeStatsWithConcurrencyInfo) SetConcurrencyInfo(infos ...*Concurren e.Lock() defer e.Unlock() e.concurrency = e.concurrency[:0] - for _, info := range infos { - e.concurrency = append(e.concurrency, info) - } + e.concurrency = append(e.concurrency, infos...) } // Clone implements the RuntimeStats interface. diff --git a/util/expensivequery/memory_usage_alarm.go b/util/expensivequery/memory_usage_alarm.go index fbe9b6c5ff438..03d53c12342c9 100644 --- a/util/expensivequery/memory_usage_alarm.go +++ b/util/expensivequery/memory_usage_alarm.go @@ -83,7 +83,6 @@ func (record *memoryUsageAlarm) initMemoryUsageAlarmRecord() { } } record.initialized = true - return } // If Performance.ServerMemoryQuota is set, use `ServerMemoryQuota * MemoryUsageAlarmRatio` to check oom risk. diff --git a/util/profile/trackerRecorder.go b/util/profile/trackerRecorder.go index c5e5390a51bff..7c449ffa9113c 100644 --- a/util/profile/trackerRecorder.go +++ b/util/profile/trackerRecorder.go @@ -29,12 +29,10 @@ func HeapProfileForGlobalMemTracker(d time.Duration) { t := time.NewTicker(d) defer t.Stop() for { - select { - case <-t.C: - err := heapProfileForGlobalMemTracker() - if err != nil { - log.Warn("profile memory into tracker failed", zap.Error(err)) - } + <-t.C + err := heapProfileForGlobalMemTracker() + if err != nil { + log.Warn("profile memory into tracker failed", zap.Error(err)) } } } diff --git a/util/rowcodec/decoder.go b/util/rowcodec/decoder.go index 69a78d1de7d43..0efd50ecaf27c 100644 --- a/util/rowcodec/decoder.go +++ b/util/rowcodec/decoder.go @@ -260,10 +260,7 @@ func (decoder *ChunkDecoder) tryAppendHandleColumn(colIdx int, col *ColInfo, han } coder := codec.NewDecoder(chk, decoder.loc) _, err := coder.DecodeOne(handle.EncodedCol(i), colIdx, col.Ft) - if err != nil { - return false - } - return true + return err == nil } } return false From e628bad698b58f8b1268c7fdf5305f25d16b1c84 Mon Sep 17 00:00:00 2001 From: cfzjywxk Date: Fri, 14 May 2021 10:31:38 +0800 Subject: [PATCH 21/42] txn: avoid the gc resolving pessimistic locks of ongoing transactions (#24601) --- store/gcworker/gc_worker.go | 13 ++++++-- store/gcworker/gc_worker_test.go | 51 +++++++++++++++++++++++++++++++- 2 files changed, 61 insertions(+), 3 deletions(-) diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index e0aa993558b6c..b408f279be98a 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -409,12 +409,21 @@ func (w *GCWorker) calcSafePointByMinStartTS(ctx context.Context, safePoint uint return safePoint } - if globalMinStartTS < safePoint { + // If the lock.ts <= max_ts(safePoint), it will be collected and resolved by the gc worker, + // the locks of ongoing pessimistic transactions could be resolved by the gc worker and then + // the transaction is aborted, decrement the value by 1 to avoid this. + globalMinStartAllowedTS := globalMinStartTS + if globalMinStartTS > 0 { + globalMinStartAllowedTS = globalMinStartTS - 1 + } + + if globalMinStartAllowedTS < safePoint { logutil.Logger(ctx).Info("[gc worker] gc safepoint blocked by a running session", zap.String("uuid", w.uuid), zap.Uint64("globalMinStartTS", globalMinStartTS), + zap.Uint64("globalMinStartAllowedTS", globalMinStartAllowedTS), zap.Uint64("safePoint", safePoint)) - safePoint = globalMinStartTS + safePoint = globalMinStartAllowedTS } return safePoint } diff --git a/store/gcworker/gc_worker_test.go b/store/gcworker/gc_worker_test.go index bc09651e0d379..39abe369f82fb 100644 --- a/store/gcworker/gc_worker_test.go +++ b/store/gcworker/gc_worker_test.go @@ -262,7 +262,7 @@ func (s *testGCWorkerSuite) TestMinStartTS(c *C) { strconv.FormatUint(now-oracle.EncodeTSO(20000), 10)) c.Assert(err, IsNil) sp = s.gcWorker.calcSafePointByMinStartTS(ctx, now-oracle.EncodeTSO(10000)) - c.Assert(sp, Equals, now-oracle.EncodeTSO(20000)) + c.Assert(sp, Equals, now-oracle.EncodeTSO(20000)-1) } func (s *testGCWorkerSuite) TestPrepareGC(c *C) { @@ -1589,3 +1589,52 @@ func (s *testGCWorkerSuite) TestGCPlacementRules(c *C) { c.Assert(pid, Equals, int64(1)) c.Assert(err, IsNil) } + +func (s *testGCWorkerSuite) TestGCWithPendingTxn(c *C) { + ctx := context.Background() + gcSafePointCacheInterval = 0 + err := s.gcWorker.saveValueToSysTable(gcEnableKey, booleanFalse) + c.Assert(err, IsNil) + + k1 := []byte("tk1") + v1 := []byte("v1") + txn, err := s.store.Begin() + c.Assert(err, IsNil) + txn.SetOption(kv.Pessimistic, true) + lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()} + + // Lock the key. + err = txn.Set(k1, v1) + c.Assert(err, IsNil) + err = txn.LockKeys(ctx, lockCtx, k1) + c.Assert(err, IsNil) + + // Prepare to run gc with txn's startTS as the safepoint ts. + spkv := s.tikvStore.GetSafePointKV() + err = spkv.Put(fmt.Sprintf("%s/%s", infosync.ServerMinStartTSPath, "a"), strconv.FormatUint(txn.StartTS(), 10)) + c.Assert(err, IsNil) + s.mustSetTiDBServiceSafePoint(c, txn.StartTS(), txn.StartTS()) + veryLong := gcDefaultLifeTime * 100 + err = s.gcWorker.saveTime(gcLastRunTimeKey, oracle.GetTimeFromTS(s.mustAllocTs(c)).Add(-veryLong)) + c.Assert(err, IsNil) + s.gcWorker.lastFinish = time.Now().Add(-veryLong) + s.oracle.AddOffset(time.Minute * 10) + err = s.gcWorker.saveValueToSysTable(gcEnableKey, booleanTrue) + c.Assert(err, IsNil) + + // Trigger the tick let the gc job start. + err = s.gcWorker.leaderTick(ctx) + c.Assert(err, IsNil) + // Wait for GC finish + select { + case err = <-s.gcWorker.done: + s.gcWorker.gcIsRunning = false + break + case <-time.After(time.Second * 10): + err = errors.New("receive from s.gcWorker.done timeout") + } + c.Assert(err, IsNil) + + err = txn.Commit(ctx) + c.Assert(err, IsNil) +} From d169a57ba5a4e677196d12b7fef797561315b29b Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Fri, 14 May 2021 11:55:38 +0800 Subject: [PATCH 22/42] util: fix wrong enum building for index range (#24632) --- expression/integration_test.go | 10 ++++++++++ util/ranger/points.go | 2 +- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 8e0f70823f327..095e21119ab30 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9344,4 +9344,14 @@ func (s *testIntegrationSuite) TestEnumIndex(c *C) { testkit.Rows( "OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A -30 ", "ZOHBSCRMZPOI`IVTSEZAIDAF7DS@1TT20AP9 -30 ")) + + // issue 24576 + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(col1 enum('a','b','c'), col2 enum('a','b','c'), col3 int, index idx(col1,col2));") + tk.MustExec("insert into t values(1,1,1),(2,2,2),(3,3,3);") + tk.MustQuery("select /*+ use_index(t,idx) */ col3 from t where col2 between 'b' and 'b' and col1 is not null;").Check( + testkit.Rows("2")) + tk.MustQuery("select /*+ use_index(t,idx) */ col3 from t where col2 = 'b' and col1 is not null;").Check( + testkit.Rows("2")) } diff --git a/util/ranger/points.go b/util/ranger/points.go index 9c33ccef7feb3..46a4283dd3222 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -459,7 +459,7 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val tmpEnum := types.Enum{} for i := range ft.Elems { tmpEnum.Name = ft.Elems[i] - tmpEnum.Value = uint64(i) + tmpEnum.Value = uint64(i) + 1 d := types.NewMysqlEnumDatum(tmpEnum) if v, err := d.CompareDatum(sc, &val); err == nil { switch op { From 80a557eb4956da36f21ba1c81ad12595dcf716b1 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Thu, 13 May 2021 22:43:38 -0600 Subject: [PATCH 23/42] sessionctx: change innodb large prefix default (#24555) --- expression/integration_test.go | 6 ++++++ sessionctx/variable/noop.go | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 095e21119ab30..b1bded2a17719 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9086,9 +9086,15 @@ func (s *testIntegrationSuite) TestEnumPushDown(c *C) { func (s *testIntegrationSuite) TestJiraSetInnoDBDefaultRowFormat(c *C) { // For issue #23541 // JIRA needs to be able to set this to be happy. + // See: https://nova.moe/run-jira-on-tidb/ tk := testkit.NewTestKit(c, s.store) tk.MustExec("set global innodb_default_row_format = dynamic") tk.MustExec("set global innodb_default_row_format = 'dynamic'") + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_default_row_format'").Check(testkit.Rows("innodb_default_row_format dynamic")) + tk.MustQuery("SHOW VARIABLES LIKE 'character_set_server'").Check(testkit.Rows("character_set_server utf8mb4")) + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_file_format'").Check(testkit.Rows("innodb_file_format Barracuda")) + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_large_prefix'").Check(testkit.Rows("innodb_large_prefix ON")) + } func (s *testIntegrationSerialSuite) TestCollationForBinaryLiteral(c *C) { diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index c510d9c73ce3a..1ad37b512d807 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -312,7 +312,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "datetime_format", Value: "%Y-%m-%d %H:%i:%s"}, {Scope: ScopeGlobal, Name: "log_syslog", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "transaction_alloc_block_size", Value: "8192"}, - {Scope: ScopeGlobal, Name: "innodb_large_prefix", Type: TypeBool, Value: Off}, + {Scope: ScopeGlobal, Name: "innodb_large_prefix", Type: TypeBool, Value: On}, {Scope: ScopeNone, Name: "performance_schema_max_cond_classes", Value: "80"}, {Scope: ScopeGlobal, Name: "innodb_io_capacity", Value: "200"}, {Scope: ScopeGlobal, Name: "max_binlog_cache_size", Value: "18446744073709547520"}, From ea7f0ca1ba80674a1bec449ca2c3d0f79d48b735 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Fri, 14 May 2021 15:49:38 +0800 Subject: [PATCH 24/42] store: fix data race about KVStore.tikvClient (#24655) --- store/tikv/commit.go | 2 +- store/tikv/kv.go | 25 ++++++++++++++++--------- store/tikv/prewrite.go | 2 +- store/tikv/scan.go | 2 +- store/tikv/split_region.go | 2 +- 5 files changed, 20 insertions(+), 13 deletions(-) diff --git a/store/tikv/commit.go b/store/tikv/commit.go index 449081860c029..10c60d9f6d4bd 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -48,7 +48,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch CommitVersion: c.commitTS, }, pb.Context{Priority: c.priority, SyncLog: c.syncLog}) - sender := NewRegionRequestSender(c.store.regionCache, c.store.client) + sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) // If we fail to receive response for the request that commits primary key, it will be undetermined whether this diff --git a/store/tikv/kv.go b/store/tikv/kv.go index f61db4168ef7d..bbf8517a42a8c 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -66,10 +66,13 @@ var oracleUpdateInterval = 2000 // KVStore contains methods to interact with a TiKV cluster. type KVStore struct { - clusterID uint64 - uuid string - oracle oracle.Oracle - client Client + clusterID uint64 + uuid string + oracle oracle.Oracle + clientMu struct { + sync.RWMutex + client Client + } pdClient pd.Client regionCache *RegionCache lockResolver *LockResolver @@ -133,7 +136,6 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client clusterID: pdClient.GetClusterID(context.TODO()), uuid: uuid, oracle: o, - client: reqCollapse{client}, pdClient: pdClient, regionCache: NewRegionCache(pdClient), kv: spkv, @@ -142,6 +144,7 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client closed: make(chan struct{}), replicaReadSeed: rand.Uint32(), } + store.clientMu.client = reqCollapse{client} store.lockResolver = newLockResolver(store) go store.runSafePointChecker() @@ -205,7 +208,7 @@ func (s *KVStore) Close() error { s.pdClient.Close() close(s.closed) - if err := s.client.Close(); err != nil { + if err := s.GetTiKVClient().Close(); err != nil { return errors.Trace(err) } @@ -312,7 +315,7 @@ func (s *KVStore) SupportDeleteRange() (supported bool) { // SendReq sends a request to region. func (s *KVStore) SendReq(bo *Backoffer, req *tikvrpc.Request, regionID RegionVerID, timeout time.Duration) (*tikvrpc.Response, error) { - sender := NewRegionRequestSender(s.regionCache, s.client) + sender := NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) return sender.SendReq(bo, req, regionID, timeout) } @@ -343,12 +346,16 @@ func (s *KVStore) SetOracle(oracle oracle.Oracle) { // SetTiKVClient resets the client instance. func (s *KVStore) SetTiKVClient(client Client) { - s.client = client + s.clientMu.Lock() + defer s.clientMu.Unlock() + s.clientMu.client = client } // GetTiKVClient gets the client instance. func (s *KVStore) GetTiKVClient() (client Client) { - return s.client + s.clientMu.RLock() + defer s.clientMu.RUnlock() + return s.clientMu.client } func (s *KVStore) getSafeTS(storeID uint64) uint64 { diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index 7097ba5dbcd3e..ffb47e1fb46fa 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -157,7 +157,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff req := c.buildPrewriteRequest(batch, txnSize) for { - sender := NewRegionRequestSender(c.store.regionCache, c.store.client) + sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) // If we fail to receive response for async commit prewrite, it will be undetermined whether this diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 6c43b7bdee7cd..035291a783aec 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -164,7 +164,7 @@ func (s *Scanner) getData(bo *Backoffer) error { zap.String("nextEndKey", kv.StrKey(s.nextEndKey)), zap.Bool("reverse", s.reverse), zap.Uint64("txnStartTS", s.startTS())) - sender := NewRegionRequestSender(s.snapshot.store.regionCache, s.snapshot.store.client) + sender := NewRegionRequestSender(s.snapshot.store.regionCache, s.snapshot.store.GetTiKVClient()) var reqEndKey, reqStartKey []byte var loc *KeyLocation var err error diff --git a/store/tikv/split_region.go b/store/tikv/split_region.go index 38ce24917d1cf..c33a89efc19be 100644 --- a/store/tikv/split_region.go +++ b/store/tikv/split_region.go @@ -123,7 +123,7 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool Priority: kvrpcpb.CommandPri_Normal, }) - sender := NewRegionRequestSender(s.regionCache, s.client) + sender := NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.regionID, ReadTimeoutShort) batchResp := singleBatchResp{resp: resp} From d9f28c7f765662a575e8a2bf6f139229cce1d360 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Fri, 14 May 2021 02:07:39 -0600 Subject: [PATCH 25/42] executor, privileges: Add dynamic privileges to SHOW PRIVILEGES (#24646) --- executor/executor_test.go | 9 ++++++++- executor/show.go | 4 ++++ privilege/privileges/privileges.go | 11 +++++++++++ privilege/privileges/privileges_test.go | 12 ++++++++++++ 4 files changed, 35 insertions(+), 1 deletion(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 65af164174e6f..d67bb6b48b8f4 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -350,7 +350,14 @@ func (s *testSuiteP1) TestShow(c *C) { "Trigger Tables To use triggers", "Create tablespace Server Admin To create/alter/drop tablespaces", "Update Tables To update existing rows", - "Usage Server Admin No privileges - allow connect only")) + "Usage Server Admin No privileges - allow connect only", + "BACKUP_ADMIN Server Admin ", + "SYSTEM_VARIABLES_ADMIN Server Admin ", + "ROLE_ADMIN Server Admin ", + "CONNECTION_ADMIN Server Admin ", + "RESTRICTED_TABLES_ADMIN Server Admin ", + "RESTRICTED_STATUS_ADMIN Server Admin ", + )) c.Assert(len(tk.MustQuery("show table status").Rows()), Equals, 1) } diff --git a/executor/show.go b/executor/show.go index 2bd9b786fcffb..ab08e5ba4cbf5 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1412,6 +1412,10 @@ func (e *ShowExec) fetchShowPrivileges() error { e.appendRow([]interface{}{"Create tablespace", "Server Admin", "To create/alter/drop tablespaces"}) e.appendRow([]interface{}{"Update", "Tables", "To update existing rows"}) e.appendRow([]interface{}{"Usage", "Server Admin", "No privileges - allow connect only"}) + + for _, priv := range privileges.GetDynamicPrivileges() { + e.appendRow([]interface{}{priv, "Server Admin", ""}) + } return nil } diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index c5ec2f8394385..6ac58e04e44e4 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -535,3 +535,14 @@ func RegisterDynamicPrivilege(privNameInUpper string) error { dynamicPrivs = append(dynamicPrivs, privNameInUpper) return nil } + +// GetDynamicPrivileges returns the list of registered DYNAMIC privileges +// for use in meta data commands (i.e. SHOW PRIVILEGES) +func GetDynamicPrivileges() []string { + dynamicPrivLock.Lock() + defer dynamicPrivLock.Unlock() + + privCopy := make([]string, len(dynamicPrivs)) + copy(privCopy, dynamicPrivs) + return privCopy +} diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 3038aad397076..c7a825a4d894d 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1427,3 +1427,15 @@ func (s *testPrivilegeSuite) TestViewDefiner(c *C) { tk.MustExec("select * from test_view") tk.MustExec("select * from test_view2") } + +func (s *testPrivilegeSuite) TestDynamicPrivsRegistration(c *C) { + se := newSession(c, s.store, s.dbName) + pm := privilege.GetPrivilegeManager(se) + + count := len(privileges.GetDynamicPrivileges()) + + c.Assert(pm.IsDynamicPrivilege("ACDC_ADMIN"), IsFalse) + privileges.RegisterDynamicPrivilege("ACDC_ADMIN") + c.Assert(pm.IsDynamicPrivilege("ACDC_ADMIN"), IsTrue) + c.Assert(len(privileges.GetDynamicPrivileges()), Equals, count+1) +} From 031a9fa18b909c1501c99d6a8a98448beb94a962 Mon Sep 17 00:00:00 2001 From: xhe Date: Fri, 14 May 2021 17:05:38 +0800 Subject: [PATCH 26/42] ddl: refactor rule [4/6] (#24007) --- ddl/ddl_api.go | 14 +-- ddl/placement/errors.go | 6 + ddl/placement/rule.go | 132 +++++++++++++++++++++ ddl/placement/rule_test.go | 206 +++++++++++++++++++++++++++++++++ ddl/placement/types.go | 36 ------ ddl/placement/types_test.go | 12 -- ddl/placement/utils.go | 4 +- ddl/placement/utils_test.go | 14 +-- ddl/placement_rule_test.go | 36 +++--- ddl/placement_sql_test.go | 6 +- executor/infoschema_reader.go | 2 +- expression/integration_test.go | 2 +- infoschema/tables_test.go | 2 +- session/session_test.go | 2 +- 14 files changed, 385 insertions(+), 89 deletions(-) create mode 100644 ddl/placement/rule.go create mode 100644 ddl/placement/rule_test.go diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 528a6087638fc..a3f8bb7f9c622 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5895,8 +5895,8 @@ func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]* } rules = append(rules, &placement.Rule{ - Count: int(replicas), - LabelConstraints: labelConstraints, + Count: int(replicas), + Constraints: labelConstraints, }) return rules, nil @@ -5925,8 +5925,8 @@ func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]* } rules = append(rules, &placement.Rule{ - Count: cnt, - LabelConstraints: labelConstraints, + Count: cnt, + Constraints: labelConstraints, }) } @@ -6051,14 +6051,14 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, newRules := bundle.Rules[:0] for i, rule := range bundle.Rules { // merge all empty constraints - if len(rule.LabelConstraints) == 0 { + if len(rule.Constraints) == 0 { extraCnt[rule.Role] += rule.Count continue } // refer to tidb#22065. // add -engine=tiflash to every rule to avoid schedules to tiflash instances. // placement rules in SQL is not compatible with `set tiflash replica` yet - if err := rule.LabelConstraints.Add(placement.Constraint{ + if err := rule.Constraints.Add(placement.Constraint{ Op: placement.NotIn, Key: placement.EngineLabelKey, Values: []string{placement.EngineLabelTiFlash}, @@ -6083,7 +6083,7 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, Count: cnt, StartKeyHex: startKey, EndKeyHex: endKey, - LabelConstraints: []placement.Constraint{{ + Constraints: []placement.Constraint{{ Op: placement.NotIn, Key: placement.EngineLabelKey, Values: []string{placement.EngineLabelTiFlash}, diff --git a/ddl/placement/errors.go b/ddl/placement/errors.go index 19797022a609c..95fce4591c961 100644 --- a/ddl/placement/errors.go +++ b/ddl/placement/errors.go @@ -24,4 +24,10 @@ var ( ErrUnsupportedConstraint = errors.New("unsupported label constraint") // ErrConflictingConstraints is from constraints.go. ErrConflictingConstraints = errors.New("conflicting label constraints") + // ErrInvalidConstraintsMapcnt is from rule.go. + ErrInvalidConstraintsMapcnt = errors.New("label constraints in map syntax have invalid replicas") + // ErrInvalidConstraintsFormat is from rule.go. + ErrInvalidConstraintsFormat = errors.New("invalid label constraints format") + // ErrInvalidConstraintsRelicas is from rule.go. + ErrInvalidConstraintsRelicas = errors.New("label constraints with invalid REPLICAS") ) diff --git a/ddl/placement/rule.go b/ddl/placement/rule.go new file mode 100644 index 0000000000000..134bdd5a610f9 --- /dev/null +++ b/ddl/placement/rule.go @@ -0,0 +1,132 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package placement + +import ( + "fmt" + "strings" + + "github.com/go-yaml/yaml" +) + +// PeerRoleType is the expected peer type of the placement rule. +type PeerRoleType string + +const ( + // Voter can either match a leader peer or follower peer. + Voter PeerRoleType = "voter" + // Leader matches a leader. + Leader PeerRoleType = "leader" + // Follower matches a follower. + Follower PeerRoleType = "follower" + // Learner matches a learner. + Learner PeerRoleType = "learner" +) + +// Rule is the core placement rule struct. Check https://github.com/tikv/pd/blob/master/server/schedule/placement/rule.go. +type Rule struct { + GroupID string `json:"group_id"` + ID string `json:"id"` + Index int `json:"index,omitempty"` + Override bool `json:"override,omitempty"` + StartKeyHex string `json:"start_key"` + EndKeyHex string `json:"end_key"` + Role PeerRoleType `json:"role"` + Count int `json:"count"` + Constraints Constraints `json:"label_constraints,omitempty"` + LocationLabels []string `json:"location_labels,omitempty"` + IsolationLevel string `json:"isolation_level,omitempty"` +} + +// NewRules constructs []*Rule from a yaml-compatible representation of +// array or map of constraints. It converts 'CONSTRAINTS' field in RFC +// https://github.com/pingcap/tidb/blob/master/docs/design/2020-06-24-placement-rules-in-sql.md to structs. +func NewRules(replicas uint64, cnstr string) ([]*Rule, error) { + rules := []*Rule{} + + cnstbytes := []byte(cnstr) + + constraints1 := []string{} + err1 := yaml.UnmarshalStrict(cnstbytes, &constraints1) + if err1 == nil { + // can not emit REPLICAS with an array or empty label + if replicas == 0 { + return rules, fmt.Errorf("%w: should be positive", ErrInvalidConstraintsRelicas) + } + + labelConstraints, err := NewConstraints(constraints1) + if err != nil { + return rules, err + } + + rules = append(rules, &Rule{ + Count: int(replicas), + Constraints: labelConstraints, + }) + + return rules, nil + } + + constraints2 := map[string]int{} + err2 := yaml.UnmarshalStrict(cnstbytes, &constraints2) + if err2 == nil { + ruleCnt := 0 + for labels, cnt := range constraints2 { + if cnt <= 0 { + return rules, fmt.Errorf("%w: count of labels '%s' should be positive, but got %d", ErrInvalidConstraintsMapcnt, labels, cnt) + } + ruleCnt += cnt + } + + if replicas == 0 { + replicas = uint64(ruleCnt) + } + + if int(replicas) < ruleCnt { + return rules, fmt.Errorf("%w: should be larger or equal to the number of total replicas, but REPLICAS=%d < total=%d", ErrInvalidConstraintsRelicas, replicas, ruleCnt) + } + + for labels, cnt := range constraints2 { + labelConstraints, err := NewConstraints(strings.Split(labels, ",")) + if err != nil { + return rules, err + } + + rules = append(rules, &Rule{ + Count: cnt, + Constraints: labelConstraints, + }) + } + + remain := int(replicas) - ruleCnt + if remain > 0 { + rules = append(rules, &Rule{ + Count: remain, + }) + } + + return rules, nil + } + + return nil, fmt.Errorf("%w: should be [constraint1, ...] (error %s), {constraint1: cnt1, ...} (error %s), or any yaml compatible representation", ErrInvalidConstraintsFormat, err1, err2) +} + +// Clone is used to duplicate a RuleOp for safe modification. +// Note that it is a shallow copy: LocationLabels and Constraints +// is not cloned. +func (r *Rule) Clone() *Rule { + n := &Rule{} + *n = *r + return n +} diff --git a/ddl/placement/rule_test.go b/ddl/placement/rule_test.go new file mode 100644 index 0000000000000..85dd492f348e7 --- /dev/null +++ b/ddl/placement/rule_test.go @@ -0,0 +1,206 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package placement + +import ( + "encoding/json" + "errors" + + . "github.com/pingcap/check" +) + +var _ = Suite(&testRuleSuite{}) + +type testRuleSuite struct{} + +func (t *testRuleSuite) TestClone(c *C) { + rule := &Rule{ID: "434"} + newRule := rule.Clone() + newRule.ID = "121" + + c.Assert(rule, DeepEquals, &Rule{ID: "434"}) + c.Assert(newRule, DeepEquals, &Rule{ID: "121"}) +} + +func matchRule(r1 *Rule, t2 []*Rule) bool { + for _, r2 := range t2 { + if ok, _ := DeepEquals.Check([]interface{}{r1, r2}, nil); ok { + return true + } + } + return false +} + +func matchRules(t1, t2 []*Rule, prefix string, c *C) { + expected, err := json.Marshal(t1) + c.Assert(err, IsNil) + got, err := json.Marshal(t2) + c.Assert(err, IsNil) + comment := Commentf("%s, expected %s\nbut got %s", prefix, expected, got) + c.Assert(len(t1), Equals, len(t2), comment) + for _, r1 := range t1 { + c.Assert(matchRule(r1, t2), IsTrue, comment) + } +} + +func (t *testRuleSuite) TestNewRules(c *C) { + type TestCase struct { + name string + input string + replicas uint64 + output []*Rule + err error + } + tests := []TestCase{} + + tests = append(tests, TestCase{ + name: "empty constraints", + input: "", + replicas: 3, + output: []*Rule{ + { + Count: 3, + Constraints: Constraints{}, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "zero replicas", + input: "", + replicas: 0, + err: ErrInvalidConstraintsRelicas, + }) + + labels, err := NewConstraints([]string{"+zone=sh", "+zone=sh"}) + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal array constraints", + input: `["+zone=sh", "+zone=sh"]`, + replicas: 3, + output: []*Rule{ + { + Count: 3, + Constraints: labels, + }, + }, + }) + + labels1, err := NewConstraints([]string{"+zone=sh", "-zone=bj"}) + c.Assert(err, IsNil) + labels2, err := NewConstraints([]string{"+zone=sh"}) + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal object constraints", + input: `{"+zone=sh,-zone=bj":2, "+zone=sh": 1}`, + replicas: 3, + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "normal object constraints, with extra count", + input: "{'+zone=sh,-zone=bj':2, '+zone=sh': 1}", + replicas: 4, + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + { + Count: 1, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "normal object constraints, without count", + input: "{'+zone=sh,-zone=bj':2, '+zone=sh': 1}", + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "zero count in object constraints", + input: `{"+zone=sh,-zone=bj":0, "+zone=sh": 1}`, + replicas: 3, + err: ErrInvalidConstraintsMapcnt, + }) + + tests = append(tests, TestCase{ + name: "overlarge total count in object constraints", + input: `{"+ne=sh,-zone=bj":1, "+zone=sh": 4}`, + replicas: 3, + err: ErrInvalidConstraintsRelicas, + }) + + tests = append(tests, TestCase{ + name: "invalid array", + input: `["+ne=sh", "+zone=sh"`, + replicas: 3, + err: ErrInvalidConstraintsFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid array constraints", + input: `["ne=sh", "+zone=sh"]`, + replicas: 3, + err: ErrInvalidConstraintFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid map", + input: `{+ne=sh,-zone=bj:1, "+zone=sh": 4`, + replicas: 5, + err: ErrInvalidConstraintsFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid map constraints", + input: `{"nesh,-zone=bj":1, "+zone=sh": 4}`, + replicas: 6, + err: ErrInvalidConstraintFormat, + }) + + for _, t := range tests { + comment := Commentf("%s", t.name) + output, err := NewRules(t.replicas, t.input) + if t.err == nil { + c.Assert(err, IsNil, comment) + matchRules(t.output, output, comment.CheckCommentString(), c) + } else { + c.Assert(errors.Is(err, t.err), IsTrue, comment) + } + } +} diff --git a/ddl/placement/types.go b/ddl/placement/types.go index 3bb9da96e3890..72093a2c19c78 100644 --- a/ddl/placement/types.go +++ b/ddl/placement/types.go @@ -22,42 +22,6 @@ import ( // After all, placement rules are communicated using an HTTP API. Loose // coupling is a good feature. -// PeerRoleType is the expected peer type of the placement rule. -type PeerRoleType string - -const ( - // Voter can either match a leader peer or follower peer. - Voter PeerRoleType = "voter" - // Leader matches a leader. - Leader PeerRoleType = "leader" - // Follower matches a follower. - Follower PeerRoleType = "follower" - // Learner matches a learner. - Learner PeerRoleType = "learner" -) - -// Rule is the placement rule. Check https://github.com/tikv/pd/blob/master/server/schedule/placement/rule.go. -type Rule struct { - GroupID string `json:"group_id"` - ID string `json:"id"` - Index int `json:"index,omitempty"` - Override bool `json:"override,omitempty"` - StartKeyHex string `json:"start_key"` - EndKeyHex string `json:"end_key"` - Role PeerRoleType `json:"role"` - Count int `json:"count"` - LabelConstraints Constraints `json:"label_constraints,omitempty"` - LocationLabels []string `json:"location_labels,omitempty"` - IsolationLevel string `json:"isolation_level,omitempty"` -} - -// Clone is used to duplicate a RuleOp for safe modification. -func (r *Rule) Clone() *Rule { - n := &Rule{} - *n = *r - return n -} - // Bundle is a group of all rules and configurations. It is used to support rule cache. type Bundle struct { ID string `json:"group_id"` diff --git a/ddl/placement/types_test.go b/ddl/placement/types_test.go index 77153cb29b692..93ed1a5a80f43 100644 --- a/ddl/placement/types_test.go +++ b/ddl/placement/types_test.go @@ -18,7 +18,6 @@ import ( ) var _ = Suite(&testBundleSuite{}) -var _ = Suite(&testRuleSuite{}) type testBundleSuite struct{} @@ -49,14 +48,3 @@ func (t *testBundleSuite) TestClone(c *C) { c.Assert(bundle, DeepEquals, &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}}) c.Assert(newBundle, DeepEquals, &Bundle{ID: GroupID(2), Rules: []*Rule{{ID: "121"}}}) } - -type testRuleSuite struct{} - -func (t *testRuleSuite) TestClone(c *C) { - rule := &Rule{ID: "434"} - newRule := rule.Clone() - newRule.ID = "121" - - c.Assert(rule, DeepEquals, &Rule{ID: "434"}) - c.Assert(newRule, DeepEquals, &Rule{ID: "121"}) -} diff --git a/ddl/placement/utils.go b/ddl/placement/utils.go index 16c0a424dde53..5b12f10e2d243 100644 --- a/ddl/placement/utils.go +++ b/ddl/placement/utils.go @@ -61,7 +61,7 @@ func BuildPlacementCopyBundle(oldBundle *Bundle, newID int64) *Bundle { func GetLeaderDCByBundle(bundle *Bundle, dcLabelKey string) (string, bool) { for _, rule := range bundle.Rules { if isValidLeaderRule(rule, dcLabelKey) { - return rule.LabelConstraints[0].Values[0], true + return rule.Constraints[0].Values[0], true } } return "", false @@ -69,7 +69,7 @@ func GetLeaderDCByBundle(bundle *Bundle, dcLabelKey string) (string, bool) { func isValidLeaderRule(rule *Rule, dcLabelKey string) bool { if rule.Role == Leader && rule.Count == 1 { - for _, con := range rule.LabelConstraints { + for _, con := range rule.Constraints { if con.Op == In && con.Key == dcLabelKey && len(con.Values) == 1 { return true } diff --git a/ddl/placement/utils_test.go b/ddl/placement/utils_test.go index 964382846485e..10941e0663455 100644 --- a/ddl/placement/utils_test.go +++ b/ddl/placement/utils_test.go @@ -58,7 +58,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -84,7 +84,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Voter, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -110,7 +110,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -127,7 +127,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Voter, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -153,7 +153,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "fake", Op: In, @@ -179,7 +179,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: NotIn, @@ -205,7 +205,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, diff --git a/ddl/placement_rule_test.go b/ddl/placement_rule_test.go index b051092a776e9..a9a916cb5a199 100644 --- a/ddl/placement_rule_test.go +++ b/ddl/placement_rule_test.go @@ -52,7 +52,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, @@ -67,9 +67,9 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { Constraints: "", }}, output: []*placement.Rule{{ - Role: placement.Voter, - Count: 3, - LabelConstraints: []placement.Constraint{}, + Role: placement.Voter, + Count: 3, + Constraints: []placement.Constraint{}, }}, }, @@ -83,14 +83,14 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, { Role: placement.Voter, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, @@ -108,7 +108,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -127,7 +127,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -154,7 +154,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -162,7 +162,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Follower, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "in", Values: []string{"bj"}}, }, @@ -189,7 +189,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "in", Values: []string{"bj"}}, }, @@ -214,14 +214,14 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { }, output: []*placement.Rule{ { - Role: placement.Voter, - Count: 1, - LabelConstraints: []placement.Constraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, + Role: placement.Voter, + Count: 1, + Constraints: []placement.Constraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, }, { - Role: placement.Voter, - Count: 1, - LabelConstraints: []placement.Constraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, + Role: placement.Voter, + Count: 1, + Constraints: []placement.Constraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, }, { Role: placement.Voter, @@ -306,7 +306,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index e77b0ba99d5cf..fb7158681714f 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -404,7 +404,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -423,7 +423,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Follower, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -619,7 +619,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 4d56cc55accac..4f788a3d7bd1d 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1915,7 +1915,7 @@ func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) er continue } for _, rule := range bundle.Rules { - constraint, err := rule.LabelConstraints.Restore() + constraint, err := rule.Constraints.Restore() if err != nil { return errors.Wrapf(err, "Restore rule %s in bundle %s failed", rule.ID, bundle.ID) } diff --git a/expression/integration_test.go b/expression/integration_test.go index b1bded2a17719..80e39b76ce746 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -8755,7 +8755,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 6cc24300c1be4..ebe4a0620256f 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1449,7 +1449,7 @@ func (s *testTableSuite) TestPlacementPolicy(c *C) { ID: "0", Role: "voter", Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: "zone", Op: "in", diff --git a/session/session_test.go b/session/session_test.go index 3baee4f0ef6f1..a8861dfddd79c 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -3319,7 +3319,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, From 392df99f04d5bc51560993408f39de81ffd3c672 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Fri, 14 May 2021 17:21:38 +0800 Subject: [PATCH 27/42] cmd: ddl_test modify retryCnt from 5 to 20 (#24662) --- cmd/ddltest/ddl_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/ddltest/ddl_test.go b/cmd/ddltest/ddl_test.go index 52009b10de142..36922638950a8 100644 --- a/cmd/ddltest/ddl_test.go +++ b/cmd/ddltest/ddl_test.go @@ -143,7 +143,7 @@ func (s *TestDDLSuite) SetUpSuite(c *C) { s.procs = make([]*server, *serverNum) // Set server restart retry count. - s.retryCount = 5 + s.retryCount = 20 createLogFiles(c, *serverNum) err = s.startServers() From 4d7fcbb3dffb7372150e47257842eb30efd33bf7 Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Fri, 14 May 2021 17:33:38 +0800 Subject: [PATCH 28/42] executor: add correctness tests about direct reading with ORDER BY and LIMIT (#24455) --- executor/partition_table_test.go | 79 ++++++++++++++++++++++++++++++++ 1 file changed, 79 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index ab5a19ca823e9..df0c52cfb55cc 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -227,6 +227,85 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows()) } +func (s *partitionTableSuite) TestOrderByandLimit(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_orderby_limit") + tk.MustExec("use test_orderby_limit") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // range partition table + tk.MustExec(`create table trange(a int, b int, index idx_a(a)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table + tk.MustExec("create table thash(a int, b int, index idx_a(a), index idx_b(b)) partition by hash(a) partitions 4;") + + // regular table + tk.MustExec("create table tregular(a int, b int, index idx_a(a))") + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000))) + } + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular values " + strings.Join(vals, ",")) + + // test indexLookUp + for i := 0; i < 100; i++ { + // explain select * from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select * from t where a > {y} use index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select * from trange use index(idx_a) where a > %v order by a, b limit %v;", x, y) + queryRegular := fmt.Sprintf("select * from tregular use index(idx_a) where a > %v order by a, b limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "IndexLookUp"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test tableReader + for i := 0; i < 100; i++ { + // explain select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select * from trange ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + queryRegular := fmt.Sprintf("select * from tregular ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "TableReader"), IsTrue) // check if tableReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test indexReader + for i := 0; i < 100; i++ { + // explain select a from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select a from t where a > {y} use index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select a from trange use index(idx_a) where a > %v order by a limit %v;", x, y) + queryRegular := fmt.Sprintf("select a from tregular use index(idx_a) where a > %v order by a limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "IndexReader"), IsTrue) // check if indexReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test indexMerge + for i := 0; i < 100; i++ { + // explain select /*+ use_index_merge(t) */ * from t where a > 2 or b < 5 order by a limit {x}; // check if IndexMerge is used + // select /*+ use_index_merge(t) */ * from t where a > 2 or b < 5 order by a limit {x}; // can return the correct value + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > 2 or b < 5 order by a, b limit %v;", y) + queryRegular := fmt.Sprintf("select * from tregular where a > 2 or b < 5 order by a, b limit %v;", y) + c.Assert(tk.HasPlan(queryPartition, "IndexMerge"), IsTrue) // check if indexMerge is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } +} + func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 42b12f7cc324c18e31e64e7c3829588cb4c96237 Mon Sep 17 00:00:00 2001 From: disksing Date: Fri, 14 May 2021 18:01:38 +0800 Subject: [PATCH 29/42] store/tikv: remove options from unionstore (#24629) --- store/driver/txn/txn_driver.go | 6 +----- store/tikv/txn.go | 16 ---------------- store/tikv/unionstore/union_store.go | 24 ------------------------ 3 files changed, 1 insertion(+), 45 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 72c1aac8c8e71..7c940561f82fe 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -168,8 +168,6 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.GetSnapshot().SetIsStatenessReadOnly(val.(bool)) case kv.MatchStoreLabels: txn.KVTxn.GetSnapshot().SetMatchStoreLabels(val.([]*metapb.StoreLabel)) - default: - txn.KVTxn.SetOption(opt, val) } } @@ -180,7 +178,7 @@ func (txn *tikvTxn) GetOption(opt int) interface{} { case kv.TxnScope: return txn.KVTxn.GetScope() default: - return txn.KVTxn.GetOption(opt) + return nil } } @@ -188,8 +186,6 @@ func (txn *tikvTxn) DelOption(opt int) { switch opt { case kv.CollectRuntimeStats: txn.KVTxn.GetSnapshot().SetRuntimeStats(nil) - default: - txn.KVTxn.DelOption(opt) } } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index e444f5adda7f6..aafaa2b323d24 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -221,22 +221,6 @@ func (txn *KVTxn) Delete(k []byte) error { return txn.us.GetMemBuffer().Delete(k) } -// SetOption sets an option with a value, when val is nil, uses the default -// value of this option. -func (txn *KVTxn) SetOption(opt int, val interface{}) { - txn.us.SetOption(opt, val) -} - -// GetOption returns the option -func (txn *KVTxn) GetOption(opt int) interface{} { - return txn.us.GetOption(opt) -} - -// DelOption deletes an option. -func (txn *KVTxn) DelOption(opt int) { - txn.us.DelOption(opt) -} - // SetSchemaLeaseChecker sets a hook to check schema version. func (txn *KVTxn) SetSchemaLeaseChecker(checker SchemaLeaseChecker) { txn.schemaLeaseChecker = checker diff --git a/store/tikv/unionstore/union_store.go b/store/tikv/unionstore/union_store.go index f9a077d1c1352..08354975e38c5 100644 --- a/store/tikv/unionstore/union_store.go +++ b/store/tikv/unionstore/union_store.go @@ -59,7 +59,6 @@ type uSnapshot interface { type KVUnionStore struct { memBuffer *MemDB snapshot uSnapshot - opts options } // NewUnionStore builds a new unionStore. @@ -67,7 +66,6 @@ func NewUnionStore(snapshot uSnapshot) *KVUnionStore { return &KVUnionStore{ snapshot: snapshot, memBuffer: newMemDB(), - opts: make(map[int]interface{}), } } @@ -131,30 +129,8 @@ func (us *KVUnionStore) UnmarkPresumeKeyNotExists(k []byte) { us.memBuffer.UpdateFlags(k, kv.DelPresumeKeyNotExists) } -// SetOption implements the unionStore SetOption interface. -func (us *KVUnionStore) SetOption(opt int, val interface{}) { - us.opts[opt] = val -} - -// DelOption implements the unionStore DelOption interface. -func (us *KVUnionStore) DelOption(opt int) { - delete(us.opts, opt) -} - -// GetOption implements the unionStore GetOption interface. -func (us *KVUnionStore) GetOption(opt int) interface{} { - return us.opts[opt] -} - // SetEntrySizeLimit sets the size limit for each entry and total buffer. func (us *KVUnionStore) SetEntrySizeLimit(entryLimit, bufferLimit uint64) { us.memBuffer.entrySizeLimit = entryLimit us.memBuffer.bufferSizeLimit = bufferLimit } - -type options map[int]interface{} - -func (opts options) Get(opt int) (interface{}, bool) { - v, ok := opts[opt] - return v, ok -} From e92df204b53310029b7c32c5143ab6cebbf7ff18 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Fri, 14 May 2021 19:09:38 +0800 Subject: [PATCH 30/42] planner: fix wrongly check for update statement (#24614) --- executor/write_test.go | 5 ++-- planner/core/integration_test.go | 14 ++++++++++ planner/core/logical_plan_builder.go | 39 +++++++++++++++------------- planner/core/logical_plan_test.go | 1 - 4 files changed, 37 insertions(+), 22 deletions(-) diff --git a/executor/write_test.go b/executor/write_test.go index 27ea70ae748a5..b832e52a9935c 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1554,7 +1554,7 @@ func (s *testSuite8) TestUpdate(c *C) { _, err = tk.Exec("UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2") c.Assert(err, IsNil) - tk.MustExec("update (select * from t) t set c1 = 1111111") + tk.MustGetErrCode("update (select * from t) t set c1 = 1111111", mysql.ErrNonUpdatableTable) // test update ignore for bad null error tk.MustExec("drop table if exists t;") @@ -1604,8 +1604,7 @@ func (s *testSuite8) TestUpdate(c *C) { tk.MustExec("drop view v") tk.MustExec("create sequence seq") - _, err = tk.Exec("update seq set minvalue=1") - c.Assert(err.Error(), Equals, "update sequence seq is not supported now.") + tk.MustGetErrCode("update seq set minvalue=1", mysql.ErrBadField) tk.MustExec("drop sequence seq") tk.MustExec("drop table if exists t1, t2") diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 9e8eaa9204af9..62e5b032c86df 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -196,6 +196,20 @@ func (s *testIntegrationSuite) TestIssue22298(c *C) { tk.MustGetErrMsg(`select * from t where 0 and c = 10;`, "[planner:1054]Unknown column 'c' in 'where clause'") } +func (s *testIntegrationSuite) TestIssue24571(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec(`create view v as select 1 as b;`) + tk.MustExec(`create table t (a int);`) + tk.MustExec(`update v, t set a=2;`) + tk.MustGetErrCode(`update v, t set b=2;`, mysql.ErrNonUpdatableTable) + tk.MustExec("create database db1") + tk.MustExec("use db1") + tk.MustExec("update test.t, (select 1 as a) as t set test.t.a=1;") + // bug in MySQL: ERROR 1288 (HY000): The target table t of the UPDATE is not updatable + tk.MustExec("update (select 1 as a) as t, test.t set test.t.a=1;") +} + func (s *testIntegrationSuite) TestIssue22828(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 7ff0e2ac6c6aa..64bc0c41407e1 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4239,17 +4239,6 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( b.popTableHints() }() - // update subquery table should be forbidden - var notUpdatableTbl []string - notUpdatableTbl = extractTableSourceAsNames(update.TableRefs.TableRefs, notUpdatableTbl, true) - for _, asName := range notUpdatableTbl { - for _, assign := range update.List { - if assign.Column.Table.L == asName { - return nil, ErrNonUpdatableTable.GenWithStackByArgs(asName, "UPDATE") - } - } - } - b.inUpdateStmt = true b.isForUpdateRead = true @@ -4265,12 +4254,6 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( if dbName == "" { dbName = b.ctx.GetSessionVars().CurrentDB } - if t.TableInfo.IsView() { - return nil, errors.Errorf("update view %s is not supported now.", t.Name.O) - } - if t.TableInfo.IsSequence() { - return nil, errors.Errorf("update sequence %s is not supported now.", t.Name.O) - } b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, dbName, t.Name.L, "", nil) } @@ -4314,6 +4297,10 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( proj.SetChildren(p) p = proj + // update subquery table should be forbidden + var notUpdatableTbl []string + notUpdatableTbl = extractTableSourceAsNames(update.TableRefs.TableRefs, notUpdatableTbl, true) + var updateTableList []*ast.TableName updateTableList = extractTableList(update.TableRefs.TableRefs, updateTableList, true) orderedList, np, allAssignmentsAreConstant, err := b.buildUpdateLists(ctx, updateTableList, update.List, p, notUpdatableTbl) @@ -4417,6 +4404,21 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab columnsIdx[assign.Column] = idx } name := p.OutputNames()[idx] + for _, tl := range tableList { + if (tl.Schema.L == "" || tl.Schema.L == name.DBName.L) && (tl.Name.L == name.TblName.L) { + if tl.TableInfo.IsView() || tl.TableInfo.IsSequence() { + return nil, nil, false, ErrNonUpdatableTable.GenWithStackByArgs(name.TblName.O, "UPDATE") + } + // may be a subquery + if tl.Schema.L == "" { + for _, nTbl := range notUpdatableTbl { + if nTbl == name.TblName.L { + return nil, nil, false, ErrNonUpdatableTable.GenWithStackByArgs(name.TblName.O, "UPDATE") + } + } + } + } + } columnFullName := fmt.Sprintf("%s.%s.%s", name.DBName.L, name.TblName.L, name.ColName.L) // We save a flag for the column in map `modifyColumns` // This flag indicated if assign keyword `DEFAULT` to the column @@ -4439,9 +4441,10 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab break } } - if !updatable { + if !updatable || tn.TableInfo.IsView() || tn.TableInfo.IsSequence() { continue } + tableInfo := tn.TableInfo tableVal, found := b.is.TableByID(tableInfo.ID) if !found { diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 11a116bb4fac8..921f1c99b34ec 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1459,7 +1459,6 @@ func (s *testPlanSuite) TestNameResolver(c *C) { {"delete a from (select * from t ) as a, t", "[planner:1288]The target table a of the DELETE is not updatable"}, {"delete b from (select * from t ) as a, t", "[planner:1109]Unknown table 'b' in MULTI DELETE"}, {"select '' as fakeCol from t group by values(fakeCol)", "[planner:1054]Unknown column '' in 'VALUES() function'"}, - {"update t, (select * from t) as b set b.a = t.a", "[planner:1288]The target table b of the UPDATE is not updatable"}, {"select row_number() over () from t group by 1", "[planner:1056]Can't group on 'row_number() over ()'"}, {"select row_number() over () as x from t group by 1", "[planner:1056]Can't group on 'x'"}, {"select sum(a) as x from t group by 1", "[planner:1056]Can't group on 'x'"}, From 557c94b9ba6e7e4d1094be7e5e2a72bf1535e0f5 Mon Sep 17 00:00:00 2001 From: disksing Date: Mon, 17 May 2021 11:13:39 +0800 Subject: [PATCH 31/42] store/tikv: remove CompareTS (#24657) --- executor/stale_txn_test.go | 2 +- store/tikv/oracle/oracle.go | 19 ------------------- store/tikv/txn.go | 2 +- 3 files changed, 2 insertions(+), 21 deletions(-) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index ce5202ae58a75..493bda06c5de2 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -312,7 +312,7 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { if testcase.useSafeTS { c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.injectSafeTS) } else { - c.Assert(oracle.CompareTS(tk.Se.GetSessionVars().TxnCtx.StartTS, testcase.injectSafeTS), Equals, 1) + c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Greater, testcase.injectSafeTS) } tk.MustExec("commit") failpoint.Disable("github.com/pingcap/tidb/store/tikv/injectSafeTS") diff --git a/store/tikv/oracle/oracle.go b/store/tikv/oracle/oracle.go index 0a6865cf59039..daf00c66814ca 100644 --- a/store/tikv/oracle/oracle.go +++ b/store/tikv/oracle/oracle.go @@ -148,25 +148,6 @@ func GoTimeToTS(t time.Time) uint64 { return uint64(ts) } -// CompareTS is used to compare two timestamps. -// If tsoOne > tsoTwo, returns 1. -// If tsoOne = tsoTwo, returns 0. -// If tsoOne < tsoTwo, returns -1. -func CompareTS(tsoOne, tsoTwo uint64) int { - tsOnePhy := ExtractPhysical(tsoOne) - tsOneLog := ExtractLogical(tsoOne) - tsTwoPhy := ExtractPhysical(tsoTwo) - tsTwoLog := ExtractLogical(tsoTwo) - - if tsOnePhy > tsTwoPhy || (tsOnePhy == tsTwoPhy && tsOneLog > tsTwoLog) { - return 1 - } - if tsOnePhy == tsTwoPhy && tsOneLog == tsTwoLog { - return 0 - } - return -1 -} - // GoTimeToLowerLimitStartTS returns the min start_ts of the uncommitted transaction. // maxTxnTimeUse means the max time a Txn May use (in ms) from its begin to commit. func GoTimeToLowerLimitStartTS(now time.Time, maxTxnTimeUse int64) uint64 { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index aafaa2b323d24..beeeafe66a063 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -119,7 +119,7 @@ func extractStartTs(store *KVStore, options kv.TransactionOption) (uint64, error startTs = *options.MinStartTS // If the safeTS is larger than the minStartTS, we will use safeTS as StartTS, otherwise we will use // minStartTS directly. - if oracle.CompareTS(startTs, safeTS) < 0 { + if startTs < safeTS { startTs = safeTS } } else if options.MaxPrevSec != nil { From f2cb145af2b893295797b36ec8e8603f784e69e6 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Sun, 16 May 2021 21:25:39 -0600 Subject: [PATCH 32/42] planner, privilege: Add security enhanced mode part 4 (#24416) --- executor/executor_test.go | 1 + planner/core/planbuilder.go | 40 ++++++++++++++++++++- privilege/privilege.go | 3 ++ privilege/privileges/privileges.go | 16 +++++++++ privilege/privileges/privileges_test.go | 47 +++++++++++++++++++++++++ 5 files changed, 106 insertions(+), 1 deletion(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index d67bb6b48b8f4..e359837d8fb92 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -357,6 +357,7 @@ func (s *testSuiteP1) TestShow(c *C) { "CONNECTION_ADMIN Server Admin ", "RESTRICTED_TABLES_ADMIN Server Admin ", "RESTRICTED_STATUS_ADMIN Server Admin ", + "RESTRICTED_USER_ADMIN Server Admin ", )) c.Assert(len(tk.MustQuery("show table status").Rows()), Equals, 1) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 6fc98bc522508..43997217da54b 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -36,6 +37,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -2276,9 +2278,16 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { p.setSchemaAndNames(buildBRIESchema()) err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or BACKUP_ADMIN") b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "BACKUP_ADMIN", false, err) - case *ast.GrantRoleStmt, *ast.RevokeRoleStmt: + case *ast.GrantRoleStmt: err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or ROLE_ADMIN") b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "ROLE_ADMIN", false, err) + case *ast.RevokeRoleStmt: + err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or ROLE_ADMIN") + b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "ROLE_ADMIN", false, err) + // Check if any of the users are RESTRICTED + for _, user := range raw.Users { + b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, user, "RESTRICTED_USER_ADMIN") + } case *ast.RevokeStmt: b.visitInfo = collectVisitInfoFromRevokeStmt(b.ctx, b.visitInfo, raw) case *ast.KillStmt: @@ -2292,12 +2301,23 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or CONNECTION_ADMIN") b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "CONNECTION_ADMIN", false, err) } + b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, &auth.UserIdentity{Username: pi.User, Hostname: pi.Host}, "RESTRICTED_CONNECTION_ADMIN") } } case *ast.UseStmt: if raw.DBName == "" { return nil, ErrNoDB } + case *ast.DropUserStmt: + // The main privilege checks for DROP USER are currently performed in executor/simple.go + // because they use complex OR conditions (not supported by visitInfo). + for _, user := range raw.UserList { + b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, user, "RESTRICTED_USER_ADMIN") + } + case *ast.SetPwdStmt: + if raw.User != nil { + b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, raw.User, "RESTRICTED_USER_ADMIN") + } case *ast.ShutdownStmt: b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShutdownPriv, "", "", "", nil) } @@ -2339,6 +2359,10 @@ func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stm for _, priv := range allPrivs { vi = appendVisitInfo(vi, priv, dbName, tableName, "", nil) } + for _, u := range stmt.Users { + // For SEM, make sure the users are not restricted + vi = appendVisitInfoIsRestrictedUser(vi, sctx, u.User, "RESTRICTED_USER_ADMIN") + } if nonDynamicPrivilege { // Dynamic privileges use their own GRANT OPTION. If there were any non-dynamic privilege requests, // we need to attach the "GLOBAL" version of the GRANT OPTION. @@ -2347,6 +2371,20 @@ func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stm return vi } +// appendVisitInfoIsRestrictedUser appends additional visitInfo if the user has a +// special privilege called "RESTRICTED_USER_ADMIN". It only applies when SEM is enabled. +func appendVisitInfoIsRestrictedUser(visitInfo []visitInfo, sctx sessionctx.Context, user *auth.UserIdentity, priv string) []visitInfo { + if !sem.IsEnabled() { + return visitInfo + } + checker := privilege.GetPrivilegeManager(sctx) + if checker != nil && checker.RequestDynamicVerificationWithUser("RESTRICTED_USER_ADMIN", false, user) { + err := ErrSpecificAccessDenied.GenWithStackByArgs(priv) + visitInfo = appendDynamicVisitInfo(visitInfo, priv, false, err) + } + return visitInfo +} + func collectVisitInfoFromGrantStmt(sctx sessionctx.Context, vi []visitInfo, stmt *ast.GrantStmt) []visitInfo { // To use GRANT, you must have the GRANT OPTION privilege, // and you must have the privileges that you are granting. diff --git a/privilege/privilege.go b/privilege/privilege.go index cf59ce9b0314e..f732d9da1199b 100644 --- a/privilege/privilege.go +++ b/privilege/privilege.go @@ -50,6 +50,9 @@ type Manager interface { // Dynamic privileges are only assignable globally, and have their own grantable attribute. RequestDynamicVerification(activeRoles []*auth.RoleIdentity, privName string, grantable bool) bool + // RequestDynamicVerification verifies a DYNAMIC privilege for a specific user. + RequestDynamicVerificationWithUser(privName string, grantable bool, user *auth.UserIdentity) bool + // ConnectionVerification verifies user privilege for connection. ConnectionVerification(user, host string, auth, salt []byte, tlsState *tls.ConnectionState) (string, string, bool) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 6ac58e04e44e4..e0c63cfb14bfc 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -45,6 +45,7 @@ var dynamicPrivs = []string{ "CONNECTION_ADMIN", "RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled "RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled. + "RESTRICTED_USER_ADMIN", // User can not have their access revoked by SUPER users. } var dynamicPrivLock sync.Mutex @@ -56,6 +57,21 @@ type UserPrivileges struct { *Handle } +// RequestDynamicVerificationWithUser implements the Manager interface. +func (p *UserPrivileges) RequestDynamicVerificationWithUser(privName string, grantable bool, user *auth.UserIdentity) bool { + if SkipWithGrant { + return true + } + + if user == nil { + return false + } + + mysqlPriv := p.Handle.Get() + roles := mysqlPriv.getDefaultRoles(user.Username, user.Hostname) + return mysqlPriv.RequestDynamicVerification(roles, user.Username, user.Hostname, privName, grantable) +} + // RequestDynamicVerification implements the Manager interface. func (p *UserPrivileges) RequestDynamicVerification(activeRoles []*auth.RoleIdentity, privName string, grantable bool) bool { if SkipWithGrant { diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index c7a825a4d894d..2af31f3699f7d 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1428,6 +1428,53 @@ func (s *testPrivilegeSuite) TestViewDefiner(c *C) { tk.MustExec("select * from test_view2") } +func (s *testPrivilegeSuite) TestSecurityEnhancedModeRestrictedUsers(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE USER ruroot1, ruroot2, ruroot3") + tk.MustExec("CREATE ROLE notimportant") + tk.MustExec("GRANT SUPER, CREATE USER ON *.* to ruroot1 WITH GRANT OPTION") + tk.MustExec("SET tidb_enable_dynamic_privileges=1") + tk.MustExec("GRANT SUPER, RESTRICTED_USER_ADMIN, CREATE USER ON *.* to ruroot2 WITH GRANT OPTION") + tk.MustExec("GRANT RESTRICTED_USER_ADMIN ON *.* to ruroot3") + tk.MustExec("GRANT notimportant TO ruroot2, ruroot3") + + sem.Enable() + defer sem.Disable() + + stmts := []string{ + "SET PASSWORD for ruroot3 = 'newpassword'", + "REVOKE notimportant FROM ruroot3", + "REVOKE SUPER ON *.* FROM ruroot3", + "DROP USER ruroot3", + } + + // ruroot1 has SUPER but in SEM will be restricted + tk.Se.Auth(&auth.UserIdentity{ + Username: "ruroot1", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + + for _, stmt := range stmts { + err := tk.ExecToErr(stmt) + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_USER_ADMIN privilege(s) for this operation") + } + + // Switch to ruroot2, it should be permitted + tk.Se.Auth(&auth.UserIdentity{ + Username: "ruroot2", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + + for _, stmt := range stmts { + err := tk.ExecToErr(stmt) + c.Assert(err, IsNil) + } +} + func (s *testPrivilegeSuite) TestDynamicPrivsRegistration(c *C) { se := newSession(c, s.store, s.dbName) pm := privilege.GetPrivilegeManager(se) From 0eefeecd1bce8aad16af6f9831dad5d85b982243 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 May 2021 11:37:39 +0800 Subject: [PATCH 33/42] executor: add some test cases about partition table dynamic-mode with split-region (#24665) --- executor/partition_table_test.go | 37 ++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index df0c52cfb55cc..5dc23e4b8e624 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -612,6 +612,43 @@ func (s *partitionTableSuite) TestAddDropPartitions(c *C) { tk.MustPartition(`select * from t where a < 20`, "p1,p2,p3").Sort().Check(testkit.Rows("12", "15", "7")) } +func (s *partitionTableSuite) TestSplitRegion(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_split_region") + tk.MustExec("use test_split_region") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table tnormal (a int, b int)`) + tk.MustExec(`create table thash (a int, b int, index(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, index(a)) partition by range(a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + vals := make([]string, 0, 1000) + for i := 0; i < 1000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into tnormal values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into trange values ` + strings.Join(vals, ", ")) + + tk.MustExec(`SPLIT TABLE thash INDEX a BETWEEN (1) AND (25000) REGIONS 10`) + tk.MustExec(`SPLIT TABLE trange INDEX a BETWEEN (1) AND (25000) REGIONS 10`) + + result := tk.MustQuery(`select * from tnormal where a>=1 and a<=15000`).Sort().Rows() + tk.MustPartition(`select * from trange where a>=1 and a<=15000`, "p0,p1").Sort().Check(result) + tk.MustPartition(`select * from thash where a>=1 and a<=15000`, "all").Sort().Check(result) + + result = tk.MustQuery(`select * from tnormal where a in (1, 10001, 20001)`).Sort().Rows() + tk.MustPartition(`select * from trange where a in (1, 10001, 20001)`, "p0,p1,p2").Sort().Check(result) + tk.MustPartition(`select * from thash where a in (1, 10001, 20001)`, "p1").Sort().Check(result) +} + func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 09e95b90284edcfffe34382d854c2b9f0c587025 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 May 2021 11:51:39 +0800 Subject: [PATCH 34/42] planner: fix wrong column offsets when processing dynamic pruning for IndexJoin (#24659) --- executor/builder.go | 33 +++++++++++++++--- executor/index_lookup_join.go | 10 +++--- executor/partition_table_test.go | 59 ++++++++++++++++++++++++++++++++ 3 files changed, 94 insertions(+), 8 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 3324e52f894ff..54e2dfb93012a 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -49,6 +49,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" @@ -2476,11 +2477,14 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) outerKeyCols[i] = v.OuterJoinKeys[i].Index } innerKeyCols := make([]int, len(v.InnerJoinKeys)) + innerKeyColIDs := make([]int64, len(v.InnerJoinKeys)) for i := 0; i < len(v.InnerJoinKeys); i++ { innerKeyCols[i] = v.InnerJoinKeys[i].Index + innerKeyColIDs[i] = v.InnerJoinKeys[i].ID } e.outerCtx.keyCols = outerKeyCols e.innerCtx.keyCols = innerKeyCols + e.innerCtx.keyColIDs = innerKeyColIDs outerHashCols, innerHashCols := make([]int, len(v.OuterHashKeys)), make([]int, len(v.InnerHashKeys)) for i := 0; i < len(v.OuterHashKeys); i++ { @@ -2785,7 +2789,6 @@ func keyColumnsIncludeAllPartitionColumns(keyColumns []int, pe *tables.Partition func prunePartitionForInnerExecutor(ctx sessionctx.Context, tbl table.Table, schema *expression.Schema, partitionInfo *plannercore.PartitionInfo, lookUpContent []*indexJoinLookUpContent) (usedPartition []table.PhysicalTable, canPrune bool, contentPos []int64, err error) { partitionTbl := tbl.(table.PartitionedTable) - locateKey := make([]types.Datum, schema.Len()) // TODO: condition based pruning can be do in advance. condPruneResult, err := partitionPruning(ctx, partitionTbl, partitionInfo.PruningConds, partitionInfo.PartitionNames, partitionInfo.Columns, partitionInfo.ColumnNames) if err != nil { @@ -2800,22 +2803,44 @@ func prunePartitionForInnerExecutor(ctx sessionctx.Context, tbl table.Table, sch if err != nil { return nil, false, nil, err } + + // recalculate key column offsets + if lookUpContent[0].keyColIDs == nil { + return nil, false, nil, + dbterror.ClassOptimizer.NewStd(mysql.ErrInternal).GenWithStack("cannot get column IDs when dynamic pruning") + } + keyColOffsets := make([]int, len(lookUpContent[0].keyColIDs)) + for i, colID := range lookUpContent[0].keyColIDs { + offset := -1 + for j, col := range partitionTbl.Cols() { + if colID == col.ID { + offset = j + break + } + } + if offset == -1 { + return nil, false, nil, + dbterror.ClassOptimizer.NewStd(mysql.ErrInternal).GenWithStack("invalid column offset when dynamic pruning") + } + keyColOffsets[i] = offset + } + offsetMap := make(map[int]bool) - for _, offset := range lookUpContent[0].keyCols { + for _, offset := range keyColOffsets { offsetMap[offset] = true } for _, offset := range pe.ColumnOffset { if _, ok := offsetMap[offset]; !ok { - logutil.BgLogger().Warn("can not runtime prune in index join") return condPruneResult, false, nil, nil } } + locateKey := make([]types.Datum, len(partitionTbl.Cols())) partitions := make(map[int64]table.PhysicalTable) contentPos = make([]int64, len(lookUpContent)) for idx, content := range lookUpContent { for i, date := range content.keys { - locateKey[content.keyCols[i]] = date + locateKey[keyColOffsets[i]] = date } p, err := partitionTbl.GetPartitionByRow(ctx, locateKey) if err != nil { diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 9bec8e118515a..0e31280b6632a 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -95,6 +95,7 @@ type innerCtx struct { readerBuilder *dataReaderBuilder rowTypes []*types.FieldType keyCols []int + keyColIDs []int64 // the original ID in its table, used by dynamic partition pruning hashCols []int colLens []int hasPrefixCol bool @@ -472,9 +473,10 @@ func (iw *innerWorker) run(ctx context.Context, wg *sync.WaitGroup) { } type indexJoinLookUpContent struct { - keys []types.Datum - row chunk.Row - keyCols []int + keys []types.Datum + row chunk.Row + keyCols []int + keyColIDs []int64 // the original ID in its table, used by dynamic partition pruning } func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) error { @@ -545,7 +547,7 @@ func (iw *innerWorker) constructLookupContent(task *lookUpJoinTask) ([]*indexJoi // dLookUpKey is sorted and deduplicated at sortAndDedupLookUpContents. // So we don't need to do it here. } - lookUpContents = append(lookUpContents, &indexJoinLookUpContent{keys: dLookUpKey, row: chk.GetRow(rowIdx), keyCols: iw.keyCols}) + lookUpContents = append(lookUpContents, &indexJoinLookUpContent{keys: dLookUpKey, row: chk.GetRow(rowIdx), keyCols: iw.keyCols, keyColIDs: iw.keyColIDs}) } } diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 5dc23e4b8e624..cf53b7cd82688 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -435,6 +435,65 @@ func (s *partitionTableSuite) TestView(c *C) { } } +func (s *partitionTableSuite) TestDynamicPruningUnderIndexJoin(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database pruing_under_index_join") + tk.MustExec("use pruing_under_index_join") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table tnormal (a int, b int, c int, primary key(a), index idx_b(b))`) + tk.MustExec(`create table thash (a int, b int, c int, primary key(a), index idx_b(b)) partition by hash(a) partitions 4`) + tk.MustExec(`create table touter (a int, b int, c int)`) + + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v, %v)", i, rand.Intn(10000), rand.Intn(10000))) + } + tk.MustExec(`insert into tnormal values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into touter values ` + strings.Join(vals, ", ")) + + // case 1: IndexReader in the inner side + tk.MustQuery(`explain format='brief' select /*+ INL_JOIN(touter, thash) */ thash.b from touter join thash use index(idx_b) on touter.b = thash.b`).Check(testkit.Rows( + `IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:pruing_under_index_join.touter.b, inner key:pruing_under_index_join.thash.b, equal cond:eq(pruing_under_index_join.touter.b, pruing_under_index_join.thash.b)`, + `├─TableReader(Build) 9990.00 root data:Selection`, + `│ └─Selection 9990.00 cop[tikv] not(isnull(pruing_under_index_join.touter.b))`, + `│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + `└─IndexReader(Probe) 1.25 root partition:all index:Selection`, + ` └─Selection 1.25 cop[tikv] not(isnull(pruing_under_index_join.thash.b))`, + ` └─IndexRangeScan 1.25 cop[tikv] table:thash, index:idx_b(b) range: decided by [eq(pruing_under_index_join.thash.b, pruing_under_index_join.touter.b)], keep order:false, stats:pseudo`)) + tk.MustQuery(`select /*+ INL_JOIN(touter, thash) */ thash.b from touter join thash use index(idx_b) on touter.b = thash.b`).Sort().Check( + tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b`).Sort().Rows()) + + // case 2: TableReader in the inner side + tk.MustQuery(`explain format='brief' select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(primary) on touter.b = thash.a`).Check(testkit.Rows( + `IndexJoin 12487.50 root inner join, inner:TableReader, outer key:pruing_under_index_join.touter.b, inner key:pruing_under_index_join.thash.a, equal cond:eq(pruing_under_index_join.touter.b, pruing_under_index_join.thash.a)`, + `├─TableReader(Build) 9990.00 root data:Selection`, + `│ └─Selection 9990.00 cop[tikv] not(isnull(pruing_under_index_join.touter.b))`, + `│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + `└─TableReader(Probe) 1.00 root partition:all data:TableRangeScan`, + ` └─TableRangeScan 1.00 cop[tikv] table:thash range: decided by [pruing_under_index_join.touter.b], keep order:false, stats:pseudo`)) + tk.MustQuery(`select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(primary) on touter.b = thash.a`).Sort().Check( + tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.* from touter join tnormal use index(primary) on touter.b = tnormal.a`).Sort().Rows()) + + // case 3: IndexLookUp in the inner side + read all inner columns + tk.MustQuery(`explain format='brief' select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(idx_b) on touter.b = thash.b`).Check(testkit.Rows( + `IndexJoin 12487.50 root inner join, inner:IndexLookUp, outer key:pruing_under_index_join.touter.b, inner key:pruing_under_index_join.thash.b, equal cond:eq(pruing_under_index_join.touter.b, pruing_under_index_join.thash.b)`, + `├─TableReader(Build) 9990.00 root data:Selection`, + `│ └─Selection 9990.00 cop[tikv] not(isnull(pruing_under_index_join.touter.b))`, + `│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + `└─IndexLookUp(Probe) 1.25 root partition:all `, + ` ├─Selection(Build) 1.25 cop[tikv] not(isnull(pruing_under_index_join.thash.b))`, + ` │ └─IndexRangeScan 1.25 cop[tikv] table:thash, index:idx_b(b) range: decided by [eq(pruing_under_index_join.thash.b, pruing_under_index_join.touter.b)], keep order:false, stats:pseudo`, + ` └─TableRowIDScan(Probe) 1.25 cop[tikv] table:thash keep order:false, stats:pseudo`)) + tk.MustQuery(`select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(idx_b) on touter.b = thash.b`).Sort().Check( + tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.* from touter join tnormal use index(idx_b) on touter.b = tnormal.b`).Sort().Rows()) +} + func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 8ad868f801fce6252e41bb2662087318980af448 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Sun, 16 May 2021 22:45:38 -0600 Subject: [PATCH 35/42] *: Add security enhanced mode part 3 (#24412) --- docs/design/2021-03-09-dynamic-privileges.md | 4 +- .../2021-03-09-security-enhanced-mode.md | 4 +- executor/executor_test.go | 1 + executor/show.go | 15 +++++- planner/core/expression_rewriter.go | 5 ++ planner/core/planbuilder.go | 4 ++ privilege/privileges/privileges.go | 7 +-- privilege/privileges/privileges_test.go | 49 +++++++++++++++++++ sessionctx/variable/sysvar.go | 2 +- sessionctx/variable/tidb_vars.go | 2 - tidb-server/main.go | 3 ++ util/sem/sem.go | 32 ++++++++++++ util/sem/sem_test.go | 25 ++++++++++ 13 files changed, 141 insertions(+), 12 deletions(-) diff --git a/docs/design/2021-03-09-dynamic-privileges.md b/docs/design/2021-03-09-dynamic-privileges.md index 7ad0d59d2c54e..c85c0dc0c5305 100644 --- a/docs/design/2021-03-09-dynamic-privileges.md +++ b/docs/design/2021-03-09-dynamic-privileges.md @@ -1,7 +1,7 @@ # Proposal: - Author(s): [morgo](https://github.com/morgo) -- Last updated: April 25, 2021 +- Last updated: May 04, 2021 - Discussion at: N/A ## Table of Contents @@ -238,7 +238,7 @@ No change | Privilege Name | Description | Notes | | --------------- | --------------- | --------------- | -| `RESTRICTED_SYSTEM_VARIABLES_ADMIN` | Allows changing a restricted `GLOBAL` system variable. | Currently in SEM all high risk variables are unloaded. TBD, it might be required in future that they are only visible/settable to those with this privilege and not SUPER. | +| `RESTRICTED_VARIABLES_ADMIN` | Allows changing a restricted `GLOBAL` system variable. | Currently in SEM all high risk variables are unloaded. TBD, it might be required in future that they are only visible/settable to those with this privilege and not SUPER. | | `RESTRICTED_STATUS_ADMIN` | Allows observing restricted status variables. | i.e. `SHOW GLOBAL STATUS` by default hides some statistics when `SEM` is enabled. | | `RESTRICTED_CONNECTION_ADMIN` | A special privilege to say that their connections, etc. can’t be killed by SUPER users AND they can kill connections by all other users. Affects `KILL`, `KILL TIDB` commands. | It is intended for the CloudAdmin user in DBaaS. | | `RESTRICTED_USER_ADMIN` | A special privilege to say that their access can’t be changed by `SUPER` users. Statements `DROP USER`, `SET PASSWORD`, `ALTER USER`, `REVOKE` are all limited. | It is intended for the CloudAdmin user in DbaaS. | diff --git a/docs/design/2021-03-09-security-enhanced-mode.md b/docs/design/2021-03-09-security-enhanced-mode.md index e939fec67c154..efc5b79f499e4 100644 --- a/docs/design/2021-03-09-security-enhanced-mode.md +++ b/docs/design/2021-03-09-security-enhanced-mode.md @@ -1,7 +1,7 @@ # Proposal: - Author(s): [morgo](https://github.com/morgo) -- Last updated: April 25, 2021 +- Last updated: May 04, 2021 - Discussion at: N/A ## Table of Contents @@ -49,7 +49,7 @@ A boolean option called `EnableEnhancedSecurity` (default `FALSE`) will be added ### System Variables -The following system variables will be hidden unless the user has the `RESTRICTED_SYSTEM_VARIABLES_ADMIN` privilege: +The following system variables will be hidden unless the user has the `RESTRICTED_VARIABLES_ADMIN` privilege: * variable.TiDBDDLSlowOprThreshold, * variable.TiDBAllowRemoveAutoInc, diff --git a/executor/executor_test.go b/executor/executor_test.go index e359837d8fb92..5e6b4490f5eb6 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -357,6 +357,7 @@ func (s *testSuiteP1) TestShow(c *C) { "CONNECTION_ADMIN Server Admin ", "RESTRICTED_TABLES_ADMIN Server Admin ", "RESTRICTED_STATUS_ADMIN Server Admin ", + "RESTRICTED_VARIABLES_ADMIN Server Admin ", "RESTRICTED_USER_ADMIN Server Admin ", )) c.Assert(len(tk.MustQuery("show table status").Rows()), Equals, 1) diff --git a/executor/show.go b/executor/show.go index ab08e5ba4cbf5..09e3d0c71e7b4 100644 --- a/executor/show.go +++ b/executor/show.go @@ -661,6 +661,17 @@ func (e *ShowExec) fetchShowMasterStatus() error { return nil } +func (e *ShowExec) sysVarHiddenForSem(sysVarNameInLower string) bool { + if !sem.IsEnabled() || !sem.IsInvisibleSysVar(sysVarNameInLower) { + return false + } + checker := privilege.GetPrivilegeManager(e.ctx) + if checker == nil || checker.RequestDynamicVerification(e.ctx.GetSessionVars().ActiveRoles, "RESTRICTED_VARIABLES_ADMIN", false) { + return false + } + return true +} + func (e *ShowExec) fetchShowVariables() (err error) { var ( value string @@ -673,7 +684,7 @@ func (e *ShowExec) fetchShowVariables() (err error) { // otherwise, fetch the value from table `mysql.Global_Variables`. for _, v := range variable.GetSysVars() { if v.Scope != variable.ScopeSession { - if v.Hidden { + if v.Hidden || e.sysVarHiddenForSem(v.Name) { continue } value, err = variable.GetGlobalSystemVar(sessionVars, v.Name) @@ -690,7 +701,7 @@ func (e *ShowExec) fetchShowVariables() (err error) { // If it is a session only variable, use the default value defined in code, // otherwise, fetch the value from table `mysql.Global_Variables`. for _, v := range variable.GetSysVars() { - if v.Hidden { + if v.Hidden || e.sysVarHiddenForSem(v.Name) { continue } value, err = variable.GetSessionOrGlobalSystemVar(sessionVars, v.Name) diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 8b52318a260a1..eb154d0201ecb 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/hint" + "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/stringutil" ) @@ -1220,6 +1221,10 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { er.err = variable.ErrUnknownSystemVar.GenWithStackByArgs(name) return } + if sem.IsEnabled() && sem.IsInvisibleSysVar(sysVar.Name) { + err := ErrSpecificAccessDenied.GenWithStackByArgs("RESTRICTED_VARIABLES_ADMIN") + er.b.visitInfo = appendDynamicVisitInfo(er.b.visitInfo, "RESTRICTED_VARIABLES_ADMIN", false, err) + } if v.ExplicitScope && !sysVar.HasNoneScope() { if v.IsGlobal && !sysVar.HasGlobalScope() { er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "GLOBAL") diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 43997217da54b..7dc2459dace33 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -722,6 +722,10 @@ func (b *PlanBuilder) buildSet(ctx context.Context, v *ast.SetStmt) (Plan, error err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or SYSTEM_VARIABLES_ADMIN") b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "SYSTEM_VARIABLES_ADMIN", false, err) } + if sem.IsEnabled() && sem.IsInvisibleSysVar(strings.ToLower(vars.Name)) { + err := ErrSpecificAccessDenied.GenWithStackByArgs("RESTRICTED_VARIABLES_ADMIN") + b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "RESTRICTED_VARIABLES_ADMIN", false, err) + } assign := &expression.VarAssignment{ Name: vars.Name, IsGlobal: vars.IsGlobal, diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index e0c63cfb14bfc..0e8d88a90c5a1 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -43,9 +43,10 @@ var dynamicPrivs = []string{ "SYSTEM_VARIABLES_ADMIN", "ROLE_ADMIN", "CONNECTION_ADMIN", - "RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled - "RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled. - "RESTRICTED_USER_ADMIN", // User can not have their access revoked by SUPER users. + "RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled + "RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled. + "RESTRICTED_VARIABLES_ADMIN", // Can see all variables when SEM is enabled + "RESTRICTED_USER_ADMIN", // User can not have their access revoked by SUPER users. } var dynamicPrivLock sync.Mutex diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 2af31f3699f7d..2f6cbef8af2cf 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1400,6 +1400,55 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeStatusVars(c *C) { }, nil, nil) } +func (s *testPrivilegeSuite) TestSecurityEnhancedModeSysVars(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE USER svroot1, svroot2") + tk.MustExec("GRANT SUPER ON *.* to svroot1 WITH GRANT OPTION") + tk.MustExec("SET tidb_enable_dynamic_privileges=1") + tk.MustExec("GRANT SUPER, RESTRICTED_VARIABLES_ADMIN ON *.* to svroot2") + + sem.Enable() + defer sem.Disable() + + // svroot1 has SUPER but in SEM will be restricted + tk.Se.Auth(&auth.UserIdentity{ + Username: "svroot1", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + + tk.MustQuery(`SHOW VARIABLES LIKE 'tidb_force_priority'`).Check(testkit.Rows()) + tk.MustQuery(`SHOW GLOBAL VARIABLES LIKE 'tidb_enable_telemetry'`).Check(testkit.Rows()) + + _, err := tk.Exec("SET tidb_force_priority = 'NO_PRIORITY'") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_VARIABLES_ADMIN privilege(s) for this operation") + _, err = tk.Exec("SET GLOBAL tidb_enable_telemetry = OFF") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_VARIABLES_ADMIN privilege(s) for this operation") + + _, err = tk.Exec("SELECT @@session.tidb_force_priority") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_VARIABLES_ADMIN privilege(s) for this operation") + _, err = tk.Exec("SELECT @@global.tidb_enable_telemetry") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_VARIABLES_ADMIN privilege(s) for this operation") + + tk.Se.Auth(&auth.UserIdentity{ + Username: "svroot2", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + + tk.MustQuery(`SHOW VARIABLES LIKE 'tidb_force_priority'`).Check(testkit.Rows("tidb_force_priority NO_PRIORITY")) + tk.MustQuery(`SHOW GLOBAL VARIABLES LIKE 'tidb_enable_telemetry'`).Check(testkit.Rows("tidb_enable_telemetry ON")) + + // should not actually make any change. + tk.MustExec("SET tidb_force_priority = 'NO_PRIORITY'") + tk.MustExec("SET GLOBAL tidb_enable_telemetry = ON") + + tk.MustQuery(`SELECT @@session.tidb_force_priority`).Check(testkit.Rows("NO_PRIORITY")) + tk.MustQuery(`SELECT @@global.tidb_enable_telemetry`).Check(testkit.Rows("1")) +} + // TestViewDefiner tests that default roles are correctly applied in the algorithm definer // See: https://github.com/pingcap/tidb/issues/24414 func (s *testPrivilegeSuite) TestViewDefiner(c *C) { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 73a8ca0066450..574e649656205 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -575,7 +575,7 @@ var defaultSysVars = []*SysVar{ } return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(ForeignKeyChecks, originalValue) }}, - {Scope: ScopeNone, Name: Hostname, Value: ServerHostname}, + {Scope: ScopeNone, Name: Hostname, Value: DefHostname}, {Scope: ScopeSession, Name: Timestamp, Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetFilesystem, Value: "binary", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCharacterSet(normalizedValue, CharacterSetFilesystem) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index e416f9a695fc3..7fdccecb5a97e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -15,7 +15,6 @@ package variable import ( "math" - "os" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" @@ -694,7 +693,6 @@ var ( // DDLSlowOprThreshold is the threshold for ddl slow operations, uint is millisecond. DDLSlowOprThreshold uint32 = DefTiDBDDLSlowOprThreshold ForcePriority = int32(DefTiDBForcePriority) - ServerHostname, _ = os.Hostname() MaxOfMaxAllowedPacket uint64 = 1073741824 ExpensiveQueryTimeThreshold uint64 = DefTiDBExpensiveQueryTimeThreshold MinExpensiveQueryTimeThreshold uint64 = 10 // 10s diff --git a/tidb-server/main.go b/tidb-server/main.go index 6429ba960a0cb..05f4ecc59c25a 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -537,6 +537,9 @@ func setGlobalVars() { variable.SetSysVar(variable.TiDBSlowQueryFile, cfg.Log.SlowQueryFile) variable.SetSysVar(variable.TiDBIsolationReadEngines, strings.Join(cfg.IsolationRead.Engines, ", ")) variable.MemoryUsageAlarmRatio.Store(cfg.Performance.MemoryUsageAlarmRatio) + if hostname, err := os.Hostname(); err != nil { + variable.SetSysVar(variable.Hostname, hostname) + } if cfg.Security.EnableSEM { sem.Enable() diff --git a/util/sem/sem.go b/util/sem/sem.go index 8c3d2b456d991..d29d29b601559 100644 --- a/util/sem/sem.go +++ b/util/sem/sem.go @@ -14,6 +14,7 @@ package sem import ( + "os" "strings" "sync/atomic" @@ -70,6 +71,7 @@ var ( func Enable() { atomic.StoreInt32(&semEnabled, 1) variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.On) + variable.SetSysVar(variable.Hostname, variable.DefHostname) // write to log so users understand why some operations are weird. logutil.BgLogger().Info("tidb-server is operating with security enhanced mode (SEM) enabled") } @@ -79,6 +81,9 @@ func Enable() { func Disable() { atomic.StoreInt32(&semEnabled, 0) variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.Off) + if hostname, err := os.Hostname(); err != nil { + variable.SetSysVar(variable.Hostname, hostname) + } } // IsEnabled checks if Security Enhanced Mode (SEM) is enabled @@ -125,6 +130,33 @@ func IsInvisibleStatusVar(varName string) bool { return varName == tidbGCLeaderDesc } +// IsInvisibleSysVar returns true if the sysvar needs to be hidden +func IsInvisibleSysVar(varNameInLower string) bool { + switch varNameInLower { + case variable.TiDBDDLSlowOprThreshold, // ddl_slow_threshold + variable.TiDBAllowRemoveAutoInc, + variable.TiDBCheckMb4ValueInUTF8, + variable.TiDBConfig, + variable.TiDBEnableSlowLog, + variable.TiDBExpensiveQueryTimeThreshold, + variable.TiDBForcePriority, + variable.TiDBGeneralLog, + variable.TiDBMetricSchemaRangeDuration, + variable.TiDBMetricSchemaStep, + variable.TiDBOptWriteRowID, + variable.TiDBPProfSQLCPU, + variable.TiDBRecordPlanInSlowLog, + variable.TiDBSlowQueryFile, + variable.TiDBSlowLogThreshold, + variable.TiDBEnableCollectExecutionInfo, + variable.TiDBMemoryUsageAlarmRatio, + variable.TiDBEnableTelemetry, + variable.TiDBRowFormatVersion: + return true + } + return false +} + // IsRestrictedPrivilege returns true if the privilege shuld not be satisfied by SUPER // As most dynamic privileges are. func IsRestrictedPrivilege(privNameInUpper string) bool { diff --git a/util/sem/sem_test.go b/util/sem/sem_test.go index c303d2195c7f4..073a195139c37 100644 --- a/util/sem/sem_test.go +++ b/util/sem/sem_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/sessionctx/variable" . "github.com/pingcap/check" ) @@ -74,3 +75,27 @@ func (s *testSecurity) TestIsInvisibleStatusVar(c *C) { c.Assert(IsInvisibleStatusVar("ddl_schema_version"), IsFalse) c.Assert(IsInvisibleStatusVar("Ssl_version"), IsFalse) } + +func (s *testSecurity) TestIsInvisibleSysVar(c *C) { + c.Assert(IsInvisibleSysVar(variable.Hostname), IsFalse) // changes the value to default, but is not invisible + c.Assert(IsInvisibleSysVar(variable.TiDBEnableEnhancedSecurity), IsFalse) // should be able to see the mode is on. + + c.Assert(IsInvisibleSysVar(variable.TiDBAllowRemoveAutoInc), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBCheckMb4ValueInUTF8), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBConfig), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBEnableSlowLog), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBExpensiveQueryTimeThreshold), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBForcePriority), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBGeneralLog), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBMetricSchemaRangeDuration), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBMetricSchemaStep), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBOptWriteRowID), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBPProfSQLCPU), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBRecordPlanInSlowLog), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBSlowQueryFile), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBSlowLogThreshold), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBEnableCollectExecutionInfo), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBMemoryUsageAlarmRatio), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBEnableTelemetry), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBRowFormatVersion), IsTrue) +} From 2df8c3bfd1b76c2cb2ec7f2bdd31b45125568ab4 Mon Sep 17 00:00:00 2001 From: disksing Date: Mon, 17 May 2021 14:09:39 +0800 Subject: [PATCH 36/42] store/tikv: resolve ReplicaReadType dependencies (#24653) --- distsql/request_builder_test.go | 18 +++++++--------- executor/analyze.go | 5 ++--- executor/analyze_test.go | 4 ++-- executor/batch_point_get.go | 2 +- executor/point_get.go | 2 +- kv/kv.go | 2 +- kv/option.go | 17 +++++++++++++++ planner/optimize.go | 3 +-- session/session.go | 5 ++--- session/session_test.go | 13 ++++++----- sessionctx/variable/session.go | 10 ++++----- sessionctx/variable/sysvar.go | 6 +++--- sessionctx/variable/varsutil_test.go | 8 +++---- store/copr/coprocessor.go | 3 ++- store/driver/options/options.go | 32 ++++++++++++++++++++++++++++ store/driver/txn/snapshot.go | 5 +++-- store/driver/txn/txn_driver.go | 4 +++- store/tikv/kv/store_vars.go | 7 +++--- 18 files changed, 96 insertions(+), 50 deletions(-) create mode 100644 store/driver/options/options.go diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 18c1ee8fc24e4..ed921c391d6a3 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -324,7 +323,7 @@ func (s *testSuite) TestRequestBuilder1(c *C) { NotFillCache: false, SyncLog: false, Streaming: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -400,7 +399,7 @@ func (s *testSuite) TestRequestBuilder2(c *C) { NotFillCache: false, SyncLog: false, Streaming: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -447,7 +446,7 @@ func (s *testSuite) TestRequestBuilder3(c *C) { NotFillCache: false, SyncLog: false, Streaming: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -494,7 +493,7 @@ func (s *testSuite) TestRequestBuilder4(c *C) { Streaming: true, NotFillCache: false, SyncLog: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -577,10 +576,10 @@ func (s *testSuite) TestRequestBuilder6(c *C) { } func (s *testSuite) TestRequestBuilder7(c *C) { - for _, replicaRead := range []tikvstore.ReplicaReadType{ - tikvstore.ReplicaReadLeader, - tikvstore.ReplicaReadFollower, - tikvstore.ReplicaReadMixed, + for _, replicaRead := range []kv.ReplicaReadType{ + kv.ReplicaReadLeader, + kv.ReplicaReadFollower, + kv.ReplicaReadMixed, } { vars := variable.NewSessionVars() vars.SetReplicaRead(replicaRead) @@ -626,7 +625,6 @@ func (s *testSuite) TestRequestBuilder8(c *C) { IsolationLevel: 0, Priority: 0, MemTracker: (*memory.Tracker)(nil), - ReplicaRead: 0x1, SchemaVar: 10000, } c.Assert(actual, DeepEquals, expect) diff --git a/executor/analyze.go b/executor/analyze.go index fec55d870bf95..633f6e4dcc69e 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -45,7 +45,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -1322,7 +1321,7 @@ func (e *AnalyzeFastExec) handleScanIter(iter kv.Iterator) (scanKeysSize int, er func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) (keysSize int, err error) { snapshot := e.ctx.GetStore().GetSnapshot(kv.MaxVersion) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } for _, t := range e.scanTasks { iter, err := snapshot.Iter(kv.Key(t.StartKey), kv.Key(t.EndKey)) @@ -1345,7 +1344,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { snapshot.SetOption(kv.IsolationLevel, kv.RC) snapshot.SetOption(kv.Priority, kv.PriorityLow) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } rander := rand.New(rand.NewSource(e.randSeed)) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 39ec524dbacc4..b527875c1498b 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" @@ -38,7 +39,6 @@ import ( "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/table" @@ -121,7 +121,7 @@ func (s *testSuite1) TestAnalyzeReplicaReadFollower(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") ctx := tk.Se.(sessionctx.Context) - ctx.GetSessionVars().SetReplicaRead(tikvstore.ReplicaReadFollower) + ctx.GetSessionVars().SetReplicaRead(kv.ReplicaReadFollower) tk.MustExec("analyze table t") } diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 23debe37404ee..2137884c69745 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -117,7 +117,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness diff --git a/executor/point_get.go b/executor/point_get.go index 8857a4d253fd0..fc8326555bf01 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -148,7 +148,7 @@ func (e *PointGetExecutor) Open(context.Context) error { e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - e.snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + e.snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } e.snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness diff --git a/kv/kv.go b/kv/kv.go index e5ab4eed6f812..572fe104024bc 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -273,7 +273,7 @@ type Request struct { // call would not corresponds to a whole region result. Streaming bool // ReplicaRead is used for reading data from replicas, only follower is supported at this time. - ReplicaRead tikvstore.ReplicaReadType + ReplicaRead ReplicaReadType // StoreType represents this request is sent to the which type of store. StoreType StoreType // Cacheable is true if the request can be cached. Currently only deterministic DAG requests can be cached. diff --git a/kv/option.go b/kv/option.go index 5b04dfba06c95..dc0d700666d5a 100644 --- a/kv/option.go +++ b/kv/option.go @@ -60,3 +60,20 @@ const ( // MatchStoreLabels indicates the labels the store should be matched MatchStoreLabels ) + +// ReplicaReadType is the type of replica to read data from +type ReplicaReadType byte + +const ( + // ReplicaReadLeader stands for 'read from leader'. + ReplicaReadLeader ReplicaReadType = iota + // ReplicaReadFollower stands for 'read from follower'. + ReplicaReadFollower + // ReplicaReadMixed stands for 'read from leader and follower and learner'. + ReplicaReadMixed +) + +// IsFollowerRead checks if follower is going to be used to read data. +func (r ReplicaReadType) IsFollowerRead() bool { + return r != ReplicaReadLeader +} diff --git a/planner/optimize.go b/planner/optimize.go index 4e0f7334ae5ae..ec9bfef67d0a7 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/logutil" @@ -533,7 +532,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin warns = append(warns, warn) } stmtHints.HasReplicaReadHint = true - stmtHints.ReplicaRead = byte(tikvstore.ReplicaReadFollower) + stmtHints.ReplicaRead = byte(kv.ReplicaReadFollower) } // Handle MAX_EXECUTION_TIME if maxExecutionTimeCnt != 0 { diff --git a/session/session.go b/session/session.go index 8fc7c4c37eac8..af3f41c863dc0 100644 --- a/session/session.go +++ b/session/session.go @@ -69,7 +69,6 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" @@ -1891,7 +1890,7 @@ func (s *session) Txn(active bool) (kv.Transaction, error) { s.sessionVars.TxnCtx.CouldRetry = s.isTxnRetryable() s.txn.SetVars(s.sessionVars.KVVars) if s.sessionVars.GetReplicaRead().IsFollowerRead() { - s.txn.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + s.txn.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } } return &s.txn, nil @@ -1955,7 +1954,7 @@ func (s *session) NewTxn(ctx context.Context) error { } txn.SetVars(s.sessionVars.KVVars) if s.GetSessionVars().GetReplicaRead().IsFollowerRead() { - txn.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + txn.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() diff --git a/session/session_test.go b/session/session_test.go index a8861dfddd79c..4870215f33c9e 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -51,7 +51,6 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" @@ -3064,11 +3063,11 @@ func (s *testSessionSuite2) TestReplicaRead(c *C) { tk := testkit.NewTestKit(c, s.store) tk.Se, err = session.CreateSession4Test(s.store) c.Assert(err, IsNil) - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadLeader) tk.MustExec("set @@tidb_replica_read = 'follower';") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) tk.MustExec("set @@tidb_replica_read = 'leader';") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadLeader) } func (s *testSessionSuite3) TestIsolationRead(c *C) { @@ -3153,12 +3152,12 @@ func (s *testSessionSuite2) TestStmtHints(c *C) { c.Assert(tk.Se.GetSessionVars().GetEnableCascadesPlanner(), IsTrue) // Test READ_CONSISTENT_REPLICA hint - tk.Se.GetSessionVars().SetReplicaRead(tikvstore.ReplicaReadLeader) + tk.Se.GetSessionVars().SetReplicaRead(kv.ReplicaReadLeader) tk.MustExec("select /*+ READ_CONSISTENT_REPLICA() */ 1;") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) tk.MustExec("select /*+ READ_CONSISTENT_REPLICA(), READ_CONSISTENT_REPLICA() */ 1;") c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) } func (s *testSessionSuite3) TestPessimisticLockOnPartition(c *C) { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 97597997b36f5..0c6c74d90a26d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -723,7 +723,7 @@ type SessionVars struct { enableIndexMerge bool // replicaRead is used for reading data from replicas, only follower is supported at this time. - replicaRead tikvstore.ReplicaReadType + replicaRead kv.ReplicaReadType // IsolationReadEngines is used to isolation read, tidb only read from the stores whose engine type is in the engines. IsolationReadEngines map[kv.StoreType]struct{} @@ -1029,7 +1029,7 @@ func NewSessionVars() *SessionVars { WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, enableIndexMerge: false, EnableNoopFuncs: DefTiDBEnableNoopFuncs, - replicaRead: tikvstore.ReplicaReadLeader, + replicaRead: kv.ReplicaReadLeader, AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, UsePlanBaselines: DefTiDBUsePlanBaselines, EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, @@ -1179,15 +1179,15 @@ func (s *SessionVars) SetEnableIndexMerge(val bool) { } // GetReplicaRead get ReplicaRead from sql hints and SessionVars.replicaRead. -func (s *SessionVars) GetReplicaRead() tikvstore.ReplicaReadType { +func (s *SessionVars) GetReplicaRead() kv.ReplicaReadType { if s.StmtCtx.HasReplicaReadHint { - return tikvstore.ReplicaReadType(s.StmtCtx.ReplicaRead) + return kv.ReplicaReadType(s.StmtCtx.ReplicaRead) } return s.replicaRead } // SetReplicaRead set SessionVars.replicaRead. -func (s *SessionVars) SetReplicaRead(val tikvstore.ReplicaReadType) { +func (s *SessionVars) SetReplicaRead(val kv.ReplicaReadType) { s.replicaRead = val } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 574e649656205..98518fe4af0f0 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1246,11 +1246,11 @@ var defaultSysVars = []*SysVar{ }}, {Scope: ScopeSession, Name: TiDBReplicaRead, Value: "leader", Type: TypeEnum, PossibleValues: []string{"leader", "follower", "leader-and-follower"}, SetSession: func(s *SessionVars, val string) error { if strings.EqualFold(val, "follower") { - s.SetReplicaRead(tikvstore.ReplicaReadFollower) + s.SetReplicaRead(kv.ReplicaReadFollower) } else if strings.EqualFold(val, "leader-and-follower") { - s.SetReplicaRead(tikvstore.ReplicaReadMixed) + s.SetReplicaRead(kv.ReplicaReadMixed) } else if strings.EqualFold(val, "leader") || len(val) == 0 { - s.SetReplicaRead(tikvstore.ReplicaReadLeader) + s.SetReplicaRead(kv.ReplicaReadLeader) } return nil }}, diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 90507f36539d6..1d8e629b7df4a 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/testleak" ) @@ -431,19 +431,19 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "follower") - c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(v.GetReplicaRead(), Equals, kv.ReplicaReadFollower) err = SetSessionSystemVar(v, TiDBReplicaRead, "leader") c.Assert(err, IsNil) val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "leader") - c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) + c.Assert(v.GetReplicaRead(), Equals, kv.ReplicaReadLeader) err = SetSessionSystemVar(v, TiDBReplicaRead, "leader-and-follower") c.Assert(err, IsNil) val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "leader-and-follower") - c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadMixed) + c.Assert(v.GetReplicaRead(), Equals, kv.ReplicaReadMixed) err = SetSessionSystemVar(v, TiDBEnableStmtSummary, "ON") c.Assert(err, IsNil) diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index e9d9e6b8f1ebb..989a6d835ce0f 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -37,6 +37,7 @@ import ( tidbmetrics "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/driver/options" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -697,7 +698,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch } } - req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, worker.req.ReplicaRead, &worker.replicaReadSeed, kvrpcpb.Context{ + req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, options.GetTiKVReplicaReadType(worker.req.ReplicaRead), &worker.replicaReadSeed, kvrpcpb.Context{ IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel), Priority: priorityToPB(worker.req.Priority), NotFillCache: worker.req.NotFillCache, diff --git a/store/driver/options/options.go b/store/driver/options/options.go new file mode 100644 index 0000000000000..dc16f7793ed91 --- /dev/null +++ b/store/driver/options/options.go @@ -0,0 +1,32 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package options + +import ( + "github.com/pingcap/tidb/kv" + storekv "github.com/pingcap/tidb/store/tikv/kv" +) + +// GetTiKVReplicaReadType maps kv.ReplicaReadType to tikv/kv.ReplicaReadType. +func GetTiKVReplicaReadType(t kv.ReplicaReadType) storekv.ReplicaReadType { + switch t { + case kv.ReplicaReadLeader: + return storekv.ReplicaReadLeader + case kv.ReplicaReadFollower: + return storekv.ReplicaReadFollower + case kv.ReplicaReadMixed: + return storekv.ReplicaReadMixed + } + return 0 +} diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 405067f5e082b..6692f45a749a3 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/driver/options" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" ) type tikvSnapshot struct { @@ -76,7 +76,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { case kv.SnapshotTS: s.KVSnapshot.SetSnapshotTS(val.(uint64)) case kv.ReplicaRead: - s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) + t := options.GetTiKVReplicaReadType(val.(kv.ReplicaReadType)) + s.KVSnapshot.SetReplicaRead(t) case kv.SampleStep: s.KVSnapshot.SetSampleStep(val.(uint32)) case kv.TaskID: diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 7c940561f82fe..7f05f80139c12 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/binloginfo" derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/driver/options" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -143,7 +144,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { case kv.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) case kv.ReplicaRead: - txn.KVTxn.GetSnapshot().SetReplicaRead(val.(tikvstore.ReplicaReadType)) + t := options.GetTiKVReplicaReadType(val.(kv.ReplicaReadType)) + txn.KVTxn.GetSnapshot().SetReplicaRead(t) case kv.TaskID: txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case kv.InfoSchema: diff --git a/store/tikv/kv/store_vars.go b/store/tikv/kv/store_vars.go index 5f65f927bffb9..02d87018213a9 100644 --- a/store/tikv/kv/store_vars.go +++ b/store/tikv/kv/store_vars.go @@ -25,15 +25,14 @@ type ReplicaReadType byte const ( // ReplicaReadLeader stands for 'read from leader'. - ReplicaReadLeader ReplicaReadType = 1 << iota + ReplicaReadLeader ReplicaReadType = iota // ReplicaReadFollower stands for 'read from follower'. ReplicaReadFollower // ReplicaReadMixed stands for 'read from leader and follower and learner'. ReplicaReadMixed ) -// IsFollowerRead checks if leader is going to be used to read data. +// IsFollowerRead checks if follower is going to be used to read data. func (r ReplicaReadType) IsFollowerRead() bool { - // In some cases the default value is 0, which should be treated as `ReplicaReadLeader`. - return r != ReplicaReadLeader && r != 0 + return r != ReplicaReadLeader } From c20d496dd780f1f3454d84716cfa60f4ac3b7eb2 Mon Sep 17 00:00:00 2001 From: rebelice Date: Mon, 17 May 2021 14:23:39 +0800 Subject: [PATCH 37/42] executor: add test cases about partition table with `expression` (#24628) --- executor/partition_table_test.go | 255 ++++++++++++++++++++++++++++++- util/testkit/testkit.go | 7 +- 2 files changed, 253 insertions(+), 9 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index cf53b7cd82688..e87e1044278e3 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -573,12 +573,12 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { } func createTable4DynamicPruneModeTestWithExpression(tk *testkit.TestKit) { - tk.MustExec("create table trange(a int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11));") - tk.MustExec("create table thash(a int) partition by hash(a) partitions 4;") - tk.MustExec("create table t(a int)") - tk.MustExec("insert into trange values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") - tk.MustExec("insert into thash values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") - tk.MustExec("insert into t values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") + tk.MustExec("create table trange(a int, b int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11));") + tk.MustExec("create table thash(a int, b int) partition by hash(a) partitions 4;") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into trange values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1);") + tk.MustExec("insert into thash values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1);") + tk.MustExec("insert into t values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1);") tk.MustExec("set session tidb_partition_prune_mode='dynamic'") tk.MustExec("analyze table trange") tk.MustExec("analyze table thash") @@ -590,7 +590,7 @@ type testData4Expression struct { partitions []string } -func (s *partitionTableSuite) TestDynamicPruneModeWithEqualExpression(c *C) { +func (s *partitionTableSuite) TestDynamicPruneModeWithExpression(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop database if exists db_equal_expression") tk.MustExec("create database db_equal_expression") @@ -627,6 +627,247 @@ func (s *partitionTableSuite) TestDynamicPruneModeWithEqualExpression(c *C) { "p0", }, }, + { + sql: "select * from %s where b is NULL", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a > -1", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a >= 4 and a <= 5", + partitions: []string{ + "p1,p2", + "p0,p1", + }, + }, + { + sql: "select * from %s where a > 10", + partitions: []string{ + "dual", + "all", + }, + }, + { + sql: "select * from %s where a >=2 and a <= 3", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a between 2 and 3", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a < 2", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where a <= 3", + partitions: []string{ + "p0,p1", + "all", + }, + }, + { + sql: "select * from %s where a in (2, 3)", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a in (1, 5)", + partitions: []string{ + "p0,p2", + "p1", + }, + }, + { + sql: "select * from %s where a not in (1, 5)", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a = 2 and a = 2", + partitions: []string{ + "p0", + "p2", + }, + }, + { + sql: "select * from %s where a = 2 and a = 3", + partitions: []string{ + // This means that we have no partition-read plan + "", + "", + }, + }, + { + sql: "select * from %s where a < 2 and a > 0", + partitions: []string{ + "p0", + "p1", + }, + }, + { + sql: "select * from %s where a < 2 and a < 3", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where a > 1 and a > 2", + partitions: []string{ + "p1,p2", + "all", + }, + }, + { + sql: "select * from %s where a = 2 or a = 3", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a = 2 or a in (3)", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a = 2 or a > 3", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a = 2 or a <= 1", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where a = 2 or a between 2 and 2", + partitions: []string{ + "p0", + "p2", + }, + }, + { + sql: "select * from %s where a != 2", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a != 2 and a > 4", + partitions: []string{ + "p2", + "all", + }, + }, + { + sql: "select * from %s where a != 2 and a != 3", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a != 2 and a = 3", + partitions: []string{ + "p1", + "p3", + }, + }, + { + sql: "select * from %s where not (a = 2)", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where not (a > 2)", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where not (a < 2)", + partitions: []string{ + "all", + "all", + }, + }, + // cases that partition pruning can not work + { + sql: "select * from %s where a + 1 > 4", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a - 1 > 0", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a * 2 < 0", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a << 1 < 0", + partitions: []string{ + "all", + "all", + }, + }, + // comparison between int column and string column + { + sql: "select * from %s where a > '10'", + partitions: []string{ + "dual", + "all", + }, + }, + { + sql: "select * from %s where a > '10ab'", + partitions: []string{ + "dual", + "all", + }, + }, } for _, t := range tests { diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index d6d6e41bb6e9f..7cacaf211375e 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -258,9 +258,12 @@ func (tk *TestKit) MustNoGlobalStats(table string) bool { // MustPartition checks if the result execution plan must read specific partitions. func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) *Result { rs := tk.MustQuery("explain "+sql, args...) - ok := false + ok := len(partitions) == 0 for i := range rs.rows { - if strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 { + if len(partitions) == 0 && strings.Contains(rs.rows[i][3], "partition:") { + ok = false + } + if len(partitions) != 0 && strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 { ok = true } } From ae36fbdb316ffd955cc72dcc8731ba1fa8b9919a Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 17 May 2021 14:35:39 +0800 Subject: [PATCH 38/42] tablecodec: fix write wrong prefix index value when collation is ascii_bin/latin1_bin (#24578) --- executor/write_test.go | 19 +++++++++++++++++++ tablecodec/tablecodec.go | 3 ++- 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/executor/write_test.go b/executor/write_test.go index b832e52a9935c..cf7a51985a450 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -3930,6 +3930,25 @@ func (s *testSerialSuite) TestIssue20840(c *C) { tk.MustExec("drop table t1") } +func (s *testSerialSuite) TestIssueInsertPrefixIndexForNonUTF8Collation(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2, t3") + tk.MustExec("create table t1 ( c_int int, c_str varchar(40) character set ascii collate ascii_bin, primary key(c_int, c_str(8)) clustered , unique key(c_str))") + tk.MustExec("create table t2 ( c_int int, c_str varchar(40) character set latin1 collate latin1_bin, primary key(c_int, c_str(8)) clustered , unique key(c_str))") + tk.MustExec("insert into t1 values (3, 'fervent brattain')") + tk.MustExec("insert into t2 values (3, 'fervent brattain')") + tk.MustExec("admin check table t1") + tk.MustExec("admin check table t2") + + tk.MustExec("create table t3 (x varchar(40) CHARACTER SET ascii COLLATE ascii_bin, UNIQUE KEY uk(x(4)))") + tk.MustExec("insert into t3 select 'abc '") + tk.MustGetErrCode("insert into t3 select 'abc d'", 1062) +} + func (s *testSerialSuite) TestIssue22496(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index 215f8d05c27fa..de766831bc245 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -1292,6 +1292,7 @@ func TruncateIndexValue(v *types.Datum, idxCol *model.IndexColumn, tblCol *model if notStringType { return } + originalKind := v.Kind() isUTF8Charset := tblCol.Charset == charset.CharsetUTF8 || tblCol.Charset == charset.CharsetUTF8MB4 if isUTF8Charset && utf8.RuneCount(v.GetBytes()) > idxCol.Length { rs := bytes.Runes(v.GetBytes()) @@ -1303,7 +1304,7 @@ func TruncateIndexValue(v *types.Datum, idxCol *model.IndexColumn, tblCol *model } } else if !isUTF8Charset && len(v.GetBytes()) > idxCol.Length { v.SetBytes(v.GetBytes()[:idxCol.Length]) - if v.Kind() == types.KindString { + if originalKind == types.KindString { v.SetString(v.GetString(), tblCol.Collate) } } From 2ca98e393cb6873727a177cb5c974fa8d3ca61c5 Mon Sep 17 00:00:00 2001 From: xhe Date: Mon, 17 May 2021 19:53:39 +0800 Subject: [PATCH 39/42] *: compatibility with staleread (#24285) --- ddl/column_change_test.go | 21 ++-- ddl/column_test.go | 3 +- ddl/ddl.go | 20 ++-- ddl/ddl_api.go | 30 +++--- ddl/ddl_test.go | 5 + ddl/ddl_worker_test.go | 10 +- ddl/index_change_test.go | 22 ++-- ddl/options.go | 16 +-- ddl/options_test.go | 6 +- ddl/partition.go | 81 +++++++-------- ddl/reorg_test.go | 2 +- ddl/restart_test.go | 4 +- ddl/schema.go | 8 +- ddl/schema_test.go | 4 +- ddl/stat_test.go | 2 +- ddl/table.go | 54 +++++----- ddl/table_test.go | 2 +- ddl/util/syncer_test.go | 7 ++ domain/domain.go | 182 +++++++++++++++------------------- domain/domain_test.go | 2 +- executor/simple.go | 21 ---- executor/stale_txn_test.go | 91 ++++------------- infoschema/builder.go | 47 +++++---- infoschema/cache.go | 95 ++++++++++++++++++ infoschema/cache_test.go | 119 ++++++++++++++++++++++ infoschema/infoschema.go | 36 ------- infoschema/infoschema_test.go | 54 ++-------- metrics/domain.go | 13 +++ metrics/metrics.go | 1 + owner/manager_test.go | 13 +++ session/session.go | 9 +- session/session_test.go | 10 +- 32 files changed, 535 insertions(+), 455 deletions(-) create mode 100644 infoschema/cache.go create mode 100644 infoschema/cache_test.go diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index 94e8787a2bdc4..6bd5a94f7235e 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -47,15 +47,18 @@ type testColumnChangeSuite struct { func (s *testColumnChangeSuite) SetUpSuite(c *C) { SetWaitTimeWhenErrorOccurred(1 * time.Microsecond) s.store = testCreateStore(c, "test_column_change") - s.dbInfo = &model.DBInfo{ - Name: model.NewCIStr("test_column_change"), - ID: 1, - } - err := kv.RunInNewTxn(context.Background(), s.store, true, func(ctx context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) - return errors.Trace(t.CreateDatabase(s.dbInfo)) - }) - c.Check(err, IsNil) + d := testNewDDLAndStart( + context.Background(), + c, + WithStore(s.store), + WithLease(testLease), + ) + defer func() { + err := d.Stop() + c.Assert(err, IsNil) + }() + s.dbInfo = testSchemaInfo(c, d, "test_index_change") + testCreateSchema(c, testNewContext(d), d, s.dbInfo) } func (s *testColumnChangeSuite) TearDownSuite(c *C) { diff --git a/ddl/column_test.go b/ddl/column_test.go index 862fb4aa04c59..f3eaa26d22385 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -54,8 +54,7 @@ func (s *testColumnSuite) SetUpSuite(c *C) { s.dbInfo = testSchemaInfo(c, d, "test_column") testCreateSchema(c, testNewContext(d), d, s.dbInfo) - err := d.Stop() - c.Assert(err, IsNil) + c.Assert(d.Stop(), IsNil) } func (s *testColumnSuite) TearDownSuite(c *C) { diff --git a/ddl/ddl.go b/ddl/ddl.go index 6f20fe25ccc07..9eb05b86741ed 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -202,7 +202,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 + infoCache *infoschema.InfoCache statsHandle *handle.Handle tableLockCkr util.DeadTableLockChecker etcdCli *clientv3.Client @@ -282,6 +282,15 @@ func newDDL(ctx context.Context, options ...Option) *ddl { deadLockCkr = util.NewDeadTableLockChecker(etcdCli) } + // TODO: make store and infoCache explicit arguments + // these two should be ensured to exist + if opt.Store == nil { + panic("store should not be nil") + } + if opt.InfoCache == nil { + panic("infoCache should not be nil") + } + ddlCtx := &ddlCtx{ uuid: id, store: opt.Store, @@ -290,7 +299,7 @@ func newDDL(ctx context.Context, options ...Option) *ddl { ownerManager: manager, schemaSyncer: syncer, binlogCli: binloginfo.GetPumpsClient(), - infoHandle: opt.InfoHandle, + infoCache: opt.InfoCache, tableLockCkr: deadLockCkr, etcdCli: opt.EtcdCli, } @@ -411,7 +420,7 @@ func (d *ddl) GetLease() time.Duration { // Please don't use this function, it is used by TestParallelDDLBeforeRunDDLJob to intercept the calling of d.infoHandle.Get(), use d.infoHandle.Get() instead. // Otherwise, the TestParallelDDLBeforeRunDDLJob will hang up forever. func (d *ddl) GetInfoSchemaWithInterceptor(ctx sessionctx.Context) infoschema.InfoSchema { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() d.mu.RLock() defer d.mu.RUnlock() @@ -649,10 +658,7 @@ func (d *ddl) startCleanDeadTableLock() { if !d.ownerManager.IsOwner() { continue } - if d.infoHandle == nil || !d.infoHandle.IsValid() { - continue - } - deadLockTables, err := d.tableLockCkr.GetDeadLockedTables(d.ctx, d.infoHandle.Get().AllSchemas()) + deadLockTables, err := d.tableLockCkr.GetDeadLockedTables(d.ctx, d.infoCache.GetLatest().AllSchemas()) if err != nil { logutil.BgLogger().Info("[ddl] get dead table lock failed.", zap.Error(err)) continue diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index a3f8bb7f9c622..d0289dc19e39f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2367,7 +2367,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A return errors.Trace(err) } - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() if is.TableIsView(ident.Schema, ident.Name) || is.TableIsSequence(ident.Schema, ident.Name) { return ErrWrongObject.GenWithStackByArgs(ident.Schema, ident.Name, "BASE TABLE") } @@ -2898,7 +2898,7 @@ func (d *ddl) AddColumns(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alte // AddTablePartitions will add a new partition to the table. func (d *ddl) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) @@ -2959,7 +2959,7 @@ func (d *ddl) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, spec * // CoalescePartitions coalesce partitions can be used with a table that is partitioned by hash or key to reduce the number of partitions by number. func (d *ddl) CoalescePartitions(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) @@ -2991,7 +2991,7 @@ func (d *ddl) CoalescePartitions(ctx sessionctx.Context, ident ast.Ident, spec * } func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) @@ -3039,7 +3039,7 @@ func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, sp } func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) @@ -3752,7 +3752,7 @@ func processAndCheckDefaultValueAndColumn(ctx sessionctx.Context, col *table.Col func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, originalColName model.CIStr, spec *ast.AlterTableSpec) (*model.Job, error) { specNewColumn := spec.NewColumns[0] - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return nil, errors.Trace(infoschema.ErrDatabaseNotExists) @@ -4203,7 +4203,7 @@ func (d *ddl) ModifyColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Al func (d *ddl) AlterColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { specNewColumn := spec.NewColumns[0] - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name) @@ -4257,7 +4257,7 @@ func (d *ddl) AlterColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Alt // AlterTableComment updates the table comment information. func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) @@ -4310,7 +4310,7 @@ func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Iden return ErrUnknownCharacterSet.GenWithStackByArgs(toCharset) } - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) @@ -4471,7 +4471,7 @@ func (d *ddl) AlterTableDropStatistics(ctx sessionctx.Context, ident ast.Ident, // UpdateTableReplicaInfo updates the table flash replica infos. func (d *ddl) UpdateTableReplicaInfo(ctx sessionctx.Context, physicalID int64, available bool) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() tb, ok := is.TableByID(physicalID) if !ok { tb, _, _ = is.FindTableByPartitionID(physicalID) @@ -4574,7 +4574,7 @@ func checkAlterTableCharset(tblInfo *model.TableInfo, dbInfo *model.DBInfo, toCh // In TiDB, indexes are case-insensitive (so index 'a' and 'A" are considered the same index), // but index names are case-sensitive (we can rename index 'a' to 'A') func (d *ddl) RenameIndex(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) @@ -5232,7 +5232,7 @@ func buildFKInfo(fkName model.CIStr, keys []*ast.IndexPartSpecification, refer * } func (d *ddl) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model.CIStr, keys []*ast.IndexPartSpecification, refer *ast.ReferenceDef) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ti.Schema) @@ -5264,7 +5264,7 @@ func (d *ddl) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName mode } func (d *ddl) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model.CIStr) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ti.Schema) @@ -5290,7 +5290,7 @@ func (d *ddl) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model. } func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CIStr, ifExists bool) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists) @@ -6036,7 +6036,7 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, return errors.Trace(err) } - oldBundle := infoschema.GetBundle(d.infoHandle.Get(), []int64{partitionID, meta.ID, schema.ID}) + oldBundle := infoschema.GetBundle(d.infoCache.GetLatest(), []int64{partitionID, meta.ID, schema.ID}) oldBundle.ID = placement.GroupID(partitionID) diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index b77c3300c2700..79635bfc0933b 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" @@ -86,6 +87,10 @@ func TestT(t *testing.T) { } func testNewDDLAndStart(ctx context.Context, c *C, options ...Option) *ddl { + // init infoCache and a stub infoSchema + ic := infoschema.NewCache(2) + ic.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) + options = append(options, WithInfoCache(ic)) d := newDDL(ctx, options...) err := d.Start(nil) c.Assert(err, IsNil) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 6e745820b04b9..72fef7c96c19e 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -247,7 +247,7 @@ func (s *testDDLSuite) TestTableError(c *C) { // Schema ID is wrong, so dropping table is failed. doDDLJobErr(c, -1, 1, model.ActionDropTable, nil, ctx, d) // Table ID is wrong, so dropping table is failed. - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_ddl") testCreateSchema(c, testNewContext(d), d, dbInfo) job := doDDLJobErr(c, dbInfo.ID, -1, model.ActionDropTable, nil, ctx, d) @@ -295,7 +295,7 @@ func (s *testDDLSuite) TestViewError(c *C) { c.Assert(err, IsNil) }() ctx := testNewContext(d) - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_ddl") testCreateSchema(c, testNewContext(d), d, dbInfo) // Table ID or schema ID is wrong, so getting table is failed. @@ -363,7 +363,7 @@ func (s *testDDLSuite) TestForeignKeyError(c *C) { doDDLJobErr(c, -1, 1, model.ActionAddForeignKey, nil, ctx, d) doDDLJobErr(c, -1, 1, model.ActionDropForeignKey, nil, ctx, d) - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_ddl") tblInfo := testTableInfo(c, d, "t", 3) testCreateSchema(c, ctx, d, dbInfo) testCreateTable(c, ctx, d, dbInfo, tblInfo) @@ -393,7 +393,7 @@ func (s *testDDLSuite) TestIndexError(c *C) { doDDLJobErr(c, -1, 1, model.ActionAddIndex, nil, ctx, d) doDDLJobErr(c, -1, 1, model.ActionDropIndex, nil, ctx, d) - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_ddl") tblInfo := testTableInfo(c, d, "t", 3) testCreateSchema(c, ctx, d, dbInfo) testCreateTable(c, ctx, d, dbInfo, tblInfo) @@ -435,7 +435,7 @@ func (s *testDDLSuite) TestColumnError(c *C) { }() ctx := testNewContext(d) - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_ddl") tblInfo := testTableInfo(c, d, "t", 3) testCreateSchema(c, ctx, d, dbInfo) testCreateTable(c, ctx, d, dbInfo, tblInfo) diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index dfdfc7111c372..6a34599137c10 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -38,15 +37,18 @@ type testIndexChangeSuite struct { func (s *testIndexChangeSuite) SetUpSuite(c *C) { s.store = testCreateStore(c, "test_index_change") - s.dbInfo = &model.DBInfo{ - Name: model.NewCIStr("test_index_change"), - ID: 1, - } - err := kv.RunInNewTxn(context.Background(), s.store, true, func(ctx context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) - return errors.Trace(t.CreateDatabase(s.dbInfo)) - }) - c.Check(err, IsNil, Commentf("err %v", errors.ErrorStack(err))) + d := testNewDDLAndStart( + context.Background(), + c, + WithStore(s.store), + WithLease(testLease), + ) + defer func() { + err := d.Stop() + c.Assert(err, IsNil) + }() + s.dbInfo = testSchemaInfo(c, d, "test_index_change") + testCreateSchema(c, testNewContext(d), d, s.dbInfo) } func (s *testIndexChangeSuite) TearDownSuite(c *C) { diff --git a/ddl/options.go b/ddl/options.go index 8613a8e9affa9..9238a7c8542ff 100644 --- a/ddl/options.go +++ b/ddl/options.go @@ -26,11 +26,11 @@ type Option func(*Options) // Options represents all the options of the DDL module needs type Options struct { - EtcdCli *clientv3.Client - Store kv.Storage - InfoHandle *infoschema.Handle - Hook Callback - Lease time.Duration + EtcdCli *clientv3.Client + Store kv.Storage + InfoCache *infoschema.InfoCache + Hook Callback + Lease time.Duration } // WithEtcdClient specifies the `clientv3.Client` of DDL used to request the etcd service @@ -47,10 +47,10 @@ func WithStore(store kv.Storage) Option { } } -// WithInfoHandle specifies the `infoschema.Handle` -func WithInfoHandle(ih *infoschema.Handle) Option { +// WithInfoCache specifies the `infoschema.InfoCache` +func WithInfoCache(ic *infoschema.InfoCache) Option { return func(options *Options) { - options.InfoHandle = ih + options.InfoCache = ic } } diff --git a/ddl/options_test.go b/ddl/options_test.go index 294d68731e4c3..22a451d622c71 100644 --- a/ddl/options_test.go +++ b/ddl/options_test.go @@ -33,14 +33,14 @@ func (s *ddlOptionsSuite) TestOptions(c *C) { callback := &ddl.BaseCallback{} lease := time.Second * 3 store := &mock.Store{} - infoHandle := infoschema.NewHandle(store) + infoHandle := infoschema.NewCache(16) options := []ddl.Option{ ddl.WithEtcdClient(client), ddl.WithHook(callback), ddl.WithLease(lease), ddl.WithStore(store), - ddl.WithInfoHandle(infoHandle), + ddl.WithInfoCache(infoHandle), } opt := &ddl.Options{} @@ -52,5 +52,5 @@ func (s *ddlOptionsSuite) TestOptions(c *C) { c.Assert(opt.Hook, Equals, callback) c.Assert(opt.Lease, Equals, lease) c.Assert(opt.Store, Equals, store) - c.Assert(opt.InfoHandle, Equals, infoHandle) + c.Assert(opt.InfoCache, Equals, infoHandle) } diff --git a/ddl/partition.go b/ddl/partition.go index 0cafa9d2ff525..4e55ec1779e21 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -911,18 +911,15 @@ func getTableInfoWithDroppingPartitions(t *model.TableInfo) *model.TableInfo { } func dropRuleBundles(d *ddlCtx, physicalTableIDs []int64) error { - if d.infoHandle != nil && d.infoHandle.IsValid() { - bundles := make([]*placement.Bundle, 0, len(physicalTableIDs)) - for _, ID := range physicalTableIDs { - oldBundle, ok := d.infoHandle.Get().BundleByName(placement.GroupID(ID)) - if ok && !oldBundle.IsEmpty() { - bundles = append(bundles, placement.BuildPlacementDropBundle(ID)) - } + bundles := make([]*placement.Bundle, 0, len(physicalTableIDs)) + for _, ID := range physicalTableIDs { + oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(ID)) + if ok && !oldBundle.IsEmpty() { + bundles = append(bundles, placement.BuildPlacementDropBundle(ID)) } - err := infosync.PutRuleBundles(context.TODO(), bundles) - return err } - return nil + err := infosync.PutRuleBundles(context.TODO(), bundles) + return err } // onDropTablePartition deletes old partition meta. @@ -1095,22 +1092,20 @@ func onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, e } } - if d.infoHandle != nil && d.infoHandle.IsValid() { - bundles := make([]*placement.Bundle, 0, len(oldIDs)) + bundles := make([]*placement.Bundle, 0, len(oldIDs)) - for i, oldID := range oldIDs { - oldBundle, ok := d.infoHandle.Get().BundleByName(placement.GroupID(oldID)) - if ok && !oldBundle.IsEmpty() { - bundles = append(bundles, placement.BuildPlacementDropBundle(oldID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newPartitions[i].ID)) - } + for i, oldID := range oldIDs { + oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(oldID)) + if ok && !oldBundle.IsEmpty() { + bundles = append(bundles, placement.BuildPlacementDropBundle(oldID)) + bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newPartitions[i].ID)) } + } - err = infosync.PutRuleBundles(context.TODO(), bundles) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Wrapf(err, "failed to notify PD the placement rules") - } + err = infosync.PutRuleBundles(context.TODO(), bundles) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } newIDs := make([]int64, len(oldIDs)) @@ -1299,27 +1294,25 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo // the follow code is a swap function for rules of two partitions // though partitions has exchanged their ID, swap still take effect - if d.infoHandle != nil && d.infoHandle.IsValid() { - bundles := make([]*placement.Bundle, 0, 2) - ptBundle, ptOK := d.infoHandle.Get().BundleByName(placement.GroupID(partDef.ID)) - ptOK = ptOK && !ptBundle.IsEmpty() - ntBundle, ntOK := d.infoHandle.Get().BundleByName(placement.GroupID(nt.ID)) - ntOK = ntOK && !ntBundle.IsEmpty() - if ptOK && ntOK { - bundles = append(bundles, placement.BuildPlacementCopyBundle(ptBundle, nt.ID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(ntBundle, partDef.ID)) - } else if ptOK { - bundles = append(bundles, placement.BuildPlacementDropBundle(partDef.ID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(ptBundle, nt.ID)) - } else if ntOK { - bundles = append(bundles, placement.BuildPlacementDropBundle(nt.ID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(ntBundle, partDef.ID)) - } - err = infosync.PutRuleBundles(context.TODO(), bundles) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Wrapf(err, "failed to notify PD the placement rules") - } + bundles := make([]*placement.Bundle, 0, 2) + ptBundle, ptOK := d.infoCache.GetLatest().BundleByName(placement.GroupID(partDef.ID)) + ptOK = ptOK && !ptBundle.IsEmpty() + ntBundle, ntOK := d.infoCache.GetLatest().BundleByName(placement.GroupID(nt.ID)) + ntOK = ntOK && !ntBundle.IsEmpty() + if ptOK && ntOK { + bundles = append(bundles, placement.BuildPlacementCopyBundle(ptBundle, nt.ID)) + bundles = append(bundles, placement.BuildPlacementCopyBundle(ntBundle, partDef.ID)) + } else if ptOK { + bundles = append(bundles, placement.BuildPlacementDropBundle(partDef.ID)) + bundles = append(bundles, placement.BuildPlacementCopyBundle(ptBundle, nt.ID)) + } else if ntOK { + bundles = append(bundles, placement.BuildPlacementDropBundle(nt.ID)) + bundles = append(bundles, placement.BuildPlacementCopyBundle(ntBundle, partDef.ID)) + } + err = infosync.PutRuleBundles(context.TODO(), bundles) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } ver, err = updateSchemaVersion(t, job) diff --git a/ddl/reorg_test.go b/ddl/reorg_test.go index 18dd9a975fceb..4c28540e7ad3b 100644 --- a/ddl/reorg_test.go +++ b/ddl/reorg_test.go @@ -217,7 +217,7 @@ func (s *testDDLSuite) TestReorgOwner(c *C) { c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d1, "test") + dbInfo := testSchemaInfo(c, d1, "test_reorg") testCreateSchema(c, ctx, d1, dbInfo) tblInfo := testTableInfo(c, d1, "t", 3) diff --git a/ddl/restart_test.go b/ddl/restart_test.go index b587d54b80cc8..b7791ef7679bd 100644 --- a/ddl/restart_test.go +++ b/ddl/restart_test.go @@ -120,7 +120,7 @@ func (s *testSchemaSuite) TestSchemaResume(c *C) { testCheckOwner(c, d1, true) - dbInfo := testSchemaInfo(c, d1, "test") + dbInfo := testSchemaInfo(c, d1, "test_restart") job := &model.Job{ SchemaID: dbInfo.ID, Type: model.ActionCreateSchema, @@ -157,7 +157,7 @@ func (s *testStatSuite) TestStat(c *C) { c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_restart") testCreateSchema(c, testNewContext(d), d, dbInfo) // TODO: Get this information from etcd. diff --git a/ddl/schema.go b/ddl/schema.go index 823e12a551900..a4b14a49bdbc3 100644 --- a/ddl/schema.go +++ b/ddl/schema.go @@ -68,16 +68,12 @@ func onCreateSchema(d *ddlCtx, 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() + is := d.infoCache.GetLatest() if is.SchemaMetaVersion() == currVer { return checkSchemaNotExistsFromInfoSchema(is, schemaID, dbInfo) } @@ -169,7 +165,7 @@ func onDropSchema(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) oldIDs := getIDs(tables) bundles := make([]*placement.Bundle, 0, len(oldIDs)+1) for _, ID := range append(oldIDs, dbInfo.ID) { - oldBundle, ok := d.infoHandle.Get().BundleByName(placement.GroupID(ID)) + oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(ID)) if ok && !oldBundle.IsEmpty() { bundles = append(bundles, placement.BuildPlacementDropBundle(ID)) } diff --git a/ddl/schema_test.go b/ddl/schema_test.go index c70a0b793bb35..b4c8efee7b089 100644 --- a/ddl/schema_test.go +++ b/ddl/schema_test.go @@ -139,7 +139,7 @@ func (s *testSchemaSuite) TestSchema(c *C) { c.Assert(err, IsNil) }() ctx := testNewContext(d) - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_schema") // create a database. job := testCreateSchema(c, ctx, d, dbInfo) @@ -228,7 +228,7 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) { // d2 must not be owner. d2.ownerManager.RetireOwner() - dbInfo := testSchemaInfo(c, d2, "test") + dbInfo := testSchemaInfo(c, d2, "test_schema") testCreateSchema(c, ctx, d2, dbInfo) testCheckSchemaState(c, d2, dbInfo, model.StatePublic) diff --git a/ddl/stat_test.go b/ddl/stat_test.go index fe562a0ae0fb8..1ed3cbfe4c7fc 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -61,7 +61,7 @@ func (s *testSerialStatSuite) TestDDLStatsInfo(c *C) { c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_stat") testCreateSchema(c, testNewContext(d), d, dbInfo) tblInfo := testTableInfo(c, d, "t", 2) ctx := testNewContext(d) diff --git a/ddl/table.go b/ddl/table.go index 668de3ac41c05..424dd040a0de9 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -487,34 +487,32 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro } } - if d.infoHandle != nil && d.infoHandle.IsValid() { - is := d.infoHandle.Get() - - bundles := make([]*placement.Bundle, 0, len(oldPartitionIDs)+1) - if oldBundle, ok := is.BundleByName(placement.GroupID(tableID)); ok { - bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newTableID)) - } - - if pi := tblInfo.GetPartitionInfo(); pi != nil { - oldIDs := make([]int64, 0, len(oldPartitionIDs)) - newIDs := make([]int64, 0, len(oldPartitionIDs)) - newDefs := pi.Definitions - for i := range oldPartitionIDs { - newID := newDefs[i].ID - if oldBundle, ok := is.BundleByName(placement.GroupID(oldPartitionIDs[i])); ok && !oldBundle.IsEmpty() { - oldIDs = append(oldIDs, oldPartitionIDs[i]) - newIDs = append(newIDs, newID) - bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newID)) - } + is := d.infoCache.GetLatest() + + bundles := make([]*placement.Bundle, 0, len(oldPartitionIDs)+1) + if oldBundle, ok := is.BundleByName(placement.GroupID(tableID)); ok { + bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newTableID)) + } + + if pi := tblInfo.GetPartitionInfo(); pi != nil { + oldIDs := make([]int64, 0, len(oldPartitionIDs)) + newIDs := make([]int64, 0, len(oldPartitionIDs)) + newDefs := pi.Definitions + for i := range oldPartitionIDs { + newID := newDefs[i].ID + if oldBundle, ok := is.BundleByName(placement.GroupID(oldPartitionIDs[i])); ok && !oldBundle.IsEmpty() { + oldIDs = append(oldIDs, oldPartitionIDs[i]) + newIDs = append(newIDs, newID) + bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newID)) } - job.CtxVars = []interface{}{oldIDs, newIDs} } + job.CtxVars = []interface{}{oldIDs, newIDs} + } - err = infosync.PutRuleBundles(context.TODO(), bundles) - if err != nil { - job.State = model.JobStateCancelled - return 0, errors.Wrapf(err, "failed to notify PD the placement rules") - } + err = infosync.PutRuleBundles(context.TODO(), bundles) + if err != nil { + job.State = model.JobStateCancelled + return 0, errors.Wrapf(err, "failed to notify PD the placement rules") } // Clear the tiflash replica available status. @@ -967,16 +965,12 @@ func onUpdateFlashReplicaStatus(t *meta.Meta, job *model.Job) (ver int64, _ erro } func checkTableNotExists(d *ddlCtx, t *meta.Meta, schemaID int64, tableName string) error { - // d.infoHandle maybe nil in some test. - if d.infoHandle == nil || !d.infoHandle.IsValid() { - 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() + is := d.infoCache.GetLatest() if is.SchemaMetaVersion() == currVer { return checkTableNotExistsFromInfoSchema(is, schemaID, tableName) } diff --git a/ddl/table_test.go b/ddl/table_test.go index 5760fc2b152b5..10927908f5289 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -355,7 +355,7 @@ func (s *testTableSuite) SetUpSuite(c *C) { WithLease(testLease), ) - s.dbInfo = testSchemaInfo(c, s.d, "test") + s.dbInfo = testSchemaInfo(c, s.d, "test_table") testCreateSchema(c, testNewContext(s.d), s.d, s.dbInfo) } diff --git a/ddl/util/syncer_test.go b/ddl/util/syncer_test.go index b552488ad49de..5a9d41d47e3b8 100644 --- a/ddl/util/syncer_test.go +++ b/ddl/util/syncer_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/parser/terror" . "github.com/pingcap/tidb/ddl" . "github.com/pingcap/tidb/ddl/util" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/store/mockstore" "go.etcd.io/etcd/clientv3" @@ -69,11 +70,14 @@ func TestSyncerSimple(t *testing.T) { defer clus.Terminate(t) cli := clus.RandClient() ctx := goctx.Background() + ic := infoschema.NewCache(2) + ic.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d := NewDDL( ctx, WithEtcdClient(cli), WithStore(store), WithLease(testLease), + WithInfoCache(ic), ) err = d.Start(nil) if err != nil { @@ -110,11 +114,14 @@ func TestSyncerSimple(t *testing.T) { t.Fatalf("client get global version result not match, err %v", err) } + ic2 := infoschema.NewCache(2) + ic2.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d1 := NewDDL( ctx, WithEtcdClient(cli), WithStore(store), WithLease(testLease), + WithInfoCache(ic2), ) err = d1.Start(nil) if err != nil { diff --git a/domain/domain.go b/domain/domain.go index f4b0ac8900f24..e6ea3d1e2d949 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -67,7 +67,7 @@ import ( // Multiple domains can be used in parallel without synchronization. type Domain struct { store kv.Storage - infoHandle *infoschema.Handle + infoCache *infoschema.InfoCache privHandle *privileges.Handle bindHandle *bindinfo.BindHandle statsHandle unsafe.Pointer @@ -92,78 +92,75 @@ type Domain struct { isLostConnectionToPD sync2.AtomicInt32 // !0: true, 0: false. } -// loadInfoSchema loads infoschema at startTS into handle, usedSchemaVersion is the currently used -// infoschema version, if it is the same as the schema version at startTS, we don't need to reload again. -// It returns the latest schema version, the changed table IDs, whether it's a full load and an error. -func (do *Domain) loadInfoSchema(handle *infoschema.Handle, usedSchemaVersion int64, - startTS uint64) (neededSchemaVersion int64, change *tikv.RelatedSchemaChange, fullLoad bool, err error) { +// loadInfoSchema loads infoschema at startTS. +// It returns: +// 1. the needed infoschema +// 2. cache hit indicator +// 3. currentSchemaVersion(before loading) +// 4. the changed table IDs if it is not full load +// 5. an error if any +func (do *Domain) loadInfoSchema(startTS uint64) (infoschema.InfoSchema, bool, int64, *tikv.RelatedSchemaChange, error) { snapshot := do.store.GetSnapshot(kv.NewVersion(startTS)) m := meta.NewSnapshotMeta(snapshot) - neededSchemaVersion, err = m.GetSchemaVersion() + neededSchemaVersion, err := m.GetSchemaVersion() if err != nil { - return 0, nil, fullLoad, err - } - if usedSchemaVersion != 0 && usedSchemaVersion == neededSchemaVersion { - return neededSchemaVersion, nil, fullLoad, nil + return nil, false, 0, nil, err } - // Update self schema version to etcd. - defer func() { - // There are two possibilities for not updating the self schema version to etcd. - // 1. Failed to loading schema information. - // 2. When users use history read feature, the neededSchemaVersion isn't the latest schema version. - if err != nil || neededSchemaVersion < do.InfoSchema().SchemaMetaVersion() { - logutil.BgLogger().Info("do not update self schema version to etcd", - zap.Int64("usedSchemaVersion", usedSchemaVersion), - zap.Int64("neededSchemaVersion", neededSchemaVersion), zap.Error(err)) - return - } + if is := do.infoCache.GetByVersion(neededSchemaVersion); is != nil { + return is, true, 0, nil, nil + } - err = do.ddl.SchemaSyncer().UpdateSelfVersion(context.Background(), neededSchemaVersion) - if err != nil { - logutil.BgLogger().Info("update self version failed", - zap.Int64("usedSchemaVersion", usedSchemaVersion), - zap.Int64("neededSchemaVersion", neededSchemaVersion), zap.Error(err)) - } - }() + currentSchemaVersion := int64(0) + if oldInfoSchema := do.infoCache.GetLatest(); oldInfoSchema != nil { + currentSchemaVersion = oldInfoSchema.SchemaMetaVersion() + } + // TODO: tryLoadSchemaDiffs has potential risks of failure. And it becomes worse in history reading cases. + // It is only kept because there is no alternative diff/partial loading solution. + // And it is only used to diff upgrading the current latest infoschema, if: + // 1. Not first time bootstrap loading, which needs a full load. + // 2. It is newer than the current one, so it will be "the current one" after this function call. + // 3. There are less 100 diffs. startTime := time.Now() - ok, relatedChanges, err := do.tryLoadSchemaDiffs(m, usedSchemaVersion, neededSchemaVersion) - if err != nil { + if currentSchemaVersion != 0 && neededSchemaVersion > currentSchemaVersion && neededSchemaVersion-currentSchemaVersion < 100 { + is, relatedChanges, err := do.tryLoadSchemaDiffs(m, currentSchemaVersion, neededSchemaVersion) + if err == nil { + do.infoCache.Insert(is) + logutil.BgLogger().Info("diff load InfoSchema success", + zap.Int64("currentSchemaVersion", currentSchemaVersion), + zap.Int64("neededSchemaVersion", neededSchemaVersion), + zap.Duration("start time", time.Since(startTime)), + zap.Int64s("phyTblIDs", relatedChanges.PhyTblIDS), + zap.Uint64s("actionTypes", relatedChanges.ActionTypes)) + return is, false, currentSchemaVersion, relatedChanges, nil + } // We can fall back to full load, don't need to return the error. logutil.BgLogger().Error("failed to load schema diff", zap.Error(err)) } - if ok { - logutil.BgLogger().Info("diff load InfoSchema success", - zap.Int64("usedSchemaVersion", usedSchemaVersion), - zap.Int64("neededSchemaVersion", neededSchemaVersion), - zap.Duration("start time", time.Since(startTime)), - zap.Int64s("phyTblIDs", relatedChanges.PhyTblIDS), - zap.Uint64s("actionTypes", relatedChanges.ActionTypes)) - return neededSchemaVersion, relatedChanges, fullLoad, nil - } - fullLoad = true schemas, err := do.fetchAllSchemasWithTables(m) if err != nil { - return 0, nil, fullLoad, err + return nil, false, currentSchemaVersion, nil, err } bundles, err := infosync.GetAllRuleBundles(context.TODO()) if err != nil { - return 0, nil, fullLoad, err + return nil, false, currentSchemaVersion, nil, err } - newISBuilder, err := infoschema.NewBuilder(handle).InitWithDBInfos(schemas, bundles, neededSchemaVersion) + newISBuilder, err := infoschema.NewBuilder(do.Store()).InitWithDBInfos(schemas, bundles, neededSchemaVersion) if err != nil { - return 0, nil, fullLoad, err + return nil, false, currentSchemaVersion, nil, err } logutil.BgLogger().Info("full load InfoSchema success", - zap.Int64("usedSchemaVersion", usedSchemaVersion), + zap.Int64("currentSchemaVersion", currentSchemaVersion), zap.Int64("neededSchemaVersion", neededSchemaVersion), zap.Duration("start time", time.Since(startTime))) - newISBuilder.Build() - return neededSchemaVersion, nil, fullLoad, nil + + is := newISBuilder.Build() + do.infoCache.Insert(is) + return is, false, currentSchemaVersion, nil, nil } func (do *Domain) fetchAllSchemasWithTables(m *meta.Meta) ([]*model.DBInfo, error) { @@ -238,48 +235,31 @@ func (do *Domain) fetchSchemasWithTables(schemas []*model.DBInfo, m *meta.Meta, done <- nil } -const ( - initialVersion = 0 - maxNumberOfDiffsToLoad = 100 -) - -func isTooOldSchema(usedVersion, newVersion int64) bool { - if usedVersion == initialVersion || newVersion-usedVersion > maxNumberOfDiffsToLoad { - return true - } - return false -} - // tryLoadSchemaDiffs tries to only load latest schema changes. // Return true if the schema is loaded successfully. // Return false if the schema can not be loaded by schema diff, then we need to do full load. // The second returned value is the delta updated table and partition IDs. -func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64) (bool, *tikv.RelatedSchemaChange, error) { - // If there isn't any used version, or used version is too old, we do full load. - // And when users use history read feature, we will set usedVersion to initialVersion, then full load is needed. - if isTooOldSchema(usedVersion, newVersion) { - return false, nil, nil - } +func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64) (infoschema.InfoSchema, *tikv.RelatedSchemaChange, error) { var diffs []*model.SchemaDiff for usedVersion < newVersion { usedVersion++ diff, err := m.GetSchemaDiff(usedVersion) if err != nil { - return false, nil, err + return nil, nil, err } if diff == nil { // If diff is missing for any version between used and new version, we fall back to full reload. - return false, nil, nil + return nil, nil, fmt.Errorf("failed to get schemadiff") } diffs = append(diffs, diff) } - builder := infoschema.NewBuilder(do.infoHandle).InitWithOldInfoSchema() + builder := infoschema.NewBuilder(do.Store()).InitWithOldInfoSchema(do.infoCache.GetLatest()) phyTblIDs := make([]int64, 0, len(diffs)) actions := make([]uint64, 0, len(diffs)) for _, diff := range diffs { IDs, err := builder.ApplyDiff(m, diff) if err != nil { - return false, nil, err + return nil, nil, err } if canSkipSchemaCheckerDDL(diff.Type) { continue @@ -289,11 +269,11 @@ func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64 actions = append(actions, uint64(1< staleVer { - return errors.New("schema version changed after the staleness startTS") - } // With START TRANSACTION, autocommit remains disabled until you end // the transaction with COMMIT or ROLLBACK. The autocommit mode then diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 493bda06c5de2..7cf235bd3c0f7 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -18,7 +18,6 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/store/tikv/oracle" @@ -26,12 +25,6 @@ import ( ) func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") - c.Assert(err, IsNil) - }() - testcases := []struct { name string preSQL string @@ -117,8 +110,6 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { } func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -155,7 +146,7 @@ func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", fmt.Sprintf(`return("%v")`, testcase.zone)) failpoint.Enable("github.com/pingcap/tidb/store/tikv/assertStoreLabels", fmt.Sprintf(`return("%v_%v")`, placement.DCLabelKey, testcase.txnScope)) failpoint.Enable("github.com/pingcap/tidb/store/tikv/assertStaleReadFlag", `return(true)`) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:20';`) + tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) tk.MustQuery(testcase.sql) tk.MustExec(`commit`) failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") @@ -165,12 +156,6 @@ func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { } func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. @@ -193,62 +178,7 @@ func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { tk.MustExec("commit") } -func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { - testcases := []struct { - name string - sql string - expectErr error - }{ - { - name: "ddl change before stale txn", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:03'`, - expectErr: errors.New("schema version changed after the staleness startTS"), - }, - { - name: "ddl change before stale txn", - sql: fmt.Sprintf("START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '%v'", - time.Now().Truncate(3*time.Second).Format("2006-01-02 15:04:05")), - expectErr: errors.New(".*schema version changed after the staleness startTS.*"), - }, - { - name: "ddl change before stale txn", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:03'`, - expectErr: nil, - }, - } - tk := testkit.NewTestKitWithInit(c, s.store) - for _, testcase := range testcases { - check := func() { - if testcase.expectErr != nil { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(true)"), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") - c.Assert(err, IsNil) - }() - - } else { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") - c.Assert(err, IsNil) - }() - - } - _, err := tk.Exec(testcase.sql) - if testcase.expectErr != nil { - c.Assert(err, NotNil) - c.Assert(err.Error(), Matches, testcase.expectErr.Error()) - } else { - c.Assert(err, IsNil) - } - } - check() - } -} - func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -318,3 +248,22 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { failpoint.Disable("github.com/pingcap/tidb/store/tikv/injectSafeTS") } } + +func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id int primary key);") + + schemaVer1 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() + time.Sleep(time.Second) + tk.MustExec("drop table if exists t") + schemaVer2 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() + // confirm schema changed + c.Assert(schemaVer1, Less, schemaVer2) + + tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:01'`) + schemaVer3 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() + // got an old infoSchema + c.Assert(schemaVer3, Equals, schemaVer1) +} diff --git a/infoschema/builder.go b/infoschema/builder.go index 28591d8679baf..88e8b71add319 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/table" @@ -35,8 +36,10 @@ import ( // Builder builds a new InfoSchema. type Builder struct { - is *infoSchema - handle *Handle + is *infoSchema + // TODO: store is only used by autoid allocators + // detach allocators from storage, use passed transaction in the feature + store kv.Storage } // ApplyDiff applies SchemaDiff to the new InfoSchema. @@ -352,14 +355,14 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i ConvertOldVersionUTF8ToUTF8MB4IfNeed(tblInfo) if len(allocs) == 0 { - allocs = autoid.NewAllocatorsFromTblInfo(b.handle.store, dbInfo.ID, tblInfo) + allocs = autoid.NewAllocatorsFromTblInfo(b.store, dbInfo.ID, tblInfo) } else { switch tp { case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType) + newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType) allocs = append(allocs, newAlloc) case model.ActionRebaseAutoRandomBase: - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) + newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) allocs = append(allocs, newAlloc) case model.ActionModifyColumn: // Change column attribute from auto_increment to auto_random. @@ -368,7 +371,7 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i allocs = allocs.Filter(func(a autoid.Allocator) bool { return a.GetType() != autoid.AutoIncrementType && a.GetType() != autoid.RowIDAllocType }) - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) + newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) allocs = append(allocs, newAlloc) } } @@ -470,9 +473,14 @@ func (b *Builder) applyPlacementUpdate(id string) error { return nil } +// Build builds and returns the built infoschema. +func (b *Builder) Build() InfoSchema { + return b.is +} + // InitWithOldInfoSchema initializes an empty new InfoSchema by copies all the data from old InfoSchema. -func (b *Builder) InitWithOldInfoSchema() *Builder { - oldIS := b.handle.Get().(*infoSchema) +func (b *Builder) InitWithOldInfoSchema(oldSchema InfoSchema) *Builder { + oldIS := oldSchema.(*infoSchema) b.is.schemaMetaVersion = oldIS.schemaMetaVersion b.copySchemasMap(oldIS) b.copyBundlesMap(oldIS) @@ -549,7 +557,7 @@ func (b *Builder) createSchemaTablesForDB(di *model.DBInfo, tableFromMeta tableF b.is.schemaMap[di.Name.L] = schTbls for _, t := range di.Tables { - allocs := autoid.NewAllocatorsFromTblInfo(b.handle.store, di.ID, t) + allocs := autoid.NewAllocatorsFromTblInfo(b.store, di.ID, t) var tbl table.Table tbl, err := tableFromMeta(allocs, t) if err != nil { @@ -574,21 +582,16 @@ func RegisterVirtualTable(dbInfo *model.DBInfo, tableFromMeta tableFromMetaFunc) drivers = append(drivers, &virtualTableDriver{dbInfo, tableFromMeta}) } -// Build sets new InfoSchema to the handle in the Builder. -func (b *Builder) Build() { - b.handle.value.Store(b.is) -} - // NewBuilder creates a new Builder with a Handle. -func NewBuilder(handle *Handle) *Builder { - b := new(Builder) - b.handle = handle - b.is = &infoSchema{ - schemaMap: map[string]*schemaTables{}, - ruleBundleMap: map[string]*placement.Bundle{}, - sortedTablesBuckets: make([]sortedTables, bucketCount), +func NewBuilder(store kv.Storage) *Builder { + return &Builder{ + store: store, + is: &infoSchema{ + schemaMap: map[string]*schemaTables{}, + ruleBundleMap: map[string]*placement.Bundle{}, + sortedTablesBuckets: make([]sortedTables, bucketCount), + }, } - return b } func tableBucketIdx(tableID int64) int { diff --git a/infoschema/cache.go b/infoschema/cache.go new file mode 100644 index 0000000000000..4c3371b1bc354 --- /dev/null +++ b/infoschema/cache.go @@ -0,0 +1,95 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package infoschema + +import ( + "sort" + "sync" + + "github.com/pingcap/tidb/metrics" +) + +// InfoCache handles information schema, including getting and setting. +// The cache behavior, however, is transparent and under automatic management. +// It only promised to cache the infoschema, if it is newer than all the cached. +type InfoCache struct { + mu sync.RWMutex + // cache is sorted by SchemaVersion in descending order + cache []InfoSchema +} + +// NewCache creates a new InfoCache. +func NewCache(capcity int) *InfoCache { + return &InfoCache{cache: make([]InfoSchema, 0, capcity)} +} + +// GetLatest gets the newest information schema. +func (h *InfoCache) GetLatest() InfoSchema { + h.mu.RLock() + defer h.mu.RUnlock() + metrics.InfoCacheCounters.WithLabelValues("get").Inc() + if len(h.cache) > 0 { + metrics.InfoCacheCounters.WithLabelValues("hit").Inc() + return h.cache[0] + } + return nil +} + +// GetByVersion gets the information schema based on schemaVersion. Returns nil if it is not loaded. +func (h *InfoCache) GetByVersion(version int64) InfoSchema { + h.mu.RLock() + defer h.mu.RUnlock() + metrics.InfoCacheCounters.WithLabelValues("get").Inc() + i := sort.Search(len(h.cache), func(i int) bool { + return h.cache[i].SchemaMetaVersion() <= version + }) + if i < len(h.cache) && h.cache[i].SchemaMetaVersion() == version { + metrics.InfoCacheCounters.WithLabelValues("hit").Inc() + return h.cache[i] + } + return nil +} + +// Insert will **TRY** to insert the infoschema into the cache. +// It only promised to cache the newest infoschema. +// It returns 'true' if it is cached, 'false' otherwise. +func (h *InfoCache) Insert(is InfoSchema) bool { + h.mu.Lock() + defer h.mu.Unlock() + + version := is.SchemaMetaVersion() + i := sort.Search(len(h.cache), func(i int) bool { + return h.cache[i].SchemaMetaVersion() <= version + }) + + // cached entry + if i < len(h.cache) && h.cache[i].SchemaMetaVersion() == version { + return true + } + + if len(h.cache) < cap(h.cache) { + // has free space, grown the slice + h.cache = h.cache[:len(h.cache)+1] + copy(h.cache[i+1:], h.cache[i:]) + h.cache[i] = is + return true + } else if i < len(h.cache) { + // drop older schema + copy(h.cache[i+1:], h.cache[i:]) + h.cache[i] = is + return true + } + // older than all cached schemas, refuse to cache it + return false +} diff --git a/infoschema/cache_test.go b/infoschema/cache_test.go new file mode 100644 index 0000000000000..a8e9ddcc0df5a --- /dev/null +++ b/infoschema/cache_test.go @@ -0,0 +1,119 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package infoschema_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/infoschema" +) + +var _ = Suite(&testInfoCacheSuite{}) + +type testInfoCacheSuite struct { +} + +func (s *testInfoCacheSuite) TestNewCache(c *C) { + ic := infoschema.NewCache(16) + c.Assert(ic, NotNil) +} + +func (s *testInfoCacheSuite) TestInsert(c *C) { + ic := infoschema.NewCache(3) + c.Assert(ic, NotNil) + + is2 := infoschema.MockInfoSchemaWithSchemaVer(nil, 2) + ic.Insert(is2) + c.Assert(ic.GetByVersion(2), NotNil) + + // newer + is5 := infoschema.MockInfoSchemaWithSchemaVer(nil, 5) + ic.Insert(is5) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(2), NotNil) + + // older + is0 := infoschema.MockInfoSchemaWithSchemaVer(nil, 0) + ic.Insert(is0) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(2), NotNil) + c.Assert(ic.GetByVersion(0), NotNil) + + // replace 5, drop 0 + is6 := infoschema.MockInfoSchemaWithSchemaVer(nil, 6) + ic.Insert(is6) + c.Assert(ic.GetByVersion(6), NotNil) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(2), NotNil) + c.Assert(ic.GetByVersion(0), IsNil) + + // replace 2, drop 2 + is3 := infoschema.MockInfoSchemaWithSchemaVer(nil, 3) + ic.Insert(is3) + c.Assert(ic.GetByVersion(6), NotNil) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(3), NotNil) + c.Assert(ic.GetByVersion(2), IsNil) + c.Assert(ic.GetByVersion(0), IsNil) + + // insert 2, but failed silently + ic.Insert(is2) + c.Assert(ic.GetByVersion(6), NotNil) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(3), NotNil) + c.Assert(ic.GetByVersion(2), IsNil) + c.Assert(ic.GetByVersion(0), IsNil) + + // insert 5, but it is already in + ic.Insert(is5) + c.Assert(ic.GetByVersion(6), NotNil) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(3), NotNil) + c.Assert(ic.GetByVersion(2), IsNil) + c.Assert(ic.GetByVersion(0), IsNil) +} + +func (s *testInfoCacheSuite) TestGetByVersion(c *C) { + ic := infoschema.NewCache(2) + c.Assert(ic, NotNil) + is1 := infoschema.MockInfoSchemaWithSchemaVer(nil, 1) + ic.Insert(is1) + is3 := infoschema.MockInfoSchemaWithSchemaVer(nil, 3) + ic.Insert(is3) + + c.Assert(ic.GetByVersion(1), Equals, is1) + c.Assert(ic.GetByVersion(3), Equals, is3) + c.Assert(ic.GetByVersion(0), IsNil, Commentf("index == 0, but not found")) + c.Assert(ic.GetByVersion(2), IsNil, Commentf("index in the middle, but not found")) + c.Assert(ic.GetByVersion(4), IsNil, Commentf("index == length, but not found")) +} + +func (s *testInfoCacheSuite) TestGetLatest(c *C) { + ic := infoschema.NewCache(16) + c.Assert(ic, NotNil) + c.Assert(ic.GetLatest(), IsNil) + + is1 := infoschema.MockInfoSchemaWithSchemaVer(nil, 1) + ic.Insert(is1) + c.Assert(ic.GetLatest(), Equals, is1) + + // newer change the newest + is2 := infoschema.MockInfoSchemaWithSchemaVer(nil, 2) + ic.Insert(is2) + c.Assert(ic.GetLatest(), Equals, is2) + + // older schema doesn't change the newest + is0 := infoschema.MockInfoSchemaWithSchemaVer(nil, 0) + ic.Insert(is0) + c.Assert(ic.GetLatest(), Equals, is2) +} diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index ac8afd14605f1..2494e89b4d57f 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -17,12 +17,10 @@ import ( "fmt" "sort" "sync" - "sync/atomic" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/ddl/placement" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util" @@ -312,40 +310,6 @@ func (is *infoSchema) SequenceByName(schema, sequence model.CIStr) (util.Sequenc return tbl.(util.SequenceTable), nil } -// Handle handles information schema, including getting and setting. -type Handle struct { - value atomic.Value - store kv.Storage -} - -// NewHandle creates a new Handle. -func NewHandle(store kv.Storage) *Handle { - h := &Handle{ - store: store, - } - return h -} - -// Get gets information schema from Handle. -func (h *Handle) Get() InfoSchema { - v := h.value.Load() - schema, _ := v.(InfoSchema) - return schema -} - -// IsValid uses to check whether handle value is valid. -func (h *Handle) IsValid() bool { - return h.value.Load() != nil -} - -// EmptyClone creates a new Handle with the same store and memSchema, but the value is not set. -func (h *Handle) EmptyClone() *Handle { - newHandle := &Handle{ - store: h.store, - } - return newHandle -} - func init() { // Initialize the information shema database and register the driver to `drivers` dbID := autoid.InformationSchemaDBID diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index 6aa0c5526f467..87276ef1452b9 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -15,7 +15,6 @@ package infoschema_test import ( "context" - "sync" "testing" . "github.com/pingcap/check" @@ -57,7 +56,6 @@ func (*testSuite) TestT(c *C) { c.Assert(err, IsNil) defer dom.Close() - handle := infoschema.NewHandle(store) dbName := model.NewCIStr("Test") tbName := model.NewCIStr("T") colName := model.NewCIStr("A") @@ -116,7 +114,7 @@ func (*testSuite) TestT(c *C) { }) c.Assert(err, IsNil) - builder, err := infoschema.NewBuilder(handle).InitWithDBInfos(dbInfos, nil, 1) + builder, err := infoschema.NewBuilder(dom.Store()).InitWithDBInfos(dbInfos, nil, 1) c.Assert(err, IsNil) txn, err := store.Begin() @@ -126,8 +124,7 @@ func (*testSuite) TestT(c *C) { err = txn.Rollback() c.Assert(err, IsNil) - builder.Build() - is := handle.Get() + is := builder.Build() schemaNames := is.AllSchemaNames() c.Assert(schemaNames, HasLen, 4) @@ -213,14 +210,10 @@ func (*testSuite) TestT(c *C) { c.Assert(err, IsNil) err = txn.Rollback() c.Assert(err, IsNil) - builder.Build() - is = handle.Get() + is = builder.Build() schema, ok = is.SchemaByID(dbID) c.Assert(ok, IsTrue) c.Assert(len(schema.Tables), Equals, 1) - - emptyHandle := handle.EmptyClone() - c.Assert(emptyHandle.Get(), IsNil) } func (testSuite) TestMockInfoSchema(c *C) { @@ -258,32 +251,6 @@ func checkApplyCreateNonExistsTableDoesNotPanic(c *C, txn kv.Transaction, builde c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) } -// TestConcurrent makes sure it is safe to concurrently create handle on multiple stores. -func (testSuite) TestConcurrent(c *C) { - defer testleak.AfterTest(c)() - storeCount := 5 - stores := make([]kv.Storage, storeCount) - for i := 0; i < storeCount; i++ { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - stores[i] = store - } - defer func() { - for _, store := range stores { - store.Close() - } - }() - var wg sync.WaitGroup - wg.Add(storeCount) - for _, store := range stores { - go func(s kv.Storage) { - defer wg.Done() - _ = infoschema.NewHandle(s) - }(store) - } - wg.Wait() -} - // TestInfoTables makes sure that all tables of information_schema could be found in infoschema handle. func (*testSuite) TestInfoTables(c *C) { defer testleak.AfterTest(c)() @@ -293,12 +260,10 @@ func (*testSuite) TestInfoTables(c *C) { err := store.Close() c.Assert(err, IsNil) }() - handle := infoschema.NewHandle(store) - builder, err := infoschema.NewBuilder(handle).InitWithDBInfos(nil, nil, 0) + + builder, err := infoschema.NewBuilder(store).InitWithDBInfos(nil, nil, 0) c.Assert(err, IsNil) - builder.Build() - is := handle.Get() - c.Assert(is, NotNil) + is := builder.Build() infoTables := []string{ "SCHEMATA", @@ -360,12 +325,9 @@ func (*testSuite) TestGetBundle(c *C) { c.Assert(err, IsNil) }() - handle := infoschema.NewHandle(store) - builder, err := infoschema.NewBuilder(handle).InitWithDBInfos(nil, nil, 0) + builder, err := infoschema.NewBuilder(store).InitWithDBInfos(nil, nil, 0) c.Assert(err, IsNil) - builder.Build() - - is := handle.Get() + is := builder.Build() bundle := &placement.Bundle{ ID: placement.PDBundleID, diff --git a/metrics/domain.go b/metrics/domain.go index dd3912555d59c..f30dbd59e5d32 100644 --- a/metrics/domain.go +++ b/metrics/domain.go @@ -38,6 +38,19 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }) + // InfoCacheCounters are the counters of get/hit. + InfoCacheCounters = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "domain", + Name: "infocache_counters", + Help: "Counters of infoCache: get/hit.", + }, []string{LblType}) + // InfoCacheCounterGet is the total number of getting entry. + InfoCacheCounterGet = "get" + // InfoCacheCounterHit is the cache hit numbers for get. + InfoCacheCounterHit = "hit" + // LoadPrivilegeCounter records the counter of load privilege. LoadPrivilegeCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ diff --git a/metrics/metrics.go b/metrics/metrics.go index ff2ac3b1aa08d..4a879b5d5423c 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -98,6 +98,7 @@ func RegisterMetrics() { prometheus.MustRegister(JobsGauge) prometheus.MustRegister(KeepAliveCounter) prometheus.MustRegister(LoadPrivilegeCounter) + prometheus.MustRegister(InfoCacheCounters) prometheus.MustRegister(LoadSchemaCounter) prometheus.MustRegister(LoadSchemaDuration) prometheus.MustRegister(MetaHistogram) diff --git a/owner/manager_test.go b/owner/manager_test.go index e25b204e6bbb4..e239419057291 100644 --- a/owner/manager_test.go +++ b/owner/manager_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/terror" . "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/logutil" @@ -72,11 +73,14 @@ func TestSingle(t *testing.T) { defer clus.Terminate(t) cli := clus.RandClient() ctx := goctx.Background() + ic := infoschema.NewCache(2) + ic.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d := NewDDL( ctx, WithEtcdClient(cli), WithStore(store), WithLease(testLease), + WithInfoCache(ic), ) err = d.Start(nil) if err != nil { @@ -142,11 +146,14 @@ func TestCluster(t *testing.T) { defer clus.Terminate(t) cli := clus.Client(0) + ic := infoschema.NewCache(2) + ic.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d := NewDDL( goctx.Background(), WithEtcdClient(cli), WithStore(store), WithLease(testLease), + WithInfoCache(ic), ) err = d.Start(nil) if err != nil { @@ -157,11 +164,14 @@ func TestCluster(t *testing.T) { t.Fatalf("expect true, got isOwner:%v", isOwner) } cli1 := clus.Client(1) + ic2 := infoschema.NewCache(2) + ic2.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d1 := NewDDL( goctx.Background(), WithEtcdClient(cli1), WithStore(store), WithLease(testLease), + WithInfoCache(ic2), ) err = d1.Start(nil) if err != nil { @@ -189,11 +199,14 @@ func TestCluster(t *testing.T) { // d3 (not owner) stop cli3 := clus.Client(3) + ic3 := infoschema.NewCache(2) + ic3.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d3 := NewDDL( goctx.Background(), WithEtcdClient(cli3), WithStore(store), WithLease(testLease), + WithInfoCache(ic3), ) err = d3.Start(nil) if err != nil { diff --git a/session/session.go b/session/session.go index af3f41c863dc0..84ff7e4eec424 100644 --- a/session/session.go +++ b/session/session.go @@ -1935,7 +1935,7 @@ func (s *session) isTxnRetryable() bool { func (s *session) NewTxn(ctx context.Context) error { if s.txn.Valid() { - txnID := s.txn.StartTS() + txnStartTS := s.txn.StartTS() txnScope := s.GetSessionVars().TxnCtx.TxnScope err := s.CommitTxn(ctx) if err != nil { @@ -1944,7 +1944,7 @@ func (s *session) NewTxn(ctx context.Context) error { vars := s.GetSessionVars() logutil.Logger(ctx).Info("NewTxn() inside a transaction auto commit", zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), - zap.Uint64("txnStartTS", txnID), + zap.Uint64("txnStartTS", txnStartTS), zap.String("txnScope", txnScope)) } @@ -2805,7 +2805,10 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc txn.SetOption(kv.IsStalenessReadOnly, true) txn.SetOption(kv.TxnScope, txnScope) s.txn.changeInvalidToValid(txn) - is := domain.GetDomain(s).InfoSchema() + is, err := domain.GetDomain(s).GetSnapshotInfoSchema(txn.StartTS()) + if err != nil { + return errors.Trace(err) + } s.sessionVars.TxnCtx = &variable.TransactionContext{ InfoSchema: is, CreateTime: time.Now(), diff --git a/session/session_test.go b/session/session_test.go index 4870215f33c9e..a897cb7db07f3 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -3903,9 +3903,7 @@ func (s *testSessionSerialSuite) TestIssue21943(c *C) { c.Assert(err.Error(), Equals, "[variable:1238]Variable 'last_plan_from_cache' is a read only variable") } -func (s *testSessionSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") +func (s *testSessionSerialSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { testcases := []struct { name string sql string @@ -4036,7 +4034,7 @@ func (s *testSessionSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { tk.MustExec(`set @@tidb_enable_noop_functions=1;`) for _, testcase := range testcases { c.Log(testcase.name) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`) + tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) if testcase.isValidate { _, err := tk.Exec(testcase.sql) c.Assert(err, IsNil) @@ -4050,8 +4048,6 @@ func (s *testSessionSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { } func (s *testSessionSerialSuite) TestSpecialSQLInStalenessTxn(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") testcases := []struct { @@ -4098,7 +4094,7 @@ func (s *testSessionSerialSuite) TestSpecialSQLInStalenessTxn(c *C) { tk.MustExec("CREATE USER 'newuser' IDENTIFIED BY 'mypassword';") for _, testcase := range testcases { comment := Commentf(testcase.name) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`) + tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, true, comment) tk.MustExec(testcase.sql) c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.sameSession, comment) From d641810f1ade5dcb5c29dd09ce0e0e2e87e28127 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Mon, 17 May 2021 21:49:39 +0800 Subject: [PATCH 40/42] session: test that temporary tables will also be retried (#24505) --- session/session.go | 50 ++++++++++++++++++++++++++--------------- session/session_test.go | 43 +++++++++++++++++++++++++++++++++++ 2 files changed, 75 insertions(+), 18 deletions(-) diff --git a/session/session.go b/session/session.go index 84ff7e4eec424..e1581d5ed4074 100644 --- a/session/session.go +++ b/session/session.go @@ -473,6 +473,9 @@ func (s *session) doCommit(ctx context.Context) error { if err != nil { return err } + if err = s.removeTempTableFromBuffer(); err != nil { + return err + } // mockCommitError and mockGetTSErrorInRetry use to test PR #8743. failpoint.Inject("mockCommitError", func(val failpoint.Value) { @@ -526,29 +529,40 @@ func (s *session) doCommit(ctx context.Context) error { s.GetSessionVars().TxnCtx.IsExplicit && s.GetSessionVars().GuaranteeLinearizability) } - // Filter out the temporary table key-values. - if tables := s.sessionVars.TxnCtx.GlobalTemporaryTables; tables != nil { - memBuffer := s.txn.GetMemBuffer() - for tid := range tables { - seekKey := tablecodec.EncodeTablePrefix(tid) - endKey := tablecodec.EncodeTablePrefix(tid + 1) - iter, err := memBuffer.Iter(seekKey, endKey) - if err != nil { + return s.txn.Commit(tikvutil.SetSessionID(ctx, s.GetSessionVars().ConnectionID)) +} + +// removeTempTableFromBuffer filters out the temporary table key-values. +func (s *session) removeTempTableFromBuffer() error { + tables := s.GetSessionVars().TxnCtx.GlobalTemporaryTables + if len(tables) == 0 { + return nil + } + memBuffer := s.txn.GetMemBuffer() + // Reset and new an empty stage buffer. + defer func() { + s.txn.cleanup() + }() + for tid := range tables { + seekKey := tablecodec.EncodeTablePrefix(tid) + endKey := tablecodec.EncodeTablePrefix(tid + 1) + iter, err := memBuffer.Iter(seekKey, endKey) + if err != nil { + return err + } + for iter.Valid() && iter.Key().HasPrefix(seekKey) { + if err = memBuffer.Delete(iter.Key()); err != nil { return err } - for iter.Valid() && iter.Key().HasPrefix(seekKey) { - if err = memBuffer.Delete(iter.Key()); err != nil { - return errors.Trace(err) - } - s.txn.UpdateEntriesCountAndSize() - if err = iter.Next(); err != nil { - return errors.Trace(err) - } + s.txn.UpdateEntriesCountAndSize() + if err = iter.Next(); err != nil { + return err } } } - - return s.txn.Commit(tikvutil.SetSessionID(ctx, s.GetSessionVars().ConnectionID)) + // Flush to the root membuffer. + s.txn.flushStmtBuf() + return nil } // errIsNoisy is used to filter DUPLCATE KEY errors. diff --git a/session/session_test.go b/session/session_test.go index a897cb7db07f3..b7cfecc9c5f4c 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -789,6 +789,49 @@ func (s *testSessionSuite) TestRetryUnion(c *C) { c.Assert(err, ErrorMatches, ".*can not retry select for update statement") } +func (s *testSessionSuite) TestRetryGlobalTempTable(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists normal_table") + tk.MustExec("create table normal_table(a int primary key, b int)") + defer tk.MustExec("drop table if exists normal_table") + tk.MustExec("drop table if exists temp_table") + tk.MustExec("create global temporary table temp_table(a int primary key, b int) on commit delete rows") + defer tk.MustExec("drop table if exists temp_table") + + // insert select + tk.MustExec("set tidb_disable_txn_auto_retry = 0") + tk.MustExec("insert normal_table value(100, 100)") + tk.MustExec("set @@autocommit = 0") + // used to make conflicts + tk.MustExec("update normal_table set b=b+1 where a=100") + tk.MustExec("insert temp_table value(1, 1)") + tk.MustExec("insert normal_table select * from temp_table") + c.Assert(session.GetHistory(tk.Se).Count(), Equals, 3) + + // try to conflict with tk + tk1 := testkit.NewTestKitWithInit(c, s.store) + tk1.MustExec("update normal_table set b=b+1 where a=100") + + // It will retry internally. + tk.MustExec("commit") + tk.MustQuery("select a, b from normal_table order by a").Check(testkit.Rows("1 1", "100 102")) + tk.MustQuery("select a, b from temp_table order by a").Check(testkit.Rows()) + + // update multi-tables + tk.MustExec("update normal_table set b=b+1 where a=100") + tk.MustExec("insert temp_table value(1, 2)") + // before update: normal_table=(1 1) (100 102), temp_table=(1 2) + tk.MustExec("update normal_table, temp_table set normal_table.b=temp_table.b where normal_table.a=temp_table.a") + c.Assert(session.GetHistory(tk.Se).Count(), Equals, 3) + + // try to conflict with tk + tk1.MustExec("update normal_table set b=b+1 where a=100") + + // It will retry internally. + tk.MustExec("commit") + tk.MustQuery("select a, b from normal_table order by a").Check(testkit.Rows("1 2", "100 104")) +} + func (s *testSessionSuite) TestRetryShow(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("set @@autocommit = 0") From 0f10bef470f45bd862b38592f51ecb92a540896a Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 17 May 2021 08:13:39 -0600 Subject: [PATCH 41/42] domain, session: Add new sysvarcache to replace global values cache (#24359) --- cmd/explaintest/main.go | 3 - ddl/db_change_test.go | 2 - domain/domain.go | 73 ++++++++- domain/global_vars_cache.go | 135 ---------------- domain/global_vars_cache_test.go | 221 -------------------------- domain/sysvar_cache.go | 167 +++++++++++++++++++ executor/executor_test.go | 2 - executor/seqtest/seq_executor_test.go | 2 - infoschema/tables_test.go | 6 - metrics/domain.go | 9 ++ metrics/metrics.go | 1 + planner/core/prepare_test.go | 3 - session/session.go | 88 +++++----- session/session_test.go | 6 - sessionctx/variable/session.go | 9 ++ sessionctx/variable/sysvar.go | 54 +++++-- 16 files changed, 343 insertions(+), 438 deletions(-) delete mode 100644 domain/global_vars_cache.go delete mode 100644 domain/global_vars_cache_test.go create mode 100644 domain/sysvar_cache.go diff --git a/cmd/explaintest/main.go b/cmd/explaintest/main.go index fa5265f7af871..a85c8ce82dd3c 100644 --- a/cmd/explaintest/main.go +++ b/cmd/explaintest/main.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/parser/ast" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/logutil" @@ -663,8 +662,6 @@ func main() { log.Fatal(fmt.Sprintf("%s failed", sql), zap.Error(err)) } } - // Wait global variables to reload. - time.Sleep(domain.GlobalVariableCacheExpiry) if _, err = mdb.Exec("set sql_mode='STRICT_TRANS_TABLES'"); err != nil { log.Fatal("set sql_mode='STRICT_TRANS_TABLES' failed", zap.Error(err)) diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 7c3a0f9ad970f..041f35c7734a8 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -1063,7 +1063,6 @@ func (s *testStateChangeSuite) TestParallelAddGeneratedColumnAndAlterModifyColum _, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0") c.Assert(err, IsNil) }() - domain.GetDomain(s.se).GetGlobalVarsCache().Disable() sql1 := "ALTER TABLE t ADD COLUMN f INT GENERATED ALWAYS AS(a+1);" sql2 := "ALTER TABLE t MODIFY COLUMN a tinyint;" @@ -1083,7 +1082,6 @@ func (s *testStateChangeSuite) TestParallelAlterModifyColumnAndAddPK(c *C) { _, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0") c.Assert(err, IsNil) }() - domain.GetDomain(s.se).GetGlobalVarsCache().Disable() sql1 := "ALTER TABLE t ADD PRIMARY KEY (b) NONCLUSTERED;" sql2 := "ALTER TABLE t MODIFY COLUMN b tinyint;" diff --git a/domain/domain.go b/domain/domain.go index e6ea3d1e2d949..44f6df1aa9086 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -79,7 +79,7 @@ type Domain struct { sysSessionPool *sessionPool exit chan struct{} etcdClient *clientv3.Client - gvc GlobalVariableCache + sysVarCache SysVarCache // replaces GlobalVariableCache slowQuery *topNSlowQueries expensiveQueryHandle *expensivequery.Handle wg sync.WaitGroup @@ -900,6 +900,55 @@ func (do *Domain) LoadPrivilegeLoop(ctx sessionctx.Context) error { return nil } +// LoadSysVarCacheLoop create a goroutine loads sysvar cache in a loop, +// it should be called only once in BootstrapSession. +func (do *Domain) LoadSysVarCacheLoop(ctx sessionctx.Context) error { + err := do.sysVarCache.RebuildSysVarCache(ctx) + if err != nil { + return err + } + var watchCh clientv3.WatchChan + duration := 30 * time.Second + if do.etcdClient != nil { + watchCh = do.etcdClient.Watch(context.Background(), sysVarCacheKey) + } + do.wg.Add(1) + go func() { + defer func() { + do.wg.Done() + logutil.BgLogger().Info("LoadSysVarCacheLoop exited.") + util.Recover(metrics.LabelDomain, "LoadSysVarCacheLoop", nil, false) + }() + var count int + for { + ok := true + select { + case <-do.exit: + return + case _, ok = <-watchCh: + case <-time.After(duration): + } + if !ok { + logutil.BgLogger().Error("LoadSysVarCacheLoop loop watch channel closed") + watchCh = do.etcdClient.Watch(context.Background(), sysVarCacheKey) + count++ + if count > 10 { + time.Sleep(time.Duration(count) * time.Second) + } + continue + } + count = 0 + logutil.BgLogger().Debug("Rebuilding sysvar cache from etcd watch event.") + err := do.sysVarCache.RebuildSysVarCache(ctx) + metrics.LoadSysVarCacheCounter.WithLabelValues(metrics.RetLabel(err)).Inc() + if err != nil { + logutil.BgLogger().Error("LoadSysVarCacheLoop failed", zap.Error(err)) + } + } + }() + return nil +} + // PrivilegeHandle returns the MySQLPrivilege. func (do *Domain) PrivilegeHandle() *privileges.Handle { return do.privHandle @@ -1278,7 +1327,10 @@ func (do *Domain) ExpensiveQueryHandle() *expensivequery.Handle { return do.expensiveQueryHandle } -const privilegeKey = "/tidb/privilege" +const ( + privilegeKey = "/tidb/privilege" + sysVarCacheKey = "/tidb/sysvars" +) // NotifyUpdatePrivilege updates privilege key in etcd, TiDB client that watches // the key will get notification. @@ -1300,6 +1352,23 @@ func (do *Domain) NotifyUpdatePrivilege(ctx sessionctx.Context) { } } +// NotifyUpdateSysVarCache updates the sysvar cache key in etcd, which other TiDB +// clients are subscribed to for updates. For the caller, the cache is also built +// synchronously so that the effect is immediate. +func (do *Domain) NotifyUpdateSysVarCache(ctx sessionctx.Context) { + if do.etcdClient != nil { + row := do.etcdClient.KV + _, err := row.Put(context.Background(), sysVarCacheKey, "") + if err != nil { + logutil.BgLogger().Warn("notify update sysvar cache failed", zap.Error(err)) + } + } + // update locally + if err := do.sysVarCache.RebuildSysVarCache(ctx); err != nil { + logutil.BgLogger().Error("rebuilding sysvar cache failed", zap.Error(err)) + } +} + // ServerID gets serverID. func (do *Domain) ServerID() uint64 { return atomic.LoadUint64(&do.serverID) diff --git a/domain/global_vars_cache.go b/domain/global_vars_cache.go deleted file mode 100644 index 52aa12a5ac955..0000000000000 --- a/domain/global_vars_cache.go +++ /dev/null @@ -1,135 +0,0 @@ -// Copyright 2018 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package domain - -import ( - "fmt" - "sync" - "time" - - "github.com/pingcap/parser/ast" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/stmtsummary" - "go.uber.org/zap" - "golang.org/x/sync/singleflight" -) - -// GlobalVariableCache caches global variables. -type GlobalVariableCache struct { - sync.RWMutex - lastModify time.Time - rows []chunk.Row - fields []*ast.ResultField - - // Unit test may like to disable it. - disable bool - SingleFight singleflight.Group -} - -// GlobalVariableCacheExpiry is the global variable cache TTL. -const GlobalVariableCacheExpiry = 2 * time.Second - -// Update updates the global variable cache. -func (gvc *GlobalVariableCache) Update(rows []chunk.Row, fields []*ast.ResultField) { - gvc.Lock() - gvc.lastModify = time.Now() - gvc.rows = rows - gvc.fields = fields - gvc.Unlock() - - checkEnableServerGlobalVar(rows) -} - -// Get gets the global variables from cache. -func (gvc *GlobalVariableCache) Get() (succ bool, rows []chunk.Row, fields []*ast.ResultField) { - gvc.RLock() - defer gvc.RUnlock() - if time.Since(gvc.lastModify) < GlobalVariableCacheExpiry { - succ, rows, fields = !gvc.disable, gvc.rows, gvc.fields - return - } - succ = false - return -} - -type loadResult struct { - rows []chunk.Row - fields []*ast.ResultField -} - -// LoadGlobalVariables will load from global cache first, loadFn will be executed if cache is not valid -func (gvc *GlobalVariableCache) LoadGlobalVariables(loadFn func() ([]chunk.Row, []*ast.ResultField, error)) ([]chunk.Row, []*ast.ResultField, error) { - succ, rows, fields := gvc.Get() - if succ { - return rows, fields, nil - } - fn := func() (interface{}, error) { - resRows, resFields, loadErr := loadFn() - if loadErr != nil { - return nil, loadErr - } - gvc.Update(resRows, resFields) - return &loadResult{resRows, resFields}, nil - } - res, err, _ := gvc.SingleFight.Do("loadGlobalVariable", fn) - if err != nil { - return nil, nil, err - } - loadRes := res.(*loadResult) - return loadRes.rows, loadRes.fields, nil -} - -// Disable disables the global variable cache, used in test only. -func (gvc *GlobalVariableCache) Disable() { - gvc.Lock() - defer gvc.Unlock() - gvc.disable = true -} - -// checkEnableServerGlobalVar processes variables that acts in server and global level. -func checkEnableServerGlobalVar(rows []chunk.Row) { - for _, row := range rows { - sVal := "" - if !row.IsNull(1) { - sVal = row.GetString(1) - } - var err error - switch row.GetString(0) { - case variable.TiDBEnableStmtSummary: - err = stmtsummary.StmtSummaryByDigestMap.SetEnabled(sVal, false) - case variable.TiDBStmtSummaryInternalQuery: - err = stmtsummary.StmtSummaryByDigestMap.SetEnabledInternalQuery(sVal, false) - case variable.TiDBStmtSummaryRefreshInterval: - err = stmtsummary.StmtSummaryByDigestMap.SetRefreshInterval(sVal, false) - case variable.TiDBStmtSummaryHistorySize: - err = stmtsummary.StmtSummaryByDigestMap.SetHistorySize(sVal, false) - case variable.TiDBStmtSummaryMaxStmtCount: - err = stmtsummary.StmtSummaryByDigestMap.SetMaxStmtCount(sVal, false) - case variable.TiDBStmtSummaryMaxSQLLength: - err = stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength(sVal, false) - case variable.TiDBCapturePlanBaseline: - variable.CapturePlanBaseline.Set(sVal, false) - } - if err != nil { - logutil.BgLogger().Error(fmt.Sprintf("load global variable %s error", row.GetString(0)), zap.Error(err)) - } - } -} - -// GetGlobalVarsCache gets the global variable cache. -func (do *Domain) GetGlobalVarsCache() *GlobalVariableCache { - return &do.gvc -} diff --git a/domain/global_vars_cache_test.go b/domain/global_vars_cache_test.go deleted file mode 100644 index 7358d709986af..0000000000000 --- a/domain/global_vars_cache_test.go +++ /dev/null @@ -1,221 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package domain - -import ( - "sync" - "sync/atomic" - "time" - - . "github.com/pingcap/check" - "github.com/pingcap/parser/ast" - "github.com/pingcap/parser/charset" - "github.com/pingcap/parser/model" - "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/stmtsummary" - "github.com/pingcap/tidb/util/testleak" -) - -var _ = SerialSuites(&testGVCSuite{}) - -type testGVCSuite struct{} - -func (gvcSuite *testGVCSuite) TestSimple(c *C) { - defer testleak.AfterTest(c)() - testleak.BeforeTest() - - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - defer func() { - err := store.Close() - c.Assert(err, IsNil) - }() - ddlLease := 50 * time.Millisecond - dom := NewDomain(store, ddlLease, 0, 0, mockFactory) - err = dom.Init(ddlLease, sysMockFactory) - c.Assert(err, IsNil) - defer dom.Close() - - // Get empty global vars cache. - gvc := dom.GetGlobalVarsCache() - succ, rows, fields := gvc.Get() - c.Assert(succ, IsFalse) - c.Assert(rows, IsNil) - c.Assert(fields, IsNil) - // Get a variable from global vars cache. - rf := getResultField("c", 1, 0) - rf1 := getResultField("c1", 2, 1) - ft := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - ft1 := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - ck := chunk.NewChunkWithCapacity([]*types.FieldType{ft, ft1}, 1024) - ck.AppendString(0, "variable1") - ck.AppendString(1, "value1") - row := ck.GetRow(0) - gvc.Update([]chunk.Row{row}, []*ast.ResultField{rf, rf1}) - succ, rows, fields = gvc.Get() - c.Assert(succ, IsTrue) - c.Assert(rows[0], Equals, row) - c.Assert(fields, DeepEquals, []*ast.ResultField{rf, rf1}) - // Disable the cache. - gvc.Disable() - succ, rows, fields = gvc.Get() - c.Assert(succ, IsFalse) - c.Assert(rows[0], Equals, row) - c.Assert(fields, DeepEquals, []*ast.ResultField{rf, rf1}) -} - -func getResultField(colName string, id, offset int) *ast.ResultField { - return &ast.ResultField{ - Column: &model.ColumnInfo{ - Name: model.NewCIStr(colName), - ID: int64(id), - Offset: offset, - FieldType: types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetUTF8, - Collate: charset.CollationUTF8, - }, - }, - TableAsName: model.NewCIStr("tbl"), - DBName: model.NewCIStr("test"), - } -} - -func (gvcSuite *testGVCSuite) TestConcurrentOneFlight(c *C) { - defer testleak.AfterTest(c)() - testleak.BeforeTest() - gvc := &GlobalVariableCache{} - succ, rows, fields := gvc.Get() - c.Assert(succ, IsFalse) - c.Assert(rows, IsNil) - c.Assert(fields, IsNil) - - // Get a variable from global vars cache. - rf := getResultField("c", 1, 0) - rf1 := getResultField("c1", 2, 1) - ft := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - ft1 := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - ckLow := chunk.NewChunkWithCapacity([]*types.FieldType{ft, ft1}, 1024) - val := "fromStorage" - val1 := "fromStorage1" - ckLow.AppendString(0, val) - ckLow.AppendString(1, val1) - - // Let cache become invalid, and try concurrent load - counter := int32(0) - waitToStart := new(sync.WaitGroup) - waitToStart.Add(1) - gvc.lastModify = time.Now().Add(time.Duration(-10) * time.Second) - loadFunc := func() ([]chunk.Row, []*ast.ResultField, error) { - time.Sleep(100 * time.Millisecond) - atomic.AddInt32(&counter, 1) - return []chunk.Row{ckLow.GetRow(0)}, []*ast.ResultField{rf, rf1}, nil - } - wg := new(sync.WaitGroup) - worker := 100 - resArray := make([]loadResult, worker) - for i := 0; i < worker; i++ { - wg.Add(1) - go func(idx int) { - defer wg.Done() - waitToStart.Wait() - resRow, resField, _ := gvc.LoadGlobalVariables(loadFunc) - resArray[idx].rows = resRow - resArray[idx].fields = resField - }(i) - } - waitToStart.Done() - wg.Wait() - succ, rows, fields = gvc.Get() - c.Assert(counter, Equals, int32(1)) - c.Assert(resArray[0].rows[0].GetString(0), Equals, val) - c.Assert(resArray[0].rows[0].GetString(1), Equals, val1) - for i := 0; i < worker; i++ { - c.Assert(resArray[0].rows[0], Equals, resArray[i].rows[0]) - c.Assert(resArray[i].rows[0].GetString(0), Equals, val) - c.Assert(resArray[i].rows[0].GetString(1), Equals, val1) - } - // Validate cache - c.Assert(succ, IsTrue) - c.Assert(rows[0], Equals, resArray[0].rows[0]) - c.Assert(fields, DeepEquals, []*ast.ResultField{rf, rf1}) -} - -func (gvcSuite *testGVCSuite) TestCheckEnableStmtSummary(c *C) { - defer testleak.AfterTest(c)() - testleak.BeforeTest() - - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - defer func() { - err := store.Close() - c.Assert(err, IsNil) - }() - ddlLease := 50 * time.Millisecond - dom := NewDomain(store, ddlLease, 0, 0, mockFactory) - err = dom.Init(ddlLease, sysMockFactory) - c.Assert(err, IsNil) - defer dom.Close() - - gvc := dom.GetGlobalVarsCache() - - rf := getResultField("c", 1, 0) - rf1 := getResultField("c1", 2, 1) - ft := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - ft1 := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - - err = stmtsummary.StmtSummaryByDigestMap.SetEnabled("0", false) - c.Assert(err, IsNil) - ck := chunk.NewChunkWithCapacity([]*types.FieldType{ft, ft1}, 1024) - ck.AppendString(0, variable.TiDBEnableStmtSummary) - ck.AppendString(1, "1") - row := ck.GetRow(0) - gvc.Update([]chunk.Row{row}, []*ast.ResultField{rf, rf1}) - c.Assert(stmtsummary.StmtSummaryByDigestMap.Enabled(), Equals, true) - - ck = chunk.NewChunkWithCapacity([]*types.FieldType{ft, ft1}, 1024) - ck.AppendString(0, variable.TiDBEnableStmtSummary) - ck.AppendString(1, "0") - row = ck.GetRow(0) - gvc.Update([]chunk.Row{row}, []*ast.ResultField{rf, rf1}) - c.Assert(stmtsummary.StmtSummaryByDigestMap.Enabled(), Equals, false) -} diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go new file mode 100644 index 0000000000000..23c9688ea2f81 --- /dev/null +++ b/domain/sysvar_cache.go @@ -0,0 +1,167 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package domain + +import ( + "context" + "fmt" + "sync" + + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/stmtsummary" + "go.uber.org/zap" +) + +// The sysvar cache replaces the GlobalVariableCache. +// It is an improvement because it operates similar to privilege cache, +// where it caches for 5 minutes instead of 2 seconds, plus it listens on etcd +// for updates from other servers. + +// SysVarCache represents the cache of system variables broken up into session and global scope. +type SysVarCache struct { + sync.RWMutex + global map[string]string + session map[string]string +} + +// GetSysVarCache gets the global variable cache. +func (do *Domain) GetSysVarCache() *SysVarCache { + return &do.sysVarCache +} + +func (svc *SysVarCache) rebuildCacheIfNeeded(ctx sessionctx.Context) (err error) { + svc.RLock() + cacheNeedsRebuild := len(svc.session) == 0 || len(svc.global) == 0 + svc.RUnlock() + if cacheNeedsRebuild { + logutil.BgLogger().Warn("sysvar cache is empty, triggering rebuild") + if err = svc.RebuildSysVarCache(ctx); err != nil { + logutil.BgLogger().Error("rebuilding sysvar cache failed", zap.Error(err)) + } + } + return err +} + +// GetSessionCache gets a copy of the session sysvar cache. +// The intention is to copy it directly to the systems[] map +// on creating a new session. +func (svc *SysVarCache) GetSessionCache(ctx sessionctx.Context) (map[string]string, error) { + if err := svc.rebuildCacheIfNeeded(ctx); err != nil { + return nil, err + } + svc.RLock() + defer svc.RUnlock() + // Perform a deep copy since this will be assigned directly to the session + newMap := make(map[string]string, len(svc.session)) + for k, v := range svc.session { + newMap[k] = v + } + return newMap, nil +} + +// GetGlobalVar gets an individual global var from the sysvar cache. +func (svc *SysVarCache) GetGlobalVar(ctx sessionctx.Context, name string) (string, error) { + if err := svc.rebuildCacheIfNeeded(ctx); err != nil { + return "", err + } + svc.RLock() + defer svc.RUnlock() + + if val, ok := svc.global[name]; ok { + return val, nil + } + logutil.BgLogger().Warn("could not find key in global cache", zap.String("name", name)) + return "", variable.ErrUnknownSystemVar.GenWithStackByArgs(name) +} + +func (svc *SysVarCache) fetchTableValues(ctx sessionctx.Context) (map[string]string, error) { + tableContents := make(map[string]string) + // Copy all variables from the table to tableContents + exec := ctx.(sqlexec.RestrictedSQLExecutor) + stmt, err := exec.ParseWithParams(context.Background(), `SELECT variable_name, variable_value FROM mysql.global_variables`) + if err != nil { + return tableContents, err + } + rows, _, err := exec.ExecRestrictedStmt(context.TODO(), stmt) + if err != nil { + return nil, err + } + for _, row := range rows { + name := row.GetString(0) + val := row.GetString(1) + tableContents[name] = val + } + return tableContents, nil +} + +// RebuildSysVarCache rebuilds the sysvar cache both globally and for session vars. +// It needs to be called when sysvars are added or removed. +func (svc *SysVarCache) RebuildSysVarCache(ctx sessionctx.Context) error { + newSessionCache := make(map[string]string) + newGlobalCache := make(map[string]string) + tableContents, err := svc.fetchTableValues(ctx) + if err != nil { + return err + } + + for _, sv := range variable.GetSysVars() { + sVal := sv.Value + if _, ok := tableContents[sv.Name]; ok { + sVal = tableContents[sv.Name] + } + if sv.HasSessionScope() { + newSessionCache[sv.Name] = sVal + } + if sv.HasGlobalScope() { + newGlobalCache[sv.Name] = sVal + } + // Propagate any changes to the server scoped variables + checkEnableServerGlobalVar(sv.Name, sVal) + } + + logutil.BgLogger().Debug("rebuilding sysvar cache") + + svc.Lock() + defer svc.Unlock() + svc.session = newSessionCache + svc.global = newGlobalCache + return nil +} + +// checkEnableServerGlobalVar processes variables that acts in server and global level. +func checkEnableServerGlobalVar(name, sVal string) { + var err error + switch name { + case variable.TiDBEnableStmtSummary: + err = stmtsummary.StmtSummaryByDigestMap.SetEnabled(sVal, false) + case variable.TiDBStmtSummaryInternalQuery: + err = stmtsummary.StmtSummaryByDigestMap.SetEnabledInternalQuery(sVal, false) + case variable.TiDBStmtSummaryRefreshInterval: + err = stmtsummary.StmtSummaryByDigestMap.SetRefreshInterval(sVal, false) + case variable.TiDBStmtSummaryHistorySize: + err = stmtsummary.StmtSummaryByDigestMap.SetHistorySize(sVal, false) + case variable.TiDBStmtSummaryMaxStmtCount: + err = stmtsummary.StmtSummaryByDigestMap.SetMaxStmtCount(sVal, false) + case variable.TiDBStmtSummaryMaxSQLLength: + err = stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength(sVal, false) + case variable.TiDBCapturePlanBaseline: + variable.CapturePlanBaseline.Set(sVal, false) + } + if err != nil { + logutil.BgLogger().Error(fmt.Sprintf("load global variable %s error", name), zap.Error(err)) + } +} diff --git a/executor/executor_test.go b/executor/executor_test.go index 5e6b4490f5eb6..7fa0d7b0d10bd 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2267,8 +2267,6 @@ func (s *testSuiteP2) TestSQLMode(c *C) { tk.MustExec("set sql_mode = 'STRICT_TRANS_TABLES'") tk.MustExec("set @@global.sql_mode = ''") - // Disable global variable cache, so load global session variable take effect immediate. - s.domain.GetGlobalVarsCache().Disable() tk2 := testkit.NewTestKit(c, s.store) tk2.MustExec("use test") tk2.MustExec("drop table if exists t2") diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 061e09dcc1315..bcecfc8d52ad4 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -1473,8 +1473,6 @@ func (s *seqTestSuite) TestMaxDeltaSchemaCount(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") c.Assert(variable.GetMaxDeltaSchemaCount(), Equals, int64(variable.DefTiDBMaxDeltaSchemaCount)) - gvc := domain.GetDomain(tk.Se).GetGlobalVarsCache() - gvc.Disable() tk.MustExec("set @@global.tidb_max_delta_schema_count= -1") tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_max_delta_schema_count value: '-1'")) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index ebe4a0620256f..1e5687928f3ad 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -965,8 +965,6 @@ func (s *testTableSuite) TestStmtSummaryTable(c *C) { tk.MustExec("set global tidb_enable_stmt_summary = 1") tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) - // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. - s.dom.GetGlobalVarsCache().Disable() // Disable refreshing summary. tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) @@ -1209,8 +1207,6 @@ func (s *testClusterTableSuite) TestStmtSummaryHistoryTable(c *C) { tk.MustExec("set global tidb_enable_stmt_summary = 1") tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) - // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. - s.dom.GetGlobalVarsCache().Disable() // Disable refreshing summary. tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) @@ -1266,8 +1262,6 @@ func (s *testTableSuite) TestStmtSummaryInternalQuery(c *C) { tk.MustExec("create global binding for select * from t where t.a = 1 using select * from t ignore index(k) where t.a = 1") tk.MustExec("set global tidb_enable_stmt_summary = 1") tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) - // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. - s.dom.GetGlobalVarsCache().Disable() // Disable refreshing summary. tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) diff --git a/metrics/domain.go b/metrics/domain.go index f30dbd59e5d32..a05b25dd6a46a 100644 --- a/metrics/domain.go +++ b/metrics/domain.go @@ -60,6 +60,15 @@ var ( Help: "Counter of load privilege", }, []string{LblType}) + // LoadSysVarCacheCounter records the counter of loading sysvars + LoadSysVarCacheCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "domain", + Name: "load_sysvarcache_total", + Help: "Counter of load sysvar cache", + }, []string{LblType}) + SchemaValidatorStop = "stop" SchemaValidatorRestart = "restart" SchemaValidatorReset = "reset" diff --git a/metrics/metrics.go b/metrics/metrics.go index 4a879b5d5423c..542398e7bbdee 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -151,6 +151,7 @@ func RegisterMetrics() { prometheus.MustRegister(TiFlashQueryTotalCounter) prometheus.MustRegister(SmallTxnWriteDuration) prometheus.MustRegister(TxnWriteThroughput) + prometheus.MustRegister(LoadSysVarCacheCounter) tikvmetrics.InitMetrics(TiDB, TiKVClient) tikvmetrics.RegisterMetrics() diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index cd43b3964d59b..d6bfe69f82b39 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -278,9 +278,6 @@ func (s *testPrepareSerialSuite) TestPrepareOverMaxPreparedStmtCount(c *C) { tk.MustExec("set @@global.max_prepared_stmt_count = 2") tk.MustQuery("select @@global.max_prepared_stmt_count").Check(testkit.Rows("2")) - // Disable global variable cache, so load global session variable take effect immediate. - dom.GetGlobalVarsCache().Disable() - // test close session to give up all prepared stmt tk.MustExec(`prepare stmt2 from "select 1"`) prePrepared = readGaugeInt(metrics.PreparedStmtGauge) diff --git a/session/session.go b/session/session.go index e1581d5ed4074..e13be2045e941 100644 --- a/session/session.go +++ b/session/session.go @@ -991,6 +991,7 @@ func (s *session) replaceTableValue(ctx context.Context, tblName string, varName return err } _, _, err = s.ExecRestrictedStmt(ctx, stmt) + domain.GetDomain(s).NotifyUpdateSysVarCache(s) return err } @@ -1011,16 +1012,27 @@ func (s *session) GetGlobalSysVar(name string) (string, error) { // When running bootstrap or upgrade, we should not access global storage. return "", nil } - sysVar, err := s.getTableValue(context.TODO(), mysql.GlobalVariablesTable, name) + + sv := variable.GetSysVar(name) + if sv == nil { + // It might be a recently unregistered sysvar. We should return unknown + // since GetSysVar is the canonical version, but we can update the cache + // so the next request doesn't attempt to load this. + logutil.BgLogger().Info("sysvar does not exist. sysvar cache may be stale", zap.String("name", name)) + return "", variable.ErrUnknownSystemVar.GenWithStackByArgs(name) + } + + sysVar, err := domain.GetDomain(s).GetSysVarCache().GetGlobalVar(s, name) if err != nil { - if errResultIsEmpty.Equal(err) { - sv := variable.GetSysVar(name) - if sv != nil { - return sv.Value, nil - } - return "", variable.ErrUnknownSystemVar.GenWithStackByArgs(name) + // The sysvar exists, but there is no cache entry yet. + // This might be because the sysvar was only recently registered. + // In which case it is safe to return the default, but we can also + // update the cache for the future. + logutil.BgLogger().Info("sysvar not in cache yet. sysvar cache may be stale", zap.String("name", name)) + sysVar, err = s.getTableValue(context.TODO(), mysql.GlobalVariablesTable, name) + if err != nil { + return sv.Value, nil } - return "", err } // Fetch mysql.tidb values if required if s.varFromTiDBTable(name) { @@ -1065,12 +1077,7 @@ func (s *session) updateGlobalSysVar(sv *variable.SysVar, value string) error { return err } } - stmt, err := s.ParseWithParams(context.TODO(), "REPLACE %n.%n VALUES (%?, %?)", mysql.SystemDB, mysql.GlobalVariablesTable, sv.Name, value) - if err != nil { - return err - } - _, _, err = s.ExecRestrictedStmt(context.TODO(), stmt) - return err + return s.replaceTableValue(context.TODO(), mysql.GlobalVariablesTable, sv.Name, value) } // setTiDBTableValue handles tikv_* sysvars which need to update mysql.tidb @@ -2330,13 +2337,18 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { } } + // Rebuild sysvar cache in a loop + err = dom.LoadSysVarCacheLoop(se) + if err != nil { + return nil, err + } + if len(cfg.Plugin.Load) > 0 { err := plugin.Init(context.Background(), plugin.Config{EtcdClient: dom.GetEtcdClient()}) if err != nil { return nil, err } } - se4, err := createSession(store) if err != nil { return nil, err @@ -2439,7 +2451,7 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) { // CreateSessionWithDomain creates a new Session and binds it with a Domain. // We need this because when we start DDL in Domain, the DDL need a session // to change some system tables. But at that time, we have been already in -// a lock context, which cause we can't call createSesion directly. +// a lock context, which cause we can't call createSession directly. func CreateSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, error) { s := &session{ store: store, @@ -2647,38 +2659,30 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { return nil } - var err error - // Use GlobalVariableCache if TiDB just loaded global variables within 2 second ago. - // When a lot of connections connect to TiDB simultaneously, it can protect TiKV meta region from overload. - gvc := domain.GetDomain(s).GetGlobalVarsCache() - loadFunc := func() ([]chunk.Row, []*ast.ResultField, error) { - vars := append(make([]string, 0, len(builtinGlobalVariable)+len(variable.PluginVarNames)), builtinGlobalVariable...) - if len(variable.PluginVarNames) > 0 { - vars = append(vars, variable.PluginVarNames...) - } - - stmt, err := s.ParseWithParams(context.TODO(), "select HIGH_PRIORITY * from mysql.global_variables where variable_name in (%?) order by VARIABLE_NAME", vars) - if err != nil { - return nil, nil, errors.Trace(err) - } + vars.CommonGlobalLoaded = true - return s.ExecRestrictedStmt(context.TODO(), stmt) - } - rows, _, err := gvc.LoadGlobalVariables(loadFunc) + // Deep copy sessionvar cache + // Eventually this whole map will be applied to systems[], which is a MySQL behavior. + sessionCache, err := domain.GetDomain(s).GetSysVarCache().GetSessionCache(s) if err != nil { - logutil.BgLogger().Warn("failed to load global variables", - zap.Uint64("conn", s.sessionVars.ConnectionID), zap.Error(err)) return err } - vars.CommonGlobalLoaded = true - - for _, row := range rows { - varName := row.GetString(0) - varVal := row.GetString(1) + for _, varName := range builtinGlobalVariable { + // The item should be in the sessionCache, but due to a strange current behavior there are some Global-only + // vars that are in builtinGlobalVariable. For compatibility we need to fall back to the Global cache on these items. + // TODO: don't load these globals into the session! + var varVal string + var ok bool + if varVal, ok = sessionCache[varName]; !ok { + varVal, err = s.GetGlobalSysVar(varName) + if err != nil { + continue // skip variables that are not loaded. + } + } // `collation_server` is related to `character_set_server`, set `character_set_server` will also set `collation_server`. // We have to make sure we set the `collation_server` with right value. if _, ok := vars.GetSystemVar(varName); !ok || varName == variable.CollationServer { - err = vars.SetSystemVar(varName, varVal) + err = vars.SetSystemVarWithRelaxedValidation(varName, varVal) if err != nil { return err } @@ -2693,8 +2697,6 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { } } } - - vars.CommonGlobalLoaded = true return nil } diff --git a/session/session_test.go b/session/session_test.go index b7cfecc9c5f4c..9845e757470f0 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -213,7 +213,6 @@ func (s *testSessionSuiteBase) SetUpSuite(c *C) { var err error s.dom, err = session.BootstrapSession(s.store) c.Assert(err, IsNil) - s.dom.GetGlobalVarsCache().Disable() } func (s *testSessionSuiteBase) TearDownSuite(c *C) { @@ -639,7 +638,6 @@ func (s *testSessionSuite) TestGlobalVarAccessor(c *C) { c.Assert(v, Equals, varValue2) // For issue 10955, make sure the new session load `max_execution_time` into sessionVars. - s.dom.GetGlobalVarsCache().Disable() tk1.MustExec("set @@global.max_execution_time = 100") tk2 := testkit.NewTestKitWithInit(c, s.store) c.Assert(tk2.Se.GetSessionVars().MaxExecutionTime, Equals, uint64(100)) @@ -2618,8 +2616,6 @@ func (s *testSessionSuite) TestSetGlobalTZ(c *C) { tk.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +08:00")) - // Disable global variable cache, so load global session variable take effect immediate. - s.dom.GetGlobalVarsCache().Disable() tk1 := testkit.NewTestKitWithInit(c, s.store) tk1.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +00:00")) } @@ -2761,8 +2757,6 @@ func (s *testSessionSuite3) TestEnablePartition(c *C) { tk.MustExec("set tidb_enable_list_partition=on") tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) - // Disable global variable cache, so load global session variable take effect immediate. - s.dom.GetGlobalVarsCache().Disable() tk1 := testkit.NewTestKitWithInit(c, s.store) tk1.MustQuery("show variables like 'tidb_enable_table_partition'").Check(testkit.Rows("tidb_enable_table_partition ON")) } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 0c6c74d90a26d..c474e7905fa7b 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1421,6 +1421,15 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { return sv.SetSessionFromHook(s, val) } +// SetSystemVarWithRelaxedValidation sets the value of a system variable for session scope. +// Validation functions are called, but scope validation is skipped. +// Errors are not expected to be returned because this could cause upgrade issues. +func (s *SessionVars) SetSystemVarWithRelaxedValidation(name string, val string) error { + sv := GetSysVar(name) + val = sv.ValidateWithRelaxedValidation(s, val, ScopeSession) + return sv.SetSessionFromHook(s, val) +} + // GetReadableTxnMode returns the session variable TxnMode but rewrites it to "OPTIMISTIC" when it's empty. func (s *SessionVars) GetReadableTxnMode() string { txnMode := s.TxnMode diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 98518fe4af0f0..99c3da8233d68 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -188,6 +188,10 @@ func (sv *SysVar) HasGlobalScope() bool { // Validate checks if system variable satisfies specific restriction. func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (string, error) { + // Check that the scope is correct first. + if err := sv.validateScope(scope); err != nil { + return value, err + } // Normalize the value and apply validation based on type. // i.e. TypeBool converts 1/on/ON to ON. normalizedValue, err := sv.validateFromType(vars, value, scope) @@ -203,17 +207,6 @@ func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (st // validateFromType provides automatic validation based on the SysVar's type func (sv *SysVar) validateFromType(vars *SessionVars, value string, scope ScopeFlag) (string, error) { - // Check that the scope is correct and return the appropriate error message. - if sv.ReadOnly || sv.Scope == ScopeNone { - return value, ErrIncorrectScope.FastGenByArgs(sv.Name, "read only") - } - if scope == ScopeGlobal && !sv.HasGlobalScope() { - return value, errLocalVariable.FastGenByArgs(sv.Name) - } - if scope == ScopeSession && !sv.HasSessionScope() { - return value, errGlobalVariable.FastGenByArgs(sv.Name) - } - // The string "DEFAULT" is a special keyword in MySQL, which restores // the compiled sysvar value. In which case we can skip further validation. if strings.EqualFold(value, "DEFAULT") { @@ -245,6 +238,37 @@ func (sv *SysVar) validateFromType(vars *SessionVars, value string, scope ScopeF return value, nil // typeString } +func (sv *SysVar) validateScope(scope ScopeFlag) error { + if sv.ReadOnly || sv.Scope == ScopeNone { + return ErrIncorrectScope.FastGenByArgs(sv.Name, "read only") + } + if scope == ScopeGlobal && !sv.HasGlobalScope() { + return errLocalVariable.FastGenByArgs(sv.Name) + } + if scope == ScopeSession && !sv.HasSessionScope() { + return errGlobalVariable.FastGenByArgs(sv.Name) + } + return nil +} + +// ValidateWithRelaxedValidation normalizes values but can not return errors. +// Normalization+validation needs to be applied when reading values because older versions of TiDB +// may be less sophisticated in normalizing values. But errors should be caught and handled, +// because otherwise there will be upgrade issues. +func (sv *SysVar) ValidateWithRelaxedValidation(vars *SessionVars, value string, scope ScopeFlag) string { + normalizedValue, err := sv.validateFromType(vars, value, scope) + if err != nil { + return normalizedValue + } + if sv.Validation != nil { + normalizedValue, err = sv.Validation(vars, normalizedValue, value, scope) + if err != nil { + return normalizedValue + } + } + return normalizedValue +} + const ( localDayTimeFormat = "15:04" // FullDayTimeFormat is the full format of analyze start time and end time. @@ -485,11 +509,15 @@ func SetSysVar(name string, value string) { sysVars[name].Value = value } -// GetSysVars returns the sysVars list under a RWLock +// GetSysVars deep copies the sysVars list under a RWLock func GetSysVars() map[string]*SysVar { sysVarsLock.RLock() defer sysVarsLock.RUnlock() - return sysVars + copy := make(map[string]*SysVar, len(sysVars)) + for name, sv := range sysVars { + copy[name] = sv + } + return copy } // PluginVarNames is global plugin var names set. From 9148ff9f44555d681a401b3f33681f7dc2414b6b Mon Sep 17 00:00:00 2001 From: djshow832 Date: Mon, 17 May 2021 22:31:39 +0800 Subject: [PATCH 42/42] ddl, transaction: DDL on temporary tables won't affect transactions (#24534) --- domain/schema_validator.go | 5 +++-- session/session.go | 6 ++++++ session/session_test.go | 39 ++++++++++++++++++++++++++++++++++++++ 3 files changed, 48 insertions(+), 2 deletions(-) diff --git a/domain/schema_validator.go b/domain/schema_validator.go index b983eff1d6203..a8baa49db93b9 100644 --- a/domain/schema_validator.go +++ b/domain/schema_validator.go @@ -234,8 +234,9 @@ func (s *schemaValidator) Check(txnTS uint64, schemaVer int64, relatedPhysicalTa // Schema changed, result decided by whether related tables change. if schemaVer < s.latestSchemaVer { - // The DDL relatedPhysicalTableIDs is empty. - if len(relatedPhysicalTableIDs) == 0 { + // When a transaction executes a DDL, relatedPhysicalTableIDs is nil. + // When a transaction only contains DML on temporary tables, relatedPhysicalTableIDs is []. + if relatedPhysicalTableIDs == nil { logutil.BgLogger().Info("the related physical table ID is empty", zap.Int64("schemaVer", schemaVer), zap.Int64("latestSchemaVer", s.latestSchemaVer)) return nil, ResultFail diff --git a/session/session.go b/session/session.go index e13be2045e941..78a60a6ebaecf 100644 --- a/session/session.go +++ b/session/session.go @@ -505,8 +505,14 @@ func (s *session) doCommit(ctx context.Context) error { // Get the related table or partition IDs. relatedPhysicalTables := s.GetSessionVars().TxnCtx.TableDeltaMap + // Get accessed global temporary tables in the transaction. + temporaryTables := s.GetSessionVars().TxnCtx.GlobalTemporaryTables physicalTableIDs := make([]int64, 0, len(relatedPhysicalTables)) for id := range relatedPhysicalTables { + // Schema change on global temporary tables doesn't affect transactions. + if _, ok := temporaryTables[id]; ok { + continue + } physicalTableIDs = append(physicalTableIDs, id) } // Set this option for 2 phase commit to validate schema lease. diff --git a/session/session_test.go b/session/session_test.go index 9845e757470f0..f7267e3a13259 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2131,6 +2131,45 @@ func (s *testSchemaSerialSuite) TestSchemaCheckerSQL(c *C) { c.Assert(err, NotNil) } +func (s *testSchemaSerialSuite) TestSchemaCheckerTempTable(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk1 := testkit.NewTestKitWithInit(c, s.store) + + // create table + tk.MustExec(`drop table if exists normal_table`) + tk.MustExec(`create table normal_table (id int, c int);`) + defer tk.MustExec(`drop table if exists normal_table`) + tk.MustExec(`drop table if exists temp_table`) + tk.MustExec(`create global temporary table temp_table (id int, c int) on commit delete rows;`) + defer tk.MustExec(`drop table if exists temp_table`) + + // The schema version is out of date in the first transaction, and the SQL can't be retried. + atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 1) + defer func() { + atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 0) + }() + + // It's fine to change the schema of temporary tables. + tk.MustExec(`begin;`) + tk1.MustExec(`alter table temp_table modify column c bigint;`) + tk.MustExec(`insert into temp_table values(3, 3);`) + tk.MustExec(`commit;`) + + // Truncate will modify table ID. + tk.MustExec(`begin;`) + tk1.MustExec(`truncate table temp_table;`) + tk.MustExec(`insert into temp_table values(3, 3);`) + tk.MustExec(`commit;`) + + // It reports error when also changing the schema of a normal table. + tk.MustExec(`begin;`) + tk1.MustExec(`alter table normal_table modify column c bigint;`) + tk.MustExec(`insert into temp_table values(3, 3);`) + tk.MustExec(`insert into normal_table values(3, 3);`) + _, err := tk.Exec(`commit;`) + c.Assert(terror.ErrorEqual(err, domain.ErrInfoSchemaChanged), IsTrue, Commentf("err %v", err)) +} + func (s *testSchemaSuite) TestPrepareStmtCommitWhenSchemaChanged(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk1 := testkit.NewTestKitWithInit(c, s.store)