From 1bd689585e36162e1a186eae01e868b0ca1fd1fd Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 27 Apr 2022 10:16:51 +0800 Subject: [PATCH 01/51] executor: fix CTE may be blocked when query report error (#33085) (#33190) close pingcap/tidb#31302 --- executor/cte.go | 3 --- executor/cte_table_reader.go | 3 --- executor/cte_test.go | 32 +++++++++++++++++++++++++++++ executor/index_lookup_hash_join.go | 9 ++++++-- executor/index_lookup_join.go | 7 ++++++- executor/index_lookup_merge_join.go | 7 ++++++- util/cteutil/storage.go | 10 --------- 7 files changed, 51 insertions(+), 20 deletions(-) diff --git a/executor/cte.go b/executor/cte.go index a9f083fa70479..7ee4a78dc417b 100644 --- a/executor/cte.go +++ b/executor/cte.go @@ -230,8 +230,6 @@ func (e *CTEExec) Close() (err error) { func (e *CTEExec) computeSeedPart(ctx context.Context) (err error) { e.curIter = 0 e.iterInTbl.SetIter(e.curIter) - // This means iterInTbl's can be read. - defer close(e.iterInTbl.GetBegCh()) chks := make([]*chunk.Chunk, 0, 10) for { if e.limitDone(e.iterInTbl) { @@ -384,7 +382,6 @@ func (e *CTEExec) setupTblsForNewIteration() (err error) { if err = e.iterInTbl.Reopen(); err != nil { return err } - defer close(e.iterInTbl.GetBegCh()) if e.isDistinct { // Already deduplicated by resTbl, adding directly is ok. for _, chk := range chks { diff --git a/executor/cte_table_reader.go b/executor/cte_table_reader.go index efd5a0387e6cb..4afd8aabbb79f 100644 --- a/executor/cte_table_reader.go +++ b/executor/cte_table_reader.go @@ -41,9 +41,6 @@ func (e *CTETableReaderExec) Open(ctx context.Context) error { func (e *CTETableReaderExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { req.Reset() - // Wait until iterInTbl can be read. This is controlled by corresponding CTEExec. - <-e.iterInTbl.GetBegCh() - // We should read `iterInTbl` from the beginning when the next iteration starts. // Can not directly judge whether to start the next iteration based on e.chkIdx, // because some operators(Selection) may use forloop to read all data in `iterInTbl`. diff --git a/executor/cte_test.go b/executor/cte_test.go index bf6d33ede4a42..92c1c861fcc23 100644 --- a/executor/cte_test.go +++ b/executor/cte_test.go @@ -22,7 +22,9 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/types" "github.com/stretchr/testify/require" ) @@ -408,3 +410,33 @@ func TestSpillToDisk(t *testing.T) { } rows.Check(testkit.Rows(resRows...)) } + +func TestCTEExecError(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists src;") + tk.MustExec("create table src(first int, second int);") + + insertStr := fmt.Sprintf("insert into src values (%d, %d)", rand.Intn(1000), rand.Intn(1000)) + for i := 0; i < 1000; i++ { + insertStr += fmt.Sprintf(",(%d, %d)", rand.Intn(1000), rand.Intn(1000)) + } + insertStr += ";" + tk.MustExec(insertStr) + + // Increase projection concurrency and decrease chunk size + // to increase the probability of reproducing the problem. + tk.MustExec("set tidb_max_chunk_size = 32") + tk.MustExec("set tidb_projection_concurrency = 20") + for i := 0; i < 10; i++ { + err := tk.QueryToErr("with recursive cte(iter, first, second, result) as " + + "(select 1, first, second, first+second from src " + + " union all " + + "select iter+1, second, result, second+result from cte where iter < 80 )" + + "select * from cte") + require.True(t, terror.ErrorEqual(err, types.ErrOverflow)) + } +} diff --git a/executor/index_lookup_hash_join.go b/executor/index_lookup_hash_join.go index 503dad70a8731..069c7b49a51bc 100644 --- a/executor/index_lookup_hash_join.go +++ b/executor/index_lookup_hash_join.go @@ -70,7 +70,8 @@ type IndexNestedLoopHashJoin struct { // taskCh is only used when `keepOuterOrder` is true. taskCh chan *indexHashJoinTask - stats *indexLookUpJoinRuntimeStats + stats *indexLookUpJoinRuntimeStats + prepared bool } type indexHashJoinOuterWorker struct { @@ -155,7 +156,6 @@ func (e *IndexNestedLoopHashJoin) Open(ctx context.Context) error { e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } e.finished.Store(false) - e.startWorkers(ctx) return nil } @@ -229,6 +229,10 @@ func (e *IndexNestedLoopHashJoin) wait4JoinWorkers() { // Next implements the IndexNestedLoopHashJoin Executor interface. func (e *IndexNestedLoopHashJoin) Next(ctx context.Context, req *chunk.Chunk) error { + if !e.prepared { + e.startWorkers(ctx) + e.prepared = true + } req.Reset() if e.keepOuterOrder { return e.runInOrder(ctx, req) @@ -321,6 +325,7 @@ func (e *IndexNestedLoopHashJoin) Close() error { } e.joinChkResourceCh = nil e.finished.Store(false) + e.prepared = false return e.baseExecutor.Close() } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 44a7bdb4b9ca1..994385566dde3 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -86,6 +86,7 @@ type IndexLookUpJoin struct { stats *indexLookUpJoinRuntimeStats finished *atomic.Value + prepared bool } type outerCtx struct { @@ -174,7 +175,6 @@ func (e *IndexLookUpJoin) Open(ctx context.Context) error { e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } e.cancelFunc = nil - e.startWorkers(ctx) return nil } @@ -258,6 +258,10 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork // Next implements the Executor interface. func (e *IndexLookUpJoin) Next(ctx context.Context, req *chunk.Chunk) error { + if !e.prepared { + e.startWorkers(ctx) + e.prepared = true + } if e.isOuterJoin { atomic.StoreInt64(&e.requiredRows, int64(req.RequiredRows())) } @@ -764,6 +768,7 @@ func (e *IndexLookUpJoin) Close() error { e.memTracker = nil e.task = nil e.finished.Store(false) + e.prepared = false return e.baseExecutor.Close() } diff --git a/executor/index_lookup_merge_join.go b/executor/index_lookup_merge_join.go index 7e4ac6a515ae9..999d22144bee9 100644 --- a/executor/index_lookup_merge_join.go +++ b/executor/index_lookup_merge_join.go @@ -73,6 +73,7 @@ type IndexLookUpMergeJoin struct { lastColHelper *plannercore.ColWithCmpFuncManager memTracker *memory.Tracker // track memory usage + prepared bool } type outerMergeCtx struct { @@ -184,7 +185,6 @@ func (e *IndexLookUpMergeJoin) Open(ctx context.Context) error { } e.memTracker = memory.NewTracker(e.id, -1) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) - e.startWorkers(ctx) return nil } @@ -271,6 +271,10 @@ func (e *IndexLookUpMergeJoin) newInnerMergeWorker(taskCh chan *lookUpMergeJoinT // Next implements the Executor interface func (e *IndexLookUpMergeJoin) Next(ctx context.Context, req *chunk.Chunk) error { + if !e.prepared { + e.startWorkers(ctx) + e.prepared = true + } if e.isOuterJoin { atomic.StoreInt64(&e.requiredRows, int64(req.RequiredRows())) } @@ -753,5 +757,6 @@ func (e *IndexLookUpMergeJoin) Close() error { // cancelFunc control the outer worker and outer worker close the task channel. e.workerWg.Wait() e.memTracker = nil + e.prepared = false return e.baseExecutor.Close() } diff --git a/util/cteutil/storage.go b/util/cteutil/storage.go index 19b1bd5151fdc..a629398000898 100644 --- a/util/cteutil/storage.go +++ b/util/cteutil/storage.go @@ -82,11 +82,6 @@ type Storage interface { SetIter(iter int) GetIter() int - // We use this channel to notify reader that Storage is ready to read. - // It exists only to solve the special implementation of IndexLookUpJoin. - // We will find a better way and remove this later. - GetBegCh() chan struct{} - GetMemTracker() *memory.Tracker GetDiskTracker() *disk.Tracker ActionSpill() *chunk.SpillDiskAction @@ -239,11 +234,6 @@ func (s *StorageRC) GetIter() int { return s.iter } -// GetBegCh impls Storage GetBegCh interface. -func (s *StorageRC) GetBegCh() chan struct{} { - return s.begCh -} - // GetMemTracker impls Storage GetMemTracker interface. func (s *StorageRC) GetMemTracker() *memory.Tracker { return s.rc.GetMemTracker() From 74cc44b5fca361bc050848f19a14cd9d00bfdb90 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 27 Apr 2022 10:36:51 +0800 Subject: [PATCH 02/51] domain: adjust the order of acquireServerID and GlobalInfoSyncerInit to fix global kill test (#33536) (#34255) close pingcap/tidb#33538 --- domain/domain.go | 37 +++++++++++++++++++------------------ 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 073a2dbb0b94a..94191d0c91394 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -811,6 +811,25 @@ func (do *Domain) Init(ddlLease time.Duration, sysExecutorFactory func(*Domain) do.ddl = d } }) + + if config.GetGlobalConfig().Experimental.EnableGlobalKill { + if do.etcdClient != nil { + err := do.acquireServerID(ctx) + if err != nil { + logutil.BgLogger().Error("acquire serverID failed", zap.Error(err)) + do.isLostConnectionToPD.Set(1) // will retry in `do.serverIDKeeper` + } else { + do.isLostConnectionToPD.Set(0) + } + + do.wg.Add(1) + go do.serverIDKeeper() + } else { + // set serverID for standalone deployment to enable 'KILL'. + atomic.StoreUint64(&do.serverID, serverIDForStandalone) + } + } + // step 1: prepare the info/schema syncer which domain reload needed. skipRegisterToDashboard := config.GetGlobalConfig().SkipRegisterToDashboard do.info, err = infosync.GlobalInfoSyncerInit(ctx, do.ddl.GetID(), do.ServerID, do.etcdClient, skipRegisterToDashboard) @@ -833,24 +852,6 @@ func (do *Domain) Init(ddlLease time.Duration, sysExecutorFactory func(*Domain) return err } - if config.GetGlobalConfig().Experimental.EnableGlobalKill { - if do.etcdClient != nil { - err := do.acquireServerID(ctx) - if err != nil { - logutil.BgLogger().Error("acquire serverID failed", zap.Error(err)) - do.isLostConnectionToPD.Set(1) // will retry in `do.serverIDKeeper` - } else { - do.isLostConnectionToPD.Set(0) - } - - do.wg.Add(1) - go do.serverIDKeeper() - } else { - // set serverID for standalone deployment to enable 'KILL'. - atomic.StoreUint64(&do.serverID, serverIDForStandalone) - } - } - // Only when the store is local that the lease value is 0. // If the store is local, it doesn't need loadSchemaInLoop. if ddlLease > 0 { From d618e6d15bb828d31e4e1fbd3254574852b10a0c Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 27 Apr 2022 15:40:51 +0800 Subject: [PATCH 03/51] session: fix tables_priv table schema (#33599) (#33605) close pingcap/tidb#33588 --- executor/infoschema_reader_test.go | 6 +++--- session/bootstrap.go | 26 +++++++++++++++++++++++++- 2 files changed, 28 insertions(+), 4 deletions(-) diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 5e40147b99939..62345610c28b7 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -918,7 +918,7 @@ func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { tk.MustQuery("select TABLE_SCHEMA, sum(TABLE_SIZE) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' group by TABLE_SCHEMA;").Check(testkit.Rows( "test 2", )) - c.Assert(len(tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Rows()), Equals, 28) + c.Assert(len(tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Rows()), Equals, 29) // More tests about the privileges. tk.MustExec("create user 'testuser'@'localhost'") @@ -944,14 +944,14 @@ func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { Hostname: "localhost", }, nil, nil), Equals, true) - tk.MustQuery("select count(1) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql'").Check(testkit.Rows("28")) + tk.MustQuery("select count(1) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql'").Check(testkit.Rows("29")) c.Assert(tk.Se.Auth(&auth.UserIdentity{ Username: "testuser3", Hostname: "localhost", }, nil, nil), Equals, true) - tk.MustQuery("select count(1) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql'").Check(testkit.Rows("28")) + tk.MustQuery("select count(1) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql'").Check(testkit.Rows("29")) } func (s *testInfoschemaTableSuite) TestSequences(c *C) { diff --git a/session/bootstrap.go b/session/bootstrap.go index a3466cb8c87cb..5a1d4edb697b5 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -376,6 +376,16 @@ const ( column_ids TEXT(19372), PRIMARY KEY (table_id) CLUSTERED );` + // CreateStatsHistory stores the historical stats. + CreateStatsHistory = `CREATE TABLE IF NOT EXISTS mysql.stats_history ( + table_id bigint(64) NOT NULL, + stats_data longblob NOT NULL, + seq_no bigint(64) NOT NULL comment 'sequence number of the gzipped data slice', + version bigint(64) NOT NULL comment 'stats version which corresponding to stats:version in EXPLAIN', + create_time datetime(6) NOT NULL, + UNIQUE KEY table_version_seq (table_id, version, seq_no), + KEY table_create_time (table_id, create_time, seq_no) + );` ) // bootstrap initiates system DB for a store. @@ -557,11 +567,14 @@ const ( version81 = 81 // version82 adds the mysql.analyze_options table version82 = 82 + // version83 adds the tables mysql.stats_history. + // (In TiDB 6.0 and later this is done in version86.) /And update mysql.tables_priv from SET('Select','Insert','Update') to SET('Select','Insert','Update','References'). + version83 = 83 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. // please make sure this is the largest version -var currentBootstrapVersion int64 = version82 +var currentBootstrapVersion int64 = version83 var ( bootstrapVersion = []func(Session, int64){ @@ -647,6 +660,7 @@ var ( upgradeToVer80, upgradeToVer81, upgradeToVer82, + upgradeToVer83, } ) @@ -1703,6 +1717,14 @@ func upgradeToVer82(s Session, ver int64) { doReentrantDDL(s, CreateAnalyzeOptionsTable) } +func upgradeToVer83(s Session, ver int64) { + if ver >= version83 { + return + } + doReentrantDDL(s, CreateStatsHistory) + doReentrantDDL(s, "ALTER TABLE mysql.tables_priv MODIFY COLUMN Column_priv SET('Select','Insert','Update','References')") +} + func writeOOMAction(s Session) { comment := "oom-action is `log` by default in v3.0.x, `cancel` by default in v4.0.11+" mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE VARIABLE_VALUE= %?`, @@ -1789,6 +1811,8 @@ func doDDLWorks(s Session) { mustExecute(s, CreateTableCacheMetaTable) // Create analyze_options table. mustExecute(s, CreateAnalyzeOptionsTable) + // Create stats_history table. + mustExecute(s, CreateStatsHistory) } // doDMLWorks executes DML statements in bootstrap stage. From 9687ea4229c4328da51b995cd06be9de78798de7 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 27 Apr 2022 15:56:51 +0800 Subject: [PATCH 04/51] planner: Fix the issue that TiDB may dispatch duplicated tasks to TiFlash (#32813) (#32841) ref pingcap/tics#4163, close pingcap/tidb#32814 --- planner/core/fragment.go | 3 +- planner/core/fragment_test.go | 53 +++++++++++++++++++++++++++++++++++ 2 files changed, 55 insertions(+), 1 deletion(-) create mode 100644 planner/core/fragment_test.go diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 491f35aed83e7..f84c1421185c3 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -126,7 +126,8 @@ func (f *Fragment) init(p PhysicalPlan) error { } f.TableScan = x case *PhysicalExchangeReceiver: - f.singleton = x.children[0].(*PhysicalExchangeSender).ExchangeType == tipb.ExchangeType_PassThrough + // TODO: after we support partial merge, we should check whether all the target exchangeReceiver is same. + f.singleton = f.singleton || x.children[0].(*PhysicalExchangeSender).ExchangeType == tipb.ExchangeType_PassThrough f.ExchangeReceivers = append(f.ExchangeReceivers, x) case *PhysicalUnionAll: return errors.New("unexpected union all detected") diff --git a/planner/core/fragment_test.go b/planner/core/fragment_test.go new file mode 100644 index 0000000000000..fa8ec9e99763c --- /dev/null +++ b/planner/core/fragment_test.go @@ -0,0 +1,53 @@ +// Copyright 2020 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "github.com/pingcap/tipb/go-tipb" + "github.com/stretchr/testify/require" + + "testing" +) + +func TestFragmentInitSingleton(t *testing.T) { + r1, r2 := &PhysicalExchangeReceiver{}, &PhysicalExchangeReceiver{} + r1.SetChildren(&PhysicalExchangeSender{ExchangeType: tipb.ExchangeType_PassThrough}) + r2.SetChildren(&PhysicalExchangeSender{ExchangeType: tipb.ExchangeType_Broadcast}) + p := &PhysicalHashJoin{} + + f := &Fragment{} + p.SetChildren(r1, r1) + err := f.init(p) + require.NoError(t, err) + require.Equal(t, f.singleton, true) + + f = &Fragment{} + p.SetChildren(r1, r2) + err = f.init(p) + require.NoError(t, err) + require.Equal(t, f.singleton, true) + + f = &Fragment{} + p.SetChildren(r2, r1) + err = f.init(p) + require.NoError(t, err) + require.Equal(t, f.singleton, true) + + f = &Fragment{} + p.SetChildren(r2, r2) + err = f.init(p) + require.NoError(t, err) + require.Equal(t, f.singleton, false) +} From 2a86e884ce93828165216e30cca120ee9e106645 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 27 Apr 2022 16:56:52 +0800 Subject: [PATCH 05/51] planner: outer merge join cannot keep the prop of its inner child (#33359) (#33375) close pingcap/tidb#33042 --- planner/core/exhaust_physical_plans.go | 11 ++++++++++- planner/core/integration_test.go | 19 +++++++++++++++++++ .../core/testdata/integration_suite_out.json | 4 ++-- planner/core/testdata/plan_suite_out.json | 2 +- 4 files changed, 32 insertions(+), 4 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index db9c16e722d78..3e6e4e4b23aec 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -287,14 +287,16 @@ func (p *LogicalJoin) getEnforcedMergeJoin(prop *property.PhysicalProperty, sche return nil } for _, item := range prop.SortItems { - isExist := false + isExist, hasLeftColInProp, hasRightColInProp := false, false, false for joinKeyPos := 0; joinKeyPos < len(leftJoinKeys); joinKeyPos++ { var key *expression.Column if item.Col.Equal(p.ctx, leftJoinKeys[joinKeyPos]) { key = leftJoinKeys[joinKeyPos] + hasLeftColInProp = true } if item.Col.Equal(p.ctx, rightJoinKeys[joinKeyPos]) { key = rightJoinKeys[joinKeyPos] + hasRightColInProp = true } if key == nil { continue @@ -314,6 +316,13 @@ func (p *LogicalJoin) getEnforcedMergeJoin(prop *property.PhysicalProperty, sche if !isExist { return nil } + // If the output wants the order of the inner side. We should reject it since we might add null-extend rows of that side. + if p.JoinType == LeftOuterJoin && hasRightColInProp { + return nil + } + if p.JoinType == RightOuterJoin && hasLeftColInProp { + return nil + } } // Generate the enforced sort merge join leftKeys := getNewJoinKeysByOffsets(leftJoinKeys, offsets) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 7e1c9c1cf9612..544a4bf0f2266 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -5227,6 +5227,25 @@ func (s *testIntegrationSuite) TestIssue31202(c *C) { tk.MustExec("drop table if exists t31202") } +func (s *testIntegrationSuite) TestIssue33042(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("create table t1(id int primary key, col1 int)") + tk.MustExec("create table t2(id int primary key, col1 int)") + tk.MustQuery("explain format='brief' SELECT /*+ merge_join(t1, t2)*/ * FROM (t1 LEFT JOIN t2 ON t1.col1=t2.id) order by t2.id;").Check( + testkit.Rows( + "Sort 12500.00 root test.t2.id", + "└─MergeJoin 12500.00 root left outer join, left key:test.t1.col1, right key:test.t2.id", + " ├─TableReader(Build) 10000.00 root data:TableFullScan", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:true, stats:pseudo", + " └─Sort(Probe) 10000.00 root test.t1.col1", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + ), + ) +} + // TestDNFCondSelectivityWithConst test selectivity calculation with DNF conditions with one is const. // Close https://github.com/pingcap/tidb/issues/31096 func (s *testIntegrationSuite) TestDNFCondSelectivityWithConst(c *C) { diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index dd4330fe1310e..0de02272836ec 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1332,7 +1332,7 @@ "└─IndexRangeScan 20.00 cop[tikv] table:tt, index:a(a) range:[10,10], [20,20], keep order:false, stats:pseudo" ], "Warnings": [ - "Warning 1105 IndexMerge is inapplicable." + "Warning 1105 IndexMerge is inapplicable" ] }, { @@ -1342,7 +1342,7 @@ "└─IndexRangeScan 6666.67 cop[tikv] table:tt, index:a(a) range:[-inf,10), [15,15], (20,+inf], keep order:false, stats:pseudo" ], "Warnings": [ - "Warning 1105 IndexMerge is inapplicable." + "Warning 1105 IndexMerge is inapplicable" ] } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 013d86e227342..7a788eaad3912 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -566,7 +566,7 @@ }, { "SQL": "select /*+ TIDB_SMJ(t1,t2,t3)*/ * from t t1 left outer join t t2 on t1.a = t2.a left outer join t t3 on t2.a = t3.a", - "Best": "MergeLeftOuterJoin{MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.t.a)->TableReader(Table(t))}(test.t.a,test.t.a)" + "Best": "MergeLeftOuterJoin{MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.t.a)->Sort->TableReader(Table(t))}(test.t.a,test.t.a)" }, { "SQL": "select /*+ TIDB_SMJ(t1,t2,t3)*/ * from t t1 left outer join t t2 on t1.a = t2.a left outer join t t3 on t1.a = t3.a", From 14e3f6bf7bc522b25e18f745a3fb11cb4c5a970c Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 27 Apr 2022 17:34:52 +0800 Subject: [PATCH 06/51] executor: reset all state when table reader executor close (#33219) (#33227) close pingcap/tidb#33214 --- executor/distsql.go | 9 +++++---- executor/table_reader.go | 7 +++---- executor/temporary_table_test.go | 17 +++++++++++++++++ 3 files changed, 25 insertions(+), 8 deletions(-) diff --git a/executor/distsql.go b/executor/distsql.go index 734074eef8baf..6ddc4a5f9157c 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -198,14 +198,14 @@ type IndexReaderExecutor struct { // Close clears all resources hold by current object. func (e *IndexReaderExecutor) Close() (err error) { - if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { - return nil - } - if e.result != nil { err = e.result.Close() } e.result = nil + e.kvRanges = e.kvRanges[:0] + if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { + return nil + } e.ctx.StoreQueryFeedback(e.feedback) return err } @@ -677,6 +677,7 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, task *lookup // Close implements Exec Close interface. func (e *IndexLookUpExecutor) Close() error { + e.kvRanges = e.kvRanges[:0] if e.table.Meta().TempTableType != model.TempTableNone { return nil } diff --git a/executor/table_reader.go b/executor/table_reader.go index 958b8cc442061..7314dde007f4b 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -263,15 +263,14 @@ func fillExtraPIDColumn(req *chunk.Chunk, extraPIDColumnIndex int, physicalID in // Close implements the Executor Close interface. func (e *TableReaderExecutor) Close() error { - if e.table.Meta() != nil && e.table.Meta().TempTableType != model.TempTableNone { - return nil - } - var err error if e.resultHandler != nil { err = e.resultHandler.Close() } e.kvRanges = e.kvRanges[:0] + if e.table.Meta() != nil && e.table.Meta().TempTableType != model.TempTableNone { + return nil + } e.ctx.StoreQueryFeedback(e.feedback) return err } diff --git a/executor/temporary_table_test.go b/executor/temporary_table_test.go index c459340bfb67e..def967c0700aa 100644 --- a/executor/temporary_table_test.go +++ b/executor/temporary_table_test.go @@ -135,3 +135,20 @@ func assertTemporaryTableNoNetwork(t *testing.T, createTable func(*testkit.TestK tk.MustExec("select * from tmp_t where id > 1 for update") tk.MustExec("rollback") } + +func TestIssue33214(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (col enum('a', 'b', 'c') default null)") + tk.MustExec("insert into t values ('a'), ('b'), ('c'), (null), ('c')") + tk.MustExec("alter table t cache") + for { + tk.MustQuery("select col from t t1 where (select count(*) from t t2 where t2.col = t1.col or t2.col = 'sdf') > 1;").Check(testkit.Rows("c", "c")) + if tk.Session().GetSessionVars().StmtCtx.ReadFromTableCache { + break + } + } +} From fa1db5c01ba754ae1a268b19a7d4e2ef6c228091 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 27 Apr 2022 17:50:52 +0800 Subject: [PATCH 07/51] executor: fix data race in the IndexNestedLoopHashJoin (#31731) (#31757) close pingcap/tidb#31467 --- executor/index_lookup_hash_join.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/executor/index_lookup_hash_join.go b/executor/index_lookup_hash_join.go index 069c7b49a51bc..8a482fa1a8836 100644 --- a/executor/index_lookup_hash_join.go +++ b/executor/index_lookup_hash_join.go @@ -503,7 +503,9 @@ func (iw *indexHashJoinInnerWorker) run(ctx context.Context, cancelFunc context. break } err := iw.handleTask(ctx, task, joinResult, h, resultCh) - if err != nil { + if err != nil && !task.keepOuterOrder { + // Only need check non-keep-outer-order case because the + // `joinResult` had been sent to the `resultCh` when err != nil. joinResult.err = err break } From 1a12c610ce1e81c5441a949c71200f24aa54757c Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 27 Apr 2022 19:40:52 +0800 Subject: [PATCH 08/51] br: fix panic during merging files (#33797) (#34019) close pingcap/tidb#33801 --- br/pkg/logutil/logging.go | 11 +++++ br/pkg/restore/merge.go | 74 ++++++++++++++++++------------- br/pkg/restore/merge_fuzz_test.go | 25 +++++++++++ 3 files changed, 80 insertions(+), 30 deletions(-) create mode 100644 br/pkg/restore/merge_fuzz_test.go diff --git a/br/pkg/logutil/logging.go b/br/pkg/logutil/logging.go index d247fe7c3375f..126de7778a401 100644 --- a/br/pkg/logutil/logging.go +++ b/br/pkg/logutil/logging.go @@ -209,8 +209,19 @@ func Keys(keys [][]byte) zap.Field { return zap.Object("keys", zapKeysMarshaler(keys)) } +// AShortError make the zap field with key to display error without verbose representation (e.g. the stack trace). +func AShortError(key string, err error) zap.Field { + if err == nil { + return zap.Skip() + } + return zap.String(key, err.Error()) +} + // ShortError make the zap field to display error without verbose representation (e.g. the stack trace). func ShortError(err error) zap.Field { + if err == nil { + return zap.Skip() + } return zap.String("error", err.Error()) } diff --git a/br/pkg/restore/merge.go b/br/pkg/restore/merge.go index 1121feaa314ac..fccff4517b2e8 100644 --- a/br/pkg/restore/merge.go +++ b/br/pkg/restore/merge.go @@ -8,7 +8,9 @@ import ( "github.com/docker/go-units" "github.com/pingcap/errors" backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/rtree" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/tablecodec" @@ -39,6 +41,47 @@ type MergeRangesStat struct { MergedRegionBytesAvg int } +// NeedsMerge checks whether two ranges needs to be merged. +func NeedsMerge(left, right *rtree.Range, splitSizeBytes, splitKeyCount uint64) bool { + leftBytes, leftKeys := left.BytesAndKeys() + rightBytes, rightKeys := right.BytesAndKeys() + if rightBytes == 0 { + return true + } + if leftBytes+rightBytes > splitSizeBytes { + return false + } + if leftKeys+rightKeys > splitKeyCount { + return false + } + tableID1, indexID1, isRecord1, err1 := tablecodec.DecodeKeyHead(kv.Key(left.StartKey)) + tableID2, indexID2, isRecord2, err2 := tablecodec.DecodeKeyHead(kv.Key(right.StartKey)) + + // Failed to decode the file key head... can this happen? + if err1 != nil || err2 != nil { + log.Warn("Failed to parse the key head for merging files, skipping", + logutil.Key("left-start-key", left.StartKey), + logutil.Key("right-start-key", right.StartKey), + logutil.AShortError("left-err", err1), + logutil.AShortError("right-err", err2), + ) + return false + } + // Merge if they are both record keys + if isRecord1 && isRecord2 { + // Do not merge ranges in different tables. + return tableID1 == tableID2 + } + // If they are all index keys... + if !isRecord1 && !isRecord2 { + // Do not merge ranges in different indexes even if they are in the same + // table, as rewrite rule only supports rewriting one pattern. + // Merge left and right if they are in the same index. + return tableID1 == tableID2 && indexID1 == indexID2 + } + return false +} + // MergeFileRanges returns ranges of the files are merged based on // splitSizeBytes and splitKeyCount. // @@ -94,38 +137,9 @@ func MergeFileRanges( } } - needMerge := func(left, right *rtree.Range) bool { - leftBytes, leftKeys := left.BytesAndKeys() - rightBytes, rightKeys := right.BytesAndKeys() - if rightBytes == 0 { - return true - } - if leftBytes+rightBytes > splitSizeBytes { - return false - } - if leftKeys+rightKeys > splitKeyCount { - return false - } - // Do not merge ranges in different tables. - if tablecodec.DecodeTableID(kv.Key(left.StartKey)) != tablecodec.DecodeTableID(kv.Key(right.StartKey)) { - return false - } - // Do not merge ranges in different indexes even if they are in the same - // table, as rewrite rule only supports rewriting one pattern. - // tableID, indexID, indexValues, err - _, indexID1, _, err1 := tablecodec.DecodeIndexKey(kv.Key(left.StartKey)) - _, indexID2, _, err2 := tablecodec.DecodeIndexKey(kv.Key(right.StartKey)) - // If both of them are index keys, ... - if err1 == nil && err2 == nil { - // Merge left and right if they are in the same index. - return indexID1 == indexID2 - } - // Otherwise, merge if they are both record keys - return err1 != nil && err2 != nil - } sortedRanges := rangeTree.GetSortedRanges() for i := 1; i < len(sortedRanges); { - if !needMerge(&sortedRanges[i-1], &sortedRanges[i]) { + if !NeedsMerge(&sortedRanges[i-1], &sortedRanges[i], splitSizeBytes, splitKeyCount) { i++ continue } diff --git a/br/pkg/restore/merge_fuzz_test.go b/br/pkg/restore/merge_fuzz_test.go new file mode 100644 index 0000000000000..624f0c3ef4378 --- /dev/null +++ b/br/pkg/restore/merge_fuzz_test.go @@ -0,0 +1,25 @@ +// Copyright 2022 PingCAP, Inc. Licensed under Apache-2.0. +//go:build go1.18 +// +build go1.18 + +package restore_test + +import ( + "testing" + + backup "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/tidb/br/pkg/restore" + "github.com/pingcap/tidb/br/pkg/rtree" + "github.com/pingcap/tidb/tablecodec" +) + +func FuzzMerge(f *testing.F) { + baseKeyA := tablecodec.EncodeIndexSeekKey(42, 1, nil) + baseKeyB := tablecodec.EncodeIndexSeekKey(42, 1, nil) + f.Add([]byte(baseKeyA), []byte(baseKeyB)) + f.Fuzz(func(t *testing.T, a, b []byte) { + left := rtree.Range{StartKey: a, Files: []*backup.File{{TotalKvs: 1, TotalBytes: 1}}} + right := rtree.Range{StartKey: b, Files: []*backup.File{{TotalKvs: 1, TotalBytes: 1}}} + restore.NeedsMerge(&left, &right, 42, 42) + }) +} From d08e21f1ba663dcdd93d8fcbbf807212d4866e07 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 01:20:51 +0800 Subject: [PATCH 09/51] statistics: improve topn pruning logic (#34282) (#34302) ref pingcap/tidb#32758, close pingcap/tidb#34256 --- statistics/builder.go | 12 ++++++----- statistics/statistics_test.go | 40 +++++++++++++++++++++++++++++++++++ 2 files changed, 47 insertions(+), 5 deletions(-) diff --git a/statistics/builder.go b/statistics/builder.go index cff117547f619..fee2f55e0860d 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -375,8 +375,8 @@ func BuildHistAndTopN( // pruneTopNItem tries to prune the least common values in the top-n list if it is not significantly more common than the values not in the list. // We assume that the ones not in the top-n list's selectivity is 1/remained_ndv which is the internal implementation of EqualRowCount func pruneTopNItem(topns []TopNMeta, ndv, nullCount, sampleRows, totalRows int64) []TopNMeta { - // If the sampleRows holds all rows. We just return the top-n directly. - if sampleRows == totalRows || totalRows <= 1 { + // If the sampleRows holds all rows, or NDV of samples equals to actual NDV, we just return the TopN directly. + if sampleRows == totalRows || totalRows <= 1 || int64(len(topns)) >= ndv { return topns } // Sum the occurrence except the least common one from the top-n list. To check whether the lest common one is worth @@ -396,7 +396,7 @@ func pruneTopNItem(topns []TopNMeta, ndv, nullCount, sampleRows, totalRows int64 if selectivity > 1 { selectivity = 1 } - otherNDV := float64(ndv) - float64(topNNum) + otherNDV := float64(ndv) - (float64(topNNum) - 1) if otherNDV > 1 { selectivity /= otherNDV } @@ -407,11 +407,13 @@ func pruneTopNItem(topns []TopNMeta, ndv, nullCount, sampleRows, totalRows int64 // Thus the variance is the following formula. variance := n * K * (N - K) * (N - n) / (N * N * (N - 1)) stddev := math.Sqrt(variance) - // We choose the bound that plus two stddev of the sample frequency, plus an additional 0.5 for the continuity correction. + // We choose the bound that plus two stddev of the sample frequency, plus an additional 0.5 for the continuity correction. // Note: // The mean + 2 * stddev is known as Wald confidence interval, plus 0.5 would be continuity-corrected Wald interval if float64(topns[topNNum-1].Count) > selectivity*n+2*stddev+0.5 { - // If the current one is worth storing, the latter ones too. So we just break here. + // Estimated selectivity of this item in the TopN is significantly higher than values not in TopN. + // So this value, and all other values in the TopN (selectivity of which is higher than this value) are + // worth being remained in the TopN list, and we stop pruning now. break } // Current one is not worth storing, remove it and subtract it from sumCount, go to next one. diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index cf753ca21b4f0..06fa72a3ca483 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -670,3 +670,43 @@ func SubTestHistogramProtoConversion() func(*testing.T) { require.True(t, HistogramEqual(col, h, true)) } } + +func TestPruneTopN(t *testing.T) { + var topnIn, topnOut []TopNMeta + var totalNDV, nullCnt, sampleRows, totalRows int64 + + // case 1 + topnIn = []TopNMeta{{[]byte{1}, 100_000}, {[]byte{2}, 10}} + totalNDV = 2 + nullCnt = 0 + sampleRows = 100_010 + totalRows = 500_050 + topnOut = pruneTopNItem(topnIn, totalNDV, nullCnt, sampleRows, totalRows) + require.Equal(t, topnIn, topnOut) + + // case 2 + topnIn = []TopNMeta{ + {[]byte{1}, 30_000}, + {[]byte{2}, 30_000}, + {[]byte{3}, 20_000}, + {[]byte{4}, 20_000}, + } + totalNDV = 5 + nullCnt = 0 + sampleRows = 100_000 + totalRows = 10_000_000 + topnOut = pruneTopNItem(topnIn, totalNDV, nullCnt, sampleRows, totalRows) + require.Equal(t, topnIn, topnOut) + + // case 3 + topnIn = nil + for i := 0; i < 100; i++ { + topnIn = append(topnIn, TopNMeta{[]byte{byte(i)}, 1_000}) + } + totalNDV = 100 + nullCnt = 0 + sampleRows = 100_000 + totalRows = 10_000_000 + topnOut = pruneTopNItem(topnIn, totalNDV, nullCnt, sampleRows, totalRows) + require.Equal(t, topnIn, topnOut) +} From 1315e32e1ba1a5757549a8bac52e966cce5b1eb9 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 15:18:52 +0800 Subject: [PATCH 10/51] executor/infoschema_reader.go: skip the unidentifiable table-attributes (#34129) (#34271) close pingcap/tidb#33665 --- executor/infoschema_reader.go | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 3dba91e42e4be..50fd7129dd362 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -2792,6 +2792,15 @@ func (e *memtableRetriever) setDataForAttributes(ctx sessionctx.Context) error { "end_key": "7480000000000000ff3a5f720000000000fa", }), }, + { + ID: "invalidIDtest", + Labels: []label.Label{{Key: "merge_option", Value: "allow"}, {Key: "db", Value: "test"}, {Key: "table", Value: "test_label"}}, + RuleType: "key-range", + Data: convert(map[string]interface{}{ + "start_key": "7480000000000000ff395f720000000000fa", + "end_key": "7480000000000000ff3a5f720000000000fa", + }), + }, } err = nil }) @@ -2803,7 +2812,8 @@ func (e *memtableRetriever) setDataForAttributes(ctx sessionctx.Context) error { skip := true dbName, tableName, err := checkRule(rule) if err != nil { - return err + logutil.BgLogger().Warn("check table-rule failed", zap.String("ID", rule.ID), zap.Error(err)) + continue } if tableName != "" && dbName != "" && (checker == nil || checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, dbName, tableName, "", mysql.SelectPriv)) { skip = false From 3f7c3411aa050791cc9b6cc34005281abe93f8c8 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 15:34:52 +0800 Subject: [PATCH 11/51] br: Ignore ddl jobs with empty query or blacklist type when exec restore (#33384) (#33517) close pingcap/tidb#33322 --- br/pkg/restore/db.go | 46 +++++++++++++++++++++++++ br/pkg/restore/db_test.go | 71 +++++++++++++++++++++++++++++++++++++++ br/pkg/task/restore.go | 1 + 3 files changed, 118 insertions(+) diff --git a/br/pkg/restore/db.go b/br/pkg/restore/db.go index 3e9d35a124dfd..7510a1fb4dd5d 100644 --- a/br/pkg/restore/db.go +++ b/br/pkg/restore/db.go @@ -28,6 +28,15 @@ type UniqueTableName struct { Table string } +type DDLJobFilterRule func(ddlJob *model.Job) bool + +var incrementalRestoreActionBlockList = map[model.ActionType]struct{}{ + model.ActionSetTiFlashReplica: {}, + model.ActionUpdateTiFlashReplicaStatus: {}, + model.ActionLockTable: {}, + model.ActionUnlockTable: {}, +} + // NewDB returns a new DB. func NewDB(g glue.Glue, store kv.Storage) (*DB, error) { se, err := g.CreateSession(store) @@ -71,6 +80,13 @@ func (db *DB) ExecDDL(ctx context.Context, ddlJob *model.Job) error { return errors.Trace(err) } + if ddlJob.Query == "" { + log.Warn("query of ddl job is empty, ignore it", + zap.Stringer("type", ddlJob.Type), + zap.String("db", ddlJob.SchemaName)) + return nil + } + if tableInfo != nil { switchDBSQL := fmt.Sprintf("use %s;", utils.EncloseName(ddlJob.SchemaName)) err = db.se.Execute(ctx, switchDBSQL) @@ -280,6 +296,31 @@ func FilterDDLJobs(allDDLJobs []*model.Job, tables []*metautil.Table) (ddlJobs [ return ddlJobs } +// FilterDDLJobByRules if one of rules returns true, the job in srcDDLJobs will be filtered. +func FilterDDLJobByRules(srcDDLJobs []*model.Job, rules ...DDLJobFilterRule) (dstDDLJobs []*model.Job) { + dstDDLJobs = make([]*model.Job, 0, len(srcDDLJobs)) + for _, ddlJob := range srcDDLJobs { + passed := true + for _, rule := range rules { + if rule(ddlJob) { + passed = false + break + } + } + + if passed { + dstDDLJobs = append(dstDDLJobs, ddlJob) + } + } + + return +} + +// DDLJobBlockListRule rule for filter ddl job with type in block list. +func DDLJobBlockListRule(ddlJob *model.Job) bool { + return checkIsInActions(ddlJob.Type, incrementalRestoreActionBlockList) +} + func getDatabases(tables []*metautil.Table) (dbs []*model.DBInfo) { dbIDs := make(map[int64]bool) for _, table := range tables { @@ -290,3 +331,8 @@ func getDatabases(tables []*metautil.Table) (dbs []*model.DBInfo) { } return } + +func checkIsInActions(action model.ActionType, actions map[model.ActionType]struct{}) bool { + _, ok := actions[action] + return ok +} diff --git a/br/pkg/restore/db_test.go b/br/pkg/restore/db_test.go index b9a1c8948f8dd..35e9f5ee7ea6d 100644 --- a/br/pkg/restore/db_test.go +++ b/br/pkg/restore/db_test.go @@ -251,3 +251,74 @@ func (s *testRestoreSchemaSuite) TestFilterDDLJobsV2(c *C) { } c.Assert(len(ddlJobs), Equals, 7) } + +func (s *testRestoreSchemaSuite) TestDB_ExecDDL(c *C) { + ctx := context.Background() + ddlJobs := []*model.Job{ + { + Type: model.ActionAddIndex, + Query: "CREATE DATABASE IF NOT EXISTS test_db;", + BinlogInfo: &model.HistoryInfo{}, + }, + { + Type: model.ActionAddIndex, + Query: "", + BinlogInfo: &model.HistoryInfo{}, + }, + } + + db, err := restore.NewDB(gluetidb.New(), s.mock.Storage) + c.Assert(err, IsNil) + + for _, ddlJob := range ddlJobs { + err = db.ExecDDL(ctx, ddlJob) + c.Assert(err, IsNil) + } +} + +func (s *testRestoreSchemaSuite) TestFilterDDLJobByRules(c *C) { + ddlJobs := []*model.Job{ + { + Type: model.ActionSetTiFlashReplica, + }, + { + Type: model.ActionAddPrimaryKey, + }, + { + Type: model.ActionUpdateTiFlashReplicaStatus, + }, + { + Type: model.ActionCreateTable, + }, + { + Type: model.ActionLockTable, + }, + { + Type: model.ActionAddIndex, + }, + { + Type: model.ActionUnlockTable, + }, + { + Type: model.ActionCreateSchema, + }, + { + Type: model.ActionModifyColumn, + }, + } + + expectedDDLTypes := []model.ActionType{ + model.ActionAddPrimaryKey, + model.ActionCreateTable, + model.ActionAddIndex, + model.ActionCreateSchema, + model.ActionModifyColumn, + } + + ddlJobs = restore.FilterDDLJobByRules(ddlJobs, restore.DDLJobBlockListRule) + + c.Assert(len(ddlJobs), Equals, len(expectedDDLTypes)) + for i, ddlJob := range ddlJobs { + c.Assert(ddlJob.Type, Equals, expectedDDLTypes[i]) + } +} diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index ae46f15b1f6ce..27df82804288a 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -322,6 +322,7 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf newTS = restoreTS } ddlJobs := restore.FilterDDLJobs(client.GetDDLJobs(), tables) + ddlJobs = restore.FilterDDLJobByRules(ddlJobs, restore.DDLJobBlockListRule) err = client.PreCheckTableTiFlashReplica(ctx, tables) if err != nil { From 95fc56bf28f43ffc0297e1f9d80bac9a4c1a1406 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 15:52:52 +0800 Subject: [PATCH 12/51] br: Fix new collaction enable check (#33500) (#33532) close pingcap/tidb#33422 --- br/pkg/glue/glue.go | 1 + br/pkg/gluetidb/glue.go | 5 + br/pkg/task/backup.go | 11 ++ br/pkg/task/common.go | 2 + br/pkg/task/restore.go | 35 +++++ br/pkg/version/version.go | 1 + .../config/new_collation_enable_false.toml | 16 +++ .../config/new_collation_enable_true.toml | 16 +++ .../br_check_new_collocation_enable/run.sh | 102 +++++++++++++++ br/tests/br_s3/run.sh | 6 + executor/brie.go | 5 + go.mod | 25 ++-- go.sum | 123 +++++++++++++++--- 13 files changed, 320 insertions(+), 28 deletions(-) create mode 100644 br/tests/br_check_new_collocation_enable/config/new_collation_enable_false.toml create mode 100644 br/tests/br_check_new_collocation_enable/config/new_collation_enable_true.toml create mode 100755 br/tests/br_check_new_collocation_enable/run.sh diff --git a/br/pkg/glue/glue.go b/br/pkg/glue/glue.go index 7f2be30a60d34..91edb4493b747 100644 --- a/br/pkg/glue/glue.go +++ b/br/pkg/glue/glue.go @@ -37,6 +37,7 @@ type Session interface { CreateDatabase(ctx context.Context, schema *model.DBInfo) error CreateTable(ctx context.Context, dbName model.CIStr, table *model.TableInfo) error Close() + GetGlobalVariable(name string) (string, error) } // Progress is an interface recording the current execution progress. diff --git a/br/pkg/gluetidb/glue.go b/br/pkg/gluetidb/glue.go index c2cb64d21328a..fd7a67539109f 100644 --- a/br/pkg/gluetidb/glue.go +++ b/br/pkg/gluetidb/glue.go @@ -151,6 +151,11 @@ func (gs *tidbSession) Close() { gs.se.Close() } +// GetGlobalVariables implements glue.Session. +func (gs *tidbSession) GetGlobalVariable(name string) (string, error) { + return gs.se.GetSessionVars().GlobalVarsAccessor.GetTiDBTableValue(name) +} + // showCreateTable shows the result of SHOW CREATE TABLE from a TableInfo. func (gs *tidbSession) showCreateTable(tbl *model.TableInfo) (string, error) { table := tbl.Clone() diff --git a/br/pkg/task/backup.go b/br/pkg/task/backup.go index 87461f53bab74..2b50a9f386f54 100644 --- a/br/pkg/task/backup.go +++ b/br/pkg/task/backup.go @@ -250,6 +250,16 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig statsHandle = mgr.GetDomain().StatsHandle() } + se, err := g.CreateSession(mgr.GetStorage()) + if err != nil { + return errors.Trace(err) + } + newCollationEnable, err := se.GetGlobalVariable(tidbNewCollationEnabled) + if err != nil { + return errors.Trace(err) + } + log.Info("get newCollationEnable for check during restore", zap.String("newCollationEnable", newCollationEnable)) + client, err := backup.NewBackupClient(ctx, mgr) if err != nil { return errors.Trace(err) @@ -339,6 +349,7 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig m.ClusterId = req.ClusterId m.ClusterVersion = clusterVersion m.BrVersion = brVersion + m.NewCollationsEnabled = newCollationEnable }) // nothing to backup diff --git a/br/pkg/task/common.go b/br/pkg/task/common.go index 357c7d267e449..dd7ffe0bb8455 100644 --- a/br/pkg/task/common.go +++ b/br/pkg/task/common.go @@ -85,6 +85,8 @@ const ( crypterAES128KeyLen = 16 crypterAES192KeyLen = 24 crypterAES256KeyLen = 32 + + tidbNewCollationEnabled = "new_collation_enabled" ) // TLSConfig is the common configuration for TLS connection. diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 27df82804288a..710be11d8e212 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -4,6 +4,7 @@ package task import ( "context" + "strings" "time" "github.com/opentracing/opentracing-go" @@ -22,6 +23,7 @@ import ( "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/br/pkg/version" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/kv" "github.com/spf13/pflag" "go.uber.org/multierr" "go.uber.org/zap" @@ -212,6 +214,35 @@ func CheckRestoreDBAndTable(client *restore.Client, cfg *RestoreConfig) error { return nil } +func CheckNewCollationEnable( + backupNewCollationEnable string, + g glue.Glue, + storage kv.Storage, +) error { + if backupNewCollationEnable == "" { + log.Warn("new_collations_enabled_on_first_bootstrap not found in backupmeta. " + + "we assume that this config is as same as backup cluster") + return nil + } + + se, err := g.CreateSession(storage) + if err != nil { + return errors.Trace(err) + } + + newCollationEnable, err := se.GetGlobalVariable(tidbNewCollationEnabled) + if err != nil { + return errors.Trace(err) + } + + if !strings.EqualFold(backupNewCollationEnable, newCollationEnable) { + return errors.Annotatef(berrors.ErrUnknown, + "newCollationEnable not match, upstream:%v, downstream: %v", + backupNewCollationEnable, newCollationEnable) + } + return nil +} + // RunRestore starts a restore task inside the current goroutine. func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConfig) error { cfg.adjustRestoreConfig() @@ -278,6 +309,10 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf return errors.Trace(versionErr) } } + if err = CheckNewCollationEnable(backupMeta.GetNewCollationsEnabled(), g, mgr.GetStorage()); err != nil { + return errors.Trace(err) + } + reader := metautil.NewMetaReader(backupMeta, s, &cfg.CipherInfo) if err = client.InitBackupMeta(c, backupMeta, u, s, reader); err != nil { return errors.Trace(err) diff --git a/br/pkg/version/version.go b/br/pkg/version/version.go index 91c1578574837..3988ffae8c195 100644 --- a/br/pkg/version/version.go +++ b/br/pkg/version/version.go @@ -102,6 +102,7 @@ func CheckClusterVersion(ctx context.Context, client pd.Client, checker VerCheck if err := checkTiFlashVersion(s); err != nil { return errors.Trace(err) } + continue } tikvVersionString := removeVAndHash(s.Version) diff --git a/br/tests/br_check_new_collocation_enable/config/new_collation_enable_false.toml b/br/tests/br_check_new_collocation_enable/config/new_collation_enable_false.toml new file mode 100644 index 0000000000000..dd82812d27156 --- /dev/null +++ b/br/tests/br_check_new_collocation_enable/config/new_collation_enable_false.toml @@ -0,0 +1,16 @@ +# config of tidb + +# Schema lease duration +# There are lot of ddl in the tests, setting this +# to 360s to test whether BR is gracefully shutdown. +lease = "360s" + +new_collations_enabled_on_first_bootstrap = false + +[security] +ssl-ca = "/tmp/backup_restore_test/certs/ca.pem" +ssl-cert = "/tmp/backup_restore_test/certs/tidb.pem" +ssl-key = "/tmp/backup_restore_test/certs/tidb.key" +cluster-ssl-ca = "/tmp/backup_restore_test/certs/ca.pem" +cluster-ssl-cert = "/tmp/backup_restore_test/certs/tidb.pem" +cluster-ssl-key = "/tmp/backup_restore_test/certs/tidb.key" diff --git a/br/tests/br_check_new_collocation_enable/config/new_collation_enable_true.toml b/br/tests/br_check_new_collocation_enable/config/new_collation_enable_true.toml new file mode 100644 index 0000000000000..d9cb1df6178f0 --- /dev/null +++ b/br/tests/br_check_new_collocation_enable/config/new_collation_enable_true.toml @@ -0,0 +1,16 @@ +# config of tidb + +# Schema lease duration +# There are lot of ddl in the tests, setting this +# to 360s to test whether BR is gracefully shutdown. +lease = "360s" + +new_collations_enabled_on_first_bootstrap = true + +[security] +ssl-ca = "/tmp/backup_restore_test/certs/ca.pem" +ssl-cert = "/tmp/backup_restore_test/certs/tidb.pem" +ssl-key = "/tmp/backup_restore_test/certs/tidb.key" +cluster-ssl-ca = "/tmp/backup_restore_test/certs/ca.pem" +cluster-ssl-cert = "/tmp/backup_restore_test/certs/tidb.pem" +cluster-ssl-key = "/tmp/backup_restore_test/certs/tidb.key" diff --git a/br/tests/br_check_new_collocation_enable/run.sh b/br/tests/br_check_new_collocation_enable/run.sh new file mode 100755 index 0000000000000..88e2c0e499ed1 --- /dev/null +++ b/br/tests/br_check_new_collocation_enable/run.sh @@ -0,0 +1,102 @@ +#!/bin/sh +# +# 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, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +set -eu +DB="$TEST_NAME" + +cur=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) +source $cur/../_utils/run_services + +PROGRESS_FILE="$TEST_DIR/progress_unit_file" +rm -rf $PROGRESS_FILE + +run_sql "CREATE DATABASE $DB;" + +run_sql "CREATE TABLE $DB.usertable1 ( \ + YCSB_KEY varchar(64) NOT NULL, \ + FIELD0 varchar(1) DEFAULT NULL, \ + PRIMARY KEY (YCSB_KEY) \ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;" + +run_sql "INSERT INTO $DB.usertable1 VALUES (\"a\", \"b\");" +run_sql "INSERT INTO $DB.usertable1 VALUES (\"aa\", \"b\");" + +run_sql "CREATE TABLE $DB.usertable2 ( \ + YCSB_KEY varchar(64) NOT NULL, \ + FIELD0 varchar(1) DEFAULT NULL, \ + PRIMARY KEY (YCSB_KEY) \ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;" + +run_sql "INSERT INTO $DB.usertable2 VALUES (\"c\", \"d\");" + +# backup db +echo "backup start ... with brv4.0.8 without NewCollactionEnable" +bin/brv4.0.8 backup db --db "$DB" -s "local://$TEST_DIR/$DB" \ + --ca "$TEST_DIR/certs/ca.pem" \ + --cert "$TEST_DIR/certs/br.pem" \ + --key "$TEST_DIR/certs/br.key" \ + --pd $PD_ADDR \ + --check-requirements=false + +# restore db from v4.0.8 version without `newCollationEnable` +echo "restore start ... without NewCollactionEnable in backupmeta" +restore_fail=0 +error_str="new_collations_enabled_on_first_bootstrap not found in backupmeta" +test_log="new_collotion_enable_test.log" +unset BR_LOG_TO_TERM +run_br restore db --db $DB -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --log-file $test_log || restore_fail=1 +if [ $restore_fail -eq 1 ]; then + echo "TEST: [$TEST_NAME] test restore failed!" + exit 1 +fi + +if ! grep -i "$error_str" $test_log; then + echo "${error_str} not found in log" + echo "TEST: [$TEST_NAME] test restore failed!" + exit 1 +fi + +rm -rf "$test_log" + +# backup with NewCollationEable = false +echo "Restart cluster with new_collation_enable=false" +start_services --tidb-cfg $cur/config/new_collation_enable_false.toml + +echo "backup start ... witch NewCollactionEnable=false in TiDB" +run_br --pd $PD_ADDR backup db --db "$DB" -s "local://$cur/${DB}_2" + +echo "Restart cluster with new_collation_enable=true" +start_services --tidb-cfg $cur/config/new_collation_enable_true.toml + +echo "restore start ... with NewCollactionEnable=True in TiDB" +restore_fail=0 +test_log2="new_collotion_enable_test2.log" +error_str="newCollationEnable not match" +unset BR_LOG_TO_TERM +run_br restore db --db $DB -s "local://$cur/${DB}_2" --pd $PD_ADDR --log-file $test_log2 || restore_fail=1 +if [ $restore_fail -ne 1 ]; then + echo "TEST: [$TEST_NAME] test restore failed!" + exit 1 +fi + +if ! grep -i "$error_str" $test_log2; then + echo "${error_str} not found in log" + echo "TEST: [$TEST_NAME] test restore failed!" + exit 1 +fi + +rm -rf "$test_log2" +rm -rf "$cur/${DB}_2" diff --git a/br/tests/br_s3/run.sh b/br/tests/br_s3/run.sh index d0c20996db7ef..9cd383de4f026 100755 --- a/br/tests/br_s3/run.sh +++ b/br/tests/br_s3/run.sh @@ -101,6 +101,12 @@ for p in $(seq 2); do exit 1 fi + target_log="get newCollationEnable for check during restore" + if ! grep -i "$target_log" $BACKUP_LOG; then + echo "${target_log} not found in log" + exit 1 + fi + for i in $(seq $DB_COUNT); do run_sql "DROP DATABASE $DB${i};" done diff --git a/executor/brie.go b/executor/brie.go index e9559d6935d50..6aa1f4e7c389b 100644 --- a/executor/brie.go +++ b/executor/brie.go @@ -505,6 +505,11 @@ func (gs *tidbGlueSession) CreateTable(ctx context.Context, dbName model.CIStr, func (gs *tidbGlueSession) Close() { } +// GetGlobalVariables implements glue.Session. +func (gs *tidbGlueSession) GetGlobalVariable(name string) (string, error) { + return gs.se.GetSessionVars().GlobalVarsAccessor.GetTiDBTableValue(name) +} + // Open implements glue.Glue func (gs *tidbGlueSession) Open(string, pd.SecurityOption) (kv.Storage, error) { return gs.se.GetStore(), nil diff --git a/go.mod b/go.mod index 560f82a83f210..32a58f2eaf5e7 100644 --- a/go.mod +++ b/go.mod @@ -28,18 +28,21 @@ require ( github.com/fsouza/fake-gcs-server v1.19.0 github.com/go-sql-driver/mysql v1.6.0 github.com/gogo/protobuf v1.3.2 + github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/mock v1.6.0 github.com/golang/protobuf v1.5.2 github.com/golang/snappy v0.0.3 - github.com/google/btree v1.0.0 + github.com/google/btree v1.0.1 github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1 github.com/google/uuid v1.1.2 github.com/gorilla/handlers v1.5.1 // indirect github.com/gorilla/mux v1.8.0 - github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 + github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 + github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect github.com/iancoleman/strcase v0.0.0-20191112232945-16388991a334 github.com/jedib0t/go-pretty/v6 v6.2.2 github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df + github.com/jonboulle/clockwork v0.2.2 // indirect github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef github.com/opentracing/basictracer-go v1.0.0 @@ -50,48 +53,52 @@ require ( github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20211207042851-78a55fb8e69c + github.com/pingcap/kvproto v0.0.0-20220428033740-e4924274acd8 github.com/pingcap/log v0.0.0-20210906054005-afc726e70354 github.com/pingcap/sysutil v0.0.0-20211208032423-041a72e5860d github.com/pingcap/tidb-tools v5.2.2-0.20211019062242-37a8bef2fa17+incompatible github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e github.com/pingcap/tipb v0.0.0-20220107024056-3b91949a18a7 - github.com/prometheus/client_golang v1.5.1 + github.com/prometheus/client_golang v1.11.1 github.com/prometheus/client_model v0.2.0 - github.com/prometheus/common v0.9.1 + github.com/prometheus/common v0.26.0 github.com/shirou/gopsutil v3.21.3+incompatible github.com/shurcooL/httpgzip v0.0.0-20190720172056-320755c1c1b0 github.com/sirupsen/logrus v1.8.1 // indirect - github.com/soheilhy/cmux v0.1.4 - github.com/spf13/cobra v1.0.0 + github.com/soheilhy/cmux v0.1.5 + github.com/spf13/cobra v1.1.3 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 github.com/tikv/client-go/v2 v2.0.0-rc.0.20220425054530-a62124d9fa75 github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee + github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 // indirect github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible github.com/uber/jaeger-lib v2.4.1+incompatible // indirect github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457 github.com/xitongsys/parquet-go-source v0.0.0-20200817004010-026bad9b25d0 + go.etcd.io/bbolt v1.3.6 // indirect go.etcd.io/etcd v0.5.0-alpha.5.0.20210512015243-d19fbe541bf9 go.uber.org/atomic v1.9.0 go.uber.org/automaxprocs v1.4.0 go.uber.org/goleak v1.1.12 go.uber.org/multierr v1.7.0 go.uber.org/zap v1.19.1 - golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f + golang.org/x/crypto v0.0.0-20220131195533-30dcbda58838 // indirect + golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2 golang.org/x/oauth2 v0.0.0-20210805134026-6f1e6394065a golang.org/x/sync v0.0.0-20210220032951-036812b2e83c golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e golang.org/x/text v0.3.7 - golang.org/x/time v0.0.0-20191024005414-555d28b269f0 + golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba golang.org/x/tools v0.1.8 google.golang.org/api v0.54.0 google.golang.org/grpc v1.40.0 gopkg.in/yaml.v2 v2.4.0 modernc.org/mathutil v1.4.1 + sigs.k8s.io/yaml v1.2.0 // 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 fbf51e97ace04..f0c9c913b5b78 100644 --- a/go.sum +++ b/go.sum @@ -31,6 +31,7 @@ cloud.google.com/go/bigquery v1.7.0/go.mod h1://okPTzCYNXSlb24MZs83e2Do+h+VXtc4g cloud.google.com/go/bigquery v1.8.0/go.mod h1:J5hqkt3O0uAFnINi6JXValWIb1v0goeZM77hZzJN/fQ= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= +cloud.google.com/go/firestore v1.1.0/go.mod h1:ulACoGHTpvq5r8rxGJ4ddJZBZqakUQqClKRT5SZwBmk= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -87,14 +88,19 @@ github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuy github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho= github.com/alvaroloes/enumer v1.1.2/go.mod h1:FxrjvuXoDAx9isTJrv4c+T410zFi0DtXIT0m65DJ+Wo= github.com/antihax/optional v0.0.0-20180407024304-ca021399b1a6/go.mod h1:V8iCPQYkqmusNa815XgQio277wI47sdRh1dUOLdyC6Q= +github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= github.com/apache/thrift v0.0.0-20181112125854-24918abba929/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714 h1:Jz3KVLYY5+JO7rDiX0sAuRGtuv2vG01r17Y9nLMWNUw= github.com/apache/thrift v0.13.1-0.20201008052519-daf620915714/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/appleboy/gin-jwt/v2 v2.6.3/go.mod h1:MfPYA4ogzvOcVkRwAxT7quHOtQmVKDpTwxyUrC2DNw0= github.com/appleboy/gofight/v2 v2.1.2/go.mod h1:frW+U1QZEdDgixycTj4CygQ48yLTUhplt43+Wczp3rw= +github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= +github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= +github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go v1.35.3 h1:r0puXncSaAfRt7Btml2swUo74Kao+vKhO3VLjwDjK54= github.com/aws/aws-sdk-go v1.35.3/go.mod h1:H7NKnBqNVzoTJpGfLrQkkD+ytBA93eiDYi/+8rV9s48= @@ -106,6 +112,7 @@ github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+Ce github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= +github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84= github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d h1:rQlvB2AYWme2bIB18r/SipGiMEVJYE9U0z+MGoU/LtQ= github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d/go.mod h1:VKt7CNAQxpFpSDz3sXyj9hY/GbVsQCr0sB3w59nE7lU= github.com/cakturk/go-netstat v0.0.0-20200220111822-e5b49efee7a5 h1:BjkPE3785EwPhhyuFkbINB+2a1xATwk8SNDWnJiD41g= @@ -151,6 +158,7 @@ github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2 h1:NnLfQ77q0G4k2Of2 github.com/coocood/rtutil v0.0.0-20190304133409-c84515f646f2/go.mod h1:7qG7YFnOALvsx6tKTNmQot8d7cGFXM9TidzvRFLWYwM= github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= @@ -240,8 +248,10 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2 github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8= github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= +github.com/go-kit/log v0.1.0/go.mod h1:zbhenjAZHb184qTLMA9ZjW7ThYL0H2mk7Q6pNt4vbaY= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= +github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A= github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas= github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= @@ -286,8 +296,9 @@ github.com/golang/groupcache v0.0.0-20160516000752-02826c3e7903/go.mod h1:cIg4er github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e h1:1r7pUrabqp18hOBcwBwiTsbnFeTZHV9eER/QT5JVZxY= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= @@ -325,8 +336,9 @@ github.com/golang/snappy v0.0.3 h1:fHPg5GQYlCeLIPB9BZqMVR5nR9A+IM5zcgeTdjMYmLA= github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/btree v1.0.0 h1:0udJVsspx3VBr5FwtLhQQtuAsVc79tTq0ocGIPAU6qo= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.1 h1:gK4Kx5IaGY9CD5sPJ36FHiBJ6ZXl0kilRiiCj+jdYp4= +github.com/google/btree v1.0.1/go.mod h1:xXMiIv4Fb/0kKde4SpL7qlzvu5cMJDRkFDxJfI9uaxA= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -387,20 +399,39 @@ github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0U github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= -github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 h1:THDBEeQ9xZ8JEaCLyLQqXMMdRqNr0QAUJTIkQAUtFjg= github.com/grpc-ecosystem/go-grpc-middleware v1.1.0/go.mod h1:f5nM7jw/oeRSadq3xCzHAvxcr8HZnzsqU6ILg/0NiiE= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= +github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway v1.12.1 h1:zCy2xE9ablevUOrUZc3Dl72Dt+ya2FNAvC2yLYMHzi4= github.com/grpc-ecosystem/grpc-gateway v1.12.1/go.mod h1:8XEsbTttt/W+VvjtQhLACqCisSPWTxCZ7sBRjU6iH9c= +github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo= +github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/gtank/cryptopasta v0.0.0-20170601214702-1f550f6f2f69/go.mod h1:YLEMZOtU+AZ7dhN9T/IpGhXVGly2bvkJQ+zxj3WeVQo= +github.com/hashicorp/consul/api v1.1.0/go.mod h1:VmuI/Lkw1nC05EYQWNKwWGbkg+FbDBtguAZLlVdkD9Q= +github.com/hashicorp/consul/sdk v0.1.1/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= +github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= +github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= +github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= +github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdvsLplgctolz4= github.com/hashicorp/go-uuid v0.0.0-20180228145832-27454136f036/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= +github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90= github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/golang-lru v0.5.1/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8= github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= +github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= +github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= +github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= +github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= github.com/iancoleman/strcase v0.0.0-20191112232945-16388991a334 h1:VHgatEHNcBFEB7inlalqfNqw65aNkM1lGX2yt3NmbS8= @@ -430,14 +461,19 @@ github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfC github.com/joho/godotenv v1.3.0/go.mod h1:7hK45KPybAkOC6peb+G5yklZfMxEjkZhHbwpqxOKXbg= github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df h1:Zrb0IbuLOGHL7nrO2WrcuNWgDTlzFv3zY69QMx4ggQE= github.com/joho/sqltocsv v0.0.0-20210428211105-a6d6801d59df/go.mod h1:mAVCUAYtW9NG31eB30umMSLKcDt6mCUWSjoSn5qBh0k= -github.com/jonboulle/clockwork v0.1.0 h1:VKV+ZcuP6l3yW9doeqz6ziZGgcynBVQO+obU0+0hcPo= github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo= +github.com/jonboulle/clockwork v0.2.2 h1:UOGuzwb1PwsrDAObMuhUnj0p5ULPj8V/xJ7Kx9qUBdQ= +github.com/jonboulle/clockwork v0.2.2/go.mod h1:Pkfl5aHPm1nk2H9h0bjmnJD/BcgbGXUBGnn1kMkgxc8= github.com/joomcode/errorx v1.0.1/go.mod h1:kgco15ekB6cs+4Xjzo7SPeXzx38PbJzBwbnu9qfVNHQ= +github.com/jpillora/backoff v1.0.0 h1:uvFg412JmmHBHw7iwprIxkPMI+sGQ4kzOWsMeHnm2EA= +github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX5e0EB2j4= github.com/json-iterator/go v1.1.5/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.9 h1:9yzud/Ht36ygwatGx56VwCZtlI/2AD15T1X2sjSuGns= github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= +github.com/json-iterator/go v1.1.11 h1:uVUAXhF2To8cbw/3xN3pxj6kk7TYKs98NIrTqPlMWAQ= +github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= @@ -447,6 +483,7 @@ github.com/juju/ratelimit v1.0.1 h1:+7AIFJVQ0EQgq/K9+0Krm7m530Du7tIz0METWzN0RgY= github.com/juju/ratelimit v1.0.1/go.mod h1:qapgC/Gy+xNh9UxzV13HGGl/6UXNN+ct+vwSgWNm/qk= github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= +github.com/julienschmidt/httprouter v1.3.0/go.mod h1:JR6WtHb+2LUe8TCKY3cZOxFyyO8IZAc4RVcycCCAKdM= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= github.com/kataras/golog v0.0.9/go.mod h1:12HJgwBIZFNGL0EJnMRhmvGA0PQGx8VFwrZtM4CqbAk= @@ -481,6 +518,7 @@ github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvf github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= github.com/leodido/go-urn v1.1.0/go.mod h1:+cyI34gQWZcE1eQU7NVgKkkzdXDQHr1dBMtdAPozLkw= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= +github.com/magiconair/properties v1.8.1/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -489,6 +527,7 @@ github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVc github.com/mattn/go-colorable v0.1.4/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= github.com/mattn/go-colorable v0.1.8 h1:c1ghPdyEDarC70ftn0y+A/Ee++9zz8ljHG1b13eJ0s8= github.com/mattn/go-colorable v0.1.8/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= +github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= @@ -511,8 +550,15 @@ github.com/mediocregopher/radix/v3 v3.3.0/go.mod h1:EmfVyvspXz1uZEyPBMyGK+kjWiKQ github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81/go.mod h1:KQ7+USdGKfpPjXk4Ga+5XxQM4Lm4e3gAogrreFAYpOg= github.com/mgechev/revive v1.0.2/go.mod h1:rb0dQy1LVAxW9SWy5R3LPUjevzUbUS316U5MFySA2lo= github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= +github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= github.com/minio/sio v0.3.0/go.mod h1:8b0yPp2avGThviy/+OCJBI6OMpvxoUuiLvE6F1lebhw= +github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc= +github.com/mitchellh/go-homedir v1.0.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI= +github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg= +github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY= +github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= @@ -525,6 +571,8 @@ github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLg github.com/montanaflynn/stats v0.5.0/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f h1:KUppIJq7/+SVif2QVs3tOP0zanoHgBEVAwHxUSIzRqU= +github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= @@ -556,6 +604,7 @@ github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7l github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/pascaldekloe/name v0.0.0-20180628100202-0fd16699aae1/go.mod h1:eD5JxqMiuNYyFNmyY9rkJ/slN8y59oEu4Ei7F8OoKWQ= github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= @@ -595,8 +644,8 @@ github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLy github.com/pingcap/kvproto v0.0.0-20210819164333-bd5706b9d9f2/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20211109071446-a8b4d34474bc/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20211122024046-03abd340988f/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20211207042851-78a55fb8e69c h1:4D/M5eYfbswv3vs0ZtbVgNKwSRMXgAcm+9a+IbC7q0o= -github.com/pingcap/kvproto v0.0.0-20211207042851-78a55fb8e69c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20220428033740-e4924274acd8 h1:b/nHmr8qwWHg0H+h52ts4GkcnoOltVNBYGc8fAbcSuw= +github.com/pingcap/kvproto v0.0.0-20220428033740-e4924274acd8/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-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= @@ -622,13 +671,16 @@ github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINE github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA= github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/posener/complete v1.1.1/go.mod h1:em0nMJCgc9GFtwrmVmEMR/ZL6WyhyjMBndrE9hABlRI= github.com/prometheus/client_golang v0.9.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDft0ttaMvbicHlPoso= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.1.0/go.mod h1:I1FGZT9+L76gKKOs5djB6ezCbFQP1xR9D75/vuwEF3g= -github.com/prometheus/client_golang v1.5.1 h1:bdHYieyGlH+6OLEk2YQha8THib30KP0/yD0YH9m6xcA= github.com/prometheus/client_golang v1.5.1/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= +github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= +github.com/prometheus/client_golang v1.11.1 h1:+4eQaD7vAZ6DsfsxB15hbE0odUjGI5ARs9yskGu1v4s= +github.com/prometheus/client_golang v1.11.1/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= @@ -639,15 +691,19 @@ github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7q github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.6.0/go.mod h1:eBmuwkDJBwy6iBfxCBob6t6dR6ENT/y+J+Zk0j9GMYc= -github.com/prometheus/common v0.9.1 h1:KOMtN28tlbam3/7ZKEYKHhKoJZYYj3gMH4uc62x7X7U= github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= +github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= +github.com/prometheus/common v0.26.0 h1:iMAkS2TDoNWnKM+Kopnx/8tnEStIfpYA0ur0xQzzhMQ= +github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.3/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= github.com/prometheus/procfs v0.0.5/go.mod h1:4A/X28fw3Fc593LaREMrKMqOKvUAntwMDaekg4FpcdQ= -github.com/prometheus/procfs v0.0.8 h1:+fpWZdT24pJBiqJdAwYBjPSk+5YmQzYNPYzQsdzLkt8= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= +github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= +github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= +github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 h1:OdAsTTz6OkFY5QxjkYwrChwuRruF69c169dPK26NUlk= github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= @@ -660,10 +716,12 @@ github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFR github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= +github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= 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/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= +github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil v3.21.2+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= @@ -684,8 +742,9 @@ github.com/sirupsen/logrus v1.8.1 h1:dJKuHgqk1NNQlqoA6BTlM1Wf9DOH3NBjQyu0h9+AZZE github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= -github.com/soheilhy/cmux v0.1.4 h1:0HKaf1o97UwFjHH9o5XsHUOF+tqmdA7KEzXLpiyaw0E= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= +github.com/soheilhy/cmux v0.1.5 h1:jjzc5WVemNEDTLwv9tlmemhC73tI08BNOIGwBOo10Js= +github.com/soheilhy/cmux v0.1.5/go.mod h1:T7TcVDs9LWfQgPlPsdngu6I6QIoyIFZDDC6sNE1GqG0= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= @@ -693,8 +752,9 @@ github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTd github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ= github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= -github.com/spf13/cobra v1.0.0 h1:6m/oheQuQ13N9ks4hubMG6BnvwOeaJrqSPLahSnczz8= github.com/spf13/cobra v1.0.0/go.mod h1:/6GTrnGXV9HjY+aR4k0oJ5tcvakLuG6EuKReYlHNrgE= +github.com/spf13/cobra v1.1.3 h1:xghbfqPkxzxP3C/f3n5DdpAbdKLj4ZE4BWQI362l53M= +github.com/spf13/cobra v1.1.3/go.mod h1:pGADOWyqRD/YMrPZigI/zbliZ2wVD/23d+is3pSWzOo= github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= @@ -702,6 +762,7 @@ github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= github.com/spf13/viper v1.4.0/go.mod h1:PTJ7Z/lr49W6bUbkmS1V3by4uWynFiR9p7+dSq/yZzE= +github.com/spf13/viper v1.7.0/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5qpdg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE= @@ -713,6 +774,7 @@ github.com/stretchr/testify v1.6.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.0 h1:nwc3DEeHmmLAfoZucVR881uASk0Mfjw8xYJ99tb5CcY= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= github.com/swaggo/files v0.0.0-20190704085106-630677cd5c14/go.mod h1:gxQT6pBGRuIGunNf/+tSOB5OHvguWi8Tbt82WOkf35E= github.com/swaggo/gin-swagger v1.2.0/go.mod h1:qlH2+W7zXGZkczuL+r2nEBR2JTT+/lX05Nn6vPhc7OI= github.com/swaggo/http-swagger v0.0.0-20200308142732-58ac5e232fba/go.mod h1:O1lAbCgAAX/KZ80LM/OXwtWFI/5TvZlwxSg8Cq08PV0= @@ -738,8 +800,9 @@ github.com/tklauser/numcpus v0.2.1 h1:ct88eFm+Q7m2ZfXJdan1xYoXKlmwsfP+k88q05KvlZ github.com/tklauser/numcpus v0.2.1/go.mod h1:9aU+wOc6WjUIZEwWMP62PL/41d65P+iks1gBkr4QyP8= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966 h1:j6JEOq5QWFker+d7mFQYOhjTZonQ7YkLTHm56dbn+yM= github.com/tmc/grpc-websocket-proxy v0.0.0-20200427203606-3cfed13b9966/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= +github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 h1:uruHq4dN7GR16kFc5fp3d1RIYzJW5onx8Ybykw2YQFA= +github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/twmb/murmur3 v1.1.3 h1:D83U0XYKcHRYwYIpBKf3Pks91Z0Byda/9SJ8B6EMRcA= github.com/twmb/murmur3 v1.1.3/go.mod h1:Qq/R7NUyOfr65zD+6Q5IHKsJLwP7exErjN6lyyq3OSQ= github.com/uber/jaeger-client-go v2.22.1+incompatible h1:NHcubEkVbahf9t3p75TOCR83gdUHXjRJvjoBh1yACsM= @@ -794,8 +857,9 @@ github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1 github.com/yuin/goldmark v1.4.1/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.5 h1:XAzx9gjCb0Rxj7EoqcClPD1d5ZBxZJk0jbuoPHenBt0= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= +go.etcd.io/bbolt v1.3.6 h1:/ecaJf0sk1l4l6V4awd65v2C3ILy7MSj+s/x1ADCIMU= +go.etcd.io/bbolt v1.3.6/go.mod h1:qXsaaIqmgQH0T+OPdb99Bf+PKfBBQVAdyD6TY9G8XM4= go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= go.etcd.io/etcd v0.5.0-alpha.5.0.20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mIZCrds/GIG4ncV9HhK5PX7jPg= go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b/go.mod h1:yVHk9ub3CSBatqGNg7GRmsnfLWtoW60w4eDYfh7vHDg= @@ -844,6 +908,7 @@ go.uber.org/zap v1.19.1 h1:ue41HOKd1vGURxrmeKIgELGb3jPW9DMUDGtsinblHwI= go.uber.org/zap v1.19.1/go.mod h1:j3DNczoxDZroyBnOT1L/Q79cfUMGZxlv/9dzN7SM1rI= golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20190510104115-cbcb75029529/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= @@ -855,8 +920,9 @@ golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8U golang.org/x/crypto v0.0.0-20200204104054-c9f3fb736b72/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= -golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897 h1:pLI5jrR7OSLijeIDcmRxNmw2api+jEfxLoykJVice/E= golang.org/x/crypto v0.0.0-20201016220609-9e8e0b390897/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/crypto v0.0.0-20220131195533-30dcbda58838 h1:71vQrMauZZhcTVK6KdYM+rklehEEwb3E+ZhaE5jrPrE= +golang.org/x/crypto v0.0.0-20220131195533-30dcbda58838/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20181106170214-d68db9428509/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -904,7 +970,9 @@ golang.org/x/mod v0.5.1/go.mod h1:5OXOZSfqPIIbmVBIIKWRFfZjPR0E5r58TLhUjH0a2Ro= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -942,6 +1010,7 @@ golang.org/x/net v0.0.0-20201010224723-4f7140c49acb/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201031054903-ff519b6c9102/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.0.0-20201202161906-c7110b5ffcbb/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201209123823-ac852fbbde11/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210119194325-5f4716e94777/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= @@ -950,8 +1019,9 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210805182204-aaa1db679c0d/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f h1:OfiFi4JbukWwe3lzw+xunroH1mnC1e2Gy5cxNJApiSY= golang.org/x/net v0.0.0-20211015210444-4f30a5c0130f/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= +golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2 h1:CIJ76btIcR3eFI5EgSo6k1qKw9KJexJuRLI9G7Hp5wE= +golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -980,8 +1050,10 @@ golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c h1:5KslGYwFpkhGh+Q16bwMP3cOontH8FOep7tGV86Y7SQ= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180816055513-1c9583448a9c/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181107165924-66b7b1311ac8/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1009,6 +1081,7 @@ golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200106162015-b016eb3dc98e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200122134326-e047566fdf82/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1023,13 +1096,17 @@ golang.org/x/sys v0.0.0-20200511232937-7e40ca221e25/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200515095857-1151b9dac4a9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200523222454-059865788121/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200803210538-64077c9b5642/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200905004654-be1d3432aa8f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200923182605-d9f96fdee20d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210217105451-b926d437f341/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210305230114-8fe3ee5dd75b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1040,7 +1117,9 @@ golang.org/x/sys v0.0.0-20210403161142-5e06dd20ab57/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210510120138-977fb7262007/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210514084401-e8d321eab015/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210603081109-ebe580a85c40/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210603125802-9665404d3644/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210806184541-e5e7981a1069/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -1062,8 +1141,9 @@ golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.0.0-20191024005414-555d28b269f0 h1:/5xXl8Y5W96D+TtHSlonuFqGHIWVuyCkGJLwGh9JJFs= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba h1:O8mE0/t419eoIwhTFpKVkHiTs/Igowgfkj25AcZrtiE= +golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= @@ -1094,6 +1174,7 @@ golang.org/x/tools v0.0.0-20191029190741-b9c20aec41a5/go.mod h1:b+2E5dAYhXwXZwtn golang.org/x/tools v0.0.0-20191030062658-86caa796c7ab/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191107010934-f79515f33823/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191108193012-7d206e10da11/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191112195655-aa38f8e97acc/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191113191852-77e3bb0ad9e7/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191114200427-caa0b0f7d508/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20191115202509-3a792d9c32b2/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= @@ -1207,8 +1288,10 @@ google.golang.org/genproto v0.0.0-20200228133532-8c2c7df3a383/go.mod h1:55QSHmfG google.golang.org/genproto v0.0.0-20200305110556-506484158171/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200312145019-da6875a35672/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= +google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA= @@ -1269,6 +1352,7 @@ gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMy gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= gopkg.in/go-playground/validator.v9 v9.29.1/go.mod h1:+c9/zcJMFNgbLvly1L1V+PpxWdVbfP1avr/N00E2vyQ= +gopkg.in/ini.v1 v1.51.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo= gopkg.in/jcmturner/dnsutils.v1 v1.0.1/go.mod h1:m3v+5svpVOhtFAP/wSz+yzh4Mc0Fg7eRhxkJMWSIz9Q= gopkg.in/jcmturner/goidentity.v3 v3.0.0/go.mod h1:oG2kH0IvSYNIu80dVAyu/yoefjq1mNfM5bm88whjWx4= @@ -1326,8 +1410,9 @@ rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8 rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= -sigs.k8s.io/yaml v1.1.0 h1:4A07+ZFc2wgJwo8YNlQpr1rVlgUDlxXHhPJciaPY5gs= sigs.k8s.io/yaml v1.1.0/go.mod h1:UJmg0vDUVViEyp3mgSv9WPwZCDxu4rQW1olrI1uml+o= +sigs.k8s.io/yaml v1.2.0 h1:kr/MCeFWJWTwyaHoR9c8EjH9OumOmoF9YGiZd7lFm/Q= +sigs.k8s.io/yaml v1.2.0/go.mod h1:yfXDCHCao9+ENCvLSE62v9VSji2MKu5jeNfTrofGhJc= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 h1:ucqkfpjg9WzSUubAO62csmucvxl4/JeW3F4I4909XkM= sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1:hI742Nqp5OhwiqlzhgfbWU4mW4yO10fP+LoT9WOswdU= sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 h1:e1sMhtVq9AfcEy8AXNb8eSg6gbzfdpYhoNqnPJa+GzI= From 8d3e2ff12905d64c4a2c1b56cb4d217c197cc8f9 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 16:14:52 +0800 Subject: [PATCH 13/51] br: modify tables that should be altered auto id or random id (#33719) (#33988) close pingcap/tidb#33596 --- br/pkg/restore/client.go | 38 +++++++++++++++++++--------------- br/pkg/restore/db.go | 6 +++--- br/tests/br_incremental/run.sh | 21 +++++++++++++++---- 3 files changed, 41 insertions(+), 24 deletions(-) diff --git a/br/pkg/restore/client.go b/br/pkg/restore/client.go index c62eb1c3af6d0..0bd5bd785b627 100644 --- a/br/pkg/restore/client.go +++ b/br/pkg/restore/client.go @@ -117,6 +117,8 @@ func NewRestoreClient( if dom != nil { statsHandle = dom.StatsHandle() } + // init backupMeta only for passing unit test + backupMeta := new(backuppb.BackupMeta) return &Client{ pdClient: pdClient, @@ -127,6 +129,7 @@ func NewRestoreClient( switchCh: make(chan struct{}), dom: dom, statsHandler: statsHandle, + backupMeta: backupMeta, }, nil } @@ -450,8 +453,7 @@ func (rc *Client) GoCreateTables( ) <-chan CreatedTable { // Could we have a smaller size of tables? log.Info("start create tables") - - ddlTables := rc.DDLJobsMap() + ddlTables := rc.GenerateRebasedTables(tables) if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("Client.GoCreateTables", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -1119,22 +1121,24 @@ func (rc *Client) IsSkipCreateSQL() bool { return rc.noSchema } -// DDLJobsMap returns a map[UniqueTableName]bool about < db table, hasCreate/hasTruncate DDL >. -// if we execute some DDLs before create table. -// we may get two situation that need to rebase auto increment/random id. -// 1. truncate table: truncate will generate new id cache. -// 2. create table/create and rename table: the first create table will lock down the id cache. -// because we cannot create onExistReplace table. -// so the final create DDL with the correct auto increment/random id won't be executed. -func (rc *Client) DDLJobsMap() map[UniqueTableName]bool { - m := make(map[UniqueTableName]bool) - for _, job := range rc.ddlJobs { - switch job.Type { - case model.ActionTruncateTable, model.ActionCreateTable, model.ActionRenameTable: - m[UniqueTableName{job.SchemaName, job.BinlogInfo.TableInfo.Name.String()}] = true - } +// GenerateRebasedTables generate a map[UniqueTableName]bool to represent tables that haven't updated table info. +// there are two situations: +// 1. tables that already exists in the restored cluster. +// 2. tables that are created by executing ddl jobs. +// so, only tables in incremental restoration will be added to the map +func (rc *Client) GenerateRebasedTables(tables []*metautil.Table) (rebasedTablesMap map[UniqueTableName]bool) { + if !rc.IsIncremental() { + // in full restoration, all tables are created by Session.CreateTable, and all tables' info is updated. + rebasedTablesMap = make(map[UniqueTableName]bool) + return + } + + rebasedTablesMap = make(map[UniqueTableName]bool, len(tables)) + for _, table := range tables { + rebasedTablesMap[UniqueTableName{DB: table.DB.Name.String(), Table: table.Info.Name.String()}] = true } - return m + + return } // PreCheckTableTiFlashReplica checks whether TiFlash replica is less than TiFlash node. diff --git a/br/pkg/restore/db.go b/br/pkg/restore/db.go index 7510a1fb4dd5d..4602ed9ad8e35 100644 --- a/br/pkg/restore/db.go +++ b/br/pkg/restore/db.go @@ -141,7 +141,7 @@ func (db *DB) CreateDatabase(ctx context.Context, schema *model.DBInfo) error { } // CreateTable executes a CREATE TABLE SQL. -func (db *DB) CreateTable(ctx context.Context, table *metautil.Table, ddlTables map[UniqueTableName]bool) error { +func (db *DB) CreateTable(ctx context.Context, table *metautil.Table, toBeCorrectedTables map[UniqueTableName]bool) error { err := db.se.CreateTable(ctx, table.DB.Name, table.Info) if err != nil { log.Error("create table failed", @@ -206,8 +206,8 @@ func (db *DB) CreateTable(ctx context.Context, table *metautil.Table, ddlTables zap.Error(err)) return errors.Trace(err) } - // only table exists in ddlJobs during incremental restoration should do alter after creation. - case ddlTables[UniqueTableName{table.DB.Name.String(), table.Info.Name.String()}]: + // only table exists in restored cluster during incremental restoration should do alter after creation. + case toBeCorrectedTables[UniqueTableName{table.DB.Name.String(), table.Info.Name.String()}]: if utils.NeedAutoID(table.Info) { restoreMetaSQL = fmt.Sprintf( "alter table %s.%s auto_increment = %d;", diff --git a/br/tests/br_incremental/run.sh b/br/tests/br_incremental/run.sh index da24ba12b4a3d..903f417672989 100755 --- a/br/tests/br_incremental/run.sh +++ b/br/tests/br_incremental/run.sh @@ -17,29 +17,37 @@ set -eu DB="$TEST_NAME" TABLE="usertable" +AUTO_ID_TABLE="a" +ROW_COUNT=10 run_sql "CREATE DATABASE $DB;" go-ycsb load mysql -P tests/$TEST_NAME/workload -p mysql.host=$TIDB_IP -p mysql.port=$TIDB_PORT -p mysql.user=root -p mysql.db=$DB row_count_ori_full=$(run_sql "SELECT COUNT(*) FROM $DB.$TABLE;" | awk '/COUNT/{print $2}') +run_sql "CREATE TABLE IF NOT EXISTS ${DB}.${AUTO_ID_TABLE} (c1 INT);" + # full backup echo "full backup start..." -run_br --pd $PD_ADDR backup table -s "local://$TEST_DIR/$DB/full" --db $DB -t $TABLE +run_br --pd $PD_ADDR backup db -s "local://$TEST_DIR/$DB/full" --db $DB go-ycsb run mysql -P tests/$TEST_NAME/workload -p mysql.host=$TIDB_IP -p mysql.port=$TIDB_PORT -p mysql.user=root -p mysql.db=$DB +for i in $(seq $ROW_COUNT); do + run_sql "INSERT INTO ${DB}.${AUTO_ID_TABLE}(c1) VALUES ($i);" +done + # incremental backup echo "incremental backup start..." last_backup_ts=$(run_br validate decode --field="end-version" -s "local://$TEST_DIR/$DB/full" | grep -oE "^[0-9]+") -run_br --pd $PD_ADDR backup table -s "local://$TEST_DIR/$DB/inc" --db $DB -t $TABLE --lastbackupts $last_backup_ts +run_br --pd $PD_ADDR backup db -s "local://$TEST_DIR/$DB/inc" --db $DB --lastbackupts $last_backup_ts row_count_ori_inc=$(run_sql "SELECT COUNT(*) FROM $DB.$TABLE;" | awk '/COUNT/{print $2}') run_sql "DROP DATABASE $DB;" # full restore echo "full restore start..." -run_br restore table --db $DB --table $TABLE -s "local://$TEST_DIR/$DB/full" --pd $PD_ADDR +run_br restore db --db $DB -s "local://$TEST_DIR/$DB/full" --pd $PD_ADDR row_count_full=$(run_sql "SELECT COUNT(*) FROM $DB.$TABLE;" | awk '/COUNT/{print $2}') # check full restore if [ "${row_count_full}" != "${row_count_ori_full}" ];then @@ -48,7 +56,7 @@ if [ "${row_count_full}" != "${row_count_ori_full}" ];then fi # incremental restore echo "incremental restore start..." -run_br restore table --db $DB --table $TABLE -s "local://$TEST_DIR/$DB/inc" --pd $PD_ADDR +run_br restore db --db $DB -s "local://$TEST_DIR/$DB/inc" --pd $PD_ADDR row_count_inc=$(run_sql "SELECT COUNT(*) FROM $DB.$TABLE;" | awk '/COUNT/{print $2}') # check full restore if [ "${row_count_inc}" != "${row_count_ori_inc}" ];then @@ -56,4 +64,9 @@ if [ "${row_count_inc}" != "${row_count_ori_inc}" ];then exit 1 fi +# check inserting records with auto increment id after incremental restore +for i in $(seq $ROW_COUNT); do + run_sql "INSERT INTO ${DB}.${AUTO_ID_TABLE}(c1) VALUES ($i);" +done + run_sql "DROP DATABASE $DB;" From 33799e61cab627042a2645806505a9b16faa645c Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 16:36:52 +0800 Subject: [PATCH 14/51] planner: misuse like2regexp and regexp when to query information.schema (#32490) (#33233) close pingcap/tidb#32451 --- executor/show.go | 25 ++++++++++++------------ planner/core/show_predicate_extractor.go | 19 +++++++++++------- planner/core/stringer_test.go | 6 +++--- 3 files changed, 28 insertions(+), 22 deletions(-) diff --git a/executor/show.go b/executor/show.go index 36d559c706173..5f1b3f8bad927 100644 --- a/executor/show.go +++ b/executor/show.go @@ -20,7 +20,6 @@ import ( gjson "encoding/json" "fmt" "reflect" - "regexp" "sort" "strconv" "strings" @@ -435,15 +434,16 @@ func (e *ShowExec) fetchShowTables() error { tableNames := make([]string, 0, len(schemaTables)) activeRoles := e.ctx.GetSessionVars().ActiveRoles var ( - tableTypes = make(map[string]string) - fieldPatternsRegexp *regexp.Regexp - FieldFilterEnable bool - fieldFilter string + tableTypes = make(map[string]string) + fieldPatternsLike collate.WildcardPattern + FieldFilterEnable bool + fieldFilter string ) if e.Extractor != nil { extractor := (e.Extractor).(*plannercore.ShowTablesTableExtractor) if extractor.FieldPatterns != "" { - fieldPatternsRegexp = regexp.MustCompile(extractor.FieldPatterns) + fieldPatternsLike = collate.GetCollatorByID(collate.CollationName2ID(mysql.UTF8MB4DefaultCollation)).Pattern() + fieldPatternsLike.Compile(extractor.FieldPatterns, byte('\\')) } FieldFilterEnable = extractor.Field != "" fieldFilter = extractor.Field @@ -455,7 +455,7 @@ func (e *ShowExec) fetchShowTables() error { continue } else if FieldFilterEnable && v.Meta().Name.L != fieldFilter { continue - } else if fieldPatternsRegexp != nil && !fieldPatternsRegexp.MatchString(v.Meta().Name.L) { + } else if fieldPatternsLike != nil && !fieldPatternsLike.DoMatch(v.Meta().Name.L) { continue } tableNames = append(tableNames, v.Meta().Name.O) @@ -538,14 +538,15 @@ func (e *ShowExec) fetchShowColumns(ctx context.Context) error { return errors.Trace(err) } var ( - fieldPatternsRegexp *regexp.Regexp - FieldFilterEnable bool - fieldFilter string + fieldPatternsLike collate.WildcardPattern + FieldFilterEnable bool + fieldFilter string ) if e.Extractor != nil { extractor := (e.Extractor).(*plannercore.ShowColumnsTableExtractor) if extractor.FieldPatterns != "" { - fieldPatternsRegexp = regexp.MustCompile(extractor.FieldPatterns) + fieldPatternsLike = collate.GetCollatorByID(collate.CollationName2ID(mysql.UTF8MB4DefaultCollation)).Pattern() + fieldPatternsLike.Compile(extractor.FieldPatterns, byte('\\')) } FieldFilterEnable = extractor.Field != "" fieldFilter = extractor.Field @@ -571,7 +572,7 @@ func (e *ShowExec) fetchShowColumns(ctx context.Context) error { for _, col := range cols { if FieldFilterEnable && col.Name.L != fieldFilter { continue - } else if fieldPatternsRegexp != nil && !fieldPatternsRegexp.MatchString(col.Name.L) { + } else if fieldPatternsLike != nil && !fieldPatternsLike.DoMatch(col.Name.L) { continue } desc := table.NewColDesc(col) diff --git a/planner/core/show_predicate_extractor.go b/planner/core/show_predicate_extractor.go index 103c4107c7f5e..37979006ec70f 100644 --- a/planner/core/show_predicate_extractor.go +++ b/planner/core/show_predicate_extractor.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/parser/ast" driver "github.com/pingcap/tidb/types/parser_driver" - "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/stringutil" ) @@ -60,12 +59,11 @@ func (e *ShowColumnsTableExtractor) Extract(show *ast.ShowStmt) bool { // It is used in `SHOW COLUMNS FROM t LIKE `abc``. ptn := pattern.Pattern.(*driver.ValueExpr).GetString() patValue, patTypes := stringutil.CompilePattern(ptn, pattern.Escape) - if !collate.NewCollationEnabled() && stringutil.IsExactMatch(patTypes) { + if stringutil.IsExactMatch(patTypes) { e.Field = strings.ToLower(string(patValue)) return true } - // (?i) mean to be case-insensitive. - e.FieldPatterns = "(?i)" + stringutil.CompileLike2Regexp(string(patValue)) + e.FieldPatterns = strings.ToLower(string(patValue)) return true case *ast.ColumnNameExpr: // It is used in `SHOW COLUMNS FROM t LIKE abc`. @@ -109,17 +107,24 @@ func (e *ShowTablesTableExtractor) Extract(show *ast.ShowStmt) bool { pattern := show.Pattern switch pattern.Pattern.(type) { case *driver.ValueExpr: - // It is used in `SHOW TABLE in t LIKE `abc``. + // It is used in `SHOW COLUMNS FROM t LIKE `abc``. ptn := pattern.Pattern.(*driver.ValueExpr).GetString() patValue, patTypes := stringutil.CompilePattern(ptn, pattern.Escape) if stringutil.IsExactMatch(patTypes) { e.Field = strings.ToLower(string(patValue)) return true } - // (?i) mean to be case-insensitive. - e.FieldPatterns = "(?i)" + stringutil.CompileLike2Regexp(string(patValue)) + e.FieldPatterns = strings.ToLower(string(patValue)) return true + case *ast.ColumnNameExpr: + // It is used in `SHOW COLUMNS FROM t LIKE abc`. + // MySQL do not support this syntax and return the error. + return false } + } else if show.Column != nil && show.Column.Name.L != "" { + // it is used in `DESCRIBE t COLUMN`. + e.Field = show.Column.Name.L + return true } return false } diff --git a/planner/core/stringer_test.go b/planner/core/stringer_test.go index e4163c7c29c8e..6433ad59b158c 100644 --- a/planner/core/stringer_test.go +++ b/planner/core/stringer_test.go @@ -42,7 +42,7 @@ func TestPlanStringer(t *testing.T) { }, { sql: "show columns from t like 'a%'", - plan: "Show(field_pattern:[(?i)a.*])", + plan: "Show(field_pattern:[a%])", }, { sql: "show columns from t where field = 'a'", @@ -66,11 +66,11 @@ func TestPlanStringer(t *testing.T) { }, { sql: "show tables in test like 't%'", - plan: "Show(table_pattern:[(?i)t.*])", + plan: "Show(table_pattern:[t%])", }, { sql: "show tables in test like '%T%'", - plan: "Show(table_pattern:[(?i).*T.*])", + plan: "Show(table_pattern:[%t%])", }, } parser := parser.New() From 73cec931160684352735ede4ea224f6a5e30ebc9 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 16:52:52 +0800 Subject: [PATCH 15/51] expression: fix greatest and least function with collation get wrong result (#31806) (#31838) close pingcap/tidb#31789 --- cmd/explaintest/r/collation.result | 22 ++++++++++++++++++++++ cmd/explaintest/t/collation.test | 20 +++++++++++++++++++- expression/collation.go | 2 +- expression/collation_test.go | 2 +- 4 files changed, 43 insertions(+), 3 deletions(-) diff --git a/cmd/explaintest/r/collation.result b/cmd/explaintest/r/collation.result index 193915a15d6c9..53a8250653ea3 100644 --- a/cmd/explaintest/r/collation.result +++ b/cmd/explaintest/r/collation.result @@ -13,3 +13,25 @@ coercibility(convert('a' using utf8mb4)) select coercibility(convert('a' using utf8mb4) collate utf8mb4_general_ci); coercibility(convert('a' using utf8mb4) collate utf8mb4_general_ci) 0 +DROP TABLE IF EXISTS t2; +CREATE TABLE t2 ( +id INT NOT NULL PRIMARY KEY auto_increment, +`col_91` char(47) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, +`col_92` int(10) unsigned DEFAULT '2478966067', +`col_97` char(32) COLLATE utf8mb4_bin NOT NULL +) collate utf8mb4_general_ci; +INSERT INTO `t2` VALUES (17,'UUtJeaV',497551109,'snRXXCZHBPW'); +SET names utf8mb4 collate utf8mb4_bin; +SELECT greatest( col_91 , col_97 ) as expr1 FROM t2 ORDER BY id; +expr1 +snRXXCZHBPW +SELECT least( col_91 , col_97 ) as expr1 FROM t2 ORDER BY id; +expr1 +UUtJeaV +SET names utf8mb4 collate utf8mb4_general_ci; +SELECT greatest( col_91 , col_97 ) as expr1 FROM t2 ORDER BY id; +expr1 +snRXXCZHBPW +SELECT least( col_91 , col_97 ) as expr1 FROM t2 ORDER BY id; +expr1 +UUtJeaV diff --git a/cmd/explaintest/t/collation.test b/cmd/explaintest/t/collation.test index 2b67de475c47c..bf5751f2c7d53 100644 --- a/cmd/explaintest/t/collation.test +++ b/cmd/explaintest/t/collation.test @@ -6,4 +6,22 @@ insert into t values ('啊', '撒旦'); select coercibility(concat(a, b)) from t; select coercibility(convert(concat(a, b) using utf8mb4) collate utf8mb4_general_ci) from t; select coercibility(convert('a' using utf8mb4)); -select coercibility(convert('a' using utf8mb4) collate utf8mb4_general_ci); \ No newline at end of file +select coercibility(convert('a' using utf8mb4) collate utf8mb4_general_ci); + +# test greatest and least function with collation. +DROP TABLE IF EXISTS t2; +CREATE TABLE t2 ( + id INT NOT NULL PRIMARY KEY auto_increment, + `col_91` char(47) CHARACTER SET utf8 COLLATE utf8_bin NOT NULL, + `col_92` int(10) unsigned DEFAULT '2478966067', + `col_97` char(32) COLLATE utf8mb4_bin NOT NULL +) collate utf8mb4_general_ci; + +INSERT INTO `t2` VALUES (17,'UUtJeaV',497551109,'snRXXCZHBPW'); + +SET names utf8mb4 collate utf8mb4_bin; +SELECT greatest( col_91 , col_97 ) as expr1 FROM t2 ORDER BY id; +SELECT least( col_91 , col_97 ) as expr1 FROM t2 ORDER BY id; +SET names utf8mb4 collate utf8mb4_general_ci; +SELECT greatest( col_91 , col_97 ) as expr1 FROM t2 ORDER BY id; +SELECT least( col_91 , col_97 ) as expr1 FROM t2 ORDER BY id; diff --git a/expression/collation.go b/expression/collation.go index d2535fe716656..40f3104dd48ae 100644 --- a/expression/collation.go +++ b/expression/collation.go @@ -183,7 +183,7 @@ func deriveCoercibilityForColumn(c *Column) Coercibility { func deriveCollation(ctx sessionctx.Context, funcName string, args []Expression, retType types.EvalType, argTps ...types.EvalType) (ec *ExprCollation, err error) { switch funcName { - case ast.Concat, ast.ConcatWS, ast.Lower, ast.Lcase, ast.Reverse, ast.Upper, ast.Ucase, ast.Quote, ast.Coalesce: + case ast.Concat, ast.ConcatWS, ast.Lower, ast.Lcase, ast.Reverse, ast.Upper, ast.Ucase, ast.Quote, ast.Coalesce, ast.Greatest, ast.Least: return CheckAndDeriveCollationFromExprs(ctx, funcName, retType, args...) case ast.Left, ast.Right, ast.Repeat, ast.Trim, ast.LTrim, ast.RTrim, ast.Substr, ast.SubstringIndex, ast.Replace, ast.Substring, ast.Mid, ast.Translate: return CheckAndDeriveCollationFromExprs(ctx, funcName, retType, args[0]) diff --git a/expression/collation_test.go b/expression/collation_test.go index 111fe1241a368..9e11c66089d12 100644 --- a/expression/collation_test.go +++ b/expression/collation_test.go @@ -508,7 +508,7 @@ func TestDeriveCollation(t *testing.T) { }, { []string{ - ast.Concat, ast.ConcatWS, ast.Coalesce, ast.In, + ast.Concat, ast.ConcatWS, ast.Coalesce, ast.In, ast.Greatest, ast.Least, }, []Expression{ newConstString("a", CoercibilityCoercible, charset.CharsetUTF8MB4, charset.CollationUTF8MB4), From d1eac99f26eaaf00a234a5e46d0f2b5196aefbe5 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 18:08:53 +0800 Subject: [PATCH 16/51] br: Fix backup rawkv failure (#32612) (#32793) close pingcap/tidb#32607 --- br/pkg/backup/client.go | 18 +++- br/pkg/backup/push.go | 17 +++ br/pkg/lightning/backend/local/local.go | 2 +- br/pkg/restore/client.go | 5 +- br/pkg/restore/client_test.go | 8 +- br/pkg/restore/pipeline_items.go | 2 +- br/pkg/restore/split.go | 13 ++- br/pkg/restore/split_client.go | 9 +- br/pkg/restore/split_test.go | 35 ++++-- br/pkg/restore/util.go | 5 +- br/pkg/task/restore.go | 2 +- br/pkg/task/restore_raw.go | 4 +- br/tests/br_rawkv/run.sh | 138 ++++++++++++++---------- 13 files changed, 164 insertions(+), 94 deletions(-) mode change 100644 => 100755 br/tests/br_rawkv/run.sh diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index 7a4b1e0e8eb66..e4584df29f83a 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -530,7 +530,7 @@ func (bc *Client) BackupRange( // TODO: test fine grained backup. err = bc.fineGrainedBackup( ctx, startKey, endKey, req.StartVersion, req.EndVersion, req.CompressionType, req.CompressionLevel, - req.RateLimit, req.Concurrency, results, progressCallBack) + req.RateLimit, req.Concurrency, req.IsRawKv, req.CipherInfo, results, progressCallBack) if err != nil { return errors.Trace(err) } @@ -574,10 +574,12 @@ func (bc *Client) BackupRange( return nil } -func (bc *Client) findRegionLeader(ctx context.Context, key []byte) (*metapb.Peer, error) { +func (bc *Client) findRegionLeader(ctx context.Context, key []byte, isRawKv bool) (*metapb.Peer, error) { // Keys are saved in encoded format in TiKV, so the key must be encoded // in order to find the correct region. - key = codec.EncodeBytes([]byte{}, key) + if !isRawKv { + key = codec.EncodeBytes([]byte{}, key) + } for i := 0; i < 5; i++ { // better backoff. region, err := bc.mgr.GetPDClient().GetRegion(ctx, key) @@ -608,6 +610,8 @@ func (bc *Client) fineGrainedBackup( compressLevel int32, rateLimit uint64, concurrency uint32, + isRawKv bool, + cipherInfo *backuppb.CipherInfo, rangeTree rtree.RangeTree, progressCallBack func(ProgressUnit), ) error { @@ -658,7 +662,7 @@ func (bc *Client) fineGrainedBackup( for rg := range retry { backoffMs, err := bc.handleFineGrained(ctx, boFork, rg, lastBackupTS, backupTS, - compressType, compressLevel, rateLimit, concurrency, respCh) + compressType, compressLevel, rateLimit, concurrency, isRawKv, cipherInfo, respCh) if err != nil { errCh <- err return @@ -803,9 +807,11 @@ func (bc *Client) handleFineGrained( compressionLevel int32, rateLimit uint64, concurrency uint32, + isRawKv bool, + cipherInfo *backuppb.CipherInfo, respCh chan<- *backuppb.BackupResponse, ) (int, error) { - leader, pderr := bc.findRegionLeader(ctx, rg.StartKey) + leader, pderr := bc.findRegionLeader(ctx, rg.StartKey, isRawKv) if pderr != nil { return 0, errors.Trace(pderr) } @@ -820,8 +826,10 @@ func (bc *Client) handleFineGrained( StorageBackend: bc.backend, RateLimit: rateLimit, Concurrency: concurrency, + IsRawKv: isRawKv, CompressionType: compressType, CompressionLevel: compressionLevel, + CipherInfo: cipherInfo, } lockResolver := bc.mgr.GetLockResolver() client, err := bc.mgr.GetBackupClient(ctx, storeID) diff --git a/br/pkg/backup/push.go b/br/pkg/backup/push.go index c365eaaa96921..c1d4cd7664be1 100644 --- a/br/pkg/backup/push.go +++ b/br/pkg/backup/push.go @@ -11,6 +11,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/metapb" berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/logutil" @@ -116,6 +117,7 @@ func (push *pushDown) pushBackup( close(push.respCh) }() + regionErrorIngestedOnce := false for { select { case respAndStore, ok := <-push.respCh: @@ -139,6 +141,21 @@ func (push *pushDown) pushBackup( Msg: msg, } }) + failpoint.Inject("tikv-region-error", func(val failpoint.Value) { + if !regionErrorIngestedOnce { + msg := val.(string) + logutil.CL(ctx).Debug("failpoint tikv-regionh-error injected.", zap.String("msg", msg)) + resp.Error = &backuppb.Error{ + // Msg: msg, + Detail: &backuppb.Error_RegionError{ + RegionError: &errorpb.Error{ + Message: msg, + }, + }, + } + } + regionErrorIngestedOnce = true + }) if resp.GetError() == nil { // None error means range has been backuped successfully. res.Put( diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index b4a5826e5651f..f34a31da8ee8a 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -258,7 +258,7 @@ func NewLocalBackend( if err != nil { return backend.MakeBackend(nil), errors.Annotate(err, "construct pd client failed") } - splitCli := split.NewSplitClient(pdCtl.GetPDClient(), tls.TLSConfig()) + splitCli := split.NewSplitClient(pdCtl.GetPDClient(), tls.TLSConfig(), false) shouldCreate := true if cfg.Checkpoint.Enable { diff --git a/br/pkg/restore/client.go b/br/pkg/restore/client.go index 0bd5bd785b627..4767a9c8323b9 100644 --- a/br/pkg/restore/client.go +++ b/br/pkg/restore/client.go @@ -102,6 +102,7 @@ func NewRestoreClient( store kv.Storage, tlsConf *tls.Config, keepaliveConf keepalive.ClientParameters, + isRawKv bool, ) (*Client, error) { db, err := NewDB(g, store) if err != nil { @@ -122,7 +123,7 @@ func NewRestoreClient( return &Client{ pdClient: pdClient, - toolClient: NewSplitClient(pdClient, tlsConf), + toolClient: NewSplitClient(pdClient, tlsConf, isRawKv), db: db, tlsConf: tlsConf, keepaliveConf: keepaliveConf, @@ -208,7 +209,7 @@ func (rc *Client) InitBackupMeta( rc.backupMeta = backupMeta log.Info("load backupmeta", zap.Int("databases", len(rc.databases)), zap.Int("jobs", len(rc.ddlJobs))) - metaClient := NewSplitClient(rc.pdClient, rc.tlsConf) + metaClient := NewSplitClient(rc.pdClient, rc.tlsConf, rc.backupMeta.IsRawKv) importCli := NewImportClient(metaClient, rc.tlsConf, rc.keepaliveConf) rc.fileImporter = NewFileImporter(metaClient, importCli, backend, rc.backupMeta.IsRawKv, rc.rateLimit) return rc.fileImporter.CheckMultiIngestSupport(c, rc.pdClient) diff --git a/br/pkg/restore/client_test.go b/br/pkg/restore/client_test.go index ef704b256f7b4..799f19e445f63 100644 --- a/br/pkg/restore/client_test.go +++ b/br/pkg/restore/client_test.go @@ -47,7 +47,7 @@ func (s *testRestoreClientSuite) TearDownTest(c *C) { } func (s *testRestoreClientSuite) TestCreateTables(c *C) { - client, err := restore.NewRestoreClient(gluetidb.New(), s.mock.PDClient, s.mock.Storage, nil, defaultKeepaliveCfg) + client, err := restore.NewRestoreClient(gluetidb.New(), s.mock.PDClient, s.mock.Storage, nil, defaultKeepaliveCfg, false) c.Assert(err, IsNil) info, err := s.mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) @@ -103,7 +103,7 @@ func (s *testRestoreClientSuite) TestCreateTables(c *C) { func (s *testRestoreClientSuite) TestIsOnline(c *C) { - client, err := restore.NewRestoreClient(gluetidb.New(), s.mock.PDClient, s.mock.Storage, nil, defaultKeepaliveCfg) + client, err := restore.NewRestoreClient(gluetidb.New(), s.mock.PDClient, s.mock.Storage, nil, defaultKeepaliveCfg, false) c.Assert(err, IsNil) c.Assert(client.IsOnline(), IsFalse) @@ -113,7 +113,7 @@ func (s *testRestoreClientSuite) TestIsOnline(c *C) { func (s *testRestoreClientSuite) TestPreCheckTableClusterIndex(c *C) { - client, err := restore.NewRestoreClient(gluetidb.New(), s.mock.PDClient, s.mock.Storage, nil, defaultKeepaliveCfg) + client, err := restore.NewRestoreClient(gluetidb.New(), s.mock.PDClient, s.mock.Storage, nil, defaultKeepaliveCfg, false) c.Assert(err, IsNil) info, err := s.mock.Domain.GetSnapshotInfoSchema(math.MaxUint64) @@ -205,7 +205,7 @@ func (s *testRestoreClientSuite) TestPreCheckTableTiFlashReplicas(c *C) { client, err := restore.NewRestoreClient(gluetidb.New(), fakePDClient{ stores: mockStores, - }, s.mock.Storage, nil, defaultKeepaliveCfg) + }, s.mock.Storage, nil, defaultKeepaliveCfg, false) c.Assert(err, IsNil) tables := make([]*metautil.Table, 4) diff --git a/br/pkg/restore/pipeline_items.go b/br/pkg/restore/pipeline_items.go index ce476b1963fa5..7d3ce107ff70b 100644 --- a/br/pkg/restore/pipeline_items.go +++ b/br/pkg/restore/pipeline_items.go @@ -289,7 +289,7 @@ func (b *tikvSender) splitWorker(ctx context.Context, // hence the checksum would fail. done := b.registerTableIsRestoring(result.TablesToSend) pool.ApplyOnErrorGroup(eg, func() error { - err := SplitRanges(ectx, b.client, result.Ranges, result.RewriteRules, b.updateCh) + err := SplitRanges(ectx, b.client, result.Ranges, result.RewriteRules, b.updateCh, false) if err != nil { log.Error("failed on split range", rtree.ZapRanges(result.Ranges), zap.Error(err)) return err diff --git a/br/pkg/restore/split.go b/br/pkg/restore/split.go index b302dfc2487d3..cd9fa405f17fd 100644 --- a/br/pkg/restore/split.go +++ b/br/pkg/restore/split.go @@ -74,6 +74,7 @@ func (rs *RegionSplitter) Split( ctx context.Context, ranges []rtree.Range, rewriteRules *RewriteRules, + isRawKv bool, onSplit OnSplitFunc, ) error { if len(ranges) == 0 { @@ -108,7 +109,7 @@ SplitRegions: } return errors.Trace(errScan) } - splitKeyMap := getSplitKeys(rewriteRules, sortedRanges, regions) + splitKeyMap := getSplitKeys(rewriteRules, sortedRanges, regions, isRawKv) regionMap := make(map[uint64]*RegionInfo) for _, region := range regions { regionMap[region.Region.GetId()] = region @@ -477,14 +478,14 @@ func (b *scanRegionBackoffer) Attempt() int { // getSplitKeys checks if the regions should be split by the end key of // the ranges, groups the split keys by region id. -func getSplitKeys(rewriteRules *RewriteRules, ranges []rtree.Range, regions []*RegionInfo) map[uint64][][]byte { +func getSplitKeys(rewriteRules *RewriteRules, ranges []rtree.Range, regions []*RegionInfo, isRawKv bool) map[uint64][][]byte { splitKeyMap := make(map[uint64][][]byte) checkKeys := make([][]byte, 0) for _, rg := range ranges { checkKeys = append(checkKeys, rg.EndKey) } for _, key := range checkKeys { - if region := NeedSplit(key, regions); region != nil { + if region := NeedSplit(key, regions, isRawKv); region != nil { splitKeys, ok := splitKeyMap[region.Region.GetId()] if !ok { splitKeys = make([][]byte, 0, 1) @@ -500,12 +501,14 @@ func getSplitKeys(rewriteRules *RewriteRules, ranges []rtree.Range, regions []*R } // NeedSplit checks whether a key is necessary to split, if true returns the split region. -func NeedSplit(splitKey []byte, regions []*RegionInfo) *RegionInfo { +func NeedSplit(splitKey []byte, regions []*RegionInfo, isRawKv bool) *RegionInfo { // If splitKey is the max key. if len(splitKey) == 0 { return nil } - splitKey = codec.EncodeBytes(splitKey) + if !isRawKv { + splitKey = codec.EncodeBytes(splitKey) + } for _, region := range regions { // If splitKey is the boundary of the region if bytes.Equal(splitKey, region.Region.GetStartKey()) { diff --git a/br/pkg/restore/split_client.go b/br/pkg/restore/split_client.go index ed24fc3984a52..a00d5dde22848 100755 --- a/br/pkg/restore/split_client.go +++ b/br/pkg/restore/split_client.go @@ -89,14 +89,17 @@ type pdClient struct { // this may mislead the scatter. needScatterVal bool needScatterInit sync.Once + + isRawKv bool } // NewSplitClient returns a client used by RegionSplitter. -func NewSplitClient(client pd.Client, tlsConf *tls.Config) SplitClient { +func NewSplitClient(client pd.Client, tlsConf *tls.Config, isRawKv bool) SplitClient { cli := &pdClient{ client: client, tlsConf: tlsConf, storeCache: make(map[uint64]*metapb.Store), + isRawKv: isRawKv, } return cli } @@ -255,6 +258,7 @@ func splitRegionWithFailpoint( peer *metapb.Peer, client tikvpb.TikvClient, keys [][]byte, + isRawKv bool, ) (*kvrpcpb.SplitRegionResponse, error) { failpoint.Inject("not-leader-error", func(injectNewLeader failpoint.Value) { log.Debug("failpoint not-leader-error injected.") @@ -285,6 +289,7 @@ func splitRegionWithFailpoint( Peer: peer, }, SplitKeys: keys, + IsRawKv: isRawKv, }) } @@ -320,7 +325,7 @@ func (c *pdClient) sendSplitRegionRequest( } defer conn.Close() client := tikvpb.NewTikvClient(conn) - resp, err := splitRegionWithFailpoint(ctx, regionInfo, peer, client, keys) + resp, err := splitRegionWithFailpoint(ctx, regionInfo, peer, client, keys, c.isRawKv) if err != nil { return nil, multierr.Append(splitErrors, err) } diff --git a/br/pkg/restore/split_test.go b/br/pkg/restore/split_test.go index fdfbba8df54d0..09288c459d601 100644 --- a/br/pkg/restore/split_test.go +++ b/br/pkg/restore/split_test.go @@ -275,7 +275,7 @@ func TestScatterFinishInTime(t *testing.T) { regionSplitter := restore.NewRegionSplitter(client) ctx := context.Background() - err := regionSplitter.Split(ctx, ranges, rewriteRules, func(key [][]byte) {}) + err := regionSplitter.Split(ctx, ranges, rewriteRules, false, func(key [][]byte) {}) require.NoError(t, err) regions := client.GetAllRegions() if !validateRegions(regions) { @@ -331,7 +331,7 @@ func runTestSplitAndScatterWith(t *testing.T, client *TestClient) { regionSplitter := restore.NewRegionSplitter(client) ctx := context.Background() - err := regionSplitter.Split(ctx, ranges, rewriteRules, func(key [][]byte) {}) + err := regionSplitter.Split(ctx, ranges, rewriteRules, false, func(key [][]byte) {}) require.NoError(t, err) regions := client.GetAllRegions() if !validateRegions(regions) { @@ -466,26 +466,39 @@ FindRegion: } func (s *testRangeSuite) TestNeedSplit(c *C) { + for _, isRawKv := range []bool{false, true} { + s.testNeedSplit(c, isRawKv) + } +} + +func (s *testRangeSuite) testNeedSplit(c *C, isRawKv bool) { + encode := func(in []byte) []byte { + if isRawKv { + return in + } + return codec.EncodeBytes([]byte{}, in) + } + regions := []*restore.RegionInfo{ { Region: &metapb.Region{ - StartKey: codec.EncodeBytes([]byte{}, []byte("b")), - EndKey: codec.EncodeBytes([]byte{}, []byte("d")), + StartKey: encode([]byte("b")), + EndKey: encode([]byte("d")), }, }, } // Out of region - c.Assert(restore.NeedSplit([]byte("a"), regions), IsNil) + c.Assert(restore.NeedSplit([]byte("a"), regions, isRawKv), IsNil) // Region start key - c.Assert(restore.NeedSplit([]byte("b"), regions), IsNil) + c.Assert(restore.NeedSplit([]byte("b"), regions, isRawKv), IsNil) // In region - region := restore.NeedSplit([]byte("c"), regions) - c.Assert(bytes.Compare(region.Region.GetStartKey(), codec.EncodeBytes([]byte{}, []byte("b"))), Equals, 0) - c.Assert(bytes.Compare(region.Region.GetEndKey(), codec.EncodeBytes([]byte{}, []byte("d"))), Equals, 0) + region := restore.NeedSplit([]byte("c"), regions, isRawKv) + c.Assert(bytes.Compare(region.Region.GetStartKey(), encode([]byte("b"))), Equals, 0) + c.Assert(bytes.Compare(region.Region.GetEndKey(), encode([]byte("d"))), Equals, 0) // Region end key - c.Assert(restore.NeedSplit([]byte("d"), regions), IsNil) + c.Assert(restore.NeedSplit([]byte("d"), regions, isRawKv), IsNil) // Out of region - c.Assert(restore.NeedSplit([]byte("e"), regions), IsNil) + c.Assert(restore.NeedSplit([]byte("e"), regions, isRawKv), IsNil) } func (s *testRangeSuite) TestRegionConsistency(c *C) { diff --git a/br/pkg/restore/util.go b/br/pkg/restore/util.go index 812d87b09cec6..a323e6006e79c 100644 --- a/br/pkg/restore/util.go +++ b/br/pkg/restore/util.go @@ -345,10 +345,11 @@ func SplitRanges( ranges []rtree.Range, rewriteRules *RewriteRules, updateCh glue.Progress, + isRawKv bool, ) error { - splitter := NewRegionSplitter(NewSplitClient(client.GetPDClient(), client.GetTLSConfig())) + splitter := NewRegionSplitter(NewSplitClient(client.GetPDClient(), client.GetTLSConfig(), isRawKv)) - return splitter.Split(ctx, ranges, rewriteRules, func(keys [][]byte) { + return splitter.Split(ctx, ranges, rewriteRules, isRawKv, func(keys [][]byte) { for range keys { updateCh.Inc() } diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 710be11d8e212..dfdfc661f75b6 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -267,7 +267,7 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf keepaliveCfg := GetKeepalive(&cfg.Config) keepaliveCfg.PermitWithoutStream = true - client, err := restore.NewRestoreClient(g, mgr.GetPDClient(), mgr.GetStorage(), mgr.GetTLSConfig(), keepaliveCfg) + client, err := restore.NewRestoreClient(g, mgr.GetPDClient(), mgr.GetStorage(), mgr.GetTLSConfig(), keepaliveCfg, false) if err != nil { return errors.Trace(err) } diff --git a/br/pkg/task/restore_raw.go b/br/pkg/task/restore_raw.go index bb1dfddc9ce2d..25409e8c28e31 100644 --- a/br/pkg/task/restore_raw.go +++ b/br/pkg/task/restore_raw.go @@ -75,7 +75,7 @@ func RunRestoreRaw(c context.Context, g glue.Glue, cmdName string, cfg *RestoreR // sometimes we have pooled the connections. // sending heartbeats in idle times is useful. keepaliveCfg.PermitWithoutStream = true - client, err := restore.NewRestoreClient(g, mgr.GetPDClient(), mgr.GetStorage(), mgr.GetTLSConfig(), keepaliveCfg) + client, err := restore.NewRestoreClient(g, mgr.GetPDClient(), mgr.GetStorage(), mgr.GetTLSConfig(), keepaliveCfg, true) if err != nil { return errors.Trace(err) } @@ -131,7 +131,7 @@ func RunRestoreRaw(c context.Context, g glue.Glue, cmdName string, cfg *RestoreR // RawKV restore does not need to rewrite keys. rewrite := &restore.RewriteRules{} - err = restore.SplitRanges(ctx, client, ranges, rewrite, updateCh) + err = restore.SplitRanges(ctx, client, ranges, rewrite, updateCh, true) if err != nil { return errors.Trace(err) } diff --git a/br/tests/br_rawkv/run.sh b/br/tests/br_rawkv/run.sh old mode 100644 new mode 100755 index 6aaaf3b52ca1b..97450d3e65fc7 --- a/br/tests/br_rawkv/run.sh +++ b/br/tests/br_rawkv/run.sh @@ -21,8 +21,7 @@ source $( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )/../_utils/run_services start_services --no-tiflash BACKUP_DIR=$TEST_DIR/"raw_backup" - -rm -rf $BACKUP_DIR +BACKUP_FULL=$TEST_DIR/"rawkv-full" checksum() { bin/rawkv --pd $PD_ADDR \ @@ -49,9 +48,12 @@ test_full_rawkv() { check_range_start=00 check_range_end=ff + rm -rf $BACKUP_FULL + checksum_full=$(checksum $check_range_start $check_range_end) # backup current state of key-values - run_br --pd $PD_ADDR backup raw -s "local://$TEST_DIR/rawkv-full" --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + # raw backup is not working with range [nil, nil]. TODO: fix it. + run_br --pd $PD_ADDR backup raw -s "local://$BACKUP_FULL" --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" --start $check_range_start --format hex clean $check_range_start $check_range_end # Ensure the data is deleted @@ -61,7 +63,7 @@ test_full_rawkv() { fail_and_exit fi - run_br --pd $PD_ADDR restore raw -s "local://$TEST_DIR/rawkv-full" --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + run_br --pd $PD_ADDR restore raw -s "local://$BACKUP_FULL" --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" --start $check_range_start --format hex checksum_new=$(checksum $check_range_start $check_range_end) if [ "$checksum_new" != "$checksum_full" ];then echo "failed to restore" @@ -71,71 +73,91 @@ test_full_rawkv() { checksum_empty=$(checksum 31 3130303030303030) -# generate raw kv randomly in range[start-key, end-key) in 10s -bin/rawkv --pd $PD_ADDR \ - --ca "$TEST_DIR/certs/ca.pem" \ - --cert "$TEST_DIR/certs/br.pem" \ - --key "$TEST_DIR/certs/br.key" \ - --mode rand-gen --start-key 31 --end-key 3130303030303030 --duration 10 +run_test() { + if [ -z "$1" ];then + echo "run test" + else + export GO_FAILPOINTS="$1" + echo "run test with failpoints: $GO_FAILPOINTS" + fi -# put some keys around 311122 to check the correctness of endKey of restoring -bin/rawkv --pd $PD_ADDR \ - --ca "$TEST_DIR/certs/ca.pem" \ - --cert "$TEST_DIR/certs/br.pem" \ - --key "$TEST_DIR/certs/br.key" \ - --mode put --put-data "311121:31, 31112100:32, 311122:33, 31112200:34, 3111220000:35, 311123:36" + rm -rf $BACKUP_DIR + clean 31 3130303030303030 -checksum_ori=$(checksum 31 3130303030303030) -checksum_partial=$(checksum 311111 311122) + # generate raw kv randomly in range[start-key, end-key) in 10s + bin/rawkv --pd $PD_ADDR \ + --ca "$TEST_DIR/certs/ca.pem" \ + --cert "$TEST_DIR/certs/br.pem" \ + --key "$TEST_DIR/certs/br.key" \ + --mode rand-gen --start-key 31 --end-key 3130303030303030 --duration 10 -# backup rawkv -echo "backup start..." -run_br --pd $PD_ADDR backup raw -s "local://$BACKUP_DIR" --start 31 --end 3130303030303030 --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + # put some keys around 311122 to check the correctness of endKey of restoring + bin/rawkv --pd $PD_ADDR \ + --ca "$TEST_DIR/certs/ca.pem" \ + --cert "$TEST_DIR/certs/br.pem" \ + --key "$TEST_DIR/certs/br.key" \ + --mode put --put-data "311121:31, 31112100:32, 311122:33, 31112200:34, 3111220000:35, 311123:36" -# delete data in range[start-key, end-key) -clean 31 3130303030303030 -# Ensure the data is deleted -checksum_new=$(checksum 31 3130303030303030) + checksum_ori=$(checksum 31 3130303030303030) + checksum_partial=$(checksum 311111 311122) -if [ "$checksum_new" != "$checksum_empty" ];then - echo "failed to delete data in range" - fail_and_exit -fi + # backup rawkv + echo "backup start..." + run_br --pd $PD_ADDR backup raw -s "local://$BACKUP_DIR" --start 31 --end 3130303030303030 --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" -# restore rawkv -echo "restore start..." -run_br --pd $PD_ADDR restore raw -s "local://$BACKUP_DIR" --start 31 --end 3130303030303030 --format hex --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + # delete data in range[start-key, end-key) + clean 31 3130303030303030 + # Ensure the data is deleted + checksum_new=$(checksum 31 3130303030303030) -checksum_new=$(checksum 31 3130303030303030) + if [ "$checksum_new" != "$checksum_empty" ];then + echo "failed to delete data in range" + fail_and_exit + fi -if [ "$checksum_new" != "$checksum_ori" ];then - echo "checksum failed after restore" - fail_and_exit -fi + # restore rawkv + echo "restore start..." + run_br --pd $PD_ADDR restore raw -s "local://$BACKUP_DIR" --start 31 --end 3130303030303030 --format hex --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" -test_full_rawkv + checksum_new=$(checksum 31 3130303030303030) -# delete data in range[start-key, end-key) -clean 31 3130303030303030 -# Ensure the data is deleted -checksum_new=$(checksum 31 3130303030303030) + if [ "$checksum_new" != "$checksum_ori" ];then + echo "checksum failed after restore" + fail_and_exit + fi -if [ "$checksum_new" != "$checksum_empty" ];then - echo "failed to delete data in range" - fail_and_exit -fi + test_full_rawkv + + # delete data in range[start-key, end-key) + clean 31 3130303030303030 + # Ensure the data is deleted + checksum_new=$(checksum 31 3130303030303030) + + if [ "$checksum_new" != "$checksum_empty" ];then + echo "failed to delete data in range" + fail_and_exit + fi + + echo "partial restore start..." + run_br --pd $PD_ADDR restore raw -s "local://$BACKUP_DIR" --start 311111 --end 311122 --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + bin/rawkv --pd $PD_ADDR \ + --ca "$TEST_DIR/certs/ca.pem" \ + --cert "$TEST_DIR/certs/br.pem" \ + --key "$TEST_DIR/certs/br.key" \ + --mode scan --start-key 311121 --end-key 33 + + checksum_new=$(checksum 31 3130303030303030) + + if [ "$checksum_new" != "$checksum_partial" ];then + echo "checksum failed after restore" + fail_and_exit + fi + + export GO_FAILPOINTS="" +} -echo "partial restore start..." -run_br --pd $PD_ADDR restore raw -s "local://$BACKUP_DIR" --start 311111 --end 311122 --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" -bin/rawkv --pd $PD_ADDR \ - --ca "$TEST_DIR/certs/ca.pem" \ - --cert "$TEST_DIR/certs/br.pem" \ - --key "$TEST_DIR/certs/br.key" \ - --mode scan --start-key 311121 --end-key 33 -checksum_new=$(checksum 31 3130303030303030) +run_test "" -if [ "$checksum_new" != "$checksum_partial" ];then - echo "checksum failed after restore" - fail_and_exit -fi +# ingest "region error" to trigger fineGrainedBackup +run_test "github.com/pingcap/tidb/br/pkg/backup/tikv-region-error=return(\"region error\")" From ddd815cbc99114e6b39e77c340eeb50958da1d71 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 18:54:52 +0800 Subject: [PATCH 17/51] ddl: build sessPool even if RunWorker == false (#33677) (#33697) close pingcap/tidb#33676 --- ddl/ddl.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index b618e21ddc9c5..5e3066621bd34 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -350,6 +350,8 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { d.wg.Add(1) go d.limitDDLJobs() + d.sessPool = newSessionPool(ctxPool) + // If RunWorker is true, we need campaign owner and do DDL job. // Otherwise, we needn't do that. if RunWorker { @@ -359,7 +361,6 @@ func (d *ddl) Start(ctxPool *pools.ResourcePool) error { } d.workers = make(map[workerType]*worker, 2) - d.sessPool = newSessionPool(ctxPool) d.delRangeMgr = d.newDeleteRangeManager(ctxPool == nil) d.workers[generalWorker] = newWorker(d.ctx, generalWorker, d.sessPool, d.delRangeMgr) d.workers[addIdxWorker] = newWorker(d.ctx, addIdxWorker, d.sessPool, d.delRangeMgr) From bd9be7228bda474ea9ca7a6300c8229709381147 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 19:10:52 +0800 Subject: [PATCH 18/51] planner: let apply be paralleled though there's order property can be used (#34238) (#34268) close pingcap/tidb#34237 --- executor/parallel_apply_test.go | 6 ++++++ planner/core/exhaust_physical_plans.go | 4 ++++ 2 files changed, 10 insertions(+) diff --git a/executor/parallel_apply_test.go b/executor/parallel_apply_test.go index 809b4c5247e5a..32c08188f97d1 100644 --- a/executor/parallel_apply_test.go +++ b/executor/parallel_apply_test.go @@ -63,6 +63,12 @@ func (s *testSuite) TestParallelApply(c *C) { q2 := "select * from t t0 where t0.b <= (select max(t1.b) from t t1 where t1.b > (select max(b) from t t2 where t1.a > t2.a and t0.a > t2.a));" checkApplyPlan(c, tk, q2, 1) // only the outside apply can be parallel tk.MustQuery(q2).Sort().Check(testkit.Rows("1 1", "2 2", "3 3", "4 4", "5 5", "6 6", "7 7", "8 8", "9 9")) + q3 := "select t1.b from t t1 where t1.b > (select max(b) from t t2 where t1.a > t2.a) order by t1.a" + checkApplyPlan(c, tk, q3, 0) + tk.MustExec("alter table t add index idx(a)") + checkApplyPlan(c, tk, q3, 1) + tk.MustQuery(q3).Sort().Check(testkit.Rows("1", "2", "3", "4", "5", "6", "7", "8", "9")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Parallel Apply rejects the possible order properties of its outer child currently")) } func (s *testSuite) TestApplyColumnType(c *C) { diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 3e6e4e4b23aec..c3429e8969044 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2259,6 +2259,10 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ "MPP mode may be blocked because operator `Apply` is not supported now.") return nil, true, nil } + if !prop.IsEmpty() && la.SCtx().GetSessionVars().EnableParallelApply { + la.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Parallel Apply rejects the possible order properties of its outer child currently")) + return nil, true, nil + } disableAggPushDownToCop(la.children[0]) join := la.GetHashJoin(prop) var columns = make([]*expression.Column, 0, len(la.CorCols)) From 573e4e6635e8f968be3c65f47c96e8887f26f5ec Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 19:26:52 +0800 Subject: [PATCH 19/51] scatter: check pending peers before scattering (#31691) (#32129) close pingcap/tidb#31034 --- br/pkg/restore/range.go | 6 ++++-- br/pkg/restore/split.go | 21 ++++++++++++++++++--- br/pkg/restore/split_client.go | 6 ++++-- 3 files changed, 26 insertions(+), 7 deletions(-) diff --git a/br/pkg/restore/range.go b/br/pkg/restore/range.go index cc81ba8423d94..63d482cf19d8d 100644 --- a/br/pkg/restore/range.go +++ b/br/pkg/restore/range.go @@ -107,8 +107,10 @@ func SortRanges(ranges []rtree.Range, rewriteRules *RewriteRules) ([]rtree.Range // RegionInfo includes a region and the leader of the region. type RegionInfo struct { - Region *metapb.Region - Leader *metapb.Peer + Region *metapb.Region + Leader *metapb.Peer + PendingPeers []*metapb.Peer + DownPeers []*metapb.Peer } // ContainsInterior returns whether the region contains the given key, and also diff --git a/br/pkg/restore/split.go b/br/pkg/restore/split.go index cd9fa405f17fd..e590c7a98d17c 100644 --- a/br/pkg/restore/split.go +++ b/br/pkg/restore/split.go @@ -183,12 +183,27 @@ SplitRegions: return nil } -func (rs *RegionSplitter) hasRegion(ctx context.Context, regionID uint64) (bool, error) { +func (rs *RegionSplitter) hasHealthyRegion(ctx context.Context, regionID uint64) (bool, error) { regionInfo, err := rs.client.GetRegionByID(ctx, regionID) if err != nil { return false, errors.Trace(err) } - return regionInfo != nil, nil + // the region hasn't get ready. + if regionInfo == nil { + return false, nil + } + + // check whether the region is healthy and report. + // TODO: the log may be too verbose. we should use Prometheus metrics once it get ready for BR. + for _, peer := range regionInfo.PendingPeers { + log.Debug("unhealthy region detected", logutil.Peer(peer), zap.String("type", "pending")) + } + for _, peer := range regionInfo.DownPeers { + log.Debug("unhealthy region detected", logutil.Peer(peer), zap.String("type", "down")) + } + // we ignore down peers for they are (normally) hard to be fixed in reasonable time. + // (or once there is a peer down, we may get stuck at waiting region get ready.) + return len(regionInfo.PendingPeers) == 0, nil } func (rs *RegionSplitter) isScatterRegionFinished(ctx context.Context, regionID uint64) (bool, error) { @@ -216,7 +231,7 @@ func (rs *RegionSplitter) isScatterRegionFinished(ctx context.Context, regionID func (rs *RegionSplitter) waitForSplit(ctx context.Context, regionID uint64) { interval := SplitCheckInterval for i := 0; i < SplitCheckMaxRetryTimes; i++ { - ok, err := rs.hasRegion(ctx, regionID) + ok, err := rs.hasHealthyRegion(ctx, regionID) if err != nil { log.Warn("wait for split failed", zap.Error(err)) return diff --git a/br/pkg/restore/split_client.go b/br/pkg/restore/split_client.go index a00d5dde22848..0b3037e3c9749 100755 --- a/br/pkg/restore/split_client.go +++ b/br/pkg/restore/split_client.go @@ -175,8 +175,10 @@ func (c *pdClient) GetRegionByID(ctx context.Context, regionID uint64) (*RegionI return nil, nil } return &RegionInfo{ - Region: region.Meta, - Leader: region.Leader, + Region: region.Meta, + Leader: region.Leader, + PendingPeers: region.PendingPeers, + DownPeers: region.DownPeers, }, nil } From da5f3728b381c16a6ef8e8faf0083e643e45311e Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 19:54:55 +0800 Subject: [PATCH 20/51] expression: fix append to result twice when vecEval compress('') (#33523) (#33557) close pingcap/tidb#33397 --- expression/builtin_encryption_vec.go | 1 + expression/integration_test.go | 13 +++++++++++++ 2 files changed, 14 insertions(+) diff --git a/expression/builtin_encryption_vec.go b/expression/builtin_encryption_vec.go index 4433463429127..1c2124b8c3001 100644 --- a/expression/builtin_encryption_vec.go +++ b/expression/builtin_encryption_vec.go @@ -567,6 +567,7 @@ func (b *builtinCompressSig) vecEvalString(input *chunk.Chunk, result *chunk.Col // According to doc: Empty strings are stored as empty strings. if len(strBytes) == 0 { result.AppendString("") + continue } compressed, err := deflate(strBytes) diff --git a/expression/integration_test.go b/expression/integration_test.go index fbf80802f5f84..99e72ff71e83f 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -6995,3 +6995,16 @@ func TestIssue30264(t *testing.T) { // compare JSON/JSON, return JSON type tk.MustQuery("select greatest(cast(a as JSON), cast('3' as JSON)) from t1;").Check(testkit.Rows("3")) } + +func TestIssue33397(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a varchar(32)) DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci;") + tk.MustExec("insert into t values(''), ('');") + tk.MustExec("set @@tidb_enable_vectorized_expression = true;") + result := tk.MustQuery("select compress(a) from t").Rows() + require.Equal(t, [][]interface{}{{""}, {""}}, result) +} From c53fe3dd54dd2a874c45a815f0d373795832656f Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 20:36:53 +0800 Subject: [PATCH 21/51] br: fix the missing retry for pd batch scan error (#33420) (#33470) close pingcap/tidb#33419 --- br/pkg/restore/split.go | 13 ++++++++++--- br/pkg/restore/split_test.go | 5 +++++ br/pkg/restore/util_test.go | 15 ++++++++++++--- 3 files changed, 27 insertions(+), 6 deletions(-) diff --git a/br/pkg/restore/split.go b/br/pkg/restore/split.go index e590c7a98d17c..4a38c5348d087 100644 --- a/br/pkg/restore/split.go +++ b/br/pkg/restore/split.go @@ -435,11 +435,18 @@ func PaginateScanRegion( } var regions []*RegionInfo - err := utils.WithRetry(ctx, func() error { + var err error + // we don't need to return multierr. since there only 3 times retry. + // in most case 3 times retry have the same error. so we just return the last error. + // actually we'd better remove all multierr in br/lightning. + // because it's not easy to check multierr equals normal error. + // see https://github.com/pingcap/tidb/issues/33419. + _ = utils.WithRetry(ctx, func() error { regions = []*RegionInfo{} scanStartKey := startKey for { - batch, err := client.ScanRegions(ctx, scanStartKey, endKey, limit) + var batch []*RegionInfo + batch, err = client.ScanRegions(ctx, scanStartKey, endKey, limit) if err != nil { return errors.Trace(err) } @@ -455,7 +462,7 @@ func PaginateScanRegion( break } } - if err := CheckRegionConsistency(startKey, endKey, regions); err != nil { + if err = CheckRegionConsistency(startKey, endKey, regions); err != nil { log.Warn("failed to scan region, retrying", logutil.ShortError(err)) return err } diff --git a/br/pkg/restore/split_test.go b/br/pkg/restore/split_test.go index 09288c459d601..17edf0b7df9dc 100644 --- a/br/pkg/restore/split_test.go +++ b/br/pkg/restore/split_test.go @@ -36,6 +36,7 @@ type TestClient struct { supportBatchScatter bool scattered map[uint64]bool + InjectErr bool } func NewTestClient( @@ -212,6 +213,10 @@ func (c *TestClient) GetOperator(ctx context.Context, regionID uint64) (*pdpb.Ge } func (c *TestClient) ScanRegions(ctx context.Context, key, endKey []byte, limit int) ([]*restore.RegionInfo, error) { + if c.InjectErr { + return nil, errors.New("mock scan error") + } + infos := c.regionsInfo.ScanRange(key, endKey, limit) regions := make([]*restore.RegionInfo, 0, len(infos)) for _, info := range infos { diff --git a/br/pkg/restore/util_test.go b/br/pkg/restore/util_test.go index e8263c6b8462c..3e584b2669ac4 100644 --- a/br/pkg/restore/util_test.go +++ b/br/pkg/restore/util_test.go @@ -10,6 +10,7 @@ import ( backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/kvproto/pkg/import_sstpb" "github.com/pingcap/kvproto/pkg/metapb" + berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/restore" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" @@ -223,9 +224,11 @@ func (s *testRestoreUtilSuite) TestPaginateScanRegion(c *C) { ctx := context.Background() regionMap := make(map[uint64]*restore.RegionInfo) - regions := []*restore.RegionInfo{} - batch, err := restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{}, []byte{}, 3) + var regions []*restore.RegionInfo + var batch []*restore.RegionInfo + _, err := restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{}, []byte{}, 3) c.Assert(err, ErrorMatches, ".*scan region return empty result.*") + c.Assert(berrors.ErrPDBatchScanRegion.Equal(err), IsTrue) regionMap, regions = makeRegions(1) batch, err = restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{}, []byte{}, 3) @@ -265,11 +268,17 @@ func (s *testRestoreUtilSuite) TestPaginateScanRegion(c *C) { _, err = restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{2}, []byte{1}, 3) c.Assert(err, ErrorMatches, ".*startKey >= endKey.*") + c.Assert(berrors.ErrRestoreInvalidRange.Equal(err), IsTrue) + + tc := NewTestClient(stores, regionMap, 0) + tc.InjectErr = true + _, err = restore.PaginateScanRegion(ctx, tc, regions[1].Region.EndKey, regions[5].Region.EndKey, 3) + c.Assert(err, ErrorMatches, ".*mock scan error.*") // make the regionMap losing some region, this will cause scan region check fails delete(regionMap, uint64(3)) _, err = restore.PaginateScanRegion( ctx, NewTestClient(stores, regionMap, 0), regions[1].Region.EndKey, regions[5].Region.EndKey, 3) c.Assert(err, ErrorMatches, ".*region endKey not equal to next region startKey.*") - + c.Assert(berrors.ErrPDBatchScanRegion.Equal(err), IsTrue) } From a6848c4d61f878e4f97d936da9488e82849b6a37 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 21:24:52 +0800 Subject: [PATCH 22/51] br: use the correct unit for pausing schedulers (#33545) (#33816) close pingcap/tidb#33546 --- br/pkg/pdutil/pd.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/br/pkg/pdutil/pd.go b/br/pkg/pdutil/pd.go index 3f4c45d1deefa..3ddd3155cc7b9 100644 --- a/br/pkg/pdutil/pd.go +++ b/br/pkg/pdutil/pd.go @@ -378,7 +378,7 @@ func (p *PdController) getStoreInfoWith( func (p *PdController) doPauseSchedulers(ctx context.Context, schedulers []string, post pdHTTPRequest) ([]string, error) { // pause this scheduler with 300 seconds - body, err := json.Marshal(pauseSchedulerBody{Delay: int64(pauseTimeout)}) + body, err := json.Marshal(pauseSchedulerBody{Delay: int64(pauseTimeout.Seconds())}) if err != nil { return nil, errors.Trace(err) } From 0282edfa16694b105e325ad538f56543dc82a4b2 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 21:42:52 +0800 Subject: [PATCH 23/51] br: download sst again with no-crypt parameters when met decryption error for downloadSST (#32847) (#33197) close pingcap/tidb#32423 --- br/pkg/restore/import.go | 15 ++++ br/pkg/utils/backoff.go | 5 +- br/tests/br_crypter2/run.sh | 79 +++++++++++++++++++ br/tests/br_crypter2/workload | 12 +++ .../data/nc.t-schema.sql | 1 + .../lightning_new_collation/data/nc.t.0.sql | 6 ++ .../lightning_new_collation/data/nc.t.1.sql | 1 + br/tests/lightning_new_collation/run.sh | 18 +---- 8 files changed, 118 insertions(+), 19 deletions(-) create mode 100755 br/tests/br_crypter2/run.sh create mode 100644 br/tests/br_crypter2/workload create mode 100644 br/tests/lightning_new_collation/data/nc.t-schema.sql create mode 100644 br/tests/lightning_new_collation/data/nc.t.0.sql create mode 100644 br/tests/lightning_new_collation/data/nc.t.1.sql diff --git a/br/pkg/restore/import.go b/br/pkg/restore/import.go index aafb144959202..556596bd4fdd0 100644 --- a/br/pkg/restore/import.go +++ b/br/pkg/restore/import.go @@ -6,6 +6,7 @@ import ( "bytes" "context" "crypto/tls" + "strings" "sync" "sync/atomic" "time" @@ -330,6 +331,14 @@ func (importer *FileImporter) Import( log.Warn("the connection to TiKV has been cut by a neko, meow :3") e = status.Error(codes.Unavailable, "the connection to TiKV has been cut by a neko, meow :3") }) + if isDecryptSstErr(e) { + log.Info("fail to decrypt when download sst, try again with no-crypt", logutil.File(f)) + if importer.isRawKvMode { + downloadMeta, e = importer.downloadRawKVSST(ctx, info, f, nil) + } else { + downloadMeta, e = importer.downloadSST(ctx, info, f, rewriteRules, nil) + } + } if e != nil { remainFiles = remainFiles[i:] return errors.Trace(e) @@ -626,3 +635,9 @@ func (importer *FileImporter) ingestSSTs( resp, err := importer.importClient.MultiIngest(ctx, leader.GetStoreId(), req) return resp, errors.Trace(err) } + +func isDecryptSstErr(err error) bool { + return err != nil && + strings.Contains(err.Error(), "Engine Engine") && + strings.Contains(err.Error(), "Corruption: Bad table magic number") +} diff --git a/br/pkg/utils/backoff.go b/br/pkg/utils/backoff.go index 5a21ad8f26a98..c69f32dc4827c 100644 --- a/br/pkg/utils/backoff.go +++ b/br/pkg/utils/backoff.go @@ -17,9 +17,10 @@ import ( ) const ( + // importSSTRetryTimes specifies the retry time. Its longest time is about 90s-100s. importSSTRetryTimes = 16 - importSSTWaitInterval = 10 * time.Millisecond - importSSTMaxWaitInterval = 1 * time.Second + importSSTWaitInterval = 40 * time.Millisecond + importSSTMaxWaitInterval = 10 * time.Second downloadSSTRetryTimes = 8 downloadSSTWaitInterval = 1 * time.Second diff --git a/br/tests/br_crypter2/run.sh b/br/tests/br_crypter2/run.sh new file mode 100755 index 0000000000000..91de74376ebce --- /dev/null +++ b/br/tests/br_crypter2/run.sh @@ -0,0 +1,79 @@ +#!/bin/sh +# +# Copyright 2022 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, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +set -eu +DB="$TEST_NAME" +TABLE="usertable" +DB_COUNT=3 + +function create_db_with_table(){ + for i in $(seq $DB_COUNT); do + run_sql "CREATE DATABASE $DB${i};" + go-ycsb load mysql -P tests/$TEST_NAME/workload -p mysql.host=$TIDB_IP -p mysql.port=$TIDB_PORT -p mysql.user=root -p mysql.db=$DB${i} + done +} + +function drop_db(){ + for i in $(seq $DB_COUNT); do + run_sql "DROP DATABASE $DB${i};" + done +} + +function check_db_row(){ + for i in $(seq $DB_COUNT); do + row_count_new[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}') + done + + fail=false + for i in $(seq $DB_COUNT); do + if [ "${row_count_ori[i]}" != "${row_count_new[i]}" ];then + fail=true + echo "TEST: [$TEST_NAME] fail on database $DB${i}" + fi + echo "database $DB${i} [original] row count: ${row_count_ori[i]}, [after br] row count: ${row_count_new[i]}" + done + + if $fail; then + echo "TEST: [$TEST_NAME] failed!" + exit 1 + fi +} + +# Create dbs with table +create_db_with_table + +# Get the original row count from dbs +for i in $(seq $DB_COUNT); do + row_count_ori[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}') +done + +# Test backup/restore with crypt for br +CRYPTER_METHOD=aes128-ctr +CRYPTER_KEY="0123456789abcdef0123456789abcdef" + +export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/backup/noop-backup=100*return(1)" +run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \ + --use-backupmeta-v2=true --check-requirements=false --crypter.method $CRYPTER_METHOD --crypter.key $CRYPTER_KEY + +drop_db + +run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \ + --check-requirements=false --crypter.method $CRYPTER_METHOD --crypter.key $CRYPTER_KEY + +check_db_row + +# Drop dbs finally +drop_db diff --git a/br/tests/br_crypter2/workload b/br/tests/br_crypter2/workload new file mode 100644 index 0000000000000..448ca3c1a477f --- /dev/null +++ b/br/tests/br_crypter2/workload @@ -0,0 +1,12 @@ +recordcount=1000 +operationcount=0 +workload=core + +readallfields=true + +readproportion=0 +updateproportion=0 +scanproportion=0 +insertproportion=0 + +requestdistribution=uniform \ No newline at end of file diff --git a/br/tests/lightning_new_collation/data/nc.t-schema.sql b/br/tests/lightning_new_collation/data/nc.t-schema.sql new file mode 100644 index 0000000000000..455c0567d1d07 --- /dev/null +++ b/br/tests/lightning_new_collation/data/nc.t-schema.sql @@ -0,0 +1 @@ +CREATE TABLE t(i INT PRIMARY KEY, s varchar(32), j TINYINT, KEY s_j (s, i)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci; diff --git a/br/tests/lightning_new_collation/data/nc.t.0.sql b/br/tests/lightning_new_collation/data/nc.t.0.sql new file mode 100644 index 0000000000000..97b01b4ab8e6d --- /dev/null +++ b/br/tests/lightning_new_collation/data/nc.t.0.sql @@ -0,0 +1,6 @@ +INSERT INTO t (s, i, j) VALUES + ("this_is_test1", 1, 1), + ("this_is_test2", 2, 2), + ("this_is_test3", 3, 3), + ("this_is_test4", 4, 4), + ("this_is_test5", 5, 5); diff --git a/br/tests/lightning_new_collation/data/nc.t.1.sql b/br/tests/lightning_new_collation/data/nc.t.1.sql new file mode 100644 index 0000000000000..734009f6bd4b6 --- /dev/null +++ b/br/tests/lightning_new_collation/data/nc.t.1.sql @@ -0,0 +1 @@ +INSERT INTO t(s, i, j) VALUES ("another test case", 6, 6); diff --git a/br/tests/lightning_new_collation/run.sh b/br/tests/lightning_new_collation/run.sh index f360ed3a94fc4..fe2c8ddda4cce 100644 --- a/br/tests/lightning_new_collation/run.sh +++ b/br/tests/lightning_new_collation/run.sh @@ -24,27 +24,11 @@ cur=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd ) # restart cluster with new collation enabled start_services --tidb-cfg $cur/tidb-new-collation.toml -# Populate the mydumper source -DBPATH="$TEST_DIR/nc.mydump" -mkdir -p $DBPATH -echo 'CREATE DATABASE nc;' > "$DBPATH/nc-schema-create.sql" -# create table with collate `utf8_general_ci`, the index key will be different between old/new collation -echo "CREATE TABLE t(i INT PRIMARY KEY, s varchar(32), j TINYINT, KEY s_j (s, i)) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_general_ci;" > "$DBPATH/nc.t-schema.sql" -cat > "$DBPATH/nc.t.0.sql" << _EOF_ -INSERT INTO t (s, i, j) VALUES - ("this_is_test1", 1, 1), - ("this_is_test2", 2, 2), - ("this_is_test3", 3, 3), - ("this_is_test4", 4, 4), - ("this_is_test5", 5, 5); -_EOF_ -echo 'INSERT INTO t(s, i, j) VALUES ("another test case", 6, 6);' > "$DBPATH/nc.t.1.sql" - for BACKEND in local importer tidb; do # Start importing the tables. run_sql 'DROP DATABASE IF EXISTS nc' - run_lightning -d "$DBPATH" --backend $BACKEND 2> /dev/null + run_lightning --backend $BACKEND 2> /dev/null run_sql 'SELECT count(*), sum(i) FROM `nc`.t' check_contains "count(*): 6" From 92be685b4ef975b92e888385c83782db1215f0e5 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 22:24:52 +0800 Subject: [PATCH 24/51] statistics: fix init topn not sorted (#34234) (#34317) ref pingcap/tidb#32758, close pingcap/tidb#34216 --- statistics/handle/bootstrap.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index bc5cff9a9084f..6a1930b44aa87 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -179,7 +179,7 @@ func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, cache *statsCache } func (h *Handle) initStatsTopN4Chunk(cache *statsCache, iter *chunk.Iterator4Chunk) { - affectedIndexes := make(map[int64]*statistics.Index) + affectedIndexes := make(map[*statistics.Index]struct{}) for row := iter.Begin(); row != iter.End(); row = iter.Next() { table, ok := cache.tables[row.GetInt64(0)] if !ok { @@ -192,12 +192,12 @@ func (h *Handle) initStatsTopN4Chunk(cache *statsCache, iter *chunk.Iterator4Chu if idx.TopN == nil { idx.TopN = statistics.NewTopN(32) } - affectedIndexes[row.GetInt64(1)] = idx + affectedIndexes[idx] = struct{}{} data := make([]byte, len(row.GetBytes(2))) copy(data, row.GetBytes(2)) idx.TopN.AppendTopN(data, row.GetUint64(3)) } - for _, idx := range affectedIndexes { + for idx := range affectedIndexes { idx.TopN.Sort() } } From 0ce290085080be58abbca465ca60532adc408ee1 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 28 Apr 2022 23:22:52 +0800 Subject: [PATCH 25/51] ddl: support index regions and updating the existed table rule when changing partition (#33925) (#34025) close pingcap/tidb#33929 --- ddl/attributes_sql_test.go | 38 ++++++++++++++++++++++++++++++++- ddl/label/rule.go | 4 ++-- ddl/label/rule_test.go | 16 +++++++------- ddl/partition.go | 42 +++++++++++++++++++++++++++++++++++++ store/gcworker/gc_worker.go | 4 ++-- 5 files changed, 91 insertions(+), 13 deletions(-) diff --git a/ddl/attributes_sql_test.go b/ddl/attributes_sql_test.go index c3295b1d518d0..b515f878cb724 100644 --- a/ddl/attributes_sql_test.go +++ b/ddl/attributes_sql_test.go @@ -107,6 +107,42 @@ PARTITION BY RANGE (c) ( c.Assert(err, IsNil) _, err = tk.Exec(`alter table alter_p partition p1 attributes " merge_option=allow , key=value ";`) c.Assert(err, IsNil) + + // reset all + tk.MustExec(`alter table alter_p partition p0 attributes default;`) + tk.MustExec(`alter table alter_p partition p1 attributes default;`) + tk.MustExec(`alter table alter_p partition p2 attributes default;`) + tk.MustExec(`alter table alter_p partition p3 attributes default;`) + + // add table level attribute + tk.MustExec(`alter table alter_p attributes="merge_option=deny";`) + rows := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows), Equals, 1) + + // add a new partition p4 + tk.MustExec(`alter table alter_p add partition (PARTITION p4 VALUES LESS THAN (60));`) + rows1 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows1), Equals, 1) + c.Assert(rows[0][3], Not(Equals), rows1[0][3]) + + // drop the new partition p4 + tk.MustExec(`alter table alter_p drop partition p4;`) + rows2 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows2), Equals, 1) + c.Assert(rows[0][3], Equals, rows2[0][3]) + + // add a new partition p5 + tk.MustExec(`alter table alter_p add partition (PARTITION p5 VALUES LESS THAN (80));`) + rows3 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows3), Equals, 1) + c.Assert(rows[0][3], Not(Equals), rows3[0][3]) + + // truncate the new partition p5 + tk.MustExec(`alter table alter_p truncate partition p5;`) + rows4 := tk.MustQuery(`select * from information_schema.attributes;`).Sort().Rows() + c.Assert(len(rows4), Equals, 1) + c.Assert(rows3[0][3], Not(Equals), rows4[0][3]) + c.Assert(rows[0][3], Not(Equals), rows4[0][3]) } func (s *testAttributesDDLSerialSuite) TestTruncateTable(c *C) { @@ -523,7 +559,7 @@ PARTITION BY RANGE (c) ( c.Assert(len(rows1), Equals, 2) c.Assert(rows1[0][0], Equals, "schema/test/part") c.Assert(rows1[0][2], Equals, `"key=value"`) - c.Assert(rows1[0][3], Equals, rows[0][3]) + c.Assert(rows1[0][3], Not(Equals), rows[0][3]) c.Assert(rows1[1][0], Equals, "schema/test/part/p1") c.Assert(rows1[1][2], Equals, `"key2=value2"`) c.Assert(rows1[1][3], Equals, rows[2][3]) diff --git a/ddl/label/rule.go b/ddl/label/rule.go index 433e4055385d9..981d9117c768b 100644 --- a/ddl/label/rule.go +++ b/ddl/label/rule.go @@ -144,8 +144,8 @@ func (r *Rule) Reset(dbName, tableName, partName string, ids ...int64) *Rule { sort.Slice(ids, func(i, j int) bool { return ids[i] < ids[j] }) for i := 0; i < len(ids); i++ { data := map[string]string{ - "start_key": hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(ids[i]))), - "end_key": hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(ids[i]+1))), + "start_key": hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(ids[i]))), + "end_key": hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(ids[i]+1))), } r.Data = append(r.Data, data) } diff --git a/ddl/label/rule_test.go b/ddl/label/rule_test.go index a0d3b02555e13..1fdc197471fe0 100644 --- a/ddl/label/rule_test.go +++ b/ddl/label/rule_test.go @@ -75,14 +75,14 @@ func TestReset(t *testing.T) { require.Equal(t, rule.Index, 2) r := rule.Data[0].(map[string]string) - require.Equal(t, "7480000000000000ff015f720000000000fa", r["start_key"]) - require.Equal(t, "7480000000000000ff025f720000000000fa", r["end_key"]) + require.Equal(t, "7480000000000000ff0100000000000000f8", r["start_key"]) + require.Equal(t, "7480000000000000ff0200000000000000f8", r["end_key"]) r = rule.Data[1].(map[string]string) - require.Equal(t, "7480000000000000ff025f720000000000fa", r["start_key"]) - require.Equal(t, "7480000000000000ff035f720000000000fa", r["end_key"]) + require.Equal(t, "7480000000000000ff0200000000000000f8", r["start_key"]) + require.Equal(t, "7480000000000000ff0300000000000000f8", r["end_key"]) r = rule.Data[2].(map[string]string) - require.Equal(t, "7480000000000000ff035f720000000000fa", r["start_key"]) - require.Equal(t, "7480000000000000ff045f720000000000fa", r["end_key"]) + require.Equal(t, "7480000000000000ff0300000000000000f8", r["start_key"]) + require.Equal(t, "7480000000000000ff0400000000000000f8", r["end_key"]) r1 := rule.Clone() require.Equal(t, r1, rule) @@ -97,8 +97,8 @@ func TestReset(t *testing.T) { require.Equal(t, rule.Index, 3) r = r2.Data[0].(map[string]string) - require.Equal(t, "7480000000000000ff025f720000000000fa", r["start_key"]) - require.Equal(t, "7480000000000000ff035f720000000000fa", r["end_key"]) + require.Equal(t, "7480000000000000ff0200000000000000f8", r["start_key"]) + require.Equal(t, "7480000000000000ff0300000000000000f8", r["end_key"]) // default case spec = &ast.AttributesSpec{Default: true} diff --git a/ddl/partition.go b/ddl/partition.go index b01f3caac07ee..efa3e75e1f2fc 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -143,6 +143,15 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } + ids := getIDs([]*model.TableInfo{tblInfo}) + for _, p := range tblInfo.Partition.AddingDefinitions { + ids = append(ids, p.ID) + } + if err := alterTableLabelRule(job.SchemaName, tblInfo, ids); err != nil { + job.State = model.JobStateCancelled + return ver, err + } + // none -> replica only job.SchemaState = model.StateReplicaOnly case model.StateReplicaOnly: @@ -184,6 +193,27 @@ func (w *worker) onAddTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (v return ver, errors.Trace(err) } +func alterTableLabelRule(schemaName string, meta *model.TableInfo, ids []int64) error { + tableRuleID := fmt.Sprintf(label.TableIDFormat, label.IDPrefix, schemaName, meta.Name.L) + oldRule, err := infosync.GetLabelRules(context.TODO(), []string{tableRuleID}) + if err != nil { + return errors.Trace(err) + } + if len(oldRule) == 0 { + return nil + } + + r, ok := oldRule[tableRuleID] + if ok { + rule := r.Reset(schemaName, meta.Name.L, "", ids...) + err = infosync.PutLabelRule(context.TODO(), rule) + if err != nil { + return errors.Wrapf(err, "failed to notify PD label rule") + } + } + return nil +} + func alterTablePartitionBundles(t *meta.Meta, tblInfo *model.TableInfo, addingDefinitions []model.PartitionDefinition) ([]*placement.Bundle, error) { var bundles []*placement.Bundle @@ -1059,6 +1089,12 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the label rules") } + + if err := alterTableLabelRule(job.SchemaName, tblInfo, getIDs([]*model.TableInfo{tblInfo})); err != nil { + job.State = model.JobStateCancelled + return ver, err + } + ver, err = updateVersionAndTableInfo(t, job, tblInfo, true) if err != nil { return ver, errors.Trace(err) @@ -1090,6 +1126,12 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( job.State = model.JobStateCancelled return ver, errors.Wrapf(err, "failed to notify PD the label rules") } + + if err := alterTableLabelRule(job.SchemaName, tblInfo, getIDs([]*model.TableInfo{tblInfo})); err != nil { + job.State = model.JobStateCancelled + return ver, err + } + job.SchemaState = model.StateDeleteOnly ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != job.SchemaState) case model.StateDeleteOnly: diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index 854408f082528..8957b9b96346a 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -1973,8 +1973,8 @@ func (w *GCWorker) doGCLabelRules(dr util.DelRangeTask) (err error) { func getGCRules(ids []int64, rules map[string]*label.Rule) []string { oldRange := make(map[string]struct{}) for _, id := range ids { - startKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(id))) - endKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTableRecordPrefix(id+1))) + startKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(id))) + endKey := hex.EncodeToString(codec.EncodeBytes(nil, tablecodec.GenTablePrefix(id+1))) oldRange[startKey+endKey] = struct{}{} } From cf3591f9269abc35a5f814515fbdc4caacfaa421 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 29 Apr 2022 11:04:52 +0800 Subject: [PATCH 26/51] codec: Don't convert set or enum datum to float64 when encoding them (#32308) (#32355) close pingcap/tidb#32302 --- executor/executor_test.go | 14 ++++++++++++++ util/codec/codec.go | 20 ++++++++++---------- 2 files changed, 24 insertions(+), 10 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 0ebc6d8c9a925..88dc938736ad1 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -9823,6 +9823,20 @@ func (s *testSerialSuite) TestIssue30971(c *C) { } } +func (s *testSerialSuite) TestEncodingSet(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("CREATE TABLE `enum-set` (`set` SET(" + + "'x00','x01','x02','x03','x04','x05','x06','x07','x08','x09','x10','x11','x12','x13','x14','x15'," + + "'x16','x17','x18','x19','x20','x21','x22','x23','x24','x25','x26','x27','x28','x29','x30','x31'," + + "'x32','x33','x34','x35','x36','x37','x38','x39','x40','x41','x42','x43','x44','x45','x46','x47'," + + "'x48','x49','x50','x51','x52','x53','x54','x55','x56','x57','x58','x59','x60','x61','x62','x63'" + + ")NOT NULL PRIMARY KEY)") + tk.MustExec("INSERT INTO `enum-set` VALUES\n(\"x00,x59\");") + tk.MustQuery("select `set` from `enum-set` use index(PRIMARY)").Check(testkit.Rows("x00,x59")) + tk.MustExec("admin check table `enum-set`") +} + func (s *testSuite) TestDeleteWithMulTbl(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/util/codec/codec.go b/util/codec/codec.go index 454e9dd28a9c9..11d0b0b9283ba 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -116,9 +116,9 @@ func encode(sc *stmtctx.StatementContext, b []byte, vals []types.Datum, comparab err = sc.HandleOverflow(err, err) } case types.KindMysqlEnum: - b = encodeUnsignedInt(b, uint64(vals[i].GetMysqlEnum().ToNumber()), comparable) + b = encodeUnsignedInt(b, vals[i].GetMysqlEnum().Value, comparable) case types.KindMysqlSet: - b = encodeUnsignedInt(b, uint64(vals[i].GetMysqlSet().ToNumber()), comparable) + b = encodeUnsignedInt(b, vals[i].GetMysqlSet().Value, comparable) case types.KindMysqlBit, types.KindBinaryLiteral: // We don't need to handle errors here since the literal is ensured to be able to store in uint64 in convertToMysqlBit. var val uint64 @@ -159,9 +159,9 @@ func EstimateValueSize(sc *stmtctx.StatementContext, val types.Datum) (int, erro case types.KindMysqlDecimal: l = valueSizeOfDecimal(val.GetMysqlDecimal(), val.Length(), val.Frac()) + 1 case types.KindMysqlEnum: - l = valueSizeOfUnsignedInt(uint64(val.GetMysqlEnum().ToNumber())) + l = valueSizeOfUnsignedInt(val.GetMysqlEnum().Value) case types.KindMysqlSet: - l = valueSizeOfUnsignedInt(uint64(val.GetMysqlSet().ToNumber())) + l = valueSizeOfUnsignedInt(val.GetMysqlSet().Value) case types.KindMysqlBit, types.KindBinaryLiteral: val, err := val.GetBinaryLiteral().ToInt(sc) terror.Log(errors.Trace(err)) @@ -355,11 +355,11 @@ func encodeHashChunkRowIdx(sc *stmtctx.StatementContext, row chunk.Row, tp *type case mysql.TypeEnum: if mysql.HasEnumSetAsIntFlag(tp.Flag) { flag = uvarintFlag - v := uint64(row.GetEnum(idx).ToNumber()) + v := row.GetEnum(idx).Value b = (*[sizeUint64]byte)(unsafe.Pointer(&v))[:] } else { flag = compactBytesFlag - v := uint64(row.GetEnum(idx).ToNumber()) + v := row.GetEnum(idx).Value str := "" if enum, err := types.ParseEnumValue(tp.Elems, v); err == nil { // str will be empty string if v out of definition of enum. @@ -570,11 +570,11 @@ func HashChunkSelected(sc *stmtctx.StatementContext, h []hash.Hash64, chk *chunk isNull[i] = !ignoreNull } else if mysql.HasEnumSetAsIntFlag(tp.Flag) { buf[0] = uvarintFlag - v := uint64(column.GetEnum(i).ToNumber()) + v := column.GetEnum(i).Value b = (*[sizeUint64]byte)(unsafe.Pointer(&v))[:] } else { buf[0] = compactBytesFlag - v := uint64(column.GetEnum(i).ToNumber()) + v := column.GetEnum(i).Value str := "" if enum, err := types.ParseEnumValue(tp.Elems, v); err == nil { // str will be empty string if v out of definition of enum. @@ -1326,9 +1326,9 @@ func HashCode(b []byte, d types.Datum) []byte { decStr := d.GetMysqlDecimal().ToString() b = encodeBytes(b, decStr, false) case types.KindMysqlEnum: - b = encodeUnsignedInt(b, uint64(d.GetMysqlEnum().ToNumber()), false) + b = encodeUnsignedInt(b, d.GetMysqlEnum().Value, false) case types.KindMysqlSet: - b = encodeUnsignedInt(b, uint64(d.GetMysqlSet().ToNumber()), false) + b = encodeUnsignedInt(b, d.GetMysqlSet().Value, false) case types.KindMysqlBit, types.KindBinaryLiteral: val := d.GetBinaryLiteral() b = encodeBytes(b, val, false) From 382775c6fa3f190e75cbcd9d361aa0af829552f0 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 29 Apr 2022 11:22:53 +0800 Subject: [PATCH 27/51] executor: handle collate for min/max(enum/set column) (#31819) (#31860) close pingcap/tidb#31638 --- executor/aggfuncs/builder.go | 4 +++- executor/aggfuncs/func_max_min.go | 12 +++++++----- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/executor/aggfuncs/builder.go b/executor/aggfuncs/builder.go index e5d9eafbd2a0c..4da578936d96e 100644 --- a/executor/aggfuncs/builder.go +++ b/executor/aggfuncs/builder.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) @@ -377,7 +378,8 @@ func buildMaxMin(aggFuncDesc *aggregation.AggFuncDesc, ordinal int, isMax bool) ordinal: ordinal, retTp: aggFuncDesc.RetTp, }, - isMax: isMax, + isMax: isMax, + collator: collate.GetCollator(aggFuncDesc.RetTp.Collate), } evalType, fieldType := aggFuncDesc.RetTp.EvalType(), aggFuncDesc.RetTp if fieldType.Tp == mysql.TypeBit { diff --git a/executor/aggfuncs/func_max_min.go b/executor/aggfuncs/func_max_min.go index 504c179516b91..956e32649eb50 100644 --- a/executor/aggfuncs/func_max_min.go +++ b/executor/aggfuncs/func_max_min.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/stringutil" ) @@ -234,7 +235,8 @@ type partialResult4MaxMinSet struct { type baseMaxMinAggFunc struct { baseAggFunc - isMax bool + isMax bool + collator collate.Collator } type maxMin4Int struct { @@ -1494,7 +1496,7 @@ func (e *maxMin4Enum) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ continue } en := d.GetMysqlEnum() - if e.isMax && en.Name > p.val.Name || !e.isMax && en.Name < p.val.Name { + if e.isMax && e.collator.Compare(en.Name, p.val.Name) > 0 || !e.isMax && e.collator.Compare(en.Name, p.val.Name) < 0 { oldMem := len(p.val.Name) newMem := len(en.Name) memDelta += int64(newMem - oldMem) @@ -1513,7 +1515,7 @@ func (e *maxMin4Enum) MergePartialResult(sctx sessionctx.Context, src, dst Parti *p2 = *p1 return 0, nil } - if e.isMax && p1.val.Name > p2.val.Name || !e.isMax && p1.val.Name < p2.val.Name { + if e.isMax && e.collator.Compare(p1.val.Name, p2.val.Name) > 0 || !e.isMax && e.collator.Compare(p1.val.Name, p2.val.Name) < 0 { p2.val, p2.isNull = p1.val, false } return 0, nil @@ -1561,7 +1563,7 @@ func (e *maxMin4Set) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [] continue } s := d.GetMysqlSet() - if e.isMax && s.Name > p.val.Name || !e.isMax && s.Name < p.val.Name { + if e.isMax && e.collator.Compare(s.Name, p.val.Name) > 0 || !e.isMax && e.collator.Compare(s.Name, p.val.Name) < 0 { oldMem := len(p.val.Name) newMem := len(s.Name) memDelta += int64(newMem - oldMem) @@ -1580,7 +1582,7 @@ func (e *maxMin4Set) MergePartialResult(sctx sessionctx.Context, src, dst Partia *p2 = *p1 return 0, nil } - if e.isMax && p1.val.Name > p2.val.Name || !e.isMax && p1.val.Name < p2.val.Name { + if e.isMax && e.collator.Compare(p1.val.Name, p2.val.Name) > 0 || !e.isMax && e.collator.Compare(p1.val.Name, p2.val.Name) < 0 { p2.val, p2.isNull = p1.val, false } return 0, nil From c679590945be5a232afc72a276d6c192ff95b861 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 29 Apr 2022 11:38:52 +0800 Subject: [PATCH 28/51] executor: Fix data race when getting snapshot ts in IndexLookupJoin and UnionExec (#30487) (#31351) close pingcap/tidb#30468 --- executor/builder.go | 11 +++++++++++ executor/index_lookup_join.go | 24 +++++++++++++++++++++++- session/txn.go | 1 + 3 files changed, 35 insertions(+), 1 deletion(-) diff --git a/executor/builder.go b/executor/builder.go index 52c8942b52932..9b10061ca21d7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1905,6 +1905,17 @@ func (b *executorBuilder) buildMaxOneRow(v *plannercore.PhysicalMaxOneRow) Execu } func (b *executorBuilder) buildUnionAll(v *plannercore.PhysicalUnionAll) Executor { + // A quick fix for avoiding a race mentioned in issue #30468. + // Fetch the snapshot ts to make the transaction's state ready. Otherwise, multiple threads in the Union executor + // may change the transaction's state concurrently, which causes race. + // This fix is a hack, but with minimal change to the current code and works. Actually, the usage of the transaction + // states and the logic to access the snapshot ts should all be refactored. + _, err := b.getSnapshotTS() + if err != nil { + b.err = err + return nil + } + childExecs := make([]Executor, len(v.Children())) for i, child := range v.Children() { childExecs[i] = b.build(child) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 994385566dde3..6337b8a01dc09 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -162,7 +162,29 @@ type innerWorker struct { // Open implements the Executor interface. func (e *IndexLookUpJoin) Open(ctx context.Context) error { - err := e.children[0].Open(ctx) + // Be careful, very dirty hack in this line!!! + // IndexLookUpJoin need to rebuild executor (the dataReaderBuilder) during + // executing. However `executor.Next()` is lazy evaluation when the RecordSet + // result is drained. + // Lazy evaluation means the saved session context may change during executor's + // building and its running. + // A specific sequence for example: + // + // e := buildExecutor() // txn at build time + // recordSet := runStmt(e) + // session.CommitTxn() // txn closed + // recordSet.Next() + // e.dataReaderBuilder.Build() // txn is used again, which is already closed + // + // The trick here is `getSnapshotTS` will cache snapshot ts in the dataReaderBuilder, + // so even txn is destroyed later, the dataReaderBuilder could still use the + // cached snapshot ts to construct DAG. + _, err := e.innerCtx.readerBuilder.getSnapshotTS() + if err != nil { + return err + } + + err = e.children[0].Open(ctx) if err != nil { return err } diff --git a/session/txn.go b/session/txn.go index 2ed83e89c2fde..ce36621ba8afe 100644 --- a/session/txn.go +++ b/session/txn.go @@ -488,6 +488,7 @@ type txnFuture struct { func (tf *txnFuture) wait() (kv.Transaction, error) { startTS, err := tf.future.Wait() + failpoint.Inject("txnFutureWait", func() {}) if err == nil { return tf.store.Begin(tikv.WithTxnScope(tf.txnScope), tikv.WithStartTS(startTS)) } else if config.GetGlobalConfig().Store == "unistore" { From c0f22ee72602da3d549149d9562d0aab2a6cdaa9 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 29 Apr 2022 11:56:52 +0800 Subject: [PATCH 29/51] planner: fix wrong range calculation for Nulleq function on Enum values (#32440) (#32496) close pingcap/tidb#32428 --- planner/core/integration_test.go | 22 ++++++++++++++++++++++ planner/core/prepare_test.go | 3 +-- util/ranger/points.go | 6 +++++- 3 files changed, 28 insertions(+), 3 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 544a4bf0f2266..465560dc69f55 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3378,6 +3378,28 @@ func (s *testIntegrationSuite) TestIssue26719(c *C) { tk.MustExec(`rollback`) } +func (s *testIntegrationSuite) TestIssue32428(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table `t1` (`a` enum('aa') DEFAULT NULL, KEY `k` (`a`))") + tk.MustExec("insert into t1 values('aa')") + tk.MustExec("insert into t1 values(null)") + tk.MustQuery("select a from t1 where a<=>'aa'").Check(testkit.Rows("aa")) + tk.MustQuery("select a from t1 where a<=>null").Check(testkit.Rows("")) + + tk.MustExec(`CREATE TABLE IDT_MULTI15860STROBJSTROBJ ( + COL1 enum('aa') DEFAULT NULL, + COL2 int(41) DEFAULT NULL, + COL3 year(4) DEFAULT NULL, + KEY U_M_COL4 (COL1,COL2), + KEY U_M_COL5 (COL3,COL2))`) + tk.MustExec(`insert into IDT_MULTI15860STROBJSTROBJ values("aa", 1013610488, 1982)`) + tk.MustQuery(`SELECT * FROM IDT_MULTI15860STROBJSTROBJ t1 RIGHT JOIN IDT_MULTI15860STROBJSTROBJ t2 ON t1.col1 <=> t2.col1 where t1.col1 is null and t2.col1 = "aa"`).Check(testkit.Rows()) // empty result + tk.MustExec(`prepare stmt from "SELECT * FROM IDT_MULTI15860STROBJSTROBJ t1 RIGHT JOIN IDT_MULTI15860STROBJSTROBJ t2 ON t1.col1 <=> t2.col1 where t1.col1 is null and t2.col1 = ?"`) + tk.MustExec(`set @a="aa"`) + tk.MustQuery(`execute stmt using @a`).Check(testkit.Rows()) // empty result +} + func (s *testIntegrationSerialSuite) TestPushDownProjectionForTiFlash(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 6b336384ef8da..baf701af22a0e 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -2390,8 +2390,7 @@ func (s *testPrepareSerialSuite) TestIssue30100(c *C) { tk.MustExec("set @a=0;") tk.MustQuery("execute stmt using @a").Check(testkit.Rows()) tk.MustQuery("execute stmt using @a").Check(testkit.Rows()) - // If the plan contains the tableDual, it can not be cached. - tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) } func (s *testPlanSerialSuite) TestPartitionTable(c *C) { diff --git a/util/ranger/points.go b/util/ranger/points.go index 3633c15690634..8b2fcf52fdf0a 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -458,6 +458,10 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val res = append(res, &point{value: d, excl: false, start: false}) } + if op == ast.NullEQ && val.IsNull() { + res = append(res, &point{start: true}, &point{}) // null point + } + tmpEnum := types.Enum{} for i := 0; i <= len(ft.Elems); i++ { if i == 0 { @@ -486,7 +490,7 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val if v >= 0 { appendPointFunc(d) } - case ast.EQ: + case ast.EQ, ast.NullEQ: if v == 0 { appendPointFunc(d) } From c9b53ee77cc958662a1c1a801622b31b8d0c75df Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 29 Apr 2022 12:12:53 +0800 Subject: [PATCH 30/51] expression: fix date format identifies '\n' as invalid separator (#32358) (#32504) close pingcap/tidb#32232 --- types/time.go | 3 ++- types/time_test.go | 5 +++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/types/time.go b/types/time.go index 3d39b76eaf6bb..7dfdb7c2d1fbe 100644 --- a/types/time.go +++ b/types/time.go @@ -786,7 +786,8 @@ func isValidSeparator(c byte, prevParts int) bool { return true } - if prevParts == 2 && (c == ' ' || c == 'T') { + // for https://github.com/pingcap/tidb/issues/32232 + if prevParts == 2 && (c == 'T' || c == ' ' || c == '\t' || c == '\n' || c == '\v' || c == '\f' || c == '\r') { return true } diff --git a/types/time_test.go b/types/time_test.go index fac16e0b289d9..15f4e3a35806b 100644 --- a/types/time_test.go +++ b/types/time_test.go @@ -1057,6 +1057,11 @@ func TestParseDateFormat(t *testing.T) { {"T10:10:10", nil}, {"2011-11-11x", []string{"2011", "11", "11x"}}, {"xxx 10:10:10", nil}, + {"2022-02-01\n16:33:00", []string{"2022", "02", "01", "16", "33", "00"}}, + {"2022-02-01\f16:33:00", []string{"2022", "02", "01", "16", "33", "00"}}, + {"2022-02-01\v16:33:00", []string{"2022", "02", "01", "16", "33", "00"}}, + {"2022-02-01\r16:33:00", []string{"2022", "02", "01", "16", "33", "00"}}, + {"2022-02-01\t16:33:00", []string{"2022", "02", "01", "16", "33", "00"}}, } for _, tt := range tbl { From de28bfc85fa8de54cf95e1f446c3d0a2079a8d0a Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 29 Apr 2022 12:28:52 +0800 Subject: [PATCH 31/51] planner: don't decorrelate the APPLY when the inner's projection reference no column (#32370) (#32576) close pingcap/tidb#32089 --- expression/util.go | 2 +- planner/core/rule_decorrelate.go | 16 ++++++++++++++++ planner/core/rule_partition_processor.go | 2 +- .../core/testdata/plan_suite_unexported_in.json | 3 ++- .../core/testdata/plan_suite_unexported_out.json | 3 ++- util/ranger/detacher.go | 2 +- 6 files changed, 23 insertions(+), 5 deletions(-) diff --git a/expression/util.go b/expression/util.go index f75775712dce1..f132d64e81f34 100644 --- a/expression/util.go +++ b/expression/util.go @@ -190,7 +190,7 @@ func ExtractColumnsAndCorColumnsFromExpressions(result []*Column, list []Express } // ExtractColumnSet extracts the different values of `UniqueId` for columns in expressions. -func ExtractColumnSet(exprs []Expression) *intsets.Sparse { +func ExtractColumnSet(exprs ...Expression) *intsets.Sparse { set := &intsets.Sparse{} for _, expr := range exprs { extractColumnSet(expr, set) diff --git a/planner/core/rule_decorrelate.go b/planner/core/rule_decorrelate.go index 182597eca6210..e8a80f722f055 100644 --- a/planner/core/rule_decorrelate.go +++ b/planner/core/rule_decorrelate.go @@ -154,6 +154,21 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo return s.optimize(ctx, p, opt) } } else if proj, ok := innerPlan.(*LogicalProjection); ok { + allConst := true + for _, expr := range proj.Exprs { + if len(expression.ExtractCorColumns(expr)) > 0 || !expression.ExtractColumnSet(expr).IsEmpty() { + allConst = false + break + } + } + if allConst && apply.JoinType == LeftOuterJoin { + // If the projection just references some constant. We cannot directly pull it up when the APPLY is an outer join. + // e.g. select (select 1 from t1 where t1.a=t2.a) from t2; When the t1.a=t2.a is false the join's output is NULL. + // But if we pull the projection upon the APPLY. It will return 1 since the projection is evaluated after the join. + // We disable the decorrelation directly for now. + // TODO: Actually, it can be optimized. We need to first push the projection down to the selection. And then the APPLY can be decorrelated. + goto NoOptimize + } for i, expr := range proj.Exprs { proj.Exprs[i] = expr.Decorrelate(outerPlan.Schema()) } @@ -315,6 +330,7 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo return s.optimize(ctx, p, opt) } } +NoOptimize: newChildren := make([]LogicalPlan, 0, len(p.Children())) for _, child := range p.Children() { np, err := s.optimize(ctx, child, opt) diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 53a655c5ed932..024bc235dad5b 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -930,7 +930,7 @@ func makePartitionByFnCol(sctx sessionctx.Context, columns []*expression.Column, monotonous = getMonotoneMode(raw.FuncName.L) // Check the partitionExpr is in the form: fn(col, ...) // There should be only one column argument, and it should be the first parameter. - if expression.ExtractColumnSet(args).Len() == 1 { + if expression.ExtractColumnSet(args...).Len() == 1 { if col1, ok := args[0].(*expression.Column); ok { col = col1 } diff --git a/planner/core/testdata/plan_suite_unexported_in.json b/planner/core/testdata/plan_suite_unexported_in.json index 06c90571e11fe..98c7b9b9b5985 100644 --- a/planner/core/testdata/plan_suite_unexported_in.json +++ b/planner/core/testdata/plan_suite_unexported_in.json @@ -131,7 +131,8 @@ "select t1.b from t t1 where t1.b in (select t2.b from t t2 where t2.a = t1.a order by t2.a)", "select t1.b from t t1 where exists(select t2.b from t t2 where t2.a = t1.a order by t2.a)", // `Sort` will not be eliminated, if it is not the top level operator. - "select t1.b from t t1 where t1.b = (select t2.b from t t2 where t2.a = t1.a order by t2.a limit 1)" + "select t1.b from t t1 where t1.b = (select t2.b from t t2 where t2.a = t1.a order by t2.a limit 1)", + "select (select 1 from t t1 where t1.a = t2.a) from t t2" ] }, { diff --git a/planner/core/testdata/plan_suite_unexported_out.json b/planner/core/testdata/plan_suite_unexported_out.json index 4fc39a9e7d4d0..44b29b2594347 100644 --- a/planner/core/testdata/plan_suite_unexported_out.json +++ b/planner/core/testdata/plan_suite_unexported_out.json @@ -117,7 +117,8 @@ "Join{DataScan(t1)->DataScan(t2)->Aggr(max(test.t.a),firstrow(test.t.b))}(test.t.b,test.t.b)->Projection->Sel([eq(test.t.b, Column#25)])->Projection", "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)(test.t.b,test.t.b)->Projection", "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->Projection", - "Apply{DataScan(t1)->DataScan(t2)->Sel([eq(test.t.a, test.t.a)])->Projection->Sort->Limit}->Projection->Sel([eq(test.t.b, test.t.b)])->Projection" + "Apply{DataScan(t1)->DataScan(t2)->Sel([eq(test.t.a, test.t.a)])->Projection->Sort->Limit}->Projection->Sel([eq(test.t.b, test.t.b)])->Projection", + "Apply{DataScan(t2)->DataScan(t1)->Sel([eq(test.t.a, test.t.a)])->Projection}->Projection" ] }, { diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index 7f64647fb792c..51bf22e9b62c1 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -197,7 +197,7 @@ func extractIndexPointRangesForCNF(sctx sessionctx.Context, conds []expression.E offset := int(-1) for i, cond := range conds { tmpConds := []expression.Expression{cond} - colSets := expression.ExtractColumnSet(tmpConds) + colSets := expression.ExtractColumnSet(cond) if colSets.Len() == 0 { continue } From cd60925897337d790469cc8293f0cbb3a2bdcb36 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 29 Apr 2022 12:44:52 +0800 Subject: [PATCH 32/51] metrics: Support multi k8s in grafana dashboards (#32546) (#32953) close pingcap/tidb#32593 --- br/metrics/grafana/br.json | 112 ++++---- br/metrics/grafana/lightning.json | 84 +++--- metrics/grafana/overview.json | 220 ++++++++-------- metrics/grafana/tidb.json | 408 ++++++++++++++++-------------- metrics/grafana/tidb_runtime.json | 101 ++++---- metrics/grafana/tidb_summary.json | 127 ++++++---- 6 files changed, 586 insertions(+), 466 deletions(-) diff --git a/br/metrics/grafana/br.json b/br/metrics/grafana/br.json index 5a2d0ad5d771f..3b48947f0eab1 100644 --- a/br/metrics/grafana/br.json +++ b/br/metrics/grafana/br.json @@ -123,7 +123,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"(backup-worker|bkwkr).*\"}[1m])) by (instance)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"(backup-worker|bkwkr).*\"}[1m])) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -133,7 +133,7 @@ "step": 4 }, { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"backup_endpoint\"}[1m])) by (instance)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"backup_endpoint\"}[1m])) by (instance)", "format": "time_series", "hide": true, "intervalFactor": 2, @@ -143,7 +143,7 @@ "step": 4 }, { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"(backup-worker|bkwkr).*\"}[1m])) by (instance, tid) > 0", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"(backup-worker|bkwkr).*\"}[1m])) by (instance, tid) > 0", "format": "time_series", "hide": true, "intervalFactor": 2, @@ -153,7 +153,7 @@ "step": 4 }, { - "expr": "sum(tikv_backup_thread_pool_size{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by(instance)", + "expr": "sum(tikv_backup_thread_pool_size{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by(instance)", "format": "time_series", "hide": true, "intervalFactor": 2, @@ -253,7 +253,7 @@ ], "targets": [ { - "expr": "sum(tikv_backup_thread_pool_size{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by(instance)", + "expr": "sum(tikv_backup_thread_pool_size{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by(instance)", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -308,7 +308,7 @@ "steppedLine": false, "targets": [ { - "expr": "delta(tikv_backup_error_counter{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])", + "expr": "delta(tikv_backup_error_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -401,7 +401,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_backup_range_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", cf=\"write\"}[1m])) by (le)", + "expr": "max(rate(tikv_backup_range_size_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", cf=\"write\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -478,7 +478,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_backup_range_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", cf=\"default\"}[1m])) by (le)", + "expr": "max(rate(tikv_backup_range_size_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", cf=\"default\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -560,7 +560,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_backup_range_size_bytes_sum{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "expr": "sum(rate(tikv_backup_range_size_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -570,7 +570,7 @@ "step": 4 }, { - "expr": "rate(tikv_backup_range_size_bytes_sum{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "rate(tikv_backup_range_size_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -666,7 +666,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_backup_range_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"snapshot\"}[1m])) by (le)", + "expr": "max(rate(tikv_backup_range_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"snapshot\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -743,7 +743,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_backup_range_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"scan\"}[1m])) by (le)", + "expr": "max(rate(tikv_backup_range_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"scan\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -820,7 +820,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_backup_range_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"save.*\"}[1m])) by (le)", + "expr": "max(rate(tikv_backup_range_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"save.*\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -897,7 +897,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tikv_backup_range_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tikv_backup_range_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}} - 99%", @@ -906,7 +906,7 @@ "step": 4 }, { - "expr": "histogram_quantile(0.95, sum(rate(tikv_backup_range_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.95, sum(rate(tikv_backup_range_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}} - 95%", @@ -914,7 +914,7 @@ "step": 4 }, { - "expr": "sum(rate(tikv_backup_range_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type) / sum(rate(tikv_backup_range_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", + "expr": "sum(rate(tikv_backup_range_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type) / sum(rate(tikv_backup_range_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}} - avg", @@ -1008,7 +1008,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_external_storage_create_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le)", + "expr": "max(rate(tikv_external_storage_create_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -1089,14 +1089,14 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tikv_external_storage_create_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le,type))", + "expr": "histogram_quantile(1, sum(rate(tikv_external_storage_create_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le,type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-100%", "refId": "E" }, { - "expr": "histogram_quantile(0.99, sum(rate(tikv_external_storage_create_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le,type))", + "expr": "histogram_quantile(0.99, sum(rate(tikv_external_storage_create_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le,type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-99%", @@ -1195,14 +1195,14 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", req=~\"checksum.*|analyze.*\"}[1m])) by (le,req))", + "expr": "histogram_quantile(1, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", req=~\"checksum.*|analyze.*\"}[1m])) by (le,req))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{req}}-100%", "refId": "E" }, { - "expr": "histogram_quantile(0.99, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", req=~\"checksum.*|analyze.*\"}[1m])) by (le,req))", + "expr": "histogram_quantile(0.99, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", req=~\"checksum.*|analyze.*\"}[1m])) by (le,req))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{req}}-99%", @@ -1300,7 +1300,7 @@ "steppedLine": false, "targets": [ { - "expr": "rate(node_disk_io_time_seconds_total{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "rate(node_disk_io_time_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}} - {{device}}", @@ -1400,7 +1400,7 @@ }, "lines": true, "linewidth": 1, - "links": [], + "links": [], "nullPointMode": "null", "percentage": false, "pointradius": 5, @@ -1417,7 +1417,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"sst_.*\"}[1m])) by (instance)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"sst_.*\"}[1m])) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -1427,7 +1427,7 @@ "step": 4 }, { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"sst_.*\"}[1m])) by (instance, tid) > 0", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"sst_.*\"}[1m])) by (instance, tid) > 0", "format": "time_series", "hide": true, "intervalFactor": 2, @@ -1437,7 +1437,7 @@ "step": 4 }, { - "expr": "count(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"sst_.*\"}[1m])) by (instance)", + "expr": "count(rate(tikv_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"sst_.*\"}[1m])) by (instance)", "format": "time_series", "hide": true, "intervalFactor": 2, @@ -1537,7 +1537,7 @@ ], "targets": [ { - "expr": "count(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"sst_.*\"}[1m])) by (instance)", + "expr": "count(rate(tikv_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", name=~\"sst_.*\"}[1m])) by (instance)", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -1593,7 +1593,7 @@ "steppedLine": false, "targets": [ { - "expr": "delta(tikv_import_error_counter{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])", + "expr": "delta(tikv_import_error_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -1681,7 +1681,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tikv_import_rpc_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, request))", + "expr": "histogram_quantile(0.99, sum(rate(tikv_import_rpc_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, request))", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -1777,7 +1777,7 @@ "steppedLine": false, "targets": [ { - "expr": "rate(node_disk_io_time_seconds_total{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "rate(node_disk_io_time_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}} - {{device}}", @@ -1872,7 +1872,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_import_rpc_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", request=~\"download|write\"}[1m])) by (le)", + "expr": "max(rate(tikv_import_rpc_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", request=~\"download|write\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -1949,7 +1949,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_import_download_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"queue\"}[1m])) by (le)", + "expr": "max(rate(tikv_import_download_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"queue\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -2026,7 +2026,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_import_download_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"read\"}[1m])) by (le)", + "expr": "max(rate(tikv_import_download_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"read\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -2103,7 +2103,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_import_download_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"rewrite\"}[1m])) by (le)", + "expr": "max(rate(tikv_import_download_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"rewrite\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -2180,7 +2180,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_import_rpc_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", request=~\"ingest\"}[1m])) by (le)", + "expr": "max(rate(tikv_import_rpc_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", request=~\"ingest\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -2257,7 +2257,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_import_ingest_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ingest\"}[1m])) by (le)", + "expr": "max(rate(tikv_import_ingest_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ingest\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -2334,7 +2334,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "max(rate(tikv_import_ingest_byte{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le)", + "expr": "max(rate(tikv_import_ingest_byte{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le)", "format": "heatmap", "instant": false, "intervalFactor": 2, @@ -2406,14 +2406,14 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_import_download_bytes_sum{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", + "expr": "sum(rate(tikv_import_download_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]))", "format": "time_series", "intervalFactor": 1, "legendFormat": "total", "refId": "A" }, { - "expr": "rate(tikv_import_download_bytes_sum{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "rate(tikv_import_download_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}", @@ -2513,7 +2513,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_raftstore_region_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"leader\"}) by (instance)", + "expr": "sum(tikv_raftstore_region_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"leader\"}) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -2522,7 +2522,7 @@ "step": 10 }, { - "expr": "delta(tikv_raftstore_region_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"leader\"}[30s]) < -10", + "expr": "delta(tikv_raftstore_region_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"leader\"}[30s]) < -10", "format": "time_series", "hide": true, "intervalFactor": 2, @@ -2619,7 +2619,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_raftstore_region_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"region\"}) by (instance)", + "expr": "sum(tikv_raftstore_region_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"region\"}) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2717,14 +2717,14 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", req=~\"checksum.*|analyze.*\"}[1m])) by (le,req))", + "expr": "histogram_quantile(1, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", req=~\"checksum.*|analyze.*\"}[1m])) by (le,req))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{req}}-100%", "refId": "E" }, { - "expr": "histogram_quantile(0.99, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", req=~\"checksum.*|analyze.*\"}[1m])) by (le,req))", + "expr": "histogram_quantile(0.99, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", req=~\"checksum.*|analyze.*\"}[1m])) by (le,req))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{req}}-99%", @@ -2822,7 +2822,7 @@ "steppedLine": false, "targets": [ { - "expr": "rate(node_disk_io_time_seconds_total{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "rate(node_disk_io_time_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}} - {{device}}", @@ -2884,6 +2884,26 @@ "tags": [], "templating": { "list": [ + { + "allValue": null, + "current": { }, + "datasource": "${DS_TEST-CLUSTER}", + "hide": 2, + "includeAll": false, + "label": "K8s-cluster", + "multi": false, + "name": "k8s_cluster", + "options": [ ], + "query": "label_values(tikv_engine_size_bytes, k8s_cluster)", + "refresh": 2, + "regex": "", + "sort": 1, + "tagValuesQuery": "", + "tags": [ ], + "tagsQuery": "", + "type": "query", + "useTags": false + }, { "allValue": null, "current": { @@ -2897,7 +2917,7 @@ "options": [ ], - "query": "label_values(tikv_engine_size_bytes, tidb_cluster)", + "query": "label_values(tikv_engine_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster)", "refresh": 2, "regex": "", "sort": 1, diff --git a/br/metrics/grafana/lightning.json b/br/metrics/grafana/lightning.json index d842f9f523a4c..624fc3e91c953 100644 --- a/br/metrics/grafana/lightning.json +++ b/br/metrics/grafana/lightning.json @@ -100,14 +100,14 @@ "steppedLine": false, "targets": [ { - "expr": "rate(tikv_import_write_chunk_bytes_sum{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "rate(tikv_import_write_chunk_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "write from lightning", "refId": "B" }, { - "expr": "sum(rate(tikv_import_upload_chunk_bytes_sum{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tikv_import_upload_chunk_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "upload to tikv", @@ -182,7 +182,7 @@ "steppedLine": false, "targets": [ { - "expr": "1/rate(lightning_chunks{tidb_cluster=\"$tidb_cluster\", state=\"finished\"}[1m]) ", + "expr": "1/rate(lightning_chunks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", state=\"finished\"}[1m]) ", "format": "time_series", "intervalFactor": 2, "legendFormat": "", @@ -294,7 +294,7 @@ "tableColumn": "", "targets": [ { - "expr": "lightning_chunks{tidb_cluster=\"$tidb_cluster\", state=\"finished\"} / ignoring(state) lightning_chunks{tidb_cluster=\"$tidb_cluster\", state=\"estimated\"}", + "expr": "lightning_chunks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", state=\"finished\"} / ignoring(state) lightning_chunks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", state=\"estimated\"}", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -370,7 +370,7 @@ "tableColumn": "", "targets": [ { - "expr": "lightning_tables{tidb_cluster=\"$tidb_cluster\", state=\"completed\"} / ignoring(state) lightning_tables{tidb_cluster=\"$tidb_cluster\", state=\"pending\"}", + "expr": "lightning_tables{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", state=\"completed\"} / ignoring(state) lightning_tables{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", state=\"pending\"}", "format": "time_series", "instant": false, "intervalFactor": 1, @@ -454,7 +454,7 @@ ], "targets": [ { - "expr": "lightning_tables{tidb_cluster=\"$tidb_cluster\", result=\"failure\"}", + "expr": "lightning_tables{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", result=\"failure\"}", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -510,14 +510,14 @@ "steppedLine": false, "targets": [ { - "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tikv-importer\"}", + "expr": "process_resident_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tikv-importer\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "importer RSS", "refId": "A" }, { - "expr": "go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", job=\"lightning\"}", + "expr": "go_memstats_heap_inuse_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"lightning\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "lightning heap", @@ -592,7 +592,7 @@ "steppedLine": false, "targets": [ { - "expr": "go_goroutines{tidb_cluster=\"$tidb_cluster\", job=\"lightning\"}", + "expr": "go_goroutines{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"lightning\"}", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -671,14 +671,14 @@ "steppedLine": false, "targets": [ { - "expr": "rate(process_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", job=\"lightning\"}[30s])*100", + "expr": "rate(process_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"lightning\"}[30s])*100", "format": "time_series", "intervalFactor": 2, "legendFormat": "Lightning", "refId": "A" }, { - "expr": "rate(process_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", job=\"tikv-importer\"}[30s])*100", + "expr": "rate(process_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tikv-importer\"}[30s])*100", "format": "time_series", "intervalFactor": 2, "legendFormat": "Importer", @@ -766,7 +766,7 @@ "steppedLine": true, "targets": [ { - "expr": "lightning_idle_workers{tidb_cluster=\"$tidb_cluster\"}", + "expr": "lightning_idle_workers{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{name}}", @@ -842,21 +842,21 @@ "steppedLine": true, "targets": [ { - "expr": "lightning_kv_encoder{tidb_cluster=\"$tidb_cluster\", type=\"open\"} - ignoring(type) lightning_kv_encoder{tidb_cluster=\"$tidb_cluster\", type=\"closed\"}", + "expr": "lightning_kv_encoder{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"open\"} - ignoring(type) lightning_kv_encoder{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"closed\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "KV Encoder", "refId": "A" }, { - "expr": "lightning_importer_engine{tidb_cluster=\"$tidb_cluster\", type=\"open\"} - ignoring(type) lightning_importer_engine{tidb_cluster=\"$tidb_cluster\", type=\"closed\"}", + "expr": "lightning_importer_engine{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"open\"} - ignoring(type) lightning_importer_engine{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"closed\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "Importer Engines (via Lightning)", "refId": "B" }, { - "expr": "tikv_import_rpc_duration_count{tidb_cluster=\"$tidb_cluster\", request=\"open_engine\",result=\"ok\"} - ignoring(request) tikv_import_rpc_duration_count{tidb_cluster=\"$tidb_cluster\", request=\"close_engine\",result=\"ok\"}", + "expr": "tikv_import_rpc_duration_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", request=\"open_engine\",result=\"ok\"} - ignoring(request) tikv_import_rpc_duration_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", request=\"close_engine\",result=\"ok\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "Importer Engines (via Importer)", @@ -958,7 +958,7 @@ ], "targets": [ { - "expr": "min(tikv_config_rocksdb{tidb_cluster=\"$tidb_cluster\", name=\"hard_pending_compaction_bytes_limit\"}) by (instance)", + "expr": "min(tikv_config_rocksdb{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name=\"hard_pending_compaction_bytes_limit\"}) by (instance)", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -1014,14 +1014,14 @@ "steppedLine": false, "targets": [ { - "expr": "rate(lightning_chunk_parser_read_block_seconds_sum{tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(lightning_chunk_parser_read_block_seconds_count{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(lightning_chunk_parser_read_block_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(lightning_chunk_parser_read_block_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 2, "legendFormat": "read block", "refId": "A" }, { - "expr": "rate(lightning_apply_worker_seconds_sum{tidb_cluster=\"$tidb_cluster\", name = \"io\"}[30s]) /rate(lightning_apply_worker_seconds_count{tidb_cluster=\"$tidb_cluster\", name = \"io\"}[30s]) ", + "expr": "rate(lightning_apply_worker_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name = \"io\"}[30s]) /rate(lightning_apply_worker_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name = \"io\"}[30s]) ", "format": "time_series", "intervalFactor": 2, "legendFormat": "apply worker", @@ -1096,14 +1096,14 @@ "steppedLine": false, "targets": [ { - "expr": "rate(lightning_row_encode_seconds_sum{tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(lightning_row_encode_seconds_count{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(lightning_row_encode_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(lightning_row_encode_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 2, "legendFormat": "row encode", "refId": "A" }, { - "expr": "rate(lightning_block_deliver_seconds_sum{tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(lightning_block_deliver_seconds_count{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(lightning_block_deliver_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(lightning_block_deliver_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 2, "legendFormat": "block deliver", @@ -1190,14 +1190,14 @@ "steppedLine": false, "targets": [ { - "expr": "rate(lightning_block_deliver_bytes_sum{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(lightning_block_deliver_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{kind}} deliver rate", "refId": "B" }, { - "expr": "sum(rate(lightning_block_deliver_bytes_sum{tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(lightning_block_deliver_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "total deliver rate", @@ -1272,21 +1272,21 @@ "steppedLine": false, "targets": [ { - "expr": "lightning_row_read_bytes_sum{tidb_cluster=\"$tidb_cluster\"}", + "expr": "lightning_row_read_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "parser read size", "refId": "A" }, { - "expr": "lightning_block_deliver_bytes_sum{tidb_cluster=\"$tidb_cluster\"}", + "expr": "lightning_block_deliver_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{kind}} deliver size", "refId": "B" }, { - "expr": "pd_cluster_status{tidb_cluster=\"$tidb_cluster\", type=\"storage_size\"} / ignoring(type) pd_config_status{tidb_cluster=\"$tidb_cluster\", type=\"max_replicas\"}", + "expr": "pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"storage_size\"} / ignoring(type) pd_config_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"max_replicas\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "storage_size / replicas", @@ -1373,14 +1373,14 @@ "steppedLine": false, "targets": [ { - "expr": "rate(tikv_import_range_delivery_duration_sum{tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(tikv_import_range_delivery_duration_count{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(tikv_import_range_delivery_duration_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(tikv_import_range_delivery_duration_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 2, "legendFormat": "range deliver", "refId": "A" }, { - "expr": "rate(tikv_import_sst_delivery_duration_sum{tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(tikv_import_sst_delivery_duration_count{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(tikv_import_sst_delivery_duration_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(tikv_import_sst_delivery_duration_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 2, "legendFormat": "SST file deliver", @@ -1461,14 +1461,14 @@ "steppedLine": false, "targets": [ { - "expr": "rate(tikv_import_split_sst_duration_sum{tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(tikv_import_split_sst_duration_count{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(tikv_import_split_sst_duration_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(tikv_import_split_sst_duration_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 2, "legendFormat": "Split SST", "refId": "C" }, { - "expr": "rate(tikv_import_sst_upload_duration_sum{tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(tikv_import_sst_upload_duration_count{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(tikv_import_sst_upload_duration_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(tikv_import_sst_upload_duration_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -1476,7 +1476,7 @@ "refId": "D" }, { - "expr": "rate(tikv_import_sst_ingest_duration_sum{tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(tikv_import_sst_ingest_duration_count{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(tikv_import_sst_ingest_duration_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]) / rate(tikv_import_sst_ingest_duration_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -1484,7 +1484,7 @@ "refId": "E" }, { - "expr": "rate(tikv_import_sst_chunk_bytes_sum{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(tikv_import_sst_chunk_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 2, "legendFormat": "SST size", @@ -1541,6 +1541,26 @@ "tags": [], "templating": { "list": [ + { + "allValue": null, + "current": { }, + "datasource": "${DS_TEST-CLUSTER}", + "hide": 2, + "includeAll": false, + "label": "K8s-cluster", + "multi": false, + "name": "k8s_cluster", + "options": [ ], + "query": "label_values(lightning_chunks, k8s_cluster)", + "refresh": 2, + "regex": "", + "sort": 1, + "tagValuesQuery": "", + "tags": [ ], + "tagsQuery": "", + "type": "query", + "useTags": false + }, { "allValue": null, "current": { @@ -1554,7 +1574,7 @@ "options": [ ], - "query": "label_values(lightning_chunks, tidb_cluster)", + "query": "label_values(lightning_chunks{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", "refresh": 2, "regex": "", "sort": 1, diff --git a/metrics/grafana/overview.json b/metrics/grafana/overview.json index ebbfb72ea361c..4017ff43a8126 100644 --- a/metrics/grafana/overview.json +++ b/metrics/grafana/overview.json @@ -153,91 +153,91 @@ ], "targets": [ { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"tidb\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"tidb\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "TiDB", "refId": "A" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"pd\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"pd\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "PD", "refId": "B" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"tikv\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"tikv\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "TiKV", "refId": "C" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"tiflash\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"tiflash\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "TiFlash", "refId": "D" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"pump\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"pump\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Pump", "refId": "E" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"drainer\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"drainer\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Drainer", "refId": "F" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"kafka\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"kafka\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Kafka", "refId": "G" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"zookeeper\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"zookeeper\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Zookeeper", "refId": "H" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"node_exporter\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"node_exporter\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Node_exporter", "refId": "I" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"blackbox_exporter\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"blackbox_exporter\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Blackbox_exporter", "refId": "J" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"grafana\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"grafana\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Grafana", "refId": "K" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", job=\"blackbox_exporter_http\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"blackbox_exporter_http\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Pushgateway", "refId": "L" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"kafka_exporter\"} == 1)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"kafka_exporter\"} == 1)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Kafka_exporter", @@ -331,91 +331,91 @@ ], "targets": [ { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"tidb\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"tidb\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "TiDB", "refId": "A" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"pd\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"pd\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "PD", "refId": "B" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"tikv\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"tikv\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "TiKV", "refId": "C" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"tiflash\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"tiflash\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "TiFlash", "refId": "D" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"pump\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"pump\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Pump", "refId": "E" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"drainer\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"drainer\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Drainer", "refId": "F" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"kafka\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"kafka\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Kafka", "refId": "G" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"zookeeper\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"zookeeper\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Zookeeper", "refId": "H" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"node_exporter\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"node_exporter\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Node_exporter", "refId": "I" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"blackbox_exporter\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"blackbox_exporter\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Blackbox_exporter", "refId": "J" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"grafana\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"grafana\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Grafana", "refId": "K" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", job=\"blackbox_exporter_http\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"blackbox_exporter_http\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Pushgateway", "refId": "L" }, { - "expr": "\ncount(probe_success{tidb_cluster=\"$tidb_cluster\", group=\"kafka_exporter\"} == 0)", + "expr": "\ncount(probe_success{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", group=\"kafka_exporter\"} == 0)", "format": "time_series", "intervalFactor": 2, "legendFormat": "Kafka_exporter", @@ -497,7 +497,7 @@ "tableColumn": "", "targets": [ { - "expr": "pd_tso_role{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", dc=\"global\"}", + "expr": "pd_tso_role{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", dc=\"global\"}", "format": "time_series", "instant": true, "intervalFactor": 2, @@ -594,7 +594,7 @@ "tableColumn": "", "targets": [ { - "expr": "pd_cluster_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\",type=\"storage_capacity\"}", + "expr": "pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\",type=\"storage_capacity\"}", "format": "time_series", "intervalFactor": 2, "refId": "A", @@ -679,7 +679,7 @@ "tableColumn": "", "targets": [ { - "expr": "pd_cluster_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\",type=\"storage_size\"}", + "expr": "pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\",type=\"storage_size\"}", "intervalFactor": 2, "refId": "A", "step": 60 @@ -759,7 +759,7 @@ "tableColumn": "", "targets": [ { - "expr": "pd_cluster_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"leader_count\"}", + "expr": "pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"leader_count\"}", "intervalFactor": 2, "refId": "A", "step": 60 @@ -839,7 +839,7 @@ "tableColumn": "", "targets": [ { - "expr": "sum(pd_cluster_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_up_count\"})", + "expr": "sum(pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_up_count\"})", "format": "time_series", "interval": "15s", "intervalFactor": 2, @@ -912,7 +912,7 @@ ], "targets": [ { - "expr": "sum(pd_cluster_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_disconnected_count\"})", + "expr": "sum(pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_disconnected_count\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "Disconnect Stores", @@ -920,7 +920,7 @@ "step": 20 }, { - "expr": "sum(pd_cluster_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_unhealth_count\"})", + "expr": "sum(pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_unhealth_count\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "Unhealth Stores", @@ -928,7 +928,7 @@ "step": 20 }, { - "expr": "sum(pd_cluster_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_low_space_count\"})", + "expr": "sum(pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_low_space_count\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "LowSpace Stores", @@ -936,7 +936,7 @@ "step": 20 }, { - "expr": "sum(pd_cluster_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_down_count\"})", + "expr": "sum(pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_down_count\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "Down Stores", @@ -944,7 +944,7 @@ "step": 20 }, { - "expr": "sum(pd_cluster_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_offline_count\"})", + "expr": "sum(pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_offline_count\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "Offline Stores", @@ -952,7 +952,7 @@ "step": 20 }, { - "expr": "sum(pd_cluster_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_tombstone_count\"})", + "expr": "sum(pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"store_tombstone_count\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "Tombstone Stores", @@ -1012,7 +1012,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(grpc_server_handling_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\"}[5m])) by (grpc_method, le))", + "expr": "histogram_quantile(0.99, sum(rate(grpc_server_handling_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\"}[5m])) by (grpc_method, le))", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -1106,7 +1106,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.98, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[30s])) by (type, le))", + "expr": "histogram_quantile(0.98, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (type, le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-98%", @@ -1114,7 +1114,7 @@ "step": 10 }, { - "expr": "avg(rate(pd_client_request_handle_requests_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\"}[30s])) by (type) / avg(rate(pd_client_request_handle_requests_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[30s])) by (type)", + "expr": "avg(rate(pd_client_request_handle_requests_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (type) / avg(rate(pd_client_request_handle_requests_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (type)", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -1204,14 +1204,14 @@ "steppedLine": false, "targets": [ { - "expr": "pd_regions_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\"}", + "expr": "pd_regions_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", "refId": "A" }, { - "expr": "sum(pd_regions_status{tidb_cluster=\"$tidb_cluster\"}) by (instance, type)", + "expr": "sum(pd_regions_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (instance, type)", "format": "time_series", "hide": true, "intervalFactor": 2, @@ -1302,7 +1302,7 @@ "steppedLine": false, "targets": [ { - "expr": "pd_hotspot_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\",type=\"hot_write_region_as_leader\"}", + "expr": "pd_hotspot_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\",type=\"hot_write_region_as_leader\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "store-{{store}}", @@ -1393,7 +1393,7 @@ "steppedLine": false, "targets": [ { - "expr": "pd_hotspot_status{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\",type=\"hot_read_region_as_leader\"}", + "expr": "pd_hotspot_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\",type=\"hot_read_region_as_leader\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "store-{{store}}", @@ -1483,7 +1483,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(delta(pd_scheduler_region_heartbeat{tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"report\", status=\"ok\"}[1m])) by (store)", + "expr": "sum(delta(pd_scheduler_region_heartbeat{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=\"$instance\", type=\"report\", status=\"ok\"}[1m])) by (store)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -1575,7 +1575,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(pd_scheduler_region_heartbeat_latency_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[5m])) by (store, le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_scheduler_region_heartbeat_latency_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[5m])) by (store, le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "store-{{store}}", @@ -1686,7 +1686,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_executor_statement_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_executor_statement_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -1775,7 +1775,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", @@ -1783,7 +1783,7 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 3, "legendFormat": "99", @@ -1791,14 +1791,14 @@ "step": 15 }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "95", "refId": "C" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80", @@ -1887,7 +1887,7 @@ "steppedLine": false, "targets": [ { - "expr": "rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "rate(tidb_server_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}} {{type}} {{result}}", @@ -1977,7 +1977,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_server_execute_error_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(increase(tidb_server_execute_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": " {{type}}", @@ -2073,7 +2073,7 @@ "steppedLine": true, "targets": [ { - "expr": "tidb_server_connections{tidb_cluster=\"$tidb_cluster\"}", + "expr": "tidb_server_connections{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2081,7 +2081,7 @@ "step": 10 }, { - "expr": "sum(tidb_server_connections{tidb_cluster=\"$tidb_cluster\"})", + "expr": "sum(tidb_server_connections{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "total", @@ -2172,7 +2172,7 @@ "steppedLine": false, "targets": [ { - "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "process_resident_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "process-{{instance}}", @@ -2180,7 +2180,7 @@ "step": 10 }, { - "expr": "go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_inuse_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "legendFormat": "HeapInuse-{{instance}}", "format": "time_series", "intervalFactor": 2, @@ -2266,7 +2266,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_session_transaction_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_session_transaction_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -2351,21 +2351,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_session_transaction_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "A" }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_transaction_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "95", "refId": "B" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_session_transaction_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80", @@ -2454,7 +2454,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -2545,7 +2545,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -2633,7 +2633,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(pd_client_cmd_handle_cmds_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m]))", + "expr": "sum(rate(pd_client_cmd_handle_cmds_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "cmd", @@ -2641,7 +2641,7 @@ "step": 10 }, { - "expr": "sum(rate(pd_client_request_handle_requests_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m]))", + "expr": "sum(rate(pd_client_request_handle_requests_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "request", @@ -2731,7 +2731,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", @@ -2739,14 +2739,14 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", @@ -2836,7 +2836,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_region_err_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_region_err_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -2927,7 +2927,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_lock_resolver_actions_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_lock_resolver_actions_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -3022,7 +3022,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_domain_load_schema_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_domain_load_schema_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -3114,7 +3114,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_backoff_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_backoff_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -3226,7 +3226,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_raftstore_region_count{tidb_cluster=\"$tidb_cluster\", type=\"leader\"}) by (instance)", + "expr": "sum(tikv_raftstore_region_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"leader\"}) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -3235,7 +3235,7 @@ "step": 10 }, { - "expr": "delta(tikv_raftstore_region_count{tidb_cluster=\"$tidb_cluster\", type=\"leader\"}[30s]) < -10", + "expr": "delta(tikv_raftstore_region_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"leader\"}[30s]) < -10", "format": "time_series", "hide": true, "intervalFactor": 2, @@ -3329,7 +3329,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_raftstore_region_count{tidb_cluster=\"$tidb_cluster\", type=\"region\"}) by (instance)", + "expr": "sum(tikv_raftstore_region_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"region\"}) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -3419,7 +3419,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", job=\"tikv\"}[1m])) by (instance)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tikv\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -3506,7 +3506,7 @@ "steppedLine": false, "targets": [ { - "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tikv\"}) by (instance)", + "expr": "avg(process_resident_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tikv\"}) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -3595,7 +3595,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_engine_size_bytes{tidb_cluster=\"$tidb_cluster\"}) by (instance)", + "expr": "sum(tikv_engine_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -3684,7 +3684,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_engine_size_bytes{tidb_cluster=\"$tidb_cluster\"}) by (type)", + "expr": "sum(tikv_engine_size_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -3778,7 +3778,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_channel_full_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance, type)", + "expr": "sum(rate(tikv_channel_full_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance, type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}} - {{type}}", @@ -3877,7 +3877,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_server_report_failure_msg_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type,instance,store_id)", + "expr": "sum(rate(tikv_server_report_failure_msg_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type,instance,store_id)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}} - {{type}} - to - {{store_id}}", @@ -3968,7 +3968,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_scheduler_contex_total{tidb_cluster=\"$tidb_cluster\"}) by (instance)", + "expr": "sum(tikv_scheduler_contex_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -4058,7 +4058,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_coprocessor_executor_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tikv_coprocessor_executor_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -4149,7 +4149,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,req))", + "expr": "histogram_quantile(0.99, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,req))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{req}}-99%", @@ -4157,7 +4157,7 @@ "step": 10 }, { - "expr": "histogram_quantile(0.95, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,req))", + "expr": "histogram_quantile(0.95, sum(rate(tikv_coprocessor_request_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,req))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{req}}-95%", @@ -4165,7 +4165,7 @@ "step": 10 }, { - "expr": " sum(rate(tikv_coprocessor_request_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", req=\"select\"}[1m])) / sum(rate(tikv_coprocessor_request_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", req=\"select\"}[1m]))", + "expr": " sum(rate(tikv_coprocessor_request_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", req=\"select\"}[1m])) / sum(rate(tikv_coprocessor_request_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", req=\"select\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "select-avg", @@ -4173,7 +4173,7 @@ "step": 10 }, { - "expr": " sum(rate(tikv_coprocessor_request_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", req=\"index\"}[1m])) / sum(rate(tikv_coprocessor_request_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", req=\"index\"}[1m]))", + "expr": " sum(rate(tikv_coprocessor_request_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", req=\"index\"}[1m])) / sum(rate(tikv_coprocessor_request_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", req=\"index\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "index-avg", @@ -4262,7 +4262,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", name=~\"raftstore_.*\"}[1m])) by (instance)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name=~\"raftstore_.*\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -4352,7 +4352,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", name=~\"cop_.*\"}[1m])) by (instance)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", name=~\"cop_.*\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -4491,7 +4491,7 @@ ], "targets": [ { - "expr": "count(node_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", mode=\"user\"}) by (instance)", + "expr": "count(node_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", mode=\"user\"}) by (instance)", "format": "table", "instant": true, "intervalFactor": 2, @@ -4603,7 +4603,7 @@ ], "targets": [ { - "expr": "node_memory_MemTotal_bytes{tidb_cluster=\"$tidb_cluster\"}", + "expr": "node_memory_MemTotal_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "table", "instant": true, "intervalFactor": 2, @@ -4657,7 +4657,7 @@ "steppedLine": false, "targets": [ { - "expr": "100 - avg by (instance) (irate(node_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", mode=\"idle\"}[1m]) ) * 100", + "expr": "100 - avg by (instance) (irate(node_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", mode=\"idle\"}[1m]) ) * 100", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -4747,7 +4747,7 @@ "steppedLine": false, "targets": [ { - "expr": "node_load1{tidb_cluster=\"$tidb_cluster\"}", + "expr": "node_load1{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -4834,7 +4834,7 @@ "steppedLine": false, "targets": [ { - "expr": "node_memory_MemAvailable_bytes{tidb_cluster=\"$tidb_cluster\"}", + "expr": "node_memory_MemAvailable_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{ instance }}", @@ -4922,14 +4922,14 @@ "steppedLine": false, "targets": [ { - "expr": "irate(node_network_receive_bytes_total{tidb_cluster=\"$tidb_cluster\", device!=\"lo\"}[5m])", + "expr": "irate(node_network_receive_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", device!=\"lo\"}[5m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "Inbound: {{instance}}-{{device}}", "refId": "A" }, { - "expr": "irate(node_network_transmit_bytes_total{tidb_cluster=\"$tidb_cluster\", device!=\"lo\"}[5m])", + "expr": "irate(node_network_transmit_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", device!=\"lo\"}[5m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "Outbound: {{instance}}-{{device}}", @@ -5018,7 +5018,7 @@ "steppedLine": false, "targets": [ { - "expr": "irate(node_netstat_Tcp_RetransSegs{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "irate(node_netstat_Tcp_RetransSegs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}} - TCPSlowStartRetrans", @@ -5108,7 +5108,7 @@ "steppedLine": false, "targets": [ { - "expr": "irate(node_disk_io_time_seconds_total{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "irate(node_disk_io_time_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}} - {{device}}", @@ -5169,6 +5169,26 @@ "tags": [], "templating": { "list": [ + { + "allValue": null, + "current": {}, + "datasource": "${DS_TEST-CLUSTER}", + "hide": 2, + "includeAll": false, + "label": "K8s-cluster", + "multi": false, + "name": "k8s_cluster", + "options": [], + "query": "label_values(pd_cluster_status, k8s_cluster)", + "refresh": 2, + "regex": "", + "sort": 1, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, { "allValue": null, "current": {}, @@ -5179,7 +5199,7 @@ "multi": false, "name": "tidb_cluster", "options": [], - "query": "label_values(pd_cluster_status, tidb_cluster)", + "query": "label_values(pd_cluster_status{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", "refresh": 2, "regex": "", "sort": 1, @@ -5200,7 +5220,7 @@ "multi": false, "name": "instance", "options": [], - "query": "label_values(pd_cluster_status{tidb_cluster=\"$tidb_cluster\"}, instance)", + "query": "label_values(pd_cluster_status{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}, instance)", "refresh": 1, "regex": "", "skipUrlSync": false, diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index b32a37fae8916..00ecaae15db85 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -107,28 +107,28 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "95", "refId": "C" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80", @@ -225,7 +225,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (result)", + "expr": "sum(rate(tidb_server_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (result)", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -234,7 +234,7 @@ "step": 60 }, { - "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\", result=\"OK\"}[1m] offset 1d))", + "expr": "sum(rate(tidb_server_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", result=\"OK\"}[1m] offset 1d))", "format": "time_series", "hide": true, "instant": false, @@ -244,7 +244,7 @@ "step": 90 }, { - "expr": "sum(tidb_server_connections{tidb_cluster=\"$tidb_cluster\"}) * sum(rate(tidb_server_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(tidb_server_connections{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) * sum(rate(tidb_server_handle_query_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_server_handle_query_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": true, "instant": false, @@ -344,7 +344,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_executor_statement_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_executor_statement_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -352,7 +352,7 @@ "step": 30 }, { - "expr": "sum(rate(tidb_executor_statement_total{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tidb_executor_statement_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "total", @@ -455,7 +455,7 @@ "steppedLine": false, "targets": [ { - "expr": "rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "rate(tidb_server_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}} {{type}} {{result}}", @@ -549,7 +549,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_server_execute_error_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, instance)", + "expr": "sum(increase(tidb_server_execute_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": " {{type}}-{{instance}}", @@ -635,21 +635,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.90, sum(rate(tidb_server_slow_query_process_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_server_slow_query_process_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "all_proc_{{sql_type}}", "refId": "A" }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_server_slow_query_cop_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_server_slow_query_cop_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "all_cop_proc_{{sql_type}}", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_server_slow_query_wait_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_server_slow_query_wait_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "all_cop_wait_{{sql_type}}", @@ -735,7 +735,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", in_txn='1'}[1m])) by (le,in_txn))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", in_txn='1'}[1m])) by (le,in_txn))", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -743,7 +743,7 @@ "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", in_txn='0'}[1m])) by (le,in_txn))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", in_txn='0'}[1m])) by (le,in_txn))", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -751,7 +751,7 @@ "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", in_txn='1'}[1m])) by (le,in_txn))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", in_txn='1'}[1m])) by (le,in_txn))", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -759,7 +759,7 @@ "refId": "C" }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", in_txn='0'}[1m])) by (le,in_txn))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", in_txn='0'}[1m])) by (le,in_txn))", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -767,7 +767,7 @@ "refId": "D" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", in_txn='1'}[1m])) by (le,in_txn))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", in_txn='1'}[1m])) by (le,in_txn))", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -775,7 +775,7 @@ "refId": "E" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", in_txn='0'}[1m])) by (le,in_txn))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_server_conn_idle_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", in_txn='0'}[1m])) by (le,in_txn))", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -862,7 +862,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{sql_type}}", @@ -948,7 +948,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{sql_type}}", @@ -1034,7 +1034,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{sql_type}}", @@ -1120,7 +1120,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.80, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le,sql_type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{sql_type}}", @@ -1226,7 +1226,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.80, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -1322,7 +1322,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{ instance }}", @@ -1417,7 +1417,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -1510,7 +1510,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -1603,7 +1603,7 @@ "steppedLine": false, "targets": [ { - "expr": "increase(tidb_server_execute_error_total{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "increase(tidb_server_execute_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}} @ {{instance}}", @@ -1694,7 +1694,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_session_restricted_sql_total{tidb_cluster=\"$tidb_cluster\"}[30s]))", + "expr": "sum(rate(tidb_session_restricted_sql_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "", @@ -1810,7 +1810,7 @@ "steppedLine": false, "targets": [ { - "expr": "(time() - process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"})", + "expr": "(time() - process_start_time_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"})", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -1907,14 +1907,14 @@ "steppedLine": false, "targets": [ { - "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "process_resident_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "intervalFactor": 1, "legendFormat": "process-{{instance}}", "refId": "A" }, { - "expr": "go_memstats_heap_sys_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_sys_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "hide": true, "intervalFactor": 1, @@ -1922,14 +1922,14 @@ "refId": "B" }, { - "expr": "go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_inuse_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "intervalFactor": 1, "legendFormat": "HeapInuse-{{instance}}", "refId": "C" }, { - "expr": "go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_alloc_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "hide": true, "intervalFactor": 1, @@ -1937,7 +1937,7 @@ "refId": "D" }, { - "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_idle_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "hide": true, "intervalFactor": 1, @@ -1945,14 +1945,14 @@ "refId": "E" }, { - "expr": "go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_released_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "hide": true, "interval": "", "legendFormat": "HeapReleased-{{instance}}", "refId": "F" }, { - "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_next_gc_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "hide": true, "interval": "", "legendFormat": "GCTrigger-{{instance}}", @@ -2056,7 +2056,7 @@ "steppedLine": false, "targets": [ { - "expr": "irate(process_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}[30s])", + "expr": "irate(process_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}[30s])", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -2065,7 +2065,7 @@ "step": 40 }, { - "expr": "tidb_server_maxprocs{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "tidb_server_maxprocs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "legendFormat": "limit-{{instance}}", "refId": "B" } @@ -2161,7 +2161,7 @@ "steppedLine": false, "targets": [ { - "expr": "tidb_server_connections{tidb_cluster=\"$tidb_cluster\"}", + "expr": "tidb_server_connections{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2169,7 +2169,7 @@ "step": 40 }, { - "expr": "sum(tidb_server_connections{tidb_cluster=\"$tidb_cluster\"})", + "expr": "sum(tidb_server_connections{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "total", @@ -2259,7 +2259,7 @@ "steppedLine": false, "targets": [ { - "expr": "process_open_fds{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "process_open_fds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2356,7 +2356,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tidb_server_disconnection_total{tidb_cluster=\"$tidb_cluster\"}) by (instance, result)", + "expr": "sum(tidb_server_disconnection_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (instance, result)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}-{{result}}", @@ -2446,7 +2446,7 @@ "steppedLine": false, "targets": [ { - "expr": " go_goroutines{tidb_cluster=\"$tidb_cluster\", job=~\"tidb.*\"}", + "expr": " go_goroutines{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=~\"tidb.*\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2534,7 +2534,7 @@ "steppedLine": false, "targets": [ { - "expr": "increase(tidb_server_event_total{tidb_cluster=\"$tidb_cluster\"}[10m])", + "expr": "increase(tidb_server_event_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[10m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}-server {{type}}", @@ -2631,7 +2631,7 @@ "steppedLine": false, "targets": [ { - "expr": "tidb_server_prepared_stmts{tidb_cluster=\"$tidb_cluster\"}", + "expr": "tidb_server_prepared_stmts{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2639,7 +2639,7 @@ "step": 40 }, { - "expr": "sum(tidb_server_prepared_stmts{tidb_cluster=\"$tidb_cluster\"})", + "expr": "sum(tidb_server_prepared_stmts{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "total", @@ -2729,7 +2729,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_monitor_keep_alive_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", + "expr": "sum(increase(tidb_monitor_keep_alive_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2817,7 +2817,7 @@ "steppedLine": false, "targets": [ { - "expr": "increase(tidb_server_panic_total{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "increase(tidb_server_panic_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -2825,7 +2825,7 @@ "refId": "A" }, { - "expr": "increase(tidb_server_critical_error_total{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "increase(tidb_server_critical_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -2914,7 +2914,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_monitor_time_jump_back_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", + "expr": "sum(increase(tidb_monitor_time_jump_back_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -3000,7 +3000,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_server_get_token_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_get_token_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 1, "legendFormat": "99", @@ -3088,7 +3088,7 @@ "steppedLine": false, "targets": [ { - "expr": "tidb_server_critical_error_total{tidb_cluster=\"$tidb_cluster\"}", + "expr": "tidb_server_critical_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -3180,14 +3180,14 @@ "steppedLine": false, "targets": [ { - "expr": "rate(tidb_server_packet_io_bytes_sum{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "rate(tidb_server_packet_io_bytes_sum{k8s_cluster=\"k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}-{{type}}-rate", "refId": "A" }, { - "expr": "tidb_server_packet_io_bytes_sum{tidb_cluster=\"$tidb_cluster\"}", + "expr": "tidb_server_packet_io_bytes_sum{k8s_cluster=\"k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}-{{type}}-total", @@ -3275,7 +3275,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_server_handshake_error_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", + "expr": "sum(increase(tidb_server_handshake_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -3386,7 +3386,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_session_transaction_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, txn_mode)", + "expr": "sum(rate(tidb_session_transaction_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, txn_mode)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-{{txn_mode}}", @@ -3480,21 +3480,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, txn_mode))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_session_transaction_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, txn_mode))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99-{{txn_mode}}", "refId": "A" }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, txn_mode))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_transaction_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, txn_mode))", "format": "time_series", "intervalFactor": 2, "legendFormat": "95-{{txn_mode}}", "refId": "B" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, txn_mode))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_session_transaction_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, txn_mode))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80-{{txn_mode}}", @@ -3720,7 +3720,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_session_retry_error_total{tidb_cluster=\"$tidb_cluster\"}[30s])) by (type, sql_type)", + "expr": "sum(rate(tidb_session_retry_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (type, sql_type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-{{sql_type}}", @@ -3814,21 +3814,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_executor_token_wait_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_executor_token_wait_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "A" }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_batch_executor_token_wait_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_batch_executor_token_wait_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "95", "refId": "B" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_tikvclient_batch_executor_token_wait_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_tikvclient_batch_executor_token_wait_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80", @@ -3923,7 +3923,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", type=\"commit\"}[1m])) by (instance)", + "expr": "sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"commit\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -4017,21 +4017,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99-{{type}}", "refId": "A" }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "95-{{type}}", "refId": "B" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80-{{type}}", @@ -4200,14 +4200,14 @@ "steppedLine": false, "targets": [ { - "expr": "rate(tidb_tikvclient_txn_write_kv_num_sum{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(tidb_tikvclient_txn_write_kv_num_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}-rate", "refId": "A" }, { - "expr": "tidb_tikvclient_txn_write_kv_num_sum{tidb_cluster=\"$tidb_cluster\"}", + "expr": "tidb_tikvclient_txn_write_kv_num_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}-sum", @@ -4440,7 +4440,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.80, sum(rate(tidb_tikvclient_txn_heart_beat_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_tikvclient_txn_heart_beat_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80-{{type}}", @@ -4448,14 +4448,14 @@ "step": 40 }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_txn_heart_beat_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_txn_heart_beat_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 1, "legendFormat": "95-{{type}}", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_txn_heart_beat_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_txn_heart_beat_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 1, "legendFormat": "99-{{type}}", @@ -4558,7 +4558,7 @@ "steppedLine": false, "targets": [ { - "expr": "rate(tidb_tikvclient_txn_write_size_bytes_sum{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "rate(tidb_tikvclient_txn_write_size_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}-rate", @@ -4566,7 +4566,7 @@ "step": 40 }, { - "expr": "tidb_tikvclient_txn_write_size_bytes_sum{tidb_cluster=\"$tidb_cluster\"}", + "expr": "tidb_tikvclient_txn_write_size_bytes_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}-sum", @@ -4731,7 +4731,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.80, sum(rate(tidb_tikvclient_pessimistic_lock_keys_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_tikvclient_pessimistic_lock_keys_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80", @@ -4739,14 +4739,14 @@ "step": 40 }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_pessimistic_lock_keys_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_pessimistic_lock_keys_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 1, "legendFormat": "95", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_pessimistic_lock_keys_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_pessimistic_lock_keys_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 1, "legendFormat": "99", @@ -4842,7 +4842,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_ttl_lifetime_reach_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", + "expr": "sum(rate(tidb_tikvclient_ttl_lifetime_reach_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}", @@ -4940,7 +4940,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_load_safepoint_total{tidb_cluster=\"$tidb_cluster\", type=\"ok\"}[1m])) by (instance)", + "expr": "sum(rate(tidb_tikvclient_load_safepoint_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"ok\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -5104,21 +5104,21 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_commit_txn_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_commit_txn_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 1, "legendFormat": "2PC-{{type}}", "refId": "C" }, { - "expr": "sum(rate(tidb_tikvclient_async_commit_txn_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_async_commit_txn_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 1, "legendFormat": "async commit-{{type}}", "refId": "A" }, { - "expr": "sum(rate(tidb_tikvclient_one_pc_txn_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_one_pc_txn_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 1, "legendFormat": "1PC-{{type}}", @@ -5225,7 +5225,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, rate(tidb_tikvclient_txn_commit_backoff_count_bucket{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "histogram_quantile(0.99, rate(tidb_tikvclient_txn_commit_backoff_count_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "intervalFactor": 1, "legendFormat": "count - {{instance}}", @@ -5233,7 +5233,7 @@ "step": 40 }, { - "expr": "histogram_quantile(0.99, rate(tidb_tikvclient_txn_commit_backoff_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "histogram_quantile(0.99, rate(tidb_tikvclient_txn_commit_backoff_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -5331,7 +5331,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_safets_update_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (result, store)", + "expr": "sum(rate(tidb_tikvclient_safets_update_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (result, store)", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{result}}-store-{{store}}", @@ -5426,7 +5426,7 @@ "steppedLine": false, "targets": [ { - "expr": "tidb_tikvclient_min_safets_gap_seconds{tidb_cluster=\"$tidb_cluster\"}", + "expr": "tidb_tikvclient_min_safets_gap_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}-store-{{store}}", @@ -5540,7 +5540,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_parse_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, sql_type))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_parse_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, sql_type))", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -5641,7 +5641,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_compile_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, sql_type))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_compile_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, sql_type))", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -5742,7 +5742,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_execute_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, sql_type))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_execute_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, sql_type))", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -5842,7 +5842,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_executor_expensive_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_executor_expensive_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -5940,7 +5940,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_server_plan_cache_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_server_plan_cache_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -6052,14 +6052,14 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999-{{type}}", "refId": "D" }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -6069,14 +6069,14 @@ "step": 4 }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90-{{type}}", "refId": "B" }, { - "expr": "histogram_quantile(0.50, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.50, sum(rate(tidb_distsql_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -6168,7 +6168,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_distsql_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (copr_type)", + "expr": "sum(rate(tidb_distsql_handle_query_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (copr_type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{copr_type}}", @@ -6261,7 +6261,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_distsql_scan_keys_partial_num_count{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tidb_distsql_scan_keys_partial_num_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "", @@ -6350,21 +6350,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_distsql_scan_keys_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(1, sum(rate(tidb_distsql_scan_keys_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "100", "refId": "A" }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_distsql_scan_keys_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_distsql_scan_keys_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", "refId": "B" }, { - "expr": "histogram_quantile(0.50, sum(rate(tidb_distsql_scan_keys_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.50, sum(rate(tidb_distsql_scan_keys_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "50", @@ -6450,21 +6450,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_distsql_scan_keys_partial_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(1, sum(rate(tidb_distsql_scan_keys_partial_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "100", "refId": "A" }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_distsql_scan_keys_partial_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_distsql_scan_keys_partial_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", "refId": "B" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_distsql_scan_keys_partial_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_distsql_scan_keys_partial_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "50", @@ -6550,21 +6550,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_distsql_partial_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(1, sum(rate(tidb_distsql_partial_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "100", "refId": "A" }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_distsql_partial_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_distsql_partial_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", "refId": "B" }, { - "expr": "histogram_quantile(0.50, sum(rate(tidb_distsql_partial_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.50, sum(rate(tidb_distsql_partial_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "50", @@ -6657,7 +6657,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_distsql_copr_cache_sum{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_distsql_copr_cache_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -6752,7 +6752,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_cop_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_cop_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -6862,7 +6862,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_backoff_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_backoff_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", @@ -6870,14 +6870,14 @@ "step": 40 }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_backoff_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_backoff_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_tikvclient_backoff_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_tikvclient_backoff_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80", @@ -6972,7 +6972,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_region_err_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_region_err_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -6981,7 +6981,7 @@ "step": 40 }, { - "expr": "sum(rate(tidb_tikvclient_region_err_total{tidb_cluster=\"$tidb_cluster\"}{EXTERNAL_LABELtype=\"server_is_busy\"}[1m]))", + "expr": "sum(rate(tidb_tikvclient_region_err_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}{EXTERNAL_LABELtype=\"server_is_busy\"}[1m]))", "format": "time_series", "hide": true, "intervalFactor": 2, @@ -7073,7 +7073,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_backoff_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_backoff_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -7166,7 +7166,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_lock_resolver_actions_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_lock_resolver_actions_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -7262,7 +7262,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_lock_cleanup_task_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_lock_cleanup_task_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "cleanup_secondary_failure_{{type}}", @@ -7271,7 +7271,7 @@ "step": 40 }, { - "expr": "sum(rate(tidb_tikvclient_load_safepoint_total{tidb_cluster=\"$tidb_cluster\", type=\"fail\"}[1m]))", + "expr": "sum(rate(tidb_tikvclient_load_safepoint_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"fail\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "load_safepoint_failure", @@ -7367,7 +7367,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_replica_selector_failure_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_replica_selector_failure_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{type}}", @@ -7474,7 +7474,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_request_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance, type)", + "expr": "sum(rate(tidb_tikvclient_request_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance, type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}-{{type}}", @@ -7569,7 +7569,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{tidb_cluster=\"$tidb_cluster\", store!=\"0\"}[1m])) by (le, store))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", store!=\"0\"}[1m])) by (le, store))", "format": "time_series", "intervalFactor": 2, "legendFormat": "store-{{store}}", @@ -7664,7 +7664,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{tidb_cluster=\"$tidb_cluster\", store!=\"0\"}[1m])) by (le,type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", store!=\"0\"}[1m])) by (le,type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -7757,7 +7757,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_forward_request_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (from_store, to_store, result)", + "expr": "sum(rate(tidb_tikvclient_forward_request_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (from_store, to_store, result)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{from_store}}-to-{{to_store}}-{{result}}", @@ -7850,7 +7850,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_forward_request_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, result)", + "expr": "sum(rate(tidb_tikvclient_forward_request_counter{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, result)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-{{result}}", @@ -7958,7 +7958,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(pd_client_cmd_handle_cmds_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", type!=\"tso\"}[1m])) by (type)", + "expr": "sum(rate(pd_client_cmd_handle_cmds_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type!=\"tso\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -8051,7 +8051,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999-{{type}}", @@ -8059,14 +8059,14 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99-{{type}}", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_tikvclient_ts_future_wait_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type!~\"tso|tso_async_wait\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90-{{type}}", @@ -8158,7 +8158,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(pd_client_cmd_handle_failed_cmds_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(pd_client_cmd_handle_failed_cmds_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -8249,14 +8249,14 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(pd_client_cmd_handle_cmds_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m]))", + "expr": "sum(rate(pd_client_cmd_handle_cmds_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "cmd", "refId": "C" }, { - "expr": "sum(rate(pd_client_request_handle_requests_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m]))", + "expr": "sum(rate(pd_client_request_handle_requests_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "request", @@ -8346,7 +8346,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", @@ -8354,14 +8354,14 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", @@ -8451,7 +8451,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", @@ -8459,14 +8459,14 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", @@ -8556,7 +8556,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso_async_wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso_async_wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", @@ -8564,14 +8564,14 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso_async_wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso_async_wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso_async_wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso_async_wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", @@ -8768,7 +8768,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_domain_load_schema_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_domain_load_schema_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -8867,7 +8867,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_domain_load_schema_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance,type)", + "expr": "sum(rate(tidb_domain_load_schema_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance,type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}-{{type}}", @@ -8964,7 +8964,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_session_schema_lease_error_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", + "expr": "sum(increase(tidb_session_schema_lease_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -9063,7 +9063,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_domain_load_privilege_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance,type)", + "expr": "sum(rate(tidb_domain_load_privilege_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance,type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}-{{type}}", @@ -9172,7 +9172,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(tidb_ddl_handle_job_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_ddl_handle_job_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -9264,7 +9264,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_ddl_batch_add_idx_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(1, sum(rate(tidb_ddl_batch_add_idx_duration_seconds_bucket{k8s_cluster=\"k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -9361,7 +9361,7 @@ "steppedLine": false, "targets": [ { - "expr": "tidb_ddl_waiting_jobs{tidb_cluster=\"$tidb_cluster\"}", + "expr": "tidb_ddl_waiting_jobs{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}-{{type}}", @@ -9449,7 +9449,7 @@ "steppedLine": false, "targets": [ { - "expr": "increase(tidb_ddl_worker_operation_total{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "increase(tidb_ddl_worker_operation_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}-{{type}}", @@ -9539,7 +9539,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(increase(tidb_ddl_worker_operation_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type, action, result))", + "expr": "histogram_quantile(0.99, sum(increase(tidb_ddl_worker_operation_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type, action, result))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-{{action}}-{{result}}", @@ -9627,7 +9627,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_ddl_deploy_syncer_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[2m])) by (le, type, result))", + "expr": "histogram_quantile(1, sum(rate(tidb_ddl_deploy_syncer_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[2m])) by (le, type, result))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-{{result}}", @@ -9715,7 +9715,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_ddl_owner_handle_syncer_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[2m])) by (le, type, result))", + "expr": "histogram_quantile(1, sum(rate(tidb_ddl_owner_handle_syncer_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[2m])) by (le, type, result))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-{{result}}", @@ -9803,7 +9803,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_ddl_update_self_ver_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[2m])) by (le, result))", + "expr": "histogram_quantile(1, sum(rate(tidb_ddl_update_self_ver_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[2m])) by (le, result))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{result}}", @@ -9889,14 +9889,14 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_ddl_handle_job_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_ddl_handle_job_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{ type }}", "refId": "A" }, { - "expr": "sum(rate(tidb_ddl_handle_job_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tidb_ddl_handle_job_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "intervalFactor": 1, "legendFormat": "total", @@ -9985,14 +9985,14 @@ "steppedLine": false, "targets": [ { - "expr": "irate(tidb_ddl_add_index_percentage_progress{tidb_cluster=\"$tidb_cluster\", type=\"add_index\"}[1m])", + "expr": "irate(tidb_ddl_add_index_percentage_progress{k8s_cluster=\"k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"add_index\"}[1m])", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}", "refId": "A" }, { - "expr": "irate(tidb_ddl_add_index_percentage_progress{tidb_cluster=\"$tidb_cluster\", type=\"modify_column\"}[1m])", + "expr": "irate(tidb_ddl_add_index_percentage_progress{k8s_cluster=\"k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"modify_column\"}[1m])", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}", @@ -10093,7 +10093,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(tidb_statistics_auto_analyze_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_statistics_auto_analyze_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "auto analyze duration", @@ -10180,7 +10180,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_statistics_auto_analyze_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_statistics_auto_analyze_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -10267,7 +10267,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_statistics_stats_inaccuracy_rate_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_statistics_stats_inaccuracy_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", @@ -10275,14 +10275,14 @@ "step": 30 }, { - "expr": "histogram_quantile(0.90, sum(rate(tidb_statistics_stats_inaccuracy_rate_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(tidb_statistics_stats_inaccuracy_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", "refId": "B" }, { - "expr": "histogram_quantile(0.50, sum(rate(tidb_statistics_stats_inaccuracy_rate_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.50, sum(rate(tidb_statistics_stats_inaccuracy_rate_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "50", @@ -10368,7 +10368,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_statistics_pseudo_estimation_total{tidb_cluster=\"$tidb_cluster\"}[30s])) by (type)", + "expr": "sum(rate(tidb_statistics_pseudo_estimation_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -10455,7 +10455,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_statistics_dump_feedback_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_statistics_dump_feedback_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -10542,7 +10542,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_statistics_store_query_feedback_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type) ", + "expr": "sum(rate(tidb_statistics_store_query_feedback_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type) ", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -10629,7 +10629,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_statistics_high_error_rate_feedback_total{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tidb_statistics_high_error_rate_feedback_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -10716,7 +10716,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_statistics_update_stats_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_statistics_update_stats_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -10810,7 +10810,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_statistics_fast_analyze_status_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(1, sum(rate(tidb_statistics_fast_analyze_status_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -10908,7 +10908,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tidb_statistics_sync_load_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_statistics_sync_load_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -10918,7 +10918,7 @@ }, { "exemplar": true, - "expr": "sum(rate(tidb_statistics_sync_load_timeout_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_statistics_sync_load_timeout_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "hide": false, "interval": "", @@ -11017,7 +11017,7 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.95, sum(rate(tidb_statistics_sync_load_latency_millis_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_statistics_sync_load_latency_millis_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -11027,7 +11027,7 @@ }, { "exemplar": true, - "expr": "histogram_quantile(0.95, sum(rate(tidb_statistics_read_stats_latency_millis_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_statistics_read_stats_latency_millis_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "hide": false, "interval": "", @@ -11136,7 +11136,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(tidb_owner_new_session_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance, result))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_owner_new_session_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance, result))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}-{{result}}", @@ -11224,7 +11224,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_owner_watch_owner_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, result, instance)", + "expr": "sum(rate(tidb_owner_watch_owner_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, result, instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-{{result}}-{{instance}}", @@ -11327,7 +11327,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_autoid_operation_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tidb_autoid_operation_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "AutoID QPS", @@ -11415,14 +11415,14 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_autoid_operation_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_autoid_operation_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99-{{type}}", "refId": "B" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_autoid_operation_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_autoid_operation_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80-{{type}}", @@ -11508,7 +11508,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_region_cache_operations_total{tidb_cluster=\"$tidb_cluster\", result=\"err\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_region_cache_operations_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", result=\"err\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -11597,7 +11597,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_meta_operation_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_meta_operation_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -11700,7 +11700,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_tikvclient_gc_worker_actions_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(increase(tidb_tikvclient_gc_worker_actions_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -11789,7 +11789,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_gc_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_gc_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", @@ -11877,7 +11877,7 @@ "steppedLine": false, "targets": [ { - "expr": "max(tidb_tikvclient_gc_config{tidb_cluster=\"$tidb_cluster\"}) by (type)", + "expr": "max(tidb_tikvclient_gc_config{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -11963,7 +11963,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_tikvclient_gc_failure{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(increase(tidb_tikvclient_gc_failure{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -12051,7 +12051,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_tikvclient_gc_unsafe_destroy_range_failures{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(increase(tidb_tikvclient_gc_unsafe_destroy_range_failures{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -12137,7 +12137,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_tikvclient_gc_region_too_many_locks{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(increase(tidb_tikvclient_gc_region_too_many_locks{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "Locks Error OPM", @@ -12223,7 +12223,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_tikvclient_gc_action_result{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(increase(tidb_tikvclient_gc_action_result{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -12320,7 +12320,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tidb_tikvclient_range_task_stats{tidb_cluster=\"$tidb_cluster\"}) by (type, result)", + "expr": "sum(tidb_tikvclient_range_task_stats{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) by (type, result)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-{{result}}", @@ -12413,7 +12413,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_range_task_push_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_range_task_push_duration_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -12516,7 +12516,7 @@ "steppedLine": false, "targets": [ { - "expr": "delta(tidb_tikvclient_batch_client_no_available_connection_total{tidb_cluster=\"$tidb_cluster\"}[30s])", + "expr": "delta(tidb_tikvclient_batch_client_no_available_connection_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}", @@ -12607,7 +12607,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_batch_client_unavailable_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[30s])) by (le, instance))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_batch_client_unavailable_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (le, instance))", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -12701,7 +12701,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.9999, sum(rate(tidb_tikvclient_batch_client_wait_connection_establish_bucket{tidb_cluster=\"$tidb_cluster\"}[30s])) by (le, instance))", + "expr": "histogram_quantile(0.9999, sum(rate(tidb_tikvclient_batch_client_wait_connection_establish_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (le, instance))", "format": "time_series", "intervalFactor": 1, "legendFormat": "{{instance}}", @@ -12802,7 +12802,7 @@ "targets": [ { "exemplar": true, - "expr": "rate(tidb_tikvclient_batch_recv_latency_sum{tidb_cluster=\"$tidb_cluster\"}[1m]) / rate(tidb_tikvclient_batch_recv_latency_count{tidb_cluster=\"$tidb_cluster\"}[1m])", + "expr": "rate(tidb_tikvclient_batch_recv_latency_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]) / rate(tidb_tikvclient_batch_recv_latency_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])", "hide": false, "interval": "", "legendFormat": "{{instance}}-{{result}}", @@ -12810,7 +12810,7 @@ }, { "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_recv_latency_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_recv_latency_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "hide": true, "interval": "", @@ -12883,6 +12883,26 @@ "tags": [], "templating": { "list": [ + { + "allValue": null, + "current": {}, + "datasource": "${DS_TEST-CLUSTER}", + "hide": 2, + "includeAll": false, + "label": "K8s-cluster", + "multi": false, + "name": "k8s_cluster", + "options": [], + "query": "label_values(pd_cluster_status, k8s_cluster)", + "refresh": 2, + "regex": "", + "sort": 1, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, { "allValue": null, "current": {}, @@ -12893,7 +12913,7 @@ "multi": false, "name": "tidb_cluster", "options": [], - "query": "label_values(pd_cluster_status, tidb_cluster)", + "query": "label_values(pd_cluster_status{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", "refresh": 2, "regex": "", "sort": 1, diff --git a/metrics/grafana/tidb_runtime.json b/metrics/grafana/tidb_runtime.json index 63136f16d0b02..fefd9d2f5b15e 100644 --- a/metrics/grafana/tidb_runtime.json +++ b/metrics/grafana/tidb_runtime.json @@ -134,7 +134,7 @@ "steppedLine": false, "targets": [ { - "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "process_resident_memory_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -142,7 +142,7 @@ "refId": "A" }, { - "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} / (1 + tidb_server_gogc{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} / 100)", + "expr": "go_memstats_next_gc_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} / (1 + tidb_server_gogc{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} / 100)", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -150,7 +150,7 @@ "refId": "H" }, { - "expr": "go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} - go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} / (1 + tidb_server_gogc{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} / 100)", + "expr": "go_memstats_heap_alloc_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} - go_memstats_next_gc_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} / (1 + tidb_server_gogc{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} / 100)", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -158,7 +158,7 @@ "refId": "C" }, { - "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} - go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "go_memstats_heap_idle_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} - go_memstats_heap_released_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_heap_inuse_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} - go_memstats_heap_alloc_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -166,7 +166,7 @@ "refId": "B" }, { - "expr": "go_memstats_stack_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_mspan_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_mcache_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_buck_hash_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_gc_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_other_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "go_memstats_stack_sys_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_mspan_sys_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_mcache_sys_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_buck_hash_sys_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_gc_sys_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} + go_memstats_other_sys_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -174,7 +174,7 @@ "refId": "D" }, { - "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "go_memstats_next_gc_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -182,7 +182,7 @@ "refId": "E" }, { - "expr": "(clamp_max(idelta(go_memstats_last_gc_time_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]), 1) * go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) > 0", + "expr": "(clamp_max(idelta(go_memstats_last_gc_time_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m]), 1) * go_memstats_next_gc_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) > 0", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -287,7 +287,7 @@ "steppedLine": false, "targets": [ { - "expr": "irate(process_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])", + "expr": "irate(process_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])", "format": "time_series", "hide": false, "interval": "", @@ -297,14 +297,14 @@ "step": 40 }, { - "expr": "(idelta((go_memstats_gc_cpu_fraction{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} * (go_memstats_last_gc_time_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} - process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) * tidb_server_maxprocs{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"})[30s:]) > 0) / 15", + "expr": "(idelta((go_memstats_gc_cpu_fraction{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} * (go_memstats_last_gc_time_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} - process_start_time_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) * tidb_server_maxprocs{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"})[30s:]) > 0) / 15", "format": "time_series", "intervalFactor": 1, "legendFormat": "gc-cpu", "refId": "C" }, { - "expr": "tidb_server_maxprocs{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "tidb_server_maxprocs{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "interval": "", "intervalFactor": 1, @@ -405,7 +405,7 @@ "steppedLine": false, "targets": [ { - "expr": "go_memstats_heap_objects{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "go_memstats_heap_objects{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, @@ -502,7 +502,7 @@ "steppedLine": false, "targets": [ { - "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", quantile=\"0\"}", + "expr": "go_gc_duration_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", quantile=\"0\"}", "format": "time_series", "hide": false, "instant": false, @@ -512,7 +512,7 @@ "step": 40 }, { - "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", quantile!~\"0|1\"}", + "expr": "go_gc_duration_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", quantile!~\"0|1\"}", "format": "time_series", "instant": false, "intervalFactor": 1, @@ -520,7 +520,7 @@ "refId": "B" }, { - "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", quantile=\"1\"}", + "expr": "go_gc_duration_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", quantile=\"1\"}", "format": "time_series", "instant": false, "intervalFactor": 1, @@ -624,28 +624,28 @@ "steppedLine": false, "targets": [ { - "expr": "irate(go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])", + "expr": "irate(go_memstats_alloc_bytes_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])", "format": "time_series", "intervalFactor": 1, "legendFormat": "alloc", "refId": "A" }, { - "expr": "irate((go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"})[30s:])", + "expr": "irate((go_memstats_alloc_bytes_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"} - go_memstats_heap_alloc_bytes{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"})[30s:])", "format": "time_series", "intervalFactor": 1, "legendFormat": "sweep", "refId": "B" }, { - "expr": "irate(go_memstats_mallocs_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])", + "expr": "irate(go_memstats_mallocs_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])", "format": "time_series", "intervalFactor": 1, "legendFormat": "alloc-ops", "refId": "C" }, { - "expr": "irate(go_memstats_frees_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])", + "expr": "irate(go_memstats_frees_total{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])", "format": "time_series", "intervalFactor": 1, "legendFormat": "swepp-ops", @@ -739,14 +739,14 @@ "steppedLine": false, "targets": [ { - "expr": " go_goroutines{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": " go_goroutines{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "intervalFactor": 1, "legendFormat": "goroutines", "refId": "A" }, { - "expr": "go_threads{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "expr": "go_threads{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "format": "time_series", "intervalFactor": 1, "legendFormat": "threads", @@ -833,14 +833,14 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{tidb_cluster=\"$tidb_cluster\", store!=\"0\",instance=~\"$instance\"}[30s])) by (le, store))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", store!=\"0\",instance=~\"$instance\"}[30s])) by (le, store))", "format": "time_series", "intervalFactor": 1, "legendFormat": "tidb-to-store{{store}}", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(tikv_grpc_msg_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!=\"kv_gc\"}[30s])) by (le, instance))", + "expr": "histogram_quantile(0.99, sum(rate(tikv_grpc_msg_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", type!=\"kv_gc\"}[30s])) by (le, instance))", "format": "time_series", "intervalFactor": 1, "legendFormat": "tikv-{{instance}}-side", @@ -926,14 +926,14 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[30s])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[30s])) by (le))", "format": "time_series", "intervalFactor": 1, "legendFormat": "tidb-side", "refId": "A" }, { - "expr": "histogram_quantile(0.99, sum(rate(pd_server_handle_tso_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[30s])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_server_handle_tso_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\"}[30s])) by (le))", "format": "time_series", "intervalFactor": 1, "legendFormat": "pd-side", @@ -1013,7 +1013,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "sum(increase(tidb_tikvclient_batch_requests_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le)", + "expr": "sum(increase(tidb_tikvclient_batch_requests_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le)", "format": "heatmap", "intervalFactor": 1, "legendFormat": "{{le}}", @@ -1079,7 +1079,7 @@ "reverseYBuckets": false, "targets": [ { - "expr": "sum(increase(tidb_tikvclient_batch_pending_requests_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le)", + "expr": "sum(increase(tidb_tikvclient_batch_pending_requests_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le)", "format": "heatmap", "intervalFactor": 1, "legendFormat": "{{le}}", @@ -1156,7 +1156,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_batch_wait_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_batch_wait_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", "format": "time_series", "intervalFactor": 1, "legendFormat": "999", @@ -1164,7 +1164,7 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_wait_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_wait_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", "format": "time_series", "intervalFactor": 1, "legendFormat": "99", @@ -1172,7 +1172,7 @@ "step": 10 }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_batch_wait_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_tikvclient_batch_wait_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", "format": "time_series", "intervalFactor": 1, "legendFormat": "95", @@ -1265,7 +1265,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.9999, sum(rate(tidb_tikvclient_batch_send_latency_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", + "expr": "histogram_quantile(0.9999, sum(rate(tidb_tikvclient_batch_send_latency_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", "format": "time_series", "intervalFactor": 1, "legendFormat": "9999", @@ -1273,7 +1273,7 @@ "step": 10 }, { - "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_batch_send_latency_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", + "expr": "histogram_quantile(0.999, sum(rate(tidb_tikvclient_batch_send_latency_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", "format": "time_series", "intervalFactor": 1, "legendFormat": "999", @@ -1281,7 +1281,7 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_send_latency_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_send_latency_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s])) by (le, instance))", "format": "time_series", "intervalFactor": 1, "legendFormat": "99", @@ -1345,25 +1345,40 @@ "list": [ { "allValue": null, - "current": { - }, + "current": {}, + "datasource": "${DS_TEST-CLUSTER}", + "hide": 2, + "includeAll": false, + "label": "K8s-cluster", + "multi": false, + "name": "k8s_cluster", + "options": [], + "query": "label_values(pd_cluster_status, k8s_cluster)", + "refresh": 2, + "regex": "", + "sort": 1, + "tagValuesQuery": "", + "tags": [], + "tagsQuery": "", + "type": "query", + "useTags": false + }, + { + "allValue": null, + "current": {}, "datasource": "${DS_TEST-CLUSTER}", "hide": 2, "includeAll": false, "label": "tidb_cluster", "multi": false, "name": "tidb_cluster", - "options": [ - - ], - "query": "label_values(pd_cluster_status, tidb_cluster)", + "options": [], + "query": "label_values(pd_cluster_status{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", "refresh": 2, "regex": "", "sort": 1, "tagValuesQuery": "", - "tags": [ - - ], + "tags": [], "tagsQuery": "", "type": "query", "useTags": false @@ -1372,14 +1387,14 @@ "allValue": null, "current": {}, "datasource": "${DS_TEST-CLUSTER}", - "definition": "label_values(process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}, instance)", + "definition": "label_values(process_start_time_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}, instance)", "hide": 0, "includeAll": true, "label": "instance", "multi": false, "name": "instance", "options": [], - "query": "label_values(process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}, instance)", + "query": "label_values(process_start_time_seconds{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}, instance)", "refresh": 1, "regex": "", "skipUrlSync": false, diff --git a/metrics/grafana/tidb_summary.json b/metrics/grafana/tidb_summary.json index 0d12628605f2e..6ce7a10087973 100644 --- a/metrics/grafana/tidb_summary.json +++ b/metrics/grafana/tidb_summary.json @@ -109,7 +109,7 @@ "steppedLine": false, "targets": [ { - "expr": "(time() - process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"})", + "expr": "(time() - process_start_time_seconds{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"})", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -209,7 +209,7 @@ "steppedLine": false, "targets": [ { - "expr": "tidb_server_connections{tidb_cluster=\"$tidb_cluster\"}", + "expr": "tidb_server_connections{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -217,7 +217,7 @@ "step": 40 }, { - "expr": "sum(tidb_server_connections{tidb_cluster=\"$tidb_cluster\"})", + "expr": "sum(tidb_server_connections{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"})", "format": "time_series", "intervalFactor": 2, "legendFormat": "total", @@ -317,7 +317,7 @@ "steppedLine": false, "targets": [ { - "expr": "rate(process_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}[1m])", + "expr": "rate(process_cpu_seconds_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}[1m])", "format": "time_series", "hide": false, "intervalFactor": 2, @@ -415,14 +415,14 @@ "steppedLine": false, "targets": [ { - "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "process_resident_memory_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "intervalFactor": 1, "legendFormat": "process-{{instance}}", "refId": "A" }, { - "expr": "go_memstats_heap_sys_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_sys_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "hide": true, "intervalFactor": 1, @@ -430,14 +430,14 @@ "refId": "B" }, { - "expr": "go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_inuse_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "intervalFactor": 1, "legendFormat": "HeapInuse-{{instance}}", "refId": "C" }, { - "expr": "go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_alloc_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "hide": true, "intervalFactor": 1, @@ -445,7 +445,7 @@ "refId": "D" }, { - "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_idle_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "format": "time_series", "hide": true, "intervalFactor": 1, @@ -453,7 +453,7 @@ "refId": "E" }, { - "expr": "go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", + "expr": "go_memstats_heap_released_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", job=\"tidb\"}", "hide": true, "interval": "", "legendFormat": "HeapReleased-{{instance}}", @@ -558,21 +558,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_server_handle_query_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "95", "refId": "C" }, { - "expr": "sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[30s])) / sum(rate(tidb_server_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[30s]))", + "expr": "sum(rate(tidb_server_handle_query_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[30s])) / sum(rate(tidb_server_handle_query_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type!=\"internal\"}[30s]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "avg", @@ -664,7 +664,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(tidb_server_execute_error_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, instance)", + "expr": "sum(increase(tidb_server_execute_error_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": " {{type}}-{{instance}}", @@ -762,7 +762,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (result)", + "expr": "sum(rate(tidb_server_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (result)", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -771,7 +771,7 @@ "step": 60 }, { - "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\", result=\"OK\"}[1m] offset 1d))", + "expr": "sum(rate(tidb_server_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", result=\"OK\"}[1m] offset 1d))", "format": "time_series", "hide": true, "instant": false, @@ -781,7 +781,7 @@ "step": 90 }, { - "expr": "sum(tidb_server_connections{tidb_cluster=\"$tidb_cluster\"}) * sum(rate(tidb_server_handle_query_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_server_handle_query_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(tidb_server_connections{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}) * sum(rate(tidb_server_handle_query_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_server_handle_query_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "hide": true, "instant": false, @@ -888,7 +888,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", + "expr": "sum(rate(tidb_server_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}} ", @@ -987,7 +987,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_executor_statement_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_executor_statement_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -995,7 +995,7 @@ "step": 30 }, { - "expr": "sum(rate(tidb_executor_statement_total{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tidb_executor_statement_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "intervalFactor": 1, "legendFormat": "total", @@ -1089,7 +1089,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_server_query_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_server_query_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": " {{type}}", @@ -1201,7 +1201,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_session_parse_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_session_parse_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -1210,7 +1210,7 @@ "step": 30 }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_parse_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_parse_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 1, "legendFormat": "95", @@ -1309,7 +1309,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_session_compile_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_session_compile_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -1318,7 +1318,7 @@ "step": 30 }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_compile_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_compile_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 1, "legendFormat": "95", @@ -1417,7 +1417,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_session_execute_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_session_execute_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", "format": "time_series", "instant": false, "intervalFactor": 2, @@ -1426,7 +1426,7 @@ "step": 30 }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_execute_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_execute_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 1, "legendFormat": "95", @@ -1524,7 +1524,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_server_plan_cache_total{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_server_plan_cache_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -1632,7 +1632,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_session_transaction_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, txn_mode)", + "expr": "sum(rate(tidb_session_transaction_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type, txn_mode)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}-{{txn_mode}}", @@ -1723,21 +1723,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_session_transaction_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99-{{txn_mode}}", "refId": "A" }, { - "expr": "histogram_quantile(0.95, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", + "expr": "histogram_quantile(0.95, sum(rate(tidb_session_transaction_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", "format": "time_series", "intervalFactor": 2, "legendFormat": "95-{{txn_mode}}", "refId": "B" }, { - "expr": "histogram_quantile(0.80, sum(rate(tidb_session_transaction_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", + "expr": "histogram_quantile(0.80, sum(rate(tidb_session_transaction_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", sql_type=\"general\"}[1m])) by (le, txn_mode))", "format": "time_series", "intervalFactor": 2, "legendFormat": "80-{{txn_mode}}", @@ -1825,7 +1825,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_session_transaction_statement_num_bucket{tidb_cluster=\"$tidb_cluster\"}[30s])) by (le))", + "expr": "histogram_quantile(1, sum(rate(tidb_session_transaction_statement_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "max", @@ -1917,7 +1917,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1.0, sum(rate(tidb_session_retry_num_bucket{tidb_cluster=\"$tidb_cluster\"}[30s])) by (le))", + "expr": "histogram_quantile(1.0, sum(rate(tidb_session_retry_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[30s])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "max", @@ -2027,7 +2027,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!=\"GC\"}[1m])) by (le, store))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type!=\"GC\"}[1m])) by (le, store))", "format": "time_series", "intervalFactor": 2, "legendFormat": "store-{{store}}", @@ -2122,7 +2122,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type!=\"GC\"}[1m])) by (le,type))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_request_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type!=\"GC\"}[1m])) by (le,type))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -2215,7 +2215,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_request_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "sum(rate(tidb_tikvclient_request_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{type}}", @@ -2309,7 +2309,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", + "expr": "sum(rate(tidb_tikvclient_txn_cmd_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2405,7 +2405,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_tikvclient_txn_write_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(1, sum(rate(tidb_tikvclient_txn_write_size_bytes_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2500,7 +2500,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_tikvclient_txn_write_kv_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(1, sum(rate(tidb_tikvclient_txn_write_kv_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2594,7 +2594,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tidb_tikvclient_txn_regions_num_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", + "expr": "histogram_quantile(1, sum(rate(tidb_tikvclient_txn_regions_num_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2686,7 +2686,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", @@ -2694,14 +2694,14 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(pd_client_cmd_handle_cmds_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"wait\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", @@ -2791,7 +2791,7 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", + "expr": "histogram_quantile(0.999, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "999", @@ -2799,14 +2799,14 @@ "step": 10 }, { - "expr": "histogram_quantile(0.99, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "histogram_quantile(0.90, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", + "expr": "histogram_quantile(0.90, sum(rate(pd_client_request_handle_requests_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", type=\"tso\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "90", @@ -2895,7 +2895,7 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tidb_autoid_operation_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", + "expr": "sum(rate(tidb_autoid_operation_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", @@ -2984,14 +2984,14 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tidb_autoid_operation_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", + "expr": "histogram_quantile(0.99, sum(rate(tidb_autoid_operation_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (le))", "format": "time_series", "intervalFactor": 2, "legendFormat": "99", "refId": "B" }, { - "expr": "sum(rate(tidb_autoid_operation_duration_seconds_sum{tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_autoid_operation_duration_seconds_count{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "expr": "sum(rate(tidb_autoid_operation_duration_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) / sum(rate(tidb_autoid_operation_duration_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m]))", "format": "time_series", "intervalFactor": 2, "legendFormat": "avg", @@ -3051,6 +3051,31 @@ "tags": [], "templating": { "list": [ + { + "allValue": null, + "current": { + }, + "datasource": "${DS_TEST-CLUSTER}", + "hide": 2, + "includeAll": false, + "label": "K8s-cluster", + "multi": false, + "name": "k8s_cluster", + "options": [ + + ], + "query": "label_values(pd_cluster_status, k8s_cluster)", + "refresh": 2, + "regex": "", + "sort": 1, + "tagValuesQuery": "", + "tags": [ + + ], + "tagsQuery": "", + "type": "query", + "useTags": false + }, { "allValue": null, "current": { @@ -3064,7 +3089,7 @@ "options": [ ], - "query": "label_values(pd_cluster_status, tidb_cluster)", + "query": "label_values(pd_cluster_status{k8s_cluster=\"$k8s_cluster\"}, tidb_cluster)", "refresh": 2, "regex": "", "sort": 1, From 52ba43507cee5259e557a05a8bb31d5e8852c7dc Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 6 May 2022 22:08:56 +0800 Subject: [PATCH 33/51] server: set 'clientConn.ctx = nil' to clean the context when changeUser. (#33703) (#33821) close pingcap/tidb#33699 --- server/conn.go | 1 + server/conn_test.go | 119 ++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 120 insertions(+) diff --git a/server/conn.go b/server/conn.go index 4511f40978566..267150ab7a63d 100644 --- a/server/conn.go +++ b/server/conn.go @@ -2321,6 +2321,7 @@ func (cc *clientConn) handleChangeUser(ctx context.Context, data []byte) error { if err := cc.ctx.Close(); err != nil { logutil.Logger(ctx).Debug("close old context failed", zap.Error(err)) } + cc.ctx = nil if err := cc.openSessionAndDoAuth(pass, ""); err != nil { return err } diff --git a/server/conn_test.go b/server/conn_test.go index a599ccee48702..438a1c4c372db 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -43,6 +43,125 @@ import ( "github.com/tikv/client-go/v2/testutils" ) +type Issue33699CheckType struct { + name string + defVal string + setVal string + isSessionVariable bool +} + +func (c *Issue33699CheckType) toSetSessionVar() string { + if c.isSessionVariable { + return fmt.Sprintf("set session %s=%s", c.name, c.setVal) + } + return fmt.Sprintf("set @%s=%s", c.name, c.setVal) +} + +func (c *Issue33699CheckType) toGetSessionVar() string { + if c.isSessionVariable { + return fmt.Sprintf("select @@session.%s", c.name) + } + return fmt.Sprintf("select @%s", c.name) +} + +func TestIssue33699(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + var outBuffer bytes.Buffer + tidbdrv := NewTiDBDriver(store) + cfg := newTestConfig() + cfg.Port, cfg.Status.StatusPort = 0, 0 + cfg.Status.ReportStatus = false + server, err := NewServer(cfg, tidbdrv) + require.NoError(t, err) + defer server.Close() + + cc := &clientConn{ + connectionID: 1, + salt: []byte{0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0A, 0x0B, 0x0C, 0x0D, 0x0E, 0x0F, 0x10, 0x11, 0x12, 0x13, 0x14}, + server: server, + pkt: &packetIO{ + bufWriter: bufio.NewWriter(&outBuffer), + }, + collation: mysql.DefaultCollationID, + peerHost: "localhost", + alloc: arena.NewAllocator(512), + chunkAlloc: chunk.NewAllocator(), + capability: mysql.ClientProtocol41, + } + + tk := testkit.NewTestKit(t, store) + ctx := &TiDBContext{Session: tk.Session()} + cc.ctx = ctx + + // change user. + doChangeUser := func() { + userData := append([]byte("root"), 0x0, 0x0) + userData = append(userData, []byte("test")...) + userData = append(userData, 0x0) + changeUserReq := dispatchInput{ + com: mysql.ComChangeUser, + in: userData, + err: nil, + out: []byte{0x7, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x2, 0x0, 0x0, 0x0}, + } + inBytes := append([]byte{changeUserReq.com}, changeUserReq.in...) + err = cc.dispatch(context.Background(), inBytes) + require.Equal(t, changeUserReq.err, err) + if err == nil { + err = cc.flush(context.TODO()) + require.NoError(t, err) + require.Equal(t, changeUserReq.out, outBuffer.Bytes()) + } else { + _ = cc.flush(context.TODO()) + } + outBuffer.Reset() + } + // check variable. + checks := []Issue33699CheckType{ + { // self define. + "a", + "", + "1", + false, + }, + { // session variable + "net_read_timeout", + "30", + "1234", + true, + }, + { + "net_write_timeout", + "60", + "1234", + true, + }, + } + + // default; + for _, ck := range checks { + tk.MustQuery(ck.toGetSessionVar()).Check(testkit.Rows(ck.defVal)) + } + // set; + for _, ck := range checks { + tk.MustExec(ck.toSetSessionVar()) + } + // check after set. + for _, ck := range checks { + tk.MustQuery(ck.toGetSessionVar()).Check(testkit.Rows(ck.setVal)) + } + doChangeUser() + require.NotEqual(t, ctx, cc.ctx) + require.NotEqual(t, ctx.Session, cc.ctx.Session) + // new session,so values is defaults; + tk.SetSession(cc.ctx.Session) // set new session. + for _, ck := range checks { + tk.MustQuery(ck.toGetSessionVar()).Check(testkit.Rows(ck.defVal)) + } +} + func TestMalformHandshakeHeader(t *testing.T) { data := []byte{0x00} var p handshakeResponse41 From f6049408b6703ce7d1f3dc6783c58da4c95535ed Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 6 May 2022 22:40:56 +0800 Subject: [PATCH 34/51] *: Revert #33519 for performance fallback and fix the #33509 in another way (#34425) (#34433) close pingcap/tidb#33509 --- planner/core/cache.go | 1 + planner/core/common_plans.go | 1 + server/driver_tidb.go | 25 +++++++++++++++++++------ session/session.go | 1 + 4 files changed, 22 insertions(+), 6 deletions(-) diff --git a/planner/core/cache.go b/planner/core/cache.go index ca0a1c6340b44..0d01f2bb98f73 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -208,6 +208,7 @@ func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.Ta type CachedPrepareStmt struct { PreparedAst *ast.Prepared VisitInfos []visitInfo + ColumnInfos interface{} Executor interface{} NormalizedSQL string NormalizedPlan string diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 39d361ee4045c..e0a4063df6417 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -282,6 +282,7 @@ func (e *Execute) OptimizePreparedPlan(ctx context.Context, sctx sessionctx.Cont // schema version like prepared plan cache key prepared.CachedPlan = nil preparedObj.Executor = nil + preparedObj.ColumnInfos = nil // If the schema version has changed we need to preprocess it again, // if this time it failed, the real reason for the error is schema changed. // Example: diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 677298c3c7fc4..6f3221cd21388 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -80,7 +80,8 @@ func (ts *TiDBStatement) Execute(ctx context.Context, args []types.Datum) (rs Re return } rs = &tidbResultSet{ - recordSet: tidbRecordset, + recordSet: tidbRecordset, + preparedStmt: ts.ctx.GetSessionVars().PreparedStmts[ts.id].(*core.CachedPrepareStmt), } return } @@ -296,10 +297,11 @@ func (tc *TiDBContext) GetStmtStats() *stmtstats.StatementStats { } type tidbResultSet struct { - recordSet sqlexec.RecordSet - columns []*ColumnInfo - rows []chunk.Row - closed int32 + recordSet sqlexec.RecordSet + columns []*ColumnInfo + rows []chunk.Row + closed int32 + preparedStmt *core.CachedPrepareStmt } func (trs *tidbResultSet) NewChunk(alloc chunk.Allocator) *chunk.Chunk { @@ -341,12 +343,23 @@ func (trs *tidbResultSet) Columns() []*ColumnInfo { if trs.columns != nil { return trs.columns } - + // for prepare statement, try to get cached columnInfo array + if trs.preparedStmt != nil { + ps := trs.preparedStmt + if colInfos, ok := ps.ColumnInfos.([]*ColumnInfo); ok { + trs.columns = colInfos + } + } if trs.columns == nil { fields := trs.recordSet.Fields() for _, v := range fields { trs.columns = append(trs.columns, convertColumnInfo(v)) } + if trs.preparedStmt != nil { + // if ColumnInfo struct has allocated object, + // here maybe we need deep copy ColumnInfo to do caching + trs.preparedStmt.ColumnInfos = trs.columns + } } return trs.columns } diff --git a/session/session.go b/session/session.go index 41a26f5704a54..5a1c2ad164c62 100644 --- a/session/session.go +++ b/session/session.go @@ -2083,6 +2083,7 @@ func (s *session) IsCachedExecOk(ctx context.Context, preparedStmt *plannercore. is := s.GetInfoSchema().(infoschema.InfoSchema) if prepared.SchemaVersion != is.SchemaMetaVersion() { prepared.CachedPlan = nil + preparedStmt.ColumnInfos = nil return false, nil } } From be484475e76b3aaa2cad520111353c3327db35fa Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sat, 7 May 2022 08:06:31 +0800 Subject: [PATCH 35/51] server: openSession instead set nil when changeUser (#33894) (#34427) close pingcap/tidb#33892 --- server/conn.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/server/conn.go b/server/conn.go index 267150ab7a63d..a0ec8796ec610 100644 --- a/server/conn.go +++ b/server/conn.go @@ -2321,7 +2321,12 @@ func (cc *clientConn) handleChangeUser(ctx context.Context, data []byte) error { if err := cc.ctx.Close(); err != nil { logutil.Logger(ctx).Debug("close old context failed", zap.Error(err)) } - cc.ctx = nil + // session was closed by `ctx.Close` and should `openSession` explicitly to renew session. + // `openSession` won't run again in `openSessionAndDoAuth` because ctx is not nil. + err := cc.openSession() + if err != nil { + return err + } if err := cc.openSessionAndDoAuth(pass, ""); err != nil { return err } From 0adc6117828202a56a0cf45f717cd4283aed3cd2 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 30 May 2022 20:38:27 +0800 Subject: [PATCH 36/51] backup: retry on internal error and make more errors can be retried (#34352) (#34378) close pingcap/tidb#34350 --- br/pkg/backup/client.go | 27 +++++++++++++++++++++------ br/pkg/utils/retry.go | 1 + br/tests/br_full/run.sh | 4 ++-- 3 files changed, 24 insertions(+), 8 deletions(-) diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index e4584df29f83a..ad26ba4b2d5c3 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -934,9 +934,17 @@ backupLoop: }) bcli, err := client.Backup(ctx, &req) failpoint.Inject("reset-retryable-error", func(val failpoint.Value) { - if val.(bool) { - logutil.CL(ctx).Debug("failpoint reset-retryable-error injected.") - err = status.Error(codes.Unavailable, "Unavailable error") + switch val.(string) { + case "Unavaiable": + { + logutil.CL(ctx).Debug("failpoint reset-retryable-error unavailable injected.") + err = status.Error(codes.Unavailable, "Unavailable error") + } + case "Internal": + { + logutil.CL(ctx).Debug("failpoint reset-retryable-error internal injected.") + err = status.Error(codes.Internal, "Internal error") + } } }) failpoint.Inject("reset-not-retryable-error", func(val failpoint.Value) { @@ -1002,9 +1010,15 @@ const ( // isRetryableError represents whether we should retry reset grpc connection. func isRetryableError(err error) bool { - - if status.Code(err) == codes.Unavailable { - return true + // some errors can be retried + // https://github.com/pingcap/tidb/issues/34350 + switch status.Code(err) { + case codes.Unavailable, codes.DeadlineExceeded, + codes.ResourceExhausted, codes.Aborted, codes.Internal: + { + log.Warn("backup met some errors, these errors can be retry 5 times", zap.Error(err)) + return true + } } // At least, there are two possible cancel() call, @@ -1012,6 +1026,7 @@ func isRetryableError(err error) bool { if status.Code(err) == codes.Canceled { if s, ok := status.FromError(err); ok { if strings.Contains(s.Message(), gRPC_Cancel) { + log.Warn("backup met grpc cancel error, this errors can be retry 5 times", zap.Error(err)) return true } } diff --git a/br/pkg/utils/retry.go b/br/pkg/utils/retry.go index 51a833d8d136c..bda305aaf11ac 100644 --- a/br/pkg/utils/retry.go +++ b/br/pkg/utils/retry.go @@ -31,6 +31,7 @@ var retryableServerError = []string{ "body write aborted", "error during dispatch", "put object timeout", + "internalerror", } // RetryableFunc presents a retryable operation. diff --git a/br/tests/br_full/run.sh b/br/tests/br_full/run.sh index 0d15794788b26..21f2e2143a002 100755 --- a/br/tests/br_full/run.sh +++ b/br/tests/br_full/run.sh @@ -30,7 +30,7 @@ done # backup full and kill tikv to test reset connection echo "backup with limit start..." -export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/backup/reset-retryable-error=1*return(true)" +export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/backup/reset-retryable-error=1*return(\"Unavailable\")->1*return(\"Internal\")" run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB-limit" --concurrency 4 export GO_FAILPOINTS="" @@ -49,7 +49,7 @@ fi # backup full echo "backup with lz4 start..." -export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/backup/backup-storage-error=1*return(\"connection refused\")" +export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/backup/backup-storage-error=1*return(\"connection refused\")->1*return(\"InternalError\")" run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB-lz4" --concurrency 4 --compression lz4 export GO_FAILPOINTS="" size_lz4=$(du -d 0 $TEST_DIR/$DB-lz4 | awk '{print $1}') From 3b3cc60cefde8f59f72376082c639ab0f7136413 Mon Sep 17 00:00:00 2001 From: Zak Zhao <57036248+joccau@users.noreply.github.com> Date: Mon, 30 May 2022 20:52:26 +0800 Subject: [PATCH 37/51] br: update package tikv client-go (#34955) close pingcap/tidb#34956 --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 32a58f2eaf5e7..4866a22539b1f 100644 --- a/go.mod +++ b/go.mod @@ -70,7 +70,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.0-rc.0.20220425054530-a62124d9fa75 + github.com/tikv/client-go/v2 v2.0.0-rc.0.20220516035110-cf67df00ad4b github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 // indirect github.com/twmb/murmur3 v1.1.3 diff --git a/go.sum b/go.sum index f0c9c913b5b78..8f6a06adb1b30 100644 --- a/go.sum +++ b/go.sum @@ -789,8 +789,8 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfK github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20220425054530-a62124d9fa75 h1:f8/xvvOtyHx9XkO8CqMHPWU95IwzJVg2MdfytBe6QNI= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20220425054530-a62124d9fa75/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20220516035110-cf67df00ad4b h1:ED0XsC7KV6SA59OLaZDrTrkMrq4g4RTjtnjgXuL5WrY= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20220516035110-cf67df00ad4b/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae/go.mod h1:varH0IE0jJ9E9WN2Ei/N6pajMlPkcXdDEf7f5mmsUVQ= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee h1:rAAdvQ8Hh36syHr92g0VmZEpkH+40RGQBpFL2121xMs= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= From 4d888ce22b08b9d2bbc2b66890d35195f666e90b Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 31 May 2022 12:52:26 +0800 Subject: [PATCH 38/51] br: add error into retry list when backup (#34930) (#34983) close pingcap/tidb#34865 --- br/pkg/backup/push.go | 7 +++++++ br/pkg/utils/retry.go | 1 + br/tests/br_full/run.sh | 16 +++++++++++++--- 3 files changed, 21 insertions(+), 3 deletions(-) diff --git a/br/pkg/backup/push.go b/br/pkg/backup/push.go index c1d4cd7664be1..f5f9ac536a63b 100644 --- a/br/pkg/backup/push.go +++ b/br/pkg/backup/push.go @@ -127,6 +127,13 @@ func (push *pushDown) pushBackup( // Finished. return res, nil } + failpoint.Inject("backup-timeout-error", func(val failpoint.Value) { + msg := val.(string) + logutil.CL(ctx).Debug("failpoint backup-timeout-error injected.", zap.String("msg", msg)) + resp.Error = &backuppb.Error{ + Msg: msg, + } + }) failpoint.Inject("backup-storage-error", func(val failpoint.Value) { msg := val.(string) logutil.CL(ctx).Debug("failpoint backup-storage-error injected.", zap.String("msg", msg)) diff --git a/br/pkg/utils/retry.go b/br/pkg/utils/retry.go index bda305aaf11ac..8d356a41eb970 100644 --- a/br/pkg/utils/retry.go +++ b/br/pkg/utils/retry.go @@ -32,6 +32,7 @@ var retryableServerError = []string{ "error during dispatch", "put object timeout", "internalerror", + "not read from or written to within the timeout period", } // RetryableFunc presents a retryable operation. diff --git a/br/tests/br_full/run.sh b/br/tests/br_full/run.sh index 21f2e2143a002..f099cebe6b014 100755 --- a/br/tests/br_full/run.sh +++ b/br/tests/br_full/run.sh @@ -46,14 +46,24 @@ if ps -q $pid ; then exit 1 fi - # backup full echo "backup with lz4 start..." -export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/backup/backup-storage-error=1*return(\"connection refused\")->1*return(\"InternalError\")" -run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB-lz4" --concurrency 4 --compression lz4 +test_log="${TEST_DIR}/${DB}_test.log" +error_str="not read from or written to within the timeout period" +unset BR_LOG_TO_TERM + +export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/backup/backup-storage-error=1*return(\"connection refused\")->1*return(\"InternalError\");github.com/pingcap/tidb/br/pkg/backup/backup-timeout-error=1*return(\"not read from or written to within the timeout period\")" +run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB-lz4" --concurrency 4 --compression lz4 --log-file $test_log + export GO_FAILPOINTS="" size_lz4=$(du -d 0 $TEST_DIR/$DB-lz4 | awk '{print $1}') +if ! grep -i "$error_str" $test_log; then + echo "${error_str} not found in log" + echo "TEST: [$TEST_NAME] test restore failed!" + exit 1 +fi + echo "backup with zstd start..." run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB-zstd" --concurrency 4 --compression zstd --compression-level 6 size_zstd=$(du -d 0 $TEST_DIR/$DB-zstd | awk '{print $1}') From d4b58380a8ca63e052f4f2a99615d2724034e445 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Wed, 22 Jun 2022 14:20:37 +0800 Subject: [PATCH 39/51] *: update client-go to improve availability on tikv failure (#35561) close pingcap/tidb#34906 --- executor/main_test.go | 3 +++ go.mod | 2 +- go.sum | 4 ++-- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/executor/main_test.go b/executor/main_test.go index 88be5b84f0750..912804496c19b 100644 --- a/executor/main_test.go +++ b/executor/main_test.go @@ -19,6 +19,7 @@ import ( "os" "testing" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/testkit" @@ -49,6 +50,8 @@ func TestMain(m *testing.M) { conf.Experimental.AllowsExpressionIndex = true }) tikv.EnableFailpoints() + failpoint.Enable("tikvclient/injectLiveness", `return("reachable")`) + defer failpoint.Disable("tikvclient/injectLiveness") tmpDir := config.GetGlobalConfig().TempStoragePath _ = os.RemoveAll(tmpDir) // clean the uncleared temp file during the last run. _ = os.MkdirAll(tmpDir, 0755) diff --git a/go.mod b/go.mod index 4866a22539b1f..99dadef17470f 100644 --- a/go.mod +++ b/go.mod @@ -70,7 +70,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.0-rc.0.20220516035110-cf67df00ad4b + github.com/tikv/client-go/v2 v2.0.0-rc.0.20220614073512-c9d27cd5a9a3 github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee github.com/tmc/grpc-websocket-proxy v0.0.0-20201229170055-e5319fda7802 // indirect github.com/twmb/murmur3 v1.1.3 diff --git a/go.sum b/go.sum index 8f6a06adb1b30..e4c1f366aa53d 100644 --- a/go.sum +++ b/go.sum @@ -789,8 +789,8 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfK github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20220516035110-cf67df00ad4b h1:ED0XsC7KV6SA59OLaZDrTrkMrq4g4RTjtnjgXuL5WrY= -github.com/tikv/client-go/v2 v2.0.0-rc.0.20220516035110-cf67df00ad4b/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20220614073512-c9d27cd5a9a3 h1:3hz+Knhrreu+lW/bV8FEdZKZXxW5WAVXhMsFiT5gFvo= +github.com/tikv/client-go/v2 v2.0.0-rc.0.20220614073512-c9d27cd5a9a3/go.mod h1:wRuh+W35daKTiYBld0oBlT6PSkzEVr+pB/vChzJZk+8= github.com/tikv/pd v1.1.0-beta.0.20211029083450-e65f0c55b6ae/go.mod h1:varH0IE0jJ9E9WN2Ei/N6pajMlPkcXdDEf7f5mmsUVQ= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee h1:rAAdvQ8Hh36syHr92g0VmZEpkH+40RGQBpFL2121xMs= github.com/tikv/pd v1.1.0-beta.0.20211118054146-02848d2660ee/go.mod h1:lRbwxBAhnTQR5vqbTzeI/Bj62bD2OvYYuFezo2vrmeI= From 93d79f8dcb2cc64d17aaa85a9658dd696e6cca05 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 22 Jun 2022 14:46:36 +0800 Subject: [PATCH 40/51] memory, kvcache : Takes process memory,compared to server-memory-quota (#34681) (#34693) close pingcap/tidb#34613 --- util/kvcache/simple_lru.go | 4 ++-- util/memory/meminfo.go | 22 ++++++++++++++++++++++ 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/util/kvcache/simple_lru.go b/util/kvcache/simple_lru.go index eb6ed426f02f3..63791333aa12b 100644 --- a/util/kvcache/simple_lru.go +++ b/util/kvcache/simple_lru.go @@ -128,7 +128,7 @@ func (l *SimpleLRUCache) Put(key Key, value Value) { return } - memUsed, err := memory.MemUsed() + memUsed, err := memory.InstanceMemUsed() if err != nil { l.DeleteAll() return @@ -148,7 +148,7 @@ func (l *SimpleLRUCache) Put(key Key, value Value) { delete(l.elements, string(lru.Value.(*cacheEntry).key.Hash())) l.size-- if memUsed > uint64(float64(l.quota)*(1.0-l.guard)) { - memUsed, err = memory.MemUsed() + memUsed, err = memory.InstanceMemUsed() if err != nil { l.DeleteAll() return diff --git a/util/memory/meminfo.go b/util/memory/meminfo.go index 67178d50fb679..dd1b0e7fc2302 100644 --- a/util/memory/meminfo.go +++ b/util/memory/meminfo.go @@ -16,6 +16,7 @@ package memory import ( "os" + "runtime" "strconv" "strings" "sync" @@ -90,6 +91,10 @@ var memLimit *memInfoCache // expiration time is 500ms var memUsage *memInfoCache +// expiration time is 500ms +// save the memory usage of the server process +var serverMemUsage *memInfoCache + // MemTotalCGroup returns the total amount of RAM on this system in container environment. func MemTotalCGroup() (uint64, error) { mem, t := memLimit.get() @@ -132,6 +137,9 @@ func init() { memUsage = &memInfoCache{ RWMutex: &sync.RWMutex{}, } + serverMemUsage = &memInfoCache{ + RWMutex: &sync.RWMutex{}, + } _, err := MemTotal() terror.MustNil(err) _, err = MemUsed() @@ -178,3 +186,17 @@ func readUint(path string) (uint64, error) { } return parseUint(strings.TrimSpace(string(v)), 10, 64) } + +// InstanceMemUsed returns the memory usage of this TiDB server +func InstanceMemUsed() (uint64, error) { + used, t := serverMemUsage.get() + if time.Since(t) < 500*time.Millisecond { + return used, nil + } + var memoryUsage uint64 + instanceStats := &runtime.MemStats{} + runtime.ReadMemStats(instanceStats) + memoryUsage = instanceStats.HeapAlloc + serverMemUsage.set(memoryUsage, time.Now()) + return memoryUsage, nil +} From ee62fb689446d7ab93bb3e4b79111a58f916a762 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 22 Jun 2022 15:22:37 +0800 Subject: [PATCH 41/51] planner: firstrow agg func should infer return type as nullable for EQAll special case (#34585) (#35037) close pingcap/tidb#34584 --- executor/tiflash_test.go | 19 ++++++++++++++++ expression/aggregation/base_func.go | 2 +- expression/aggregation/base_func_test.go | 29 ++++++++++++++++++++++++ planner/core/expression_rewriter.go | 11 +++++++++ 4 files changed, 60 insertions(+), 1 deletion(-) diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index 5aefde2d562c1..5435463657677 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -128,6 +128,25 @@ func (s *tiflashTestSuite) TestReadPartitionTable(c *C) { tk.MustExec("commit") } +func (s *tiflashTestSuite) TestAggPushDownApplyAll(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists foo") + tk.MustExec("drop table if exists bar") + tk.MustExec("create table foo(a int, b int)") + tk.MustExec("create table bar(a double not null, b decimal(65,0) not null)") + tk.MustExec("alter table foo set tiflash replica 1") + tk.MustExec("alter table bar set tiflash replica 1") + tk.MustExec("insert into foo values(0, NULL)") + tk.MustExec("insert into bar values(0, 0)") + + tk.MustExec("set @@session.tidb_allow_mpp=1") + tk.MustExec("set @@session.tidb_enforce_mpp=1") + + tk.MustQuery("select * from foo where a=all(select a from bar where bar.b=foo.b)").Check(testkit.Rows("0 ")) +} + func (s *tiflashTestSuite) TestReadUnsigedPK(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/expression/aggregation/base_func.go b/expression/aggregation/base_func.go index a185910c82b97..63239f94085d3 100644 --- a/expression/aggregation/base_func.go +++ b/expression/aggregation/base_func.go @@ -274,7 +274,7 @@ func (a *baseFuncDesc) typeInfer4MaxMin(ctx sessionctx.Context) { a.Args[0] = expression.BuildCastFunction(ctx, a.Args[0], tp) } a.RetTp = a.Args[0].GetType() - if (a.Name == ast.AggFuncMax || a.Name == ast.AggFuncMin) && a.RetTp.Tp != mysql.TypeBit { + if a.Name == ast.AggFuncMax || a.Name == ast.AggFuncMin { a.RetTp = a.Args[0].GetType().Clone() a.RetTp.Flag &^= mysql.NotNullFlag } diff --git a/expression/aggregation/base_func_test.go b/expression/aggregation/base_func_test.go index 2066b7773a1ad..d4ea76c66db6b 100644 --- a/expression/aggregation/base_func_test.go +++ b/expression/aggregation/base_func_test.go @@ -45,3 +45,32 @@ func TestClone(t *testing.T) { require.Equal(t, col, desc.Args[0]) require.False(t, desc.equal(ctx, cloned)) } + +func TestBaseFunc_InferAggRetType(t *testing.T) { + ctx := mock.NewContext() + doubleType := types.NewFieldType(mysql.TypeDouble) + bitType := types.NewFieldType(mysql.TypeBit) + + funcNames := []string{ + ast.AggFuncMax, ast.AggFuncMin, + } + dataTypes := []*types.FieldType{ + doubleType, bitType, + } + + for _, dataType := range dataTypes { + notNullType := dataType.Clone() + notNullType.Flag |= mysql.NotNullFlag + col := &expression.Column{ + UniqueID: 0, + RetType: notNullType, + } + for _, name := range funcNames { + desc, err := newBaseFuncDesc(ctx, name, []expression.Expression{col}) + require.NoError(t, err) + err = desc.TypeInfer(ctx) + require.NoError(t, err) + require.Equal(t, dataType, desc.RetTp) + } + } +} diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 91ac0dfdc1b2f..c088e0f48d582 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -774,6 +774,17 @@ func (er *expressionRewriter) handleEQAll(lexpr, rexpr expression.Expression, np } plan4Agg.SetChildren(np) plan4Agg.names = append(plan4Agg.names, types.EmptyName) + + // Currently, firstrow agg function is treated like the exact representation of aggregate group key, + // so the data type is the same with group key, even if the group key is not null. + // However, the return type of firstrow should be nullable, we clear the null flag here instead of + // during invoking NewAggFuncDesc, in order to keep compatibility with the existing presumption + // that the return type firstrow does not change nullability, whatsoever. + // Cloning it because the return type is the same object with argument's data type. + newRetTp := firstRowFunc.RetTp.Clone() + newRetTp.Flag &= ^mysql.NotNullFlag + firstRowFunc.RetTp = newRetTp + firstRowResultCol := &expression.Column{ UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(), RetType: firstRowFunc.RetTp, From b67cbf9cf404be64aa2eab66b67e741d6af72661 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 22 Jun 2022 15:40:37 +0800 Subject: [PATCH 42/51] planner: fix bug that aggregate push down may generate wrong plan for outer joins (#34468) (#34649) close pingcap/tidb#34465 --- planner/core/integration_test.go | 42 ++++++++++++++++++++ planner/core/rule_aggregation_elimination.go | 4 +- planner/core/rule_aggregation_push_down.go | 33 ++++++++++++--- 3 files changed, 73 insertions(+), 6 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 465560dc69f55..d6cb766d448ca 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -155,6 +155,48 @@ func (s *testIntegrationSuite) TestBitColErrorMessage(c *C) { tk.MustGetErrCode("create table bit_col_t (a bit(65))", mysql.ErrTooBigDisplaywidth) } +func (s *testIntegrationSuite) TestAggPushDownLeftJoin(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists customer") + tk.MustExec("create table customer (C_CUSTKEY bigint(20) NOT NULL, C_NAME varchar(25) NOT NULL, " + + "C_ADDRESS varchar(25) NOT NULL, PRIMARY KEY (`C_CUSTKEY`) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("drop table if exists orders") + tk.MustExec("create table orders (O_ORDERKEY bigint(20) NOT NULL, O_CUSTKEY bigint(20) NOT NULL, " + + "O_TOTALPRICE decimal(15,2) NOT NULL, PRIMARY KEY (`O_ORDERKEY`) /*T![clustered_index] CLUSTERED */)") + tk.MustExec("insert into customer values (6, \"xiao zhang\", \"address1\");") + tk.MustExec("set @@tidb_opt_agg_push_down=1;") + + tk.MustQuery("select c_custkey, count(o_orderkey) as c_count from customer left outer join orders " + + "on c_custkey = o_custkey group by c_custkey").Check(testkit.Rows("6 0")) + tk.MustQuery("explain format='brief' select c_custkey, count(o_orderkey) as c_count from customer left outer join orders " + + "on c_custkey = o_custkey group by c_custkey").Check(testkit.Rows( + "Projection 10000.00 root test.customer.c_custkey, Column#7", + "└─Projection 10000.00 root if(isnull(Column#8), 0, 1)->Column#7, test.customer.c_custkey", + " └─HashJoin 10000.00 root left outer join, equal:[eq(test.customer.c_custkey, test.orders.o_custkey)]", + " ├─HashAgg(Build) 8000.00 root group by:test.orders.o_custkey, funcs:count(Column#9)->Column#8, funcs:firstrow(test.orders.o_custkey)->test.orders.o_custkey", + " │ └─TableReader 8000.00 root data:HashAgg", + " │ └─HashAgg 8000.00 cop[tikv] group by:test.orders.o_custkey, funcs:count(test.orders.o_orderkey)->Column#9", + " │ └─TableFullScan 10000.00 cop[tikv] table:orders keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:customer keep order:false, stats:pseudo")) + + tk.MustQuery("select c_custkey, count(o_orderkey) as c_count from orders right outer join customer " + + "on c_custkey = o_custkey group by c_custkey").Check(testkit.Rows("6 0")) + tk.MustQuery("explain format='brief' select c_custkey, count(o_orderkey) as c_count from orders right outer join customer " + + "on c_custkey = o_custkey group by c_custkey").Check(testkit.Rows( + "Projection 10000.00 root test.customer.c_custkey, Column#7", + "└─Projection 10000.00 root if(isnull(Column#8), 0, 1)->Column#7, test.customer.c_custkey", + " └─HashJoin 10000.00 root right outer join, equal:[eq(test.orders.o_custkey, test.customer.c_custkey)]", + " ├─HashAgg(Build) 8000.00 root group by:test.orders.o_custkey, funcs:count(Column#9)->Column#8, funcs:firstrow(test.orders.o_custkey)->test.orders.o_custkey", + " │ └─TableReader 8000.00 root data:HashAgg", + " │ └─HashAgg 8000.00 cop[tikv] group by:test.orders.o_custkey, funcs:count(test.orders.o_orderkey)->Column#9", + " │ └─TableFullScan 10000.00 cop[tikv] table:orders keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:customer keep order:false, stats:pseudo")) +} + func (s *testIntegrationSuite) TestPushLimitDownIndexLookUpReader(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/planner/core/rule_aggregation_elimination.go b/planner/core/rule_aggregation_elimination.go index 9adca6936099d..db4cf873619ec 100644 --- a/planner/core/rule_aggregation_elimination.go +++ b/planner/core/rule_aggregation_elimination.go @@ -158,7 +158,9 @@ func ConvertAggToProj(agg *LogicalAggregation, schema *expression.Schema) (bool, func rewriteExpr(ctx sessionctx.Context, aggFunc *aggregation.AggFuncDesc) (bool, expression.Expression) { switch aggFunc.Name { case ast.AggFuncCount: - if aggFunc.Mode == aggregation.FinalMode { + if aggFunc.Mode == aggregation.FinalMode && + len(aggFunc.Args) == 1 && + mysql.HasNotNullFlag(aggFunc.Args[0].GetType().Flag) { return true, wrapCastFunction(ctx, aggFunc.Args[0], aggFunc.RetTp) } return true, rewriteCount(ctx, aggFunc.Args, aggFunc.RetTp) diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 5f927befb8ee3..7e874369e6027 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -193,7 +193,8 @@ func (a *aggregationPushDownSolver) checkValidJoin(join *LogicalJoin) bool { // decompose splits an aggregate function to two parts: a final mode function and a partial mode function. Currently // there are no differences between partial mode and complete mode, so we can confuse them. -func (a *aggregationPushDownSolver) decompose(ctx sessionctx.Context, aggFunc *aggregation.AggFuncDesc, schema *expression.Schema) ([]*aggregation.AggFuncDesc, *expression.Schema) { +func (a *aggregationPushDownSolver) decompose(ctx sessionctx.Context, aggFunc *aggregation.AggFuncDesc, + schema *expression.Schema, nullGenerating bool) ([]*aggregation.AggFuncDesc, *expression.Schema) { // Result is a slice because avg should be decomposed to sum and count. Currently we don't process this case. result := []*aggregation.AggFuncDesc{aggFunc.Clone()} for _, aggFunc := range result { @@ -202,7 +203,21 @@ func (a *aggregationPushDownSolver) decompose(ctx sessionctx.Context, aggFunc *a RetType: aggFunc.RetTp, }) } - aggFunc.Args = expression.Column2Exprs(schema.Columns[schema.Len()-len(result):]) + cols := schema.Columns[schema.Len()-len(result):] + aggFunc.Args = make([]expression.Expression, 0, len(cols)) + // if the partial aggregation is on the null generating side, we have to clear the NOT NULL flag + // for the final aggregate functions' arguments + for _, col := range cols { + if nullGenerating { + arg := *col + newFieldType := *arg.RetType + newFieldType.Flag &= ^mysql.NotNullFlag + arg.RetType = &newFieldType + aggFunc.Args = append(aggFunc.Args, &arg) + } else { + aggFunc.Args = append(aggFunc.Args, col) + } + } aggFunc.Mode = aggregation.FinalMode return result, schema } @@ -226,7 +241,9 @@ func (a *aggregationPushDownSolver) tryToPushDownAgg(oldAgg *LogicalAggregation, return child, nil } } - agg, err := a.makeNewAgg(join.ctx, aggFuncs, gbyCols, aggHints, blockOffset) + nullGenerating := (join.JoinType == LeftOuterJoin && childIdx == 1) || + (join.JoinType == RightOuterJoin && childIdx == 0) + agg, err := a.makeNewAgg(join.ctx, aggFuncs, gbyCols, aggHints, blockOffset, nullGenerating) if err != nil { return nil, err } @@ -273,7 +290,8 @@ func (a *aggregationPushDownSolver) checkAnyCountAndSum(aggFuncs []*aggregation. // TODO: // 1. https://github.com/pingcap/tidb/issues/16355, push avg & distinct functions across join // 2. remove this method and use splitPartialAgg instead for clean code. -func (a *aggregationPushDownSolver) makeNewAgg(ctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc, gbyCols []*expression.Column, aggHints aggHintInfo, blockOffset int) (*LogicalAggregation, error) { +func (a *aggregationPushDownSolver) makeNewAgg(ctx sessionctx.Context, aggFuncs []*aggregation.AggFuncDesc, + gbyCols []*expression.Column, aggHints aggHintInfo, blockOffset int, nullGenerating bool) (*LogicalAggregation, error) { agg := LogicalAggregation{ GroupByItems: expression.Column2Exprs(gbyCols), aggHints: aggHints, @@ -283,7 +301,7 @@ func (a *aggregationPushDownSolver) makeNewAgg(ctx sessionctx.Context, aggFuncs schema := expression.NewSchema(make([]*expression.Column, 0, aggLen)...) for _, aggFunc := range aggFuncs { var newFuncs []*aggregation.AggFuncDesc - newFuncs, schema = a.decompose(ctx, aggFunc, schema) + newFuncs, schema = a.decompose(ctx, aggFunc, schema, nullGenerating) newAggFuncDescs = append(newAggFuncDescs, newFuncs...) } for _, gbyCol := range gbyCols { @@ -436,6 +454,11 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan, opt *logicalOptim } join.SetChildren(lChild, rChild) join.SetSchema(expression.MergeSchema(lChild.Schema(), rChild.Schema())) + if join.JoinType == LeftOuterJoin { + resetNotNullFlag(join.schema, lChild.Schema().Len(), join.schema.Len()) + } else if join.JoinType == RightOuterJoin { + resetNotNullFlag(join.schema, 0, lChild.Schema().Len()) + } buildKeyInfo(join) proj := a.tryToEliminateAggregation(agg, opt) if proj != nil { From d40a30bec7624caec469ae1b591ad9781b2ee9bf Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 22 Jun 2022 21:04:36 +0800 Subject: [PATCH 43/51] executor: fix unstable test TestSavedAnalyzeOptions (#32017) (#35640) close pingcap/tidb#32018 --- executor/analyze_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index de35933562e5a..6ef3875c96de2 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -1332,7 +1332,7 @@ func TestSavedAnalyzeOptions(t *testing.T) { require.Equal(t, 2, len(col0.Buckets)) // manual analyze uses the table-level persisted options by merging the new options - tk.MustExec("analyze table t columns a,b with 0.9 samplerate, 3 buckets") + tk.MustExec("analyze table t columns a,b with 1 samplerate, 3 buckets") tbl = h.GetTableStats(tableInfo) require.Greater(t, tbl.Version, lastVersion) lastVersion = tbl.Version @@ -1347,7 +1347,7 @@ func TestSavedAnalyzeOptions(t *testing.T) { // The columns are: table_id, sample_num, sample_rate, buckets, topn, column_choice, column_ids. rs = tk.MustQuery("select * from mysql.analyze_options where table_id=" + strconv.FormatInt(tbl.PhysicalID, 10)) require.Equal(t, 1, len(rs.Rows())) - require.Equal(t, "0.9", rs.Rows()[0][2]) + require.Equal(t, "1", rs.Rows()[0][2]) require.Equal(t, "3", rs.Rows()[0][3]) require.Equal(t, "1", rs.Rows()[0][4]) require.Equal(t, "LIST", rs.Rows()[0][5]) From 9c22c6cc76d43c594b89a329e04c6ab20dc56af6 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 23 Jun 2022 19:28:37 +0800 Subject: [PATCH 44/51] server: fix connection close on network timeout/read error (#34757) (#35353) close pingcap/tidb#34722 --- server/conn.go | 12 +++++++----- server/server.go | 1 + 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/server/conn.go b/server/conn.go index a0ec8796ec610..769fe7eb7e80f 100644 --- a/server/conn.go +++ b/server/conn.go @@ -318,8 +318,14 @@ func closeConn(cc *clientConn, connections int) error { metrics.ConnGauge.Set(float64(connections)) if cc.bufReadConn != nil { err := cc.bufReadConn.Close() - terror.Log(err) + if err != nil { + // We need to expect connection might have already disconnected. + // This is because closeConn() might be called after a connection read-timeout. + logutil.Logger(context.Background()).Debug("could not close connection", zap.Error(err)) + } } + // Close statements and session + // This will release advisory locks, row locks, etc. if cc.ctx != nil { return cc.ctx.Close() } @@ -1037,10 +1043,6 @@ func (cc *clientConn) Run(ctx context.Context) { terror.Log(err) metrics.PanicCounter.WithLabelValues(metrics.LabelSession).Inc() } - if atomic.LoadInt32(&cc.status) != connStatusShutdown { - err := cc.Close() - terror.Log(err) - } }() // Usually, client connection status changes between [dispatching] <=> [reading]. diff --git a/server/server.go b/server/server.go index ce4ef41a2151d..2be15b20b7bf9 100644 --- a/server/server.go +++ b/server/server.go @@ -521,6 +521,7 @@ func (s *Server) onConn(conn *clientConn) { logutil.Logger(ctx).Debug("new connection", zap.String("remoteAddr", conn.bufReadConn.RemoteAddr().String())) defer func() { + terror.Log(conn.Close()) logutil.Logger(ctx).Debug("connection closed") }() s.rwlock.Lock() From 1f8716d5451af536b04be5eb609dfad4614f5104 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 23 Jun 2022 19:44:37 +0800 Subject: [PATCH 45/51] br: Fix stuck when meeting error (#33201) (#33268) close pingcap/tidb#33200 --- br/pkg/restore/client.go | 10 +++ br/pkg/restore/pipeline_items.go | 40 +++++++--- br/pkg/restore/split_test.go | 125 +++++++++++++++++++++++++++++++ 3 files changed, 166 insertions(+), 9 deletions(-) diff --git a/br/pkg/restore/client.go b/br/pkg/restore/client.go index 4767a9c8323b9..50d6429382161 100644 --- a/br/pkg/restore/client.go +++ b/br/pkg/restore/client.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/br/pkg/metautil" "github.com/pingcap/tidb/br/pkg/pdutil" "github.com/pingcap/tidb/br/pkg/redact" + "github.com/pingcap/tidb/br/pkg/rtree" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils" @@ -598,6 +599,15 @@ func drainFilesByRange(files []*backuppb.File, supportMulti bool) ([]*backuppb.F return files[:idx], files[idx:] } +// SplitRanges implements TiKVRestorer. +func (rc *Client) SplitRanges(ctx context.Context, + ranges []rtree.Range, + rewriteRules *RewriteRules, + updateCh glue.Progress, + isRawKv bool) error { + return SplitRanges(ctx, rc, ranges, rewriteRules, updateCh, isRawKv) +} + // RestoreFiles tries to restore the files. func (rc *Client) RestoreFiles( ctx context.Context, diff --git a/br/pkg/restore/pipeline_items.go b/br/pkg/restore/pipeline_items.go index 7d3ce107ff70b..26da3824b9a4b 100644 --- a/br/pkg/restore/pipeline_items.go +++ b/br/pkg/restore/pipeline_items.go @@ -8,8 +8,10 @@ import ( "time" "github.com/pingcap/errors" + backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" "github.com/pingcap/tidb/br/pkg/glue" + "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/metautil" "github.com/pingcap/tidb/br/pkg/rtree" "github.com/pingcap/tidb/br/pkg/summary" @@ -183,8 +185,27 @@ type BatchSender interface { Close() } +// TiKVRestorer is the minimal methods required for restoring. +// It contains the primitive APIs extract from `restore.Client`, so some of arguments may seem redundant. +// Maybe TODO: make a better abstraction? +type TiKVRestorer interface { + // SplitRanges split regions implicated by the ranges and rewrite rules. + // After spliting, it also scatters the fresh regions. + SplitRanges(ctx context.Context, + ranges []rtree.Range, + rewriteRules *RewriteRules, + updateCh glue.Progress, + isRawKv bool) error + // RestoreFiles import the files to the TiKV. + RestoreFiles(ctx context.Context, + files []*backuppb.File, + rewriteRules *RewriteRules, + updateCh glue.Progress) error +} + type tikvSender struct { - client *Client + client TiKVRestorer + updateCh glue.Progress sink TableSink @@ -209,7 +230,7 @@ func (b *tikvSender) RestoreBatch(ranges DrainResult) { // NewTiKVSender make a sender that send restore requests to TiKV. func NewTiKVSender( ctx context.Context, - cli *Client, + cli TiKVRestorer, updateCh glue.Progress, splitConcurrency uint, ) (BatchSender, error) { @@ -252,9 +273,9 @@ func (b *tikvSender) splitWorker(ctx context.Context, b.wg.Done() if err := eg.Wait(); err != nil { b.sink.EmitError(err) - return } close(next) + log.Info("TiKV Sender: split worker exits.") }() start := time.Now() @@ -266,7 +287,7 @@ func (b *tikvSender) splitWorker(ctx context.Context, pool := utils.NewWorkerPool(concurrency, "split") for { select { - case <-ctx.Done(): + case <-ectx.Done(): return case result, ok := <-ranges: if !ok { @@ -289,7 +310,7 @@ func (b *tikvSender) splitWorker(ctx context.Context, // hence the checksum would fail. done := b.registerTableIsRestoring(result.TablesToSend) pool.ApplyOnErrorGroup(eg, func() error { - err := SplitRanges(ectx, b.client, result.Ranges, result.RewriteRules, b.updateCh, false) + err := b.client.SplitRanges(ectx, result.Ranges, result.RewriteRules, b.updateCh, false) if err != nil { log.Error("failed on split range", rtree.ZapRanges(result.Ranges), zap.Error(err)) return err @@ -338,17 +359,17 @@ func (b *tikvSender) waitTablesDone(ts []CreatedTable) { func (b *tikvSender) restoreWorker(ctx context.Context, ranges <-chan drainResultAndDone) { eg, ectx := errgroup.WithContext(ctx) defer func() { - log.Debug("restore worker closed") + log.Info("TiKV Sender: restore worker prepare to close.") if err := eg.Wait(); err != nil { b.sink.EmitError(err) - return } - b.wg.Done() b.sink.Close() + b.wg.Done() + log.Info("TiKV Sender: restore worker exits.") }() for { select { - case <-ctx.Done(): + case <-ectx.Done(): return case r, ok := <-ranges: if !ok { @@ -360,6 +381,7 @@ func (b *tikvSender) restoreWorker(ctx context.Context, ranges <-chan drainResul eg.Go(func() error { e := b.client.RestoreFiles(ectx, files, r.result.RewriteRules, b.updateCh) if e != nil { + log.Error("restore batch meet error", logutil.ShortError(e), logutil.Files(files)) r.done() return e } diff --git a/br/pkg/restore/split_test.go b/br/pkg/restore/split_test.go index 17edf0b7df9dc..7ec3953569b60 100644 --- a/br/pkg/restore/split_test.go +++ b/br/pkg/restore/split_test.go @@ -11,9 +11,14 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" + backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/kvproto/pkg/import_sstpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/pingcap/log" + berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/glue" + "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/restore" "github.com/pingcap/tidb/br/pkg/rtree" "github.com/pingcap/tidb/br/pkg/utils" @@ -572,3 +577,123 @@ func (s *testRangeSuite) TestRegionConsistency(c *C) { ca.err) } } + +type fakeRestorer struct { + errorInSplit bool + splitRanges []rtree.Range + restoredFiles []*backuppb.File +} + +func (f *fakeRestorer) SplitRanges(ctx context.Context, ranges []rtree.Range, rewriteRules *restore.RewriteRules, updateCh glue.Progress, isRawKv bool) error { + if ctx.Err() != nil { + return ctx.Err() + } + f.splitRanges = append(f.splitRanges, ranges...) + if f.errorInSplit { + err := errors.Annotatef(berrors.ErrRestoreSplitFailed, + "the key space takes many efforts and finally get together, how dare you split them again... :<") + log.Error("error happens :3", logutil.ShortError(err)) + return err + } + return nil +} + +func (f *fakeRestorer) RestoreFiles(ctx context.Context, files []*backuppb.File, rewriteRules *restore.RewriteRules, updateCh glue.Progress) error { + if ctx.Err() != nil { + return ctx.Err() + } + f.restoredFiles = append(f.restoredFiles, files...) + err := errors.Annotatef(berrors.ErrRestoreWriteAndIngest, "the files to restore are taken by a hijacker, meow :3") + log.Error("error happens :3", logutil.ShortError(err)) + return err +} + +func fakeRanges(keys ...string) (r restore.DrainResult) { + for i := range keys { + if i+1 == len(keys) { + return + } + r.Ranges = append(r.Ranges, rtree.Range{ + StartKey: []byte(keys[i]), + EndKey: []byte(keys[i+1]), + Files: []*backuppb.File{{Name: "fake.sst"}}, + }) + } + return +} + +type errorInTimeSink struct { + ctx context.Context + errCh chan error + t *testing.T +} + +func (e errorInTimeSink) EmitTables(tables ...restore.CreatedTable) {} + +func (e errorInTimeSink) EmitError(err error) { + e.errCh <- err +} + +func (e errorInTimeSink) Close() {} + +func (e errorInTimeSink) Wait() { + select { + case <-e.ctx.Done(): + e.t.Logf("The context is canceled but no error happen") + e.t.FailNow() + case <-e.errCh: + } +} + +func assertErrorEmitInTime(ctx context.Context, t *testing.T) errorInTimeSink { + errCh := make(chan error, 1) + return errorInTimeSink{ + ctx: ctx, + errCh: errCh, + t: t, + } +} + +func TestRestoreFailed(t *testing.T) { + ranges := []restore.DrainResult{ + fakeRanges("aax", "abx", "abz"), + fakeRanges("abz", "bbz", "bcy"), + fakeRanges("bcy", "cad", "xxy"), + } + r := &fakeRestorer{} + sender, err := restore.NewTiKVSender(context.TODO(), r, nil, 1) + require.NoError(t, err) + dctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + sink := assertErrorEmitInTime(dctx, t) + sender.PutSink(sink) + for _, r := range ranges { + sender.RestoreBatch(r) + } + sink.Wait() + sink.Close() + sender.Close() + require.GreaterOrEqual(t, len(r.restoredFiles), 1) +} + +func TestSplitFailed(t *testing.T) { + ranges := []restore.DrainResult{ + fakeRanges("aax", "abx", "abz"), + fakeRanges("abz", "bbz", "bcy"), + fakeRanges("bcy", "cad", "xxy"), + } + r := &fakeRestorer{errorInSplit: true} + sender, err := restore.NewTiKVSender(context.TODO(), r, nil, 1) + require.NoError(t, err) + dctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + sink := assertErrorEmitInTime(dctx, t) + sender.PutSink(sink) + for _, r := range ranges { + sender.RestoreBatch(r) + } + sink.Wait() + sender.Close() + require.GreaterOrEqual(t, len(r.splitRanges), 2) + require.Len(t, r.restoredFiles, 0) +} From 10dd152ac06d7e5e39025fbedd9b987fa5234d96 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 23 Jun 2022 22:48:37 +0800 Subject: [PATCH 46/51] parser: skip restoring schema name of CTE table columns (#33991) (#34081) close pingcap/tidb#33965 --- executor/cte_test.go | 20 ++++++++++++++++++++ parser/ast/dml.go | 40 ++++++++------------------------------- parser/ast/expressions.go | 2 +- parser/format/format.go | 34 +++++++++++++++++++++++++++++++-- 4 files changed, 61 insertions(+), 35 deletions(-) diff --git a/executor/cte_test.go b/executor/cte_test.go index 92c1c861fcc23..e8a0710fc73bb 100644 --- a/executor/cte_test.go +++ b/executor/cte_test.go @@ -440,3 +440,23 @@ func TestCTEExecError(t *testing.T) { require.True(t, terror.ErrorEqual(err, types.ErrOverflow)) } } + +// https://github.com/pingcap/tidb/issues/33965. +func TestCTEsInView(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + + tk.MustExec("create database if not exists test1;") + tk.MustExec("create table test.t (a int);") + tk.MustExec("create table test1.t (a int);") + tk.MustExec("insert into test.t values (1);") + tk.MustExec("insert into test1.t values (2);") + + tk.MustExec("use test;") + tk.MustExec("create definer='root'@'localhost' view test.v as with tt as (select * from t) select * from tt;") + tk.MustQuery("select * from test.v;").Check(testkit.Rows("1")) + tk.MustExec("use test1;") + tk.MustQuery("select * from test.v;").Check(testkit.Rows("1")) +} diff --git a/parser/ast/dml.go b/parser/ast/dml.go index cd29e293ba875..5b8c633ec61b0 100644 --- a/parser/ast/dml.go +++ b/parser/ast/dml.go @@ -14,8 +14,6 @@ package ast import ( - "strings" - "github.com/pingcap/errors" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/format" @@ -286,14 +284,7 @@ func (n *TableName) restoreName(ctx *format.RestoreCtx) { ctx.WritePlain(".") } else if ctx.DefaultDB != "" { // Try CTE, for a CTE table name, we shouldn't write the database name. - ok := false - for _, name := range ctx.CTENames { - if strings.EqualFold(name, n.Name.String()) { - ok = true - break - } - } - if !ok { + if !ctx.IsCTETableName(n.Name.L) { ctx.WriteName(ctx.DefaultDB) ctx.WritePlain(".") } @@ -1119,7 +1110,7 @@ func (n *WithClause) Restore(ctx *format.RestoreCtx) error { if n.IsRecursive { // If the CTE is recursive, we should make it visible for the CTE's query. // Otherwise, we should put it to stack after building the CTE's query. - ctx.CTENames = append(ctx.CTENames, cte.Name.L) + ctx.RecordCTEName(cte.Name.L) } if len(cte.ColNameList) > 0 { ctx.WritePlain(" (") @@ -1137,7 +1128,7 @@ func (n *WithClause) Restore(ctx *format.RestoreCtx) error { return err } if !n.IsRecursive { - ctx.CTENames = append(ctx.CTENames, cte.Name.L) + ctx.RecordCTEName(cte.Name.L) } } ctx.WritePlain(" ") @@ -1163,10 +1154,7 @@ func (n *WithClause) Accept(v Visitor) (Node, bool) { // Restore implements Node interface. func (n *SelectStmt) Restore(ctx *format.RestoreCtx) error { if n.WithBeforeBraces { - l := len(ctx.CTENames) - defer func() { - ctx.CTENames = ctx.CTENames[:l] - }() + defer ctx.RestoreCTEFunc()() err := n.With.Restore(ctx) if err != nil { return err @@ -1514,10 +1502,7 @@ type SetOprSelectList struct { // Restore implements Node interface. func (n *SetOprSelectList) Restore(ctx *format.RestoreCtx) error { if n.With != nil { - l := len(ctx.CTENames) - defer func() { - ctx.CTENames = ctx.CTENames[:l] - }() + defer ctx.RestoreCTEFunc()() if err := n.With.Restore(ctx); err != nil { return errors.Annotate(err, "An error occurred while restore SetOprSelectList.With") } @@ -1618,10 +1603,7 @@ func (*SetOprStmt) resultSet() {} // Restore implements Node interface. func (n *SetOprStmt) Restore(ctx *format.RestoreCtx) error { if n.With != nil { - l := len(ctx.CTENames) - defer func() { - ctx.CTENames = ctx.CTENames[:l] - }() + defer ctx.RestoreCTEFunc()() if err := n.With.Restore(ctx); err != nil { return errors.Annotate(err, "An error occurred while restore UnionStmt.With") } @@ -2203,10 +2185,7 @@ type DeleteStmt struct { // Restore implements Node interface. func (n *DeleteStmt) Restore(ctx *format.RestoreCtx) error { if n.With != nil { - l := len(ctx.CTENames) - defer func() { - ctx.CTENames = ctx.CTENames[:l] - }() + defer ctx.RestoreCTEFunc()() err := n.With.Restore(ctx) if err != nil { return err @@ -2367,10 +2346,7 @@ type UpdateStmt struct { // Restore implements Node interface. func (n *UpdateStmt) Restore(ctx *format.RestoreCtx) error { if n.With != nil { - l := len(ctx.CTENames) - defer func() { - ctx.CTENames = ctx.CTENames[:l] - }() + defer ctx.RestoreCTEFunc()() err := n.With.Restore(ctx) if err != nil { return err diff --git a/parser/ast/expressions.go b/parser/ast/expressions.go index 6a46ab332c831..66f40eb205952 100644 --- a/parser/ast/expressions.go +++ b/parser/ast/expressions.go @@ -512,7 +512,7 @@ type ColumnName struct { // Restore implements Node interface. func (n *ColumnName) Restore(ctx *format.RestoreCtx) error { - if n.Schema.O != "" { + if n.Schema.O != "" && !ctx.IsCTETableName(n.Table.L) { ctx.WriteName(n.Schema.O) ctx.WritePlain(".") } diff --git a/parser/format/format.go b/parser/format/format.go index ef003d6a78d6d..4141b0baf119a 100644 --- a/parser/format/format.go +++ b/parser/format/format.go @@ -305,12 +305,12 @@ type RestoreCtx struct { Flags RestoreFlags In io.Writer DefaultDB string - CTENames []string + CTERestorer } // NewRestoreCtx returns a new `RestoreCtx`. func NewRestoreCtx(flags RestoreFlags, in io.Writer) *RestoreCtx { - return &RestoreCtx{flags, in, "", make([]string, 0)} + return &RestoreCtx{Flags: flags, In: in, DefaultDB: ""} } // WriteKeyWord writes the `keyWord` into writer. @@ -387,3 +387,33 @@ func (ctx *RestoreCtx) WritePlain(plainText string) { func (ctx *RestoreCtx) WritePlainf(format string, a ...interface{}) { fmt.Fprintf(ctx.In, format, a...) } + +// CTERestorer is used by WithClause related nodes restore. +type CTERestorer struct { + CTENames []string +} + +// IsCTETableName returns true if the given tableName comes from CTE. +func (c *CTERestorer) IsCTETableName(nameL string) bool { + for _, n := range c.CTENames { + if n == nameL { + return true + } + } + return false +} + +func (c *CTERestorer) RecordCTEName(nameL string) { + c.CTENames = append(c.CTENames, nameL) +} + +func (c *CTERestorer) RestoreCTEFunc() func() { + l := len(c.CTENames) + return func() { + if l == 0 { + c.CTENames = nil + } else { + c.CTENames = c.CTENames[:l] + } + } +} From 6aacaf5f71b84a5a0c310c400e2dfe8af37f11ea Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 24 Jun 2022 16:46:38 +0800 Subject: [PATCH 47/51] stmtsummary: fix issue of concurrent map read and write (#35367) (#35385) close pingcap/tidb#35340 --- infoschema/cluster_tables_test.go | 37 ++++++++++++++++++++++++++----- util/stmtsummary/reader.go | 28 +++++++++++++---------- 2 files changed, 48 insertions(+), 17 deletions(-) diff --git a/infoschema/cluster_tables_test.go b/infoschema/cluster_tables_test.go index 958ed02caf6c8..73cbb1f3352eb 100644 --- a/infoschema/cluster_tables_test.go +++ b/infoschema/cluster_tables_test.go @@ -23,6 +23,7 @@ import ( "runtime" "strconv" "strings" + "sync" "testing" "time" @@ -81,6 +82,7 @@ func TestClusterTables(t *testing.T) { t.Run("SelectClusterTablePrivilege", SubTestSelectClusterTablePrivilege(s)) t.Run("StmtSummaryEvictedCountTable", SubTestStmtSummaryEvictedCountTable(s)) t.Run("StmtSummaryHistoryTable", SubTestStmtSummaryHistoryTable(s)) + t.Run("SubTestStmtSummaryIssue35340", SubTestStmtSummaryIssue35340(s)) t.Run("Issue26379", SubTestIssue26379(s)) t.Run("SubTestStmtSummaryResultRows", SubTestStmtSummaryResultRows(s)) } @@ -322,7 +324,6 @@ func SubTestStmtSummaryEvictedCountTable(s *clusterTablesSuite) func(*testing.T) // statements_summary is off, statements_summary_evicted is empty. tk.MustQuery("select count(*) from information_schema.cluster_statements_summary_evicted;").Check(testkit.Rows("0")) tk.MustExec("set global tidb_enable_stmt_summary=1") - // make a new session for test... tk = s.newTestKitWithRoot(t) // first sql @@ -334,23 +335,19 @@ func SubTestStmtSummaryEvictedCountTable(s *clusterTablesSuite) func(*testing.T) tk.MustQuery("select evicted_count from information_schema.cluster_statements_summary_evicted;"). Check(testkit.Rows("2")) // TODO: Add more tests. - tk.MustExec("create user 'testuser'@'localhost'") tk.MustExec("create user 'testuser2'@'localhost'") tk.MustExec("grant process on *.* to 'testuser2'@'localhost'") tk1 := s.newTestKitWithRoot(t) defer tk1.MustExec("drop user 'testuser'@'localhost'") defer tk1.MustExec("drop user 'testuser2'@'localhost'") - require.True(t, tk.Session().Auth(&auth.UserIdentity{ Username: "testuser", Hostname: "localhost", }, nil, nil)) - err := tk.QueryToErr("select * from information_schema.CLUSTER_STATEMENTS_SUMMARY_EVICTED") // This error is come from cop(TiDB) fetch from rpc server. require.EqualError(t, err, "other error: [planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation") - require.True(t, tk.Session().Auth(&auth.UserIdentity{ Username: "testuser2", Hostname: "localhost", @@ -359,6 +356,36 @@ func SubTestStmtSummaryEvictedCountTable(s *clusterTablesSuite) func(*testing.T) } } +func SubTestStmtSummaryIssue35340(s *clusterTablesSuite) func(t *testing.T) { + return func(t *testing.T) { + tk := s.newTestKitWithRoot(t) + tk.MustExec("set global tidb_stmt_summary_refresh_interval=1800") + tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 3000") + for i := 0; i < 100; i++ { + user := "user" + strconv.Itoa(i) + tk.MustExec(fmt.Sprintf("create user '%v'@'localhost'", user)) + } + tk.MustExec("flush privileges") + var wg sync.WaitGroup + for i := 0; i < 10; i++ { + wg.Add(1) + go func() { + defer wg.Done() + tk := s.newTestKitWithRoot(t) + for j := 0; j < 100; j++ { + user := "user" + strconv.Itoa(j) + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: user, + Hostname: "localhost", + }, nil, nil)) + tk.MustQuery("select count(*) from information_schema.statements_summary;") + } + }() + } + wg.Wait() + } +} + func SubTestStmtSummaryHistoryTable(s *clusterTablesSuite) func(*testing.T) { return func(t *testing.T) { tk := s.newTestKitWithRoot(t) diff --git a/util/stmtsummary/reader.go b/util/stmtsummary/reader.go index c849e2765da4c..46d52e99bfd66 100644 --- a/util/stmtsummary/reader.go +++ b/util/stmtsummary/reader.go @@ -138,11 +138,7 @@ func (ssr *stmtSummaryReader) getStmtByDigestRow(ssbd *stmtSummaryByDigest, begi // `ssElement` is lazy expired, so expired elements could also be read. // `beginTime` won't change since `ssElement` is created, so locking is not needed here. - isAuthed := true - if ssr.user != nil && !ssr.hasProcessPriv && ssElement != nil { - _, isAuthed = ssElement.authUsers[ssr.user.Username] - } - if ssElement == nil || ssElement.beginTime < beginTimeForCurInterval || !isAuthed { + if ssElement == nil || ssElement.beginTime < beginTimeForCurInterval { return nil } return ssr.getStmtByDigestElementRow(ssElement, ssbd) @@ -151,6 +147,14 @@ func (ssr *stmtSummaryReader) getStmtByDigestRow(ssbd *stmtSummaryByDigest, begi func (ssr *stmtSummaryReader) getStmtByDigestElementRow(ssElement *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) []types.Datum { ssElement.Lock() defer ssElement.Unlock() + isAuthed := true + if ssr.user != nil && !ssr.hasProcessPriv { + _, isAuthed = ssElement.authUsers[ssr.user.Username] + } + if !isAuthed { + return nil + } + datums := make([]types.Datum, len(ssr.columnValueFactories)) for i, factory := range ssr.columnValueFactories { datums[i] = types.NewDatum(factory(ssElement, ssbd)) @@ -164,12 +168,9 @@ func (ssr *stmtSummaryReader) getStmtByDigestHistoryRow(ssbd *stmtSummaryByDiges rows := make([][]types.Datum, 0, len(ssElements)) for _, ssElement := range ssElements { - isAuthed := true - if ssr.user != nil && !ssr.hasProcessPriv { - _, isAuthed = ssElement.authUsers[ssr.user.Username] - } - if isAuthed { - rows = append(rows, ssr.getStmtByDigestElementRow(ssElement, ssbd)) + record := ssr.getStmtByDigestElementRow(ssElement, ssbd) + if record != nil { + rows = append(rows, record) } } return rows @@ -200,7 +201,10 @@ func (ssr *stmtSummaryReader) getStmtEvictedOtherHistoryRow(ssbde *stmtSummaryBy ssbd := new(stmtSummaryByDigest) for _, seElement := range seElements { - rows = append(rows, ssr.getStmtByDigestElementRow(seElement.otherSummary, ssbd)) + record := ssr.getStmtByDigestElementRow(seElement.otherSummary, ssbd) + if record != nil { + rows = append(rows, record) + } } return rows } From d16bcd8987242b47836b191b058dfa598b06c144 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 24 Jun 2022 17:06:38 +0800 Subject: [PATCH 48/51] load_data: fix the bug that column list does not work in load data. (#35222) (#35464) close pingcap/tidb#35198 --- errno/errcode.go | 1 + errno/errname.go | 1 + errors.toml | 5 ++ executor/errors.go | 1 + executor/load_data.go | 37 ++++++++++ server/server_test.go | 138 +++++++++++++++++++++++++++++++++++++ server/tidb_serial_test.go | 1 + 7 files changed, 184 insertions(+) diff --git a/errno/errcode.go b/errno/errcode.go index 046a7614bdb32..91d3e21789100 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -975,6 +975,7 @@ const ( ErrWarnOptimizerHintParseError = 8064 ErrWarnOptimizerHintInvalidInteger = 8065 ErrUnsupportedSecondArgumentType = 8066 + ErrColumnNotMatched = 8067 ErrInvalidPluginID = 8101 ErrInvalidPluginManifest = 8102 ErrInvalidPluginName = 8103 diff --git a/errno/errname.go b/errno/errname.go index 5163dab1f11bd..9cce3f0a8df1c 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1038,6 +1038,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrInvalidWildCard: mysql.Message("Wildcard fields without any table name appears in wrong place", nil), ErrMixOfGroupFuncAndFieldsIncompatible: mysql.Message("In aggregated query without GROUP BY, expression #%d of SELECT list contains nonaggregated column '%s'; this is incompatible with sql_mode=only_full_group_by", nil), ErrUnsupportedSecondArgumentType: mysql.Message("JSON_OBJECTAGG: unsupported second argument type %v", nil), + ErrColumnNotMatched: mysql.Message("Load data: unmatched columns", nil), ErrLockExpire: mysql.Message("TTL manager has timed out, pessimistic locks may expire, please commit or rollback this transaction", nil), ErrTableOptionUnionUnsupported: mysql.Message("CREATE/ALTER table with union option is not supported", nil), ErrTableOptionInsertMethodUnsupported: mysql.Message("CREATE/ALTER table with insert method option is not supported", nil), diff --git a/errors.toml b/errors.toml index 229ad4df3847c..83390510861e9 100644 --- a/errors.toml +++ b/errors.toml @@ -861,6 +861,11 @@ error = ''' TiDB admin check table failed. ''' +["executor:8067"] +error = ''' +Load data: unmatched columns +''' + ["executor:8114"] error = ''' Unknown plan diff --git a/executor/errors.go b/executor/errors.go index f6e0f87d08e6b..5aeea2c4da092 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -32,6 +32,7 @@ var ( ErrUnsupportedPs = dbterror.ClassExecutor.NewStd(mysql.ErrUnsupportedPs) ErrSubqueryMoreThan1Row = dbterror.ClassExecutor.NewStd(mysql.ErrSubqueryNo1Row) ErrIllegalGrantForTable = dbterror.ClassExecutor.NewStd(mysql.ErrIllegalGrantForTable) + ErrColumnsNotMatched = dbterror.ClassExecutor.NewStd(mysql.ErrColumnNotMatched) ErrCantCreateUserWithGrant = dbterror.ClassExecutor.NewStd(mysql.ErrCantCreateUserWithGrant) ErrPasswordNoMatch = dbterror.ClassExecutor.NewStd(mysql.ErrPasswordNoMatch) diff --git a/executor/load_data.go b/executor/load_data.go index 7d124c0cacf3d..5d65e01595ccb 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -131,6 +131,36 @@ type FieldMapping struct { UserVar *ast.VariableExpr } +// reorderColumns reorder the e.insertColumns according to the order of columnNames +// Note: We must ensure there must be one-to-one mapping between e.insertColumns and columnNames in terms of column name. +func (e *LoadDataInfo) reorderColumns(columnNames []string) error { + cols := e.insertColumns + + if len(cols) != len(columnNames) { + return ErrColumnsNotMatched + } + + reorderedColumns := make([]*table.Column, len(cols)) + + if columnNames == nil { + return nil + } + + mapping := make(map[string]int) + for idx, colName := range columnNames { + mapping[strings.ToLower(colName)] = idx + } + + for _, col := range cols { + idx := mapping[col.Name.L] + reorderedColumns[idx] = col + } + + e.insertColumns = reorderedColumns + + return nil +} + // initLoadColumns sets columns which the input fields loaded to. func (e *LoadDataInfo) initLoadColumns(columnNames []string) error { var cols []*table.Column @@ -163,6 +193,13 @@ func (e *LoadDataInfo) initLoadColumns(columnNames []string) error { break } } + + // e.insertColumns is appended according to the original tables' column sequence. + // We have to reorder it to follow the use-specified column order which is shown in the columnNames. + if err = e.reorderColumns(columnNames); err != nil { + return err + } + e.rowLen = len(e.insertColumns) // Check column whether is specified only once. err = table.CheckOnce(cols) diff --git a/server/server_test.go b/server/server_test.go index 034587d7be1f7..f81d8ff5c9069 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -850,6 +850,144 @@ func (cli *testServerClient) checkRows(t *testing.T, rows *sql.Rows, expectedRow require.Equal(t, strings.Join(expectedRows, "\n"), strings.Join(result, "\n")) } +func (cli *testServerClient) runTestLoadDataWithColumnList(t *testing.T, _ *Server) { + fp, err := os.CreateTemp("", "load_data_test.csv") + require.NoError(t, err) + path := fp.Name() + require.NotNil(t, fp) + defer func() { + err = fp.Close() + require.NoError(t, err) + err = os.Remove(path) + require.NoError(t, err) + }() + + _, err = fp.WriteString("dsadasdas\n" + + "\"1\",\"1\",,\"2022-04-19\",\"a\",\"2022-04-19 00:00:01\"\n" + + "\"1\",\"2\",\"a\",\"2022-04-19\",\"a\",\"2022-04-19 00:00:01\"\n" + + "\"1\",\"3\",\"a\",\"2022-04-19\",\"a\",\"2022-04-19 00:00:01\"\n" + + "\"1\",\"4\",\"a\",\"2022-04-19\",\"a\",\"2022-04-19 00:00:01\"") + + cli.runTestsOnNewDB(t, func(config *mysql.Config) { + config.AllowAllFiles = true + config.Params["sql_mode"] = "''" + }, "LoadData", func(db *testkit.DBTestKit) { + db.MustExec("use test") + db.MustExec("drop table if exists t66") + db.MustExec("create table t66 (id int primary key,k int,c varchar(10),dt date,vv char(1),ts datetime)") + db.MustExec(fmt.Sprintf("LOAD DATA LOCAL INFILE '%s' INTO TABLE t66 FIELDS TERMINATED BY ',' ENCLOSED BY '\\\"' IGNORE 1 LINES (k,id,c,dt,vv,ts)", path)) + rows := db.MustQuery("select * from t66") + var ( + id sql.NullString + k sql.NullString + c sql.NullString + dt sql.NullString + vv sql.NullString + ts sql.NullString + ) + columns := []*sql.NullString{&k, &id, &c, &dt, &vv, &ts} + require.Truef(t, rows.Next(), "unexpected data") + err := rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,1,,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,2,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,3,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,4,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + }) + + // Also test cases where column list only specifies partial columns + cli.runTestsOnNewDB(t, func(config *mysql.Config) { + config.AllowAllFiles = true + config.Params["sql_mode"] = "''" + }, "LoadData", func(db *testkit.DBTestKit) { + db.MustExec("use test") + db.MustExec("drop table if exists t66") + db.MustExec("create table t66 (id int primary key,k int,c varchar(10),dt date,vv char(1),ts datetime)") + db.MustExec(fmt.Sprintf("LOAD DATA LOCAL INFILE '%s' INTO TABLE t66 FIELDS TERMINATED BY ',' ENCLOSED BY '\\\"' IGNORE 1 LINES (k,id,c)", path)) + rows := db.MustQuery("select * from t66") + var ( + id sql.NullString + k sql.NullString + c sql.NullString + dt sql.NullString + vv sql.NullString + ts sql.NullString + ) + columns := []*sql.NullString{&k, &id, &c, &dt, &vv, &ts} + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,1,,,,", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,2,a,,,", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,3,a,,,", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,4,a,,,", ",")) + }) + + // Also test for case-insensitivity + cli.runTestsOnNewDB(t, func(config *mysql.Config) { + config.AllowAllFiles = true + config.Params["sql_mode"] = "''" + }, "LoadData", func(db *testkit.DBTestKit) { + db.MustExec("use test") + db.MustExec("drop table if exists t66") + db.MustExec("create table t66 (id int primary key,k int,c varchar(10),dt date,vv char(1),ts datetime)") + // We modify the upper case and lower case in the column list to test the case-insensitivity + db.MustExec(fmt.Sprintf("LOAD DATA LOCAL INFILE '%s' INTO TABLE t66 FIELDS TERMINATED BY ',' ENCLOSED BY '\\\"' IGNORE 1 LINES (K,Id,c,dT,Vv,Ts)", path)) + rows := db.MustQuery("select * from t66") + var ( + id sql.NullString + k sql.NullString + c sql.NullString + dt sql.NullString + vv sql.NullString + ts sql.NullString + ) + columns := []*sql.NullString{&k, &id, &c, &dt, &vv, &ts} + require.Truef(t, rows.Next(), "unexpected data") + err := rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,1,,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,2,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,3,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,4,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + }) +} + +func columnsAsExpected(t *testing.T, columns []*sql.NullString, expected []string) { + require.Equal(t, len(columns), len(expected)) + + for i := 0; i < len(columns); i++ { + require.Equal(t, expected[i], columns[i].String) + } +} + func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { // create a file and write data. path := "/tmp/load_data_test.csv" diff --git a/server/tidb_serial_test.go b/server/tidb_serial_test.go index 141681e1df24e..ec323fcaec0f3 100644 --- a/server/tidb_serial_test.go +++ b/server/tidb_serial_test.go @@ -41,6 +41,7 @@ func TestLoadData(t *testing.T) { ts, cleanup := createTidbTestSuite(t) defer cleanup() + ts.runTestLoadDataWithColumnList(t, ts.server) ts.runTestLoadData(t, ts.server) ts.runTestLoadDataWithSelectIntoOutfile(t, ts.server) ts.runTestLoadDataForSlowLog(t, ts.server) From 73df8dafc85e509eaae17383002a226232dbcca0 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 24 Jun 2022 18:38:38 +0800 Subject: [PATCH 49/51] raw_restore: fix the issue that raw restore rewrite the t prefix keys (#35641) (#35695) close pingcap/tidb#35279 --- br/pkg/task/restore_raw.go | 3 +-- br/tests/br_rawkv/run.sh | 28 +++++++++++++++++++++++++++- 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/br/pkg/task/restore_raw.go b/br/pkg/task/restore_raw.go index 25409e8c28e31..10d4c6ed54841 100644 --- a/br/pkg/task/restore_raw.go +++ b/br/pkg/task/restore_raw.go @@ -130,8 +130,7 @@ func RunRestoreRaw(c context.Context, g glue.Glue, cmdName string, cfg *RestoreR !cfg.LogProgress) // RawKV restore does not need to rewrite keys. - rewrite := &restore.RewriteRules{} - err = restore.SplitRanges(ctx, client, ranges, rewrite, updateCh, true) + err = restore.SplitRanges(ctx, client, ranges, nil, updateCh, true) if err != nil { return errors.Trace(err) } diff --git a/br/tests/br_rawkv/run.sh b/br/tests/br_rawkv/run.sh index 97450d3e65fc7..b32cca0f8e41f 100755 --- a/br/tests/br_rawkv/run.sh +++ b/br/tests/br_rawkv/run.sh @@ -98,12 +98,23 @@ run_test() { --key "$TEST_DIR/certs/br.key" \ --mode put --put-data "311121:31, 31112100:32, 311122:33, 31112200:34, 3111220000:35, 311123:36" + + # put some keys starts with t. https://github.com/pingcap/tidb/issues/35279 + # t_128_r_12 ----> 745f3132385f725f3132 + # t_128_r_13 ----> 745f3132385f725f3133 + bin/rawkv --pd $PD_ADDR \ + --ca "$TEST_DIR/certs/ca.pem" \ + --cert "$TEST_DIR/certs/br.pem" \ + --key "$TEST_DIR/certs/br.key" \ + --mode put --put-data "745f3132385f725f3132:31, 745f3132385f725f3133:32" + checksum_ori=$(checksum 31 3130303030303030) checksum_partial=$(checksum 311111 311122) + checksum_t_prefix=$(checksum 745f3132385f725f3131 745f3132385f725f3134) # backup rawkv echo "backup start..." - run_br --pd $PD_ADDR backup raw -s "local://$BACKUP_DIR" --start 31 --end 3130303030303030 --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + run_br --pd $PD_ADDR backup raw -s "local://$BACKUP_DIR" --start 31 --end 745f3132385f725f3134 --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" # delete data in range[start-key, end-key) clean 31 3130303030303030 @@ -153,6 +164,21 @@ run_test() { fail_and_exit fi + echo "t prefix restore start..." + run_br --pd $PD_ADDR restore raw -s "local://$BACKUP_DIR" --start "745f3132385f725f3131" --end "745f3132385f725f3134" --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + bin/rawkv --pd $PD_ADDR \ + --ca "$TEST_DIR/certs/ca.pem" \ + --cert "$TEST_DIR/certs/br.pem" \ + --key "$TEST_DIR/certs/br.key" \ + --mode scan --start-key 745f3132385f725f3131 --end-key 745f3132385f725f3134 + + checksum_new=$(checksum 745f3132385f725f3131 745f3132385f725f3134) + + if [ "$checksum_new" != "$checksum_t_prefix" ];then + echo "checksum failed after restore" + fail_and_exit + fi + export GO_FAILPOINTS="" } From 216f74660d6826e62e1d257902793ad0ef5b8a72 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 24 Jun 2022 23:26:37 +0800 Subject: [PATCH 50/51] planner: fix bug that planner generates wrong 2 phase aggregate plan for TiFlash (#34779) (#35589) close pingcap/tidb#34682 --- executor/tiflash_test.go | 25 ++++++ planner/core/enforce_mpp_test.go | 51 ++++++++++++ planner/core/exhaust_physical_plans.go | 13 +++- .../core/testdata/enforce_mpp_suite_in.json | 9 +++ .../core/testdata/enforce_mpp_suite_out.json | 78 +++++++++++++++++++ 5 files changed, 175 insertions(+), 1 deletion(-) diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index 5435463657677..930e316680251 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -1090,3 +1090,28 @@ func (s *tiflashTestSuite) TestForbidTiflashDuringStaleRead(c *C) { c.Assert(strings.Contains(res, "tiflash"), IsFalse) c.Assert(strings.Contains(res, "tikv"), IsTrue) } + +func (s *tiflashTestSuite) TestAggPushDownCountStar(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists c") + tk.MustExec("drop table if exists o") + tk.MustExec("create table c(c_id bigint primary key)") + tk.MustExec("create table o(o_id bigint primary key, c_id bigint not null)") + tk.MustExec("alter table c set tiflash replica 1") + tb := testGetTableByName(c, tk.Se, "test", "c") + err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + tk.MustExec("alter table o set tiflash replica 1") + tb = testGetTableByName(c, tk.Se, "test", "o") + err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + tk.MustExec("insert into c values(1),(2),(3),(4),(5)") + tk.MustExec("insert into o values(1,1),(2,1),(3,2),(4,2),(5,2)") + + tk.MustExec("set @@tidb_enforce_mpp=1") + tk.MustExec("set @@tidb_opt_agg_push_down=1") + + tk.MustQuery("select count(*) from c, o where c.c_id=o.c_id").Check(testkit.Rows("5")) +} diff --git a/planner/core/enforce_mpp_test.go b/planner/core/enforce_mpp_test.go index f9eb265c1313f..8be5ddcb163ad 100644 --- a/planner/core/enforce_mpp_test.go +++ b/planner/core/enforce_mpp_test.go @@ -360,3 +360,54 @@ func (s *testEnforceMPPSuite) TestEnforceMPPWarning4(c *C) { c.Assert(s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()), DeepEquals, output[i].Warn) } } + +// Test agg push down for MPP mode +func (s *testEnforceMPPSuite) TestMPP2PhaseAggPushDown(c *C) { + tk := testkit.NewTestKit(c, s.store) + + // test table + tk.MustExec("use test") + tk.MustExec("drop table if exists c") + tk.MustExec("drop table if exists o") + tk.MustExec("create table c(c_id bigint)") + tk.MustExec("create table o(o_id bigint, c_id bigint not null)") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Se) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + c.Assert(exists, IsTrue) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "c" || tblInfo.Name.L == "o" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + c.Assert(s.testData.ConvertSQLWarnToStrings(tk.Se.GetSessionVars().StmtCtx.GetWarnings()), DeepEquals, output[i].Warn) + } +} diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index c3429e8969044..5dd497f2385e0 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2561,6 +2561,11 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert if prop.MPPPartitionTp == property.BroadcastType { return nil } + + // Is this aggregate a final stage aggregate? + // Final agg can't be split into multi-stage aggregate + hasFinalAgg := len(la.AggFuncs) > 0 && la.AggFuncs[0].Mode == aggregation.FinalMode + if len(la.GroupByItems) > 0 { partitionCols := la.GetPotentialPartitionKeys() // trying to match the required parititions. @@ -2584,6 +2589,11 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert hashAggs = append(hashAggs, agg) } + // Final agg can't be split into multi-stage aggregate, so exit early + if hasFinalAgg { + return + } + // 2-phase agg childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, RejectSort: true} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) @@ -2600,7 +2610,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert agg.MppRunMode = MppTiDB hashAggs = append(hashAggs, agg) } - } else { + } else if !hasFinalAgg { // TODO: support scalar agg in MPP, merge the final result to one node childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) @@ -2649,6 +2659,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy if prop.IsFlashProp() { taskTypes = []property.TaskType{prop.TaskTp} } + for _, taskTp := range taskTypes { if taskTp == property.MppTaskType { mppAggs := la.tryToGetMppHashAggs(prop) diff --git a/planner/core/testdata/enforce_mpp_suite_in.json b/planner/core/testdata/enforce_mpp_suite_in.json index 70f519b80192f..df637a715419f 100644 --- a/planner/core/testdata/enforce_mpp_suite_in.json +++ b/planner/core/testdata/enforce_mpp_suite_in.json @@ -85,5 +85,14 @@ "explain select a from t where t.a>1 or t.a not in (select a from t); -- 8. anti left outer semi join", "explain select a from t where t.a not in (select a from s where t.a<1); -- 9. non left join has left conditions" ] + }, + { + "name": "TestMPP2PhaseAggPushDown", + "cases": [ + "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_opt_agg_push_down=1;", + "EXPLAIN select count(*) from c, o where c.c_id=o.c_id; -- 1. test agg push down, scalar aggregate", + "EXPLAIN select o.o_id, count(*) from c, o where c.c_id=o.c_id group by o.o_id; -- 2. test agg push down, group by non-join column", + "EXPLAIN select o.c_id, count(*) from c, o where c.c_id=o.c_id group by o.c_id; -- 3. test agg push down, group by join column" + ] } ] diff --git a/planner/core/testdata/enforce_mpp_suite_out.json b/planner/core/testdata/enforce_mpp_suite_out.json index d93fd15e63968..a3089682a1227 100644 --- a/planner/core/testdata/enforce_mpp_suite_out.json +++ b/planner/core/testdata/enforce_mpp_suite_out.json @@ -632,5 +632,83 @@ ] } ] + }, + { + "Name": "TestMPP2PhaseAggPushDown", + "Cases": [ + { + "SQL": "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_opt_agg_push_down=1;", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN select count(*) from c, o where c.c_id=o.c_id; -- 1. test agg push down, scalar aggregate", + "Plan": [ + "HashAgg_11 1.00 root funcs:count(Column#7)->Column#6", + "└─TableReader_34 9990.00 root data:ExchangeSender_33", + " └─ExchangeSender_33 9990.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin_13 9990.00 batchCop[tiflash] inner join, equal:[eq(test.c.c_id, test.o.c_id)]", + " ├─ExchangeReceiver_32(Build) 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender_31 8000.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection_27 8000.00 batchCop[tiflash] Column#7, test.o.c_id", + " │ └─HashAgg_28 8000.00 batchCop[tiflash] group by:test.o.c_id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.o.c_id)->test.o.c_id", + " │ └─ExchangeReceiver_30 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender_29 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.o.c_id, collate: binary]", + " │ └─HashAgg_19 8000.00 batchCop[tiflash] group by:test.o.c_id, funcs:count(1)->Column#8", + " │ └─TableFullScan_26 10000.00 batchCop[tiflash] table:o keep order:false, stats:pseudo", + " └─Selection_17(Probe) 9990.00 batchCop[tiflash] not(isnull(test.c.c_id))", + " └─TableFullScan_16 10000.00 batchCop[tiflash] table:c keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select o.o_id, count(*) from c, o where c.c_id=o.c_id group by o.o_id; -- 2. test agg push down, group by non-join column", + "Plan": [ + "TableReader_77 8000.00 root data:ExchangeSender_76", + "└─ExchangeSender_76 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection_10 8000.00 batchCop[tiflash] test.o.o_id, Column#6", + " └─Projection_75 8000.00 batchCop[tiflash] Column#6, test.o.o_id", + " └─HashAgg_74 8000.00 batchCop[tiflash] group by:test.o.o_id, funcs:count(Column#7)->Column#6, funcs:firstrow(Column#8)->test.o.o_id", + " └─ExchangeReceiver_70 9990.00 batchCop[tiflash] ", + " └─ExchangeSender_69 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.o.o_id, collate: binary]", + " └─HashJoin_68 9990.00 batchCop[tiflash] inner join, equal:[eq(test.c.c_id, test.o.c_id)]", + " ├─ExchangeReceiver_33(Build) 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender_32 8000.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection_28 8000.00 batchCop[tiflash] Column#7, Column#8, test.o.o_id, test.o.c_id", + " │ └─HashAgg_29 8000.00 batchCop[tiflash] group by:test.o.c_id, test.o.o_id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.o.o_id)->Column#8, funcs:firstrow(test.o.o_id)->test.o.o_id, funcs:firstrow(test.o.c_id)->test.o.c_id", + " │ └─ExchangeReceiver_31 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender_30 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.o.o_id, collate: binary], [name: test.o.c_id, collate: binary]", + " │ └─HashAgg_20 8000.00 batchCop[tiflash] group by:test.o.c_id, test.o.o_id, funcs:count(1)->Column#9", + " │ └─TableFullScan_27 10000.00 batchCop[tiflash] table:o keep order:false, stats:pseudo", + " └─Selection_18(Probe) 9990.00 batchCop[tiflash] not(isnull(test.c.c_id))", + " └─TableFullScan_17 10000.00 batchCop[tiflash] table:c keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select o.c_id, count(*) from c, o where c.c_id=o.c_id group by o.c_id; -- 3. test agg push down, group by join column", + "Plan": [ + "TableReader_77 8000.00 root data:ExchangeSender_76", + "└─ExchangeSender_76 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection_10 8000.00 batchCop[tiflash] test.o.c_id, Column#6", + " └─Projection_75 8000.00 batchCop[tiflash] Column#6, test.o.c_id", + " └─HashAgg_74 8000.00 batchCop[tiflash] group by:test.o.c_id, funcs:count(Column#7)->Column#6, funcs:firstrow(Column#8)->test.o.c_id", + " └─ExchangeReceiver_70 9990.00 batchCop[tiflash] ", + " └─ExchangeSender_69 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.o.c_id, collate: binary]", + " └─HashJoin_68 9990.00 batchCop[tiflash] inner join, equal:[eq(test.c.c_id, test.o.c_id)]", + " ├─ExchangeReceiver_33(Build) 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender_32 8000.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection_28 8000.00 batchCop[tiflash] Column#7, Column#8, test.o.c_id", + " │ └─HashAgg_29 8000.00 batchCop[tiflash] group by:test.o.c_id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.o.c_id)->Column#8, funcs:firstrow(test.o.c_id)->test.o.c_id", + " │ └─ExchangeReceiver_31 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender_30 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.o.c_id, collate: binary]", + " │ └─HashAgg_20 8000.00 batchCop[tiflash] group by:test.o.c_id, funcs:count(1)->Column#9", + " │ └─TableFullScan_27 10000.00 batchCop[tiflash] table:o keep order:false, stats:pseudo", + " └─Selection_18(Probe) 9990.00 batchCop[tiflash] not(isnull(test.c.c_id))", + " └─TableFullScan_17 10000.00 batchCop[tiflash] table:c keep order:false, stats:pseudo" + ], + "Warn": null + } + ] } ] From 8f13676e0e07fcc9b6c7b4924e33a24194f9c67d Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 24 Jun 2022 23:46:37 +0800 Subject: [PATCH 51/51] planner: fix the issue that the optimizer caches wrong TableDual plans under binary protocol (#34709) (#34737) close pingcap/tidb#34678, close pingcap/tidb#34690 --- executor/builder.go | 2 +- planner/core/cache.go | 23 +++++++++++--- planner/core/common_plans.go | 60 +++++++++++++++++++++++------------- planner/core/planbuilder.go | 4 +-- 4 files changed, 59 insertions(+), 30 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 9b10061ca21d7..cf3c6978fbe87 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -729,7 +729,7 @@ func (b *executorBuilder) buildExecute(v *plannercore.Execute) Executor { baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), is: b.is, name: v.Name, - usingVars: v.UsingVars, + usingVars: v.TxtProtoVars, id: v.ExecID, stmt: v.Stmt, plan: v.Plan, diff --git a/planner/core/cache.go b/planner/core/cache.go index 0d01f2bb98f73..e07ac56757b42 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -15,6 +15,7 @@ package core import ( + "bytes" "math" "sync/atomic" "time" @@ -181,25 +182,37 @@ type PlanCacheValue struct { Plan Plan OutPutNames []*types.FieldName TblInfo2UnionScan map[*model.TableInfo]bool - UserVarTypes FieldSlice + TxtVarTypes FieldSlice // variable types under text protocol + BinVarTypes []byte // variable types under binary protocol + IsBinProto bool // whether this plan is under binary protocol BindSQL string } +func (v *PlanCacheValue) varTypesUnchanged(binVarTps []byte, txtVarTps []*types.FieldType) bool { + if v.IsBinProto { + return bytes.Equal(v.BinVarTypes, binVarTps) + } + return v.TxtVarTypes.Equal(txtVarTps) +} + // NewPlanCacheValue creates a SQLCacheValue. -func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool, userVarTps []*types.FieldType, bindSQL string) *PlanCacheValue { +func NewPlanCacheValue(plan Plan, names []*types.FieldName, srcMap map[*model.TableInfo]bool, + isBinProto bool, binVarTypes []byte, txtVarTps []*types.FieldType, bindSQL string) *PlanCacheValue { dstMap := make(map[*model.TableInfo]bool) for k, v := range srcMap { dstMap[k] = v } - userVarTypes := make([]types.FieldType, len(userVarTps)) - for i, tp := range userVarTps { + userVarTypes := make([]types.FieldType, len(txtVarTps)) + for i, tp := range txtVarTps { userVarTypes[i] = *tp } return &PlanCacheValue{ Plan: plan, OutPutNames: names, TblInfo2UnionScan: dstMap, - UserVarTypes: userVarTypes, + TxtVarTypes: userVarTypes, + BinVarTypes: binVarTypes, + IsBinProto: isBinProto, BindSQL: bindSQL, } } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index e0a4063df6417..dca5d8cfe8138 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -184,12 +184,15 @@ type Prepare struct { type Execute struct { baseSchemaProducer - Name string - UsingVars []expression.Expression - PrepareParams []types.Datum - ExecID uint32 - SnapshotTS uint64 - IsStaleness bool + Name string + TxtProtoVars []expression.Expression // parsed variables under text protocol + BinProtoVars []types.Datum // parsed variables under binary protocol + ExecID uint32 + // Deprecated: SnapshotTS now is only used for asserting after refactoring stale read, it will be removed later. + SnapshotTS uint64 + // Deprecated: IsStaleness now is only used for asserting after refactoring stale read, it will be removed later. + IsStaleness bool + // Deprecated: ReadReplicaScope now is only used for asserting after refactoring stale read, it will be removed later. ReadReplicaScope string Stmt ast.StmtNode StmtType string @@ -228,25 +231,25 @@ func (e *Execute) OptimizePreparedPlan(ctx context.Context, sctx sessionctx.Cont prepared := preparedObj.PreparedAst vars.StmtCtx.StmtType = prepared.StmtType - paramLen := len(e.PrepareParams) + paramLen := len(e.BinProtoVars) if paramLen > 0 { - // for binary protocol execute, argument is placed in vars.PrepareParams + // for binary protocol execute, argument is placed in vars.BinProtoVars if len(prepared.Params) != paramLen { return errors.Trace(ErrWrongParamCount) } - vars.PreparedParams = e.PrepareParams + vars.PreparedParams = e.BinProtoVars for i, val := range vars.PreparedParams { param := prepared.Params[i].(*driver.ParamMarkerExpr) param.Datum = val param.InExecute = true } } else { - // for `execute stmt using @a, @b, @c`, using value in e.UsingVars - if len(prepared.Params) != len(e.UsingVars) { + // for `execute stmt using @a, @b, @c`, using value in e.TxtProtoVars + if len(prepared.Params) != len(e.TxtProtoVars) { return errors.Trace(ErrWrongParamCount) } - for i, usingVar := range e.UsingVars { + for i, usingVar := range e.TxtProtoVars { val, err := usingVar.Eval(chunk.Row{}) if err != nil { return err @@ -449,15 +452,28 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, bindSQL = GetBindSQL4PlanCache(sctx, preparedStmt) cacheKey = NewPlanCacheKey(sctx.GetSessionVars(), e.ExecID, prepared.SchemaVersion) } - tps := make([]*types.FieldType, len(e.UsingVars)) - varsNum := len(e.UsingVars) - for i, param := range e.UsingVars { - name := param.(*expression.ScalarFunction).GetArgs()[0].String() - tps[i] = sctx.GetSessionVars().UserVarTypes[name] - if tps[i] == nil { - tps[i] = types.NewFieldType(mysql.TypeNull) + + var varsNum int + var binVarTypes []byte + var txtVarTypes []*types.FieldType + isBinProtocol := len(e.BinProtoVars) > 0 + if isBinProtocol { // binary protocol + varsNum = len(e.BinProtoVars) + for _, param := range e.BinProtoVars { + binVarTypes = append(binVarTypes, param.Kind()) + } + } else { // txt protocol + varsNum = len(e.TxtProtoVars) + for _, param := range e.TxtProtoVars { + name := param.(*expression.ScalarFunction).GetArgs()[0].String() + tp := sctx.GetSessionVars().UserVarTypes[name] + if tp == nil { + tp = types.NewFieldType(mysql.TypeNull) + } + txtVarTypes = append(txtVarTypes, tp) } } + if prepared.CachedPlan != nil { // Rewriting the expression in the select.where condition will convert its // type from "paramMarker" to "Constant".When Point Select queries are executed, @@ -498,7 +514,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, sctx.PreparedPlanCache().Delete(cacheKey) break } - if !cachedVal.UserVarTypes.Equal(tps) { + if !cachedVal.varTypesUnchanged(binVarTypes, txtVarTypes) { continue } planValid := true @@ -567,13 +583,13 @@ REBUILD: cacheKey = NewPlanCacheKey(sessVars, e.ExecID, prepared.SchemaVersion) sessVars.IsolationReadEngines[kv.TiFlash] = struct{}{} } - cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, tps, sessVars.StmtCtx.BindSQL) + cached := NewPlanCacheValue(p, names, stmtCtx.TblInfo2UnionScan, isBinProtocol, binVarTypes, txtVarTypes, sessVars.StmtCtx.BindSQL) preparedStmt.NormalizedPlan, preparedStmt.PlanDigest = NormalizePlan(p) stmtCtx.SetPlanDigest(preparedStmt.NormalizedPlan, preparedStmt.PlanDigest) if cacheVals, exists := sctx.PreparedPlanCache().Get(cacheKey); exists { hitVal := false for i, cacheVal := range cacheVals.([]*PlanCacheValue) { - if cacheVal.UserVarTypes.Equal(tps) { + if cacheVal.varTypesUnchanged(binVarTypes, txtVarTypes) { hitVal = true cacheVals.([]*PlanCacheValue)[i] = cached break diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 131cbe9c35466..84954e37575ee 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -772,9 +772,9 @@ func (b *PlanBuilder) buildExecute(ctx context.Context, v *ast.ExecuteStmt) (Pla } vars = append(vars, newExpr) } - exe := &Execute{Name: v.Name, UsingVars: vars, ExecID: v.ExecID} + exe := &Execute{Name: v.Name, TxtProtoVars: vars, ExecID: v.ExecID} if v.BinaryArgs != nil { - exe.PrepareParams = v.BinaryArgs.([]types.Datum) + exe.BinProtoVars = v.BinaryArgs.([]types.Datum) } return exe, nil }