From 0581cb4ed9267a202c771fc969d432c4f1535c81 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 10 May 2021 18:47:27 +0800 Subject: [PATCH 001/343] executor: add some test cases about partition-table dynamic mode with view (#24448) --- executor/partition_table_test.go | 72 ++++++++++++++++++++++++++++++++ 1 file changed, 72 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index ee7d807ef4e8f..58d6293b4ce54 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -227,6 +227,78 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows()) } +func (s *partitionTableSuite) TestView(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_view") + tk.MustExec("use test_view") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table thash (a int, b int, key(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a varchar(10), b varchar(10), key(a)) partition by range columns(a) ( + partition p0 values less than ('300'), + partition p1 values less than ('600'), + partition p2 values less than ('900'), + partition p3 values less than ('9999'))`) + tk.MustExec(`create table t1 (a int, b int, key(a))`) + tk.MustExec(`create table t2 (a varchar(10), b varchar(10), key(a))`) + + // insert the same data into thash and t1 + vals := make([]string, 0, 3000) + for i := 0; i < 3000; i++ { + vals = append(vals, fmt.Sprintf(`(%v, %v)`, rand.Intn(10000), rand.Intn(10000))) + } + tk.MustExec(fmt.Sprintf(`insert into thash values %v`, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf(`insert into t1 values %v`, strings.Join(vals, ", "))) + + // insert the same data into trange and t2 + vals = vals[:0] + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf(`("%v", "%v")`, rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec(fmt.Sprintf(`insert into trange values %v`, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf(`insert into t2 values %v`, strings.Join(vals, ", "))) + + // test views on a single table + tk.MustExec(`create definer='root'@'localhost' view vhash as select a*2 as a, a+b as b from thash`) + tk.MustExec(`create definer='root'@'localhost' view v1 as select a*2 as a, a+b as b from t1`) + tk.MustExec(`create definer='root'@'localhost' view vrange as select concat(a, b) as a, a+b as b from trange`) + tk.MustExec(`create definer='root'@'localhost' view v2 as select concat(a, b) as a, a+b as b from t2`) + for i := 0; i < 100; i++ { + xhash := rand.Intn(10000) + tk.MustQuery(fmt.Sprintf(`select * from vhash where a>=%v`, xhash)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v1 where a>=%v`, xhash)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vhash where b>=%v`, xhash)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v1 where b>=%v`, xhash)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vhash where a>=%v and b>=%v`, xhash, xhash)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v1 where a>=%v and b>=%v`, xhash, xhash)).Sort().Rows()) + + xrange := fmt.Sprintf(`"%v"`, rand.Intn(1000)) + tk.MustQuery(fmt.Sprintf(`select * from vrange where a>=%v`, xrange)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v2 where a>=%v`, xrange)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vrange where b>=%v`, xrange)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v2 where b>=%v`, xrange)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vrange where a>=%v and b<=%v`, xrange, xrange)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from v2 where a>=%v and b<=%v`, xrange, xrange)).Sort().Rows()) + } + + // test views on both tables + tk.MustExec(`create definer='root'@'localhost' view vboth as select thash.a+trange.a as a, thash.b+trange.b as b from thash, trange where thash.a=trange.a`) + tk.MustExec(`create definer='root'@'localhost' view vt as select t1.a+t2.a as a, t1.b+t2.b as b from t1, t2 where t1.a=t2.a`) + for i := 0; i < 100; i++ { + x := rand.Intn(10000) + tk.MustQuery(fmt.Sprintf(`select * from vboth where a>=%v`, x)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from vt where a>=%v`, x)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vboth where b>=%v`, x)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from vt where b>=%v`, x)).Sort().Rows()) + tk.MustQuery(fmt.Sprintf(`select * from vboth where a>=%v and b>=%v`, x, x)).Sort().Check( + tk.MustQuery(fmt.Sprintf(`select * from vt where a>=%v and b>=%v`, x, x)).Sort().Rows()) + } +} + func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From a0fad5a9ef886296c8b6ebbf1551ec763c64a098 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 10 May 2021 19:57:30 +0800 Subject: [PATCH 002/343] planner: add more test cases about dynamic-mode with new-collation (#24454) --- planner/core/integration_test.go | 40 ++++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 816c94fa32a66..f717888caf2c9 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -806,6 +806,46 @@ func (s *testIntegrationSerialSuite) TestMPPWithBroadcastExchangeUnderNewCollati } } +func (s *testIntegrationSerialSuite) TestPartitionTableDynamicModeUnderNewCollation(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_new_collation") + tk.MustExec("use test_new_collation") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash + range partition + tk.MustExec(`CREATE TABLE thash (a int, c varchar(20) charset utf8mb4 collate utf8mb4_general_ci, key(a)) partition by hash(a) partitions 4`) + tk.MustExec(`CREATE TABLE trange (a int, c varchar(20) charset utf8mb4 collate utf8mb4_general_ci, key(a)) partition by range(a) ( + partition p0 values less than (10), + partition p1 values less than (20), + partition p2 values less than (30), + partition p3 values less than (40))`) + tk.MustExec(`insert into thash values (1, 'a'), (1, 'A'), (11, 'a'), (11, 'A'), (21, 'a'), (21, 'A'), (31, 'a'), (31, 'A')`) + tk.MustExec(`insert into trange values (1, 'a'), (1, 'A'), (11, 'a'), (11, 'A'), (21, 'a'), (21, 'A'), (31, 'a'), (31, 'A')`) + tk.MustQuery(`select * from thash use index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + tk.MustQuery(`select * from thash ignore index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + tk.MustQuery(`select * from trange use index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + tk.MustQuery(`select * from trange ignore index(a) where a in (1, 11, 31) and c='a'`).Sort().Check(testkit.Rows("1 A", "1 a", "11 A", "11 a", "31 A", "31 a")) + + // range partition and partitioned by utf8mb4_general_ci + tk.MustExec(`create table strrange(a varchar(10) charset utf8mb4 collate utf8mb4_general_ci, b int) partition by range columns(a) ( + partition p0 values less than ('a'), + partition p1 values less than ('k'), + partition p2 values less than ('z'))`) + tk.MustExec("insert into strrange values ('a', 1), ('A', 1), ('y', 1), ('Y', 1), ('q', 1)") + tk.MustQuery("select * from strrange where a in ('a', 'y')").Sort().Check(testkit.Rows("A 1", "Y 1", "a 1", "y 1")) + + // list partition and partitioned by utf8mb4_general_ci + tk.MustExec(`create table strlist(a varchar(10) charset utf8mb4 collate utf8mb4_general_ci, b int) partition by list(a) ( + partition p0 values in ('a', 'b'), + partition p1 values in ('c', 'd'), + partition p2 values in ('e', 'f'))`) + tk.MustExec("insert into strlist values ('a', 1), ('A', 1), ('d', 1), ('D', 1), ('e', 1)") + tk.MustQuery(`select * from strlist where a='a'`).Sort().Check(testkit.Rows("A 1", "a 1")) + tk.MustQuery(`select * from strlist where a in ('D', 'e')`).Sort().Check(testkit.Rows("D 1", "d 1", "e 1")) +} + func (s *testIntegrationSerialSuite) TestMPPAvgRewrite(c *C) { defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) From 5e9e0e6e37be14a84f22ac86a2e98dd36b092f66 Mon Sep 17 00:00:00 2001 From: xhe Date: Mon, 10 May 2021 21:00:36 +0800 Subject: [PATCH 003/343] *: consitent get infoschema (#24230) --- distsql/request_builder.go | 7 +++---- executor/adapter.go | 2 +- executor/analyze.go | 4 ++-- executor/analyze_test.go | 12 ++++++------ executor/builder.go | 2 +- executor/compiler.go | 2 +- executor/coprocessor.go | 2 +- executor/distsql.go | 18 ++++++++++++------ executor/executor_test.go | 4 ++-- executor/grant.go | 2 +- executor/index_merge_reader.go | 2 +- executor/infoschema_reader.go | 16 ++++++++-------- executor/load_stats.go | 2 +- executor/metrics_reader_test.go | 2 +- executor/partition_table_test.go | 2 +- executor/point_get.go | 2 +- executor/prepared.go | 2 +- executor/simple.go | 2 +- executor/table_reader.go | 11 +++++++---- expression/builtin_info.go | 6 +++--- infoschema/infoschema.go | 26 -------------------------- infoschema/tables.go | 2 +- planner/core/cacheable_checker_test.go | 2 +- planner/core/explain.go | 8 ++++---- planner/core/expression_rewriter.go | 5 +++-- planner/core/integration_test.go | 2 +- planner/core/point_get_plan.go | 6 +++--- planner/core/prepare_test.go | 2 +- session/session.go | 10 +++++----- sessionctx/variable/session.go | 19 +++++++++++++++++++ statistics/handle/ddl.go | 2 +- statistics/handle/update.go | 2 +- 32 files changed, 95 insertions(+), 93 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index ce577b993d009..69a6da548ec60 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -230,10 +230,9 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req builder.Request.TaskID = sv.StmtCtx.TaskID builder.Request.Priority = builder.getKVPriority(sv) builder.Request.ReplicaRead = sv.GetReplicaRead() - if sv.SnapshotInfoschema != nil { - builder.Request.SchemaVar = infoschema.GetInfoSchemaBySessionVars(sv).SchemaMetaVersion() - } else { - builder.Request.SchemaVar = sv.TxnCtx.SchemaVersion + // in tests, it may be null + if is, ok := sv.GetInfoSchema().(infoschema.InfoSchema); ok { + builder.Request.SchemaVar = is.SchemaMetaVersion() } builder.txnScope = sv.TxnCtx.TxnScope builder.IsStaleness = sv.TxnCtx.IsStaleness diff --git a/executor/adapter.go b/executor/adapter.go index 5e5b7990f61d9..5f5229195c3f9 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -268,7 +268,7 @@ func (a *ExecStmt) IsReadOnly(vars *variable.SessionVars) bool { // RebuildPlan rebuilds current execute statement plan. // It returns the current information schema version that 'a' is using. func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { - is := infoschema.GetInfoSchema(a.Ctx) + is := a.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) a.InfoSchema = is if err := plannercore.Preprocess(a.Ctx, a.StmtNode, is, plannercore.InTxnRetry); err != nil { return 0, err diff --git a/executor/analyze.go b/executor/analyze.go index 2d3187842845c..b7e9e51fec7da 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -187,7 +187,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } if needGlobalStats { for globalStatsID, info := range globalStatsMap { - globalStats, err := statsHandle.MergePartitionStats2GlobalStatsByTableID(e.ctx, e.opts, infoschema.GetInfoSchema(e.ctx), globalStatsID.tableID, info.isIndex, info.idxID) + globalStats, err := statsHandle.MergePartitionStats2GlobalStatsByTableID(e.ctx, e.opts, e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), globalStatsID.tableID, info.isIndex, info.idxID) if err != nil { if types.ErrPartitionStatsMissing.Equal(err) { // When we find some partition-level stats are missing, we need to report warning. @@ -205,7 +205,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } } } - return statsHandle.Update(infoschema.GetInfoSchema(e.ctx)) + return statsHandle.Update(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)) } func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) { diff --git a/executor/analyze_test.go b/executor/analyze_test.go index f8eff902fcb3d..39ec524dbacc4 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -69,7 +69,7 @@ PARTITION BY RANGE ( a ) ( } tk.MustExec("analyze table t") - is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pi := table.Meta().GetPartitionInfo() @@ -96,7 +96,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "hello")`, i, i)) } tk.MustExec("alter table t analyze partition p0") - is = infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is = tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pi = table.Meta().GetPartitionInfo() @@ -176,7 +176,7 @@ func (s *testSuite1) TestAnalyzeParameters(c *C) { tk.MustExec("set @@tidb_enable_fast_analyze = 1") tk.MustExec("analyze table t with 30 samples") - is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -227,7 +227,7 @@ func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) { tk.MustExec(fmt.Sprintf("insert into t values ('%s')", value)) tk.MustExec("analyze table t") - is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -259,7 +259,7 @@ func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("analyze table t with 10 cmsketch width") - is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -435,7 +435,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { } tk.MustExec("analyze table t with 5 buckets, 6 samples") - is := infoschema.GetInfoSchema(tk.Se.(sessionctx.Context)) + is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() diff --git a/executor/builder.go b/executor/builder.go index e82db0d6aaccc..40282d1030b2c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3432,7 +3432,7 @@ func (builder *dataReaderBuilder) buildTableReaderBase(ctx context.Context, e *T SetKeepOrder(e.keepOrder). SetStreaming(e.streaming). SetFromSessionVars(e.ctx.GetSessionVars()). - SetFromInfoSchema(infoschema.GetInfoSchema(e.ctx)). + SetFromInfoSchema(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)). Build() if err != nil { return nil, err diff --git a/executor/compiler.go b/executor/compiler.go index bb0f5274a159e..bb00bfe14602d 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -53,7 +53,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm ctx = opentracing.ContextWithSpan(ctx, span1) } - infoSchema := infoschema.GetInfoSchema(c.Ctx) + infoSchema := c.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil { return nil, err } diff --git a/executor/coprocessor.go b/executor/coprocessor.go index 25959e5454655..490b981add461 100644 --- a/executor/coprocessor.go +++ b/executor/coprocessor.go @@ -159,7 +159,7 @@ func (h *CoprocessorDAGHandler) buildDAGExecutor(req *coprocessor.Request) (Exec return nil, errors.Trace(err) } h.dagReq = dagReq - is := h.sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema) + is := h.sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) // Build physical plan. bp := core.NewPBPlanBuilder(h.sctx, is, req.Ranges) plan, err := bp.Build(dagReq.Executors) diff --git a/executor/distsql.go b/executor/distsql.go index 6e3105a30142d..bd422a0458ef1 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -271,16 +271,19 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) e.memTracker = memory.NewTracker(e.id, -1) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) var builder distsql.RequestBuilder - kvReq, err := builder.SetKeyRanges(kvRanges). + builder.SetKeyRanges(kvRanges). SetDAGRequest(e.dagPB). SetStartTS(e.startTS). SetDesc(e.desc). SetKeepOrder(e.keepOrder). SetStreaming(e.streaming). SetFromSessionVars(e.ctx.GetSessionVars()). - SetMemTracker(e.memTracker). - SetFromInfoSchema(infoschema.GetInfoSchema(e.ctx)). - Build() + SetMemTracker(e.memTracker) + // for tests, infoschema may be null + if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { + builder.SetFromInfoSchema(is) + } + kvReq, err := builder.Build() if err != nil { e.feedback.Invalidate() return err @@ -527,8 +530,11 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< SetKeepOrder(e.keepOrder). SetStreaming(e.indexStreaming). SetFromSessionVars(e.ctx.GetSessionVars()). - SetMemTracker(tracker). - SetFromInfoSchema(infoschema.GetInfoSchema(e.ctx)) + SetMemTracker(tracker) + // for tests, infoschema may be null + if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { + builder.SetFromInfoSchema(is) + } for partTblIdx, kvRange := range kvRanges { // check if executor is closed diff --git a/executor/executor_test.go b/executor/executor_test.go index 80056439ec7c6..823967f24f217 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2338,7 +2338,7 @@ func (s *testSuiteP2) TestIsPointGet(c *C) { "select * from help_topic where help_topic_id=1": true, "select * from help_topic where help_category_id=1": false, } - infoSchema := infoschema.GetInfoSchema(ctx) + infoSchema := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") @@ -2370,7 +2370,7 @@ func (s *testSuiteP2) TestClusteredIndexIsPointGet(c *C) { "select * from t where a='x' and c='x'": true, "select * from t where a='x' and c='x' and b=1": false, } - infoSchema := infoschema.GetInfoSchema(ctx) + infoSchema := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") c.Check(err, IsNil) diff --git a/executor/grant.go b/executor/grant.go index f6a8453ba72dd..6c715758b6c7f 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -74,7 +74,7 @@ func (e *GrantExec) Next(ctx context.Context, req *chunk.Chunk) error { // Make sure the table exist. if e.Level.Level == ast.GrantLevelTable { dbNameStr := model.NewCIStr(dbName) - schema := infoschema.GetInfoSchema(e.ctx) + schema := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tbl, err := schema.TableByName(dbNameStr, model.NewCIStr(e.Level.TableName)) if err != nil { return err diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index c769d2705c44b..84b4b810b059d 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -245,7 +245,7 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, SetStreaming(e.partialStreamings[workID]). SetFromSessionVars(e.ctx.GetSessionVars()). SetMemTracker(e.memTracker). - SetFromInfoSchema(infoschema.GetInfoSchema(e.ctx)) + SetFromInfoSchema(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)) worker := &partialIndexWorker{ stats: e.stats, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 368d8838a777e..a39992799b27a 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -79,7 +79,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex // Cache the ret full rows in schemataRetriever if !e.initialized { - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) dbs := is.AllSchemas() sort.Sort(infoschema.SchemasSorter(dbs)) var err error @@ -295,7 +295,7 @@ func (c *statsCache) get(ctx sessionctx.Context) (map[int64]uint64, map[tableHis } func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *model.TableInfo) (int64, error) { - is := infoschema.GetInfoSchema(ctx) + is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(schema.Name, tblInfo.Name) if err != nil { return 0, err @@ -583,7 +583,7 @@ func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sess } func (e *hugeMemTableRetriever) dataForColumnsInTable(ctx context.Context, sctx sessionctx.Context, schema *model.DBInfo, tbl *model.TableInfo) { - if err := tryFillViewColumnType(ctx, sctx, infoschema.GetInfoSchema(sctx), schema.Name, tbl); err != nil { + if err := tryFillViewColumnType(ctx, sctx, sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), schema.Name, tbl); err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(err) return } @@ -1330,7 +1330,7 @@ func (e *memtableRetriever) setDataForTiKVRegionStatus(ctx sessionctx.Context) e if err != nil { return err } - allSchemas := ctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema).AllSchemas() + allSchemas := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema).AllSchemas() tableInfos := tikvHelper.GetRegionsTableInfo(regionsInfo, allSchemas) for _, region := range regionsInfo.Regions { tableList := tableInfos[region.ID] @@ -1442,7 +1442,7 @@ func (e *memtableRetriever) setDataForTiDBHotRegions(ctx sessionctx.Context) err if !ok { return errors.New("Information about hot region can be gotten only when the storage is TiKV") } - allSchemas := ctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema).AllSchemas() + allSchemas := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema).AllSchemas() tikvHelper := &helper.Helper{ Store: tikvStore, RegionCache: tikvStore.GetRegionCache(), @@ -1591,7 +1591,7 @@ type initialTable struct { } func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) var databases []string schemas := e.extractor.TableSchema tables := e.extractor.TableName @@ -1883,7 +1883,7 @@ func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) error { checker := privilege.GetPrivilegeManager(ctx) - is := infoschema.GetInfoSchema(ctx) + is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) var rows [][]types.Datum for _, bundle := range is.RuleBundles() { id, err := placement.ObjectIDFromGroupID(bundle.ID) @@ -2030,7 +2030,7 @@ func (e *hugeMemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Co } if !e.initialized { - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) dbs := is.AllSchemas() sort.Sort(infoschema.SchemasSorter(dbs)) e.dbs = dbs diff --git a/executor/load_stats.go b/executor/load_stats.go index 83fbb3ad188f7..984f649e1291f 100644 --- a/executor/load_stats.go +++ b/executor/load_stats.go @@ -86,5 +86,5 @@ func (e *LoadStatsInfo) Update(data []byte) error { if h == nil { return errors.New("Load Stats: handle is nil") } - return h.LoadStatsFromJSON(infoschema.GetInfoSchema(e.Ctx), jsonTbl) + return h.LoadStatsFromJSON(e.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), jsonTbl) } diff --git a/executor/metrics_reader_test.go b/executor/metrics_reader_test.go index 19000b4faee5d..662c3b917ce9c 100644 --- a/executor/metrics_reader_test.go +++ b/executor/metrics_reader_test.go @@ -62,7 +62,7 @@ func (s *testSuite7) TestStmtLabel(c *C) { for _, tt := range tests { stmtNode, err := parser.New().ParseOneStmt(tt.sql, "", "") c.Check(err, IsNil) - is := infoschema.GetInfoSchema(tk.Se) + is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) err = plannercore.Preprocess(tk.Se.(sessionctx.Context), stmtNode, is) c.Assert(err, IsNil) _, _, err = planner.Optimize(context.TODO(), tk.Se, stmtNode, is) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 58d6293b4ce54..55b8fa51786d7 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -209,7 +209,7 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { PARTITION p202010 VALUES LESS THAN ("2020-11-01"), PARTITION p202011 VALUES LESS THAN ("2020-12-01") )`) - is := infoschema.GetInfoSchema(tk.Se) + is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t_info_null")) c.Assert(err, IsNil) diff --git a/executor/point_get.go b/executor/point_get.go index 241f52d421344..b0cd700c5c920 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -398,7 +398,7 @@ func (e *PointGetExecutor) verifyTxnScope() error { var tblID int64 var tblName string var partName string - is := infoschema.GetInfoSchema(e.ctx) + is := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) if e.partInfo != nil { tblID = e.partInfo.ID tblInfo, _, partInfo := is.FindTableByPartitionID(tblID) diff --git a/executor/prepared.go b/executor/prepared.go index c5fdd5c1bf404..448ee3b7fdc66 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -320,7 +320,7 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, return nil, false, false, err } execStmt.BinaryArgs = args - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) execPlan, names, err := planner.Optimize(ctx, sctx, execStmt, is) if err != nil { return nil, false, false, err diff --git a/executor/simple.go b/executor/simple.go index 5ed8ced28af48..65df5ca43117f 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -1406,7 +1406,7 @@ func (e *SimpleExec) executeDropStats(s *ast.DropStatsStmt) (err error) { if err := h.DeleteTableStatsFromKV(statsIDs); err != nil { return err } - return h.Update(infoschema.GetInfoSchema(e.ctx)) + return h.Update(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)) } func (e *SimpleExec) autoNewTxn() bool { diff --git a/executor/table_reader.go b/executor/table_reader.go index 767826f0c3b6c..1a76598fb2250 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -221,7 +221,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra } else { reqBuilder = builder.SetHandleRanges(e.ctx.GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.table.Meta() != nil && e.table.Meta().IsCommonHandle, ranges, e.feedback) } - kvReq, err := reqBuilder. + reqBuilder. SetDAGRequest(e.dagPB). SetStartTS(e.startTS). SetDesc(e.desc). @@ -230,9 +230,12 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra SetFromSessionVars(e.ctx.GetSessionVars()). SetMemTracker(e.memTracker). SetStoreType(e.storeType). - SetAllowBatchCop(e.batchCop). - SetFromInfoSchema(infoschema.GetInfoSchema(e.ctx)). - Build() + SetAllowBatchCop(e.batchCop) + // infoschema maybe null for tests + if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { + reqBuilder.SetFromInfoSchema(is) + } + kvReq, err := reqBuilder.Build() if err != nil { return nil, err } diff --git a/expression/builtin_info.go b/expression/builtin_info.go index 6a41b20ef75af..fda57a884f1d8 100644 --- a/expression/builtin_info.go +++ b/expression/builtin_info.go @@ -847,7 +847,7 @@ func (b *builtinNextValSig) evalInt(row chunk.Row) (int64, bool, error) { db = b.ctx.GetSessionVars().CurrentDB } // Check the tableName valid. - sequence, err := b.ctx.GetSessionVars().TxnCtx.InfoSchema.(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) + sequence, err := b.ctx.GetSessionVars().GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) if err != nil { return 0, false, err } @@ -903,7 +903,7 @@ func (b *builtinLastValSig) evalInt(row chunk.Row) (int64, bool, error) { db = b.ctx.GetSessionVars().CurrentDB } // Check the tableName valid. - sequence, err := b.ctx.GetSessionVars().TxnCtx.InfoSchema.(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) + sequence, err := b.ctx.GetSessionVars().GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) if err != nil { return 0, false, err } @@ -953,7 +953,7 @@ func (b *builtinSetValSig) evalInt(row chunk.Row) (int64, bool, error) { db = b.ctx.GetSessionVars().CurrentDB } // Check the tableName valid. - sequence, err := b.ctx.GetSessionVars().TxnCtx.InfoSchema.(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) + sequence, err := b.ctx.GetSessionVars().GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) if err != nil { return 0, false, err } diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 4fcbdc042de85..ac8afd14605f1 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -24,12 +24,8 @@ import ( "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/logutil" - "go.uber.org/zap" ) // InfoSchema is the interface used to retrieve the schema information. @@ -386,28 +382,6 @@ func HasAutoIncrementColumn(tbInfo *model.TableInfo) (bool, string) { return false, "" } -// GetInfoSchema gets TxnCtx InfoSchema if snapshot schema is not set, -// Otherwise, snapshot schema is returned. -func GetInfoSchema(ctx sessionctx.Context) InfoSchema { - return GetInfoSchemaBySessionVars(ctx.GetSessionVars()) -} - -// GetInfoSchemaBySessionVars gets TxnCtx InfoSchema if snapshot schema is not set, -// Otherwise, snapshot schema is returned. -func GetInfoSchemaBySessionVars(sessVar *variable.SessionVars) InfoSchema { - var is InfoSchema - if snap := sessVar.SnapshotInfoschema; snap != nil { - is = snap.(InfoSchema) - logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", sessVar.ConnectionID), zap.Int64("schemaVersion", is.SchemaMetaVersion())) - } else { - if sessVar.TxnCtx == nil || sessVar.TxnCtx.InfoSchema == nil { - return nil - } - is = sessVar.TxnCtx.InfoSchema.(InfoSchema) - } - return is -} - func (is *infoSchema) BundleByName(name string) (*placement.Bundle, bool) { is.ruleBundleMutex.RLock() defer is.ruleBundleMutex.RUnlock() diff --git a/infoschema/tables.go b/infoschema/tables.go index 085bc6a96a77d..bfca649e89fdd 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1737,7 +1737,7 @@ func (s SchemasSorter) Less(i, j int) bool { } func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) (fullRows [][]types.Datum, err error) { - is := GetInfoSchema(ctx) + is := ctx.GetSessionVars().GetInfoSchema().(InfoSchema) dbs := is.AllSchemas() sort.Sort(SchemasSorter(dbs)) switch it.meta.Name.O { diff --git a/planner/core/cacheable_checker_test.go b/planner/core/cacheable_checker_test.go index eb33790dfd74f..fb9d05d528ec0 100644 --- a/planner/core/cacheable_checker_test.go +++ b/planner/core/cacheable_checker_test.go @@ -42,7 +42,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { tk.MustExec("create table t2(a int, b int) partition by hash(a) partitions 11") tk.MustExec("create table t3(a int, b int)") tbl := &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t3")} - is := infoschema.GetInfoSchema(tk.Se) + is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) // test non-SelectStmt/-InsertStmt/-DeleteStmt/-UpdateStmt/-SetOprStmt var stmt ast.Node = &ast.ShowStmt{} c.Assert(core.Cacheable(stmt, is), IsFalse) diff --git a/planner/core/explain.go b/planner/core/explain.go index e3e7e4e06d0b3..913b4a88b5dac 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -306,7 +306,7 @@ func (p *PhysicalTableReader) accessObject(sctx sessionctx.Context) string { return "" } - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { return "partition table not found" + strconv.FormatInt(ts.Table.ID, 10) @@ -366,7 +366,7 @@ func (p *PhysicalIndexReader) accessObject(sctx sessionctx.Context) string { } var buffer bytes.Buffer - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { fmt.Fprintf(&buffer, "partition table not found: %d", ts.Table.ID) @@ -394,7 +394,7 @@ func (p *PhysicalIndexLookUpReader) accessObject(sctx sessionctx.Context) string } var buffer bytes.Buffer - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { fmt.Fprintf(&buffer, "partition table not found: %d", ts.Table.ID) @@ -417,7 +417,7 @@ func (p *PhysicalIndexMergeReader) accessObject(sctx sessionctx.Context) string return "" } - is := infoschema.GetInfoSchema(sctx) + is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { return "partition table not found" + strconv.FormatInt(ts.Table.ID, 10) diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 5d6a11ad982a9..25ba4a21460fe 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -63,8 +63,9 @@ func evalAstExpr(sctx sessionctx.Context, expr ast.ExprNode) (types.Datum, error // rewriteAstExpr rewrites ast expression directly. func rewriteAstExpr(sctx sessionctx.Context, expr ast.ExprNode, schema *expression.Schema, names types.NameSlice) (expression.Expression, error) { var is infoschema.InfoSchema - if sctx.GetSessionVars().TxnCtx.InfoSchema != nil { - is = sctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema) + // in tests, it may be null + if s, ok := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { + is = s } b, savedBlockNames := NewPlanBuilder(sctx, is, &hint.BlockHintProcessor{}) fakePlan := LogicalTableDual{}.Init(sctx, 0) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index f717888caf2c9..2d167906c7177 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -1205,7 +1205,7 @@ func (s *testIntegrationSuite) TestPartitionPruningForEQ(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a datetime, b int) partition by range(weekday(a)) (partition p0 values less than(10), partition p1 values less than (100))") - is := infoschema.GetInfoSchema(tk.Se) + is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pt := tbl.(table.PartitionedTable) diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index e2d74d92376ad..77a853a4568b6 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -1007,7 +1007,7 @@ func checkFastPlanPrivilege(ctx sessionctx.Context, dbName, tableName string, ch }) } - infoSchema := infoschema.GetInfoSchema(ctx) + infoSchema := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) return CheckTableLock(ctx, infoSchema, visitInfos) } @@ -1313,7 +1313,7 @@ func buildPointUpdatePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName VirtualAssignmentsOffset: len(orderedList), }.Init(ctx) updatePlan.names = pointPlan.OutputNames() - is := infoschema.GetInfoSchema(ctx) + is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) t, _ := is.TableByID(tbl.ID) updatePlan.tblID2Table = map[int64]table.Table{ tbl.ID: t, @@ -1509,7 +1509,7 @@ func getHashPartitionColumnName(ctx sessionctx.Context, tbl *model.TableInfo) *a if pi.Type != model.PartitionTypeHash { return nil } - is := infoschema.GetInfoSchema(ctx) + is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) table, ok := is.TableByID(tbl.ID) if !ok { return nil diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 537f1e06468bd..cd43b3964d59b 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -201,7 +201,7 @@ func (s *testPlanSerialSuite) TestPrepareCacheDeferredFunction(c *C) { for i := 0; i < 2; i++ { stmt, err := s.ParseOneStmt(sql1, "", "") c.Check(err, IsNil) - is := tk.Se.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema) + is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) builder, _ := core.NewPlanBuilder(tk.Se, is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Check(err, IsNil) diff --git a/session/session.go b/session/session.go index 13df91510f61a..94582fbcb6886 100644 --- a/session/session.go +++ b/session/session.go @@ -406,7 +406,7 @@ func (s *session) StoreIndexUsage(tblID int64, idxID int64, rowsSelected int64) // FieldList returns fields list of a table. func (s *session) FieldList(tableName string) ([]*ast.ResultField, error) { - is := infoschema.GetInfoSchema(s) + is := s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) dbName := model.NewCIStr(s.GetSessionVars().CurrentDB) tName := model.NewCIStr(tableName) pm := privilege.GetPrivilegeManager(s) @@ -1660,7 +1660,7 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields // So we have to call PrepareTxnCtx here. s.PrepareTxnCtx(ctx) s.PrepareTSFuture(ctx) - prepareExec := executor.NewPrepareExec(s, infoschema.GetInfoSchema(s), sql) + prepareExec := executor.NewPrepareExec(s, s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), sql) err = prepareExec.Next(ctx, nil) if err != nil { return @@ -1701,7 +1701,7 @@ func (s *session) cachedPlanExec(ctx context.Context, if prepareStmt.ForUpdateRead { is = domain.GetDomain(s).InfoSchema() } else { - is = infoschema.GetInfoSchema(s) + is = s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) } execAst := &ast.ExecuteStmt{ExecID: stmtID} if err := executor.ResetContextOfStmt(s, execAst); err != nil { @@ -1781,7 +1781,7 @@ func (s *session) IsCachedExecOk(ctx context.Context, preparedStmt *plannercore. return false, nil } // check schema version - is := infoschema.GetInfoSchema(s) + is := s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) if prepared.SchemaVersion != is.SchemaMetaVersion() { prepared.CachedPlan = nil return false, nil @@ -2892,7 +2892,7 @@ func (s *session) checkPlacementPolicyBeforeCommit() error { txnScope = oracle.GlobalTxnScope } if txnScope != oracle.GlobalTxnScope { - is := infoschema.GetInfoSchema(s) + is := s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) deltaMap := s.GetSessionVars().TxnCtx.TableDeltaMap for physicalTableID := range deltaMap { var tableName string diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 1554f3c429d65..89b31da2e8bbd 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -46,11 +46,13 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tidb/util/timeutil" "github.com/twmb/murmur3" atomic2 "go.uber.org/atomic" + "go.uber.org/zap" ) // PreparedStmtCount is exported for test. @@ -868,6 +870,23 @@ func (s *SessionVars) BuildParserConfig() parser.ParserConfig { } } +// GetInfoSchema returns snapshotInfoSchema if snapshot schema is set. +// Otherwise, transaction infoschema is returned. +// Nil if there is no available infoschema. +func (s *SessionVars) GetInfoSchema() interface{} { + type IS interface { + SchemaMetaVersion() int64 + } + if snap, ok := s.SnapshotInfoschema.(IS); ok { + logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", s.ConnectionID), zap.Int64("schemaVersion", snap.SchemaMetaVersion())) + return snap + } + if s.TxnCtx != nil && s.TxnCtx.InfoSchema != nil { + return s.TxnCtx.InfoSchema + } + return nil +} + // PartitionPruneMode presents the prune mode used. type PartitionPruneMode string diff --git a/statistics/handle/ddl.go b/statistics/handle/ddl.go index 168c8a07daad1..34f30c1241e1c 100644 --- a/statistics/handle/ddl.go +++ b/statistics/handle/ddl.go @@ -74,7 +74,7 @@ var analyzeOptionDefault = map[ast.AnalyzeOptionType]uint64{ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error { // We need to merge the partition-level stats to global-stats when we drop table partition in dynamic mode. tableID := tblInfo.ID - is := infoschema.GetInfoSchema(h.mu.ctx) + is := h.mu.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) globalStats, err := h.TableStatsFromStorage(tblInfo, tableID, true, 0) if err != nil { return err diff --git a/statistics/handle/update.go b/statistics/handle/update.go index c65f0885877f6..6f472fc61fdc7 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -480,7 +480,7 @@ func (h *Handle) dumpTableStatCountToKV(id int64, delta variable.TableDelta) (up affectedRows := h.mu.ctx.GetSessionVars().StmtCtx.AffectedRows() // if it's a partitioned table and its global-stats exists, update its count and modify_count as well. - is := infoschema.GetInfoSchema(h.mu.ctx) + is := h.mu.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) if is == nil { return false, errors.New("cannot get the information schema") } From b8cad01bef3019dd6ba44ded294b990be73b022d Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 10 May 2021 08:08:42 -0600 Subject: [PATCH 004/343] privilege: fix RequestVerificationWithUser use of default roles (#24442) --- privilege/privileges/privileges.go | 3 ++- privilege/privileges/privileges_test.go | 28 +++++++++++++++++++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 5b7917e802aac..c5ec2f8394385 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -140,7 +140,8 @@ func (p *UserPrivileges) RequestVerificationWithUser(db, table, column string, p } mysqlPriv := p.Handle.Get() - return mysqlPriv.RequestVerification(nil, user.Username, user.Hostname, db, table, column, priv) + roles := mysqlPriv.getDefaultRoles(user.Username, user.Hostname) + return mysqlPriv.RequestVerification(roles, user.Username, user.Hostname, db, table, column, priv) } // GetEncodedPassword implements the Manager interface. diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 816fe5a59d0bd..2efb565b3ed2b 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1393,3 +1393,31 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeStatusVars(c *C) { AuthHostname: "%", }, nil, nil) } + +// TestViewDefiner tests that default roles are correctly applied in the algorithm definer +// See: https://github.com/pingcap/tidb/issues/24414 +func (s *testPrivilegeSuite) TestViewDefiner(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE DATABASE issue24414") + tk.MustExec("USE issue24414") + tk.MustExec(`create table table1( + col1 int, + col2 int, + col3 int + )`) + tk.MustExec(`insert into table1 values (1,1,1),(2,2,2)`) + tk.MustExec(`CREATE ROLE 'ACL-mobius-admin'`) + tk.MustExec(`CREATE USER 'mobius-admin'`) + tk.MustExec(`CREATE USER 'mobius-admin-no-role'`) + tk.MustExec(`GRANT Select,Insert,Update,Delete,Create,Drop,Alter,Index,Create View,Show View ON issue24414.* TO 'ACL-mobius-admin'@'%'`) + tk.MustExec(`GRANT Select,Insert,Update,Delete,Create,Drop,Alter,Index,Create View,Show View ON issue24414.* TO 'mobius-admin-no-role'@'%'`) + tk.MustExec(`GRANT 'ACL-mobius-admin'@'%' to 'mobius-admin'@'%'`) + tk.MustExec(`SET DEFAULT ROLE ALL TO 'mobius-admin'`) + // create tables + tk.MustExec(`CREATE ALGORITHM = UNDEFINED DEFINER = 'mobius-admin'@'127.0.0.1' SQL SECURITY DEFINER VIEW test_view (col1 , col2 , col3) AS SELECT * from table1`) + tk.MustExec(`CREATE ALGORITHM = UNDEFINED DEFINER = 'mobius-admin-no-role'@'127.0.0.1' SQL SECURITY DEFINER VIEW test_view2 (col1 , col2 , col3) AS SELECT * from table1`) + + // all examples should work + tk.MustExec("select * from test_view") + tk.MustExec("select * from test_view2") +} From c6c8265e098b65339a85be2e35678b1c13f0f53a Mon Sep 17 00:00:00 2001 From: disksing Date: Mon, 10 May 2021 22:51:37 +0800 Subject: [PATCH 005/343] store/tikv: remove use of TaskID transaction option in store/tikv (#24407) --- store/driver/txn/snapshot.go | 2 ++ store/driver/txn/txn_driver.go | 2 ++ store/tikv/snapshot.go | 12 ++++++++---- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 1a3f7bf9c3bbc..6064db02cdeba 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -73,6 +73,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetNotFillCache(val.(bool)) case tikvstore.SnapshotTS: s.KVSnapshot.SetSnapshotTS(val.(uint64)) + case tikvstore.TaskID: + s.KVSnapshot.SetTaskID(val.(uint64)) default: s.KVSnapshot.SetOption(opt, val) } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 8595b2fe874ae..50bba80d2b54e 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -144,6 +144,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetPessimistic(val.(bool)) case tikvstore.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) + case tikvstore.TaskID: + txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case tikvstore.InfoSchema: txn.SetSchemaVer(val.(tikv.SchemaVer)) case tikvstore.CommitHook: diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 2b9926c7a2b9a..a37e1d8343c5e 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -569,10 +569,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.mu.Lock() s.mu.replicaRead = val.(kv.ReplicaReadType) s.mu.Unlock() - case kv.TaskID: - s.mu.Lock() - s.mu.taskID = val.(uint64) - s.mu.Unlock() case kv.CollectRuntimeStats: s.mu.Lock() s.mu.stats = val.(*SnapshotRuntimeStats) @@ -625,6 +621,14 @@ func (s *KVSnapshot) SetPriority(pri Priority) { s.priority = pri } +// SetTaskID marks current task's unique ID to allow TiKV to schedule +// tasks more fairly. +func (s *KVSnapshot) SetTaskID(id uint64) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.taskID = id +} + // SnapCacheHitCount gets the snapshot cache hit count. Only for test. func (s *KVSnapshot) SnapCacheHitCount() int { return int(atomic.LoadInt64(&s.mu.hitCnt)) From f135c534a50cc3b3242047aee8860e81cc1a5a05 Mon Sep 17 00:00:00 2001 From: Zhuhe Fang Date: Mon, 10 May 2021 23:25:37 +0800 Subject: [PATCH 006/343] plan: merge continuous selections and delete surely true expressions (#24214) --- cmd/explaintest/r/explain_easy.result | 6 +- executor/executor_test.go | 2 +- expression/integration_test.go | 10 +- expression/testdata/expression_suite_out.json | 4 +- go.sum | 1 + planner/core/integration_test.go | 38 +++++++ planner/core/optimizer.go | 26 +++++ planner/core/rule_predicate_push_down.go | 31 ++++++ .../testdata/integration_serial_suite_in.json | 6 + .../integration_serial_suite_out.json | 28 +++++ .../core/testdata/partition_pruner_out.json | 103 +++++++++--------- planner/core/testdata/plan_suite_out.json | 6 +- planner/core/testdata/point_get_plan_out.json | 3 +- 13 files changed, 195 insertions(+), 69 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index a33fe5a791bbc..927e25c8024f2 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -108,9 +108,9 @@ HashJoin 9990.00 root inner join, equal:[eq(test.t1.c1, test.t2.c2)] └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select (select count(1) k from t1 s where s.c1 = t1.c1 having k != 0) from t1; id estRows task access object operator info -Projection 10000.00 root ifnull(Column#10, 0)->Column#10 -└─MergeJoin 10000.00 root left outer join, left key:test.t1.c1, right key:test.t1.c1 - ├─Projection(Build) 8000.00 root 1->Column#10, test.t1.c1 +Projection 12500.00 root ifnull(Column#10, 0)->Column#10 +└─MergeJoin 12500.00 root left outer join, left key:test.t1.c1, right key:test.t1.c1 + ├─Projection(Build) 10000.00 root 1->Column#10, test.t1.c1 │ └─TableReader 10000.00 root data:TableFullScan │ └─TableFullScan 10000.00 cop[tikv] table:s keep order:true, stats:pseudo └─TableReader(Probe) 10000.00 root data:TableFullScan diff --git a/executor/executor_test.go b/executor/executor_test.go index 823967f24f217..e69b956f8d82b 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8128,7 +8128,7 @@ func (s *testSerialSuite) TestIssue24210(c *C) { // for SelectionExec c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError", `return(true)`), IsNil) - _, err = tk.Exec("select * from (select 1 as a) t where a > 0") + _, err = tk.Exec("select * from (select rand() as a) t where a > 0") c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "mock SelectionExec.baseExecutor.Open returned error") err = failpoint.Disable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError") diff --git a/expression/integration_test.go b/expression/integration_test.go index 76ba37b49d4bd..d4b6a031087e4 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -5474,16 +5474,14 @@ func (s *testIntegrationSuite) TestExprPushdownBlacklist(c *C) { // > pushed to both TiKV and TiFlash rows := tk.MustQuery("explain format = 'brief' select * from test.t where b > date'1988-01-01' and b < date'1994-01-01' " + "and cast(a as decimal(10,2)) > 10.10 and date_format(b,'%m') = '11'").Rows() - c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "lt(test.t.b, 1994-01-01)") - c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10)") - c.Assert(fmt.Sprintf("%v", rows[3][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\"), gt(test.t.b, 1988-01-01)") + c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10), lt(test.t.b, 1994-01-01)") + c.Assert(fmt.Sprintf("%v", rows[2][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\"), gt(test.t.b, 1988-01-01)") tk.MustExec("set @@session.tidb_isolation_read_engines = 'tikv'") rows = tk.MustQuery("explain format = 'brief' select * from test.t where b > date'1988-01-01' and b < date'1994-01-01' " + "and cast(a as decimal(10,2)) > 10.10 and date_format(b,'%m') = '11'").Rows() - c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "lt(test.t.b, 1994-01-01)") - c.Assert(fmt.Sprintf("%v", rows[1][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\")") - c.Assert(fmt.Sprintf("%v", rows[3][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10), gt(test.t.b, 1988-01-01)") + c.Assert(fmt.Sprintf("%v", rows[0][4]), Equals, "eq(date_format(test.t.b, \"%m\"), \"11\"), lt(test.t.b, 1994-01-01)") + c.Assert(fmt.Sprintf("%v", rows[2][4]), Equals, "gt(cast(test.t.a, decimal(10,2) BINARY), 10.10), gt(test.t.b, 1988-01-01)") tk.MustExec("delete from mysql.expr_pushdown_blacklist where name = '<' and store_type = 'tikv,tiflash,tidb' and reason = 'for test'") tk.MustExec("delete from mysql.expr_pushdown_blacklist where name = 'date_format' and store_type = 'tikv' and reason = 'for test'") diff --git a/expression/testdata/expression_suite_out.json b/expression/testdata/expression_suite_out.json index ea36fc2923764..8b89ee1e4dc81 100644 --- a/expression/testdata/expression_suite_out.json +++ b/expression/testdata/expression_suite_out.json @@ -186,7 +186,7 @@ { "SQL": "explain format = 'brief' select * from t1 left join t2 on true where t1.a = 1 and t1.a = 1", "Result": [ - "HashJoin 80000.00 root CARTESIAN left outer join", + "HashJoin 100000.00 root CARTESIAN left outer join", "├─TableReader(Build) 10.00 root data:Selection", "│ └─Selection 10.00 cop[tikv] eq(test.t1.a, 1)", "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", @@ -253,7 +253,7 @@ { "SQL": "explain format = 'brief' select * from t1 left join t2 on true where t1.a = 1 or (t1.a = 2 and t1.a = 3)", "Result": [ - "HashJoin 80000.00 root CARTESIAN left outer join", + "HashJoin 100000.00 root CARTESIAN left outer join", "├─TableReader(Build) 10.00 root data:Selection", "│ └─Selection 10.00 cop[tikv] or(eq(test.t1.a, 1), 0)", "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", diff --git a/go.sum b/go.sum index 74b4f623789b8..a3ebad580db64 100644 --- a/go.sum +++ b/go.sum @@ -500,6 +500,7 @@ github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQD github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= +github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A= diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 2d167906c7177..4ae7342a4f7d7 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3651,3 +3651,41 @@ func (s *testIntegrationSuite) TestSequenceAsDataSource(c *C) { tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) } } + +func (s *testIntegrationSerialSuite) TestMergeContinuousSelections(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists ts") + tk.MustExec("create table ts (col_char_64 char(64), col_varchar_64_not_null varchar(64) not null, col_varchar_key varchar(1), id int primary key, col_varchar_64 varchar(64),col_char_64_not_null char(64) 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 == "ts" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + tk.MustExec(" set @@tidb_allow_mpp=1;") + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 2ad7fc1136d5b..d79d83331723b 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/lock" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/privilege" @@ -156,9 +157,34 @@ func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic return finalPlan, cost, nil } +// mergeContinuousSelections merge continuous selections which may occur after changing plans. +func mergeContinuousSelections(p PhysicalPlan) { + if sel, ok := p.(*PhysicalSelection); ok { + for { + childSel := sel.children[0] + if tmp, ok := childSel.(*PhysicalSelection); ok { + sel.Conditions = append(sel.Conditions, tmp.Conditions...) + sel.SetChild(0, tmp.children[0]) + } else { + break + } + } + } + for _, child := range p.Children() { + mergeContinuousSelections(child) + } + // merge continuous selections in a coprocessor task of tiflash + tableReader, isTableReader := p.(*PhysicalTableReader) + if isTableReader && tableReader.StoreType == kv.TiFlash { + mergeContinuousSelections(tableReader.tablePlan) + tableReader.TablePlans = flattenPushDownPlan(tableReader.tablePlan) + } +} + func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan { plan = eliminatePhysicalProjection(plan) plan = InjectExtraProjection(plan) + mergeContinuousSelections(plan) plan = eliminateUnionScanAndLock(sctx, plan) plan = enableParallelApply(sctx, plan) return plan diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index 936c4720cbc87..a3c6737db74d7 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -42,6 +42,12 @@ func addSelection(p LogicalPlan, child LogicalPlan, conditions []expression.Expr p.Children()[chIdx] = dual return } + + conditions = DeleteTrueExprs(p, conditions) + if len(conditions) == 0 { + p.Children()[chIdx] = child + return + } selection := LogicalSelection{Conditions: conditions}.Init(p.SCtx(), p.SelectBlockOffset()) selection.SetChildren(child) p.Children()[chIdx] = selection @@ -73,6 +79,8 @@ func splitSetGetVarFunc(filters []expression.Expression) ([]expression.Expressio // PredicatePushDown implements LogicalPlan PredicatePushDown interface. func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { + predicates = DeleteTrueExprs(p, predicates) + p.Conditions = DeleteTrueExprs(p, p.Conditions) canBePushDown, canNotBePushDown := splitSetGetVarFunc(p.Conditions) retConditions, child := p.children[0].PredicatePushDown(append(canBePushDown, predicates...)) retConditions = append(retConditions, canNotBePushDown...) @@ -100,6 +108,7 @@ func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression) // PredicatePushDown implements LogicalPlan PredicatePushDown interface. func (ds *DataSource) PredicatePushDown(predicates []expression.Expression) ([]expression.Expression, LogicalPlan) { predicates = expression.PropagateConstant(ds.ctx, predicates) + predicates = DeleteTrueExprs(ds, predicates) ds.allConds = predicates ds.pushedDownConds, predicates = expression.PushDownExprs(ds.ctx.GetSessionVars().StmtCtx, predicates, ds.ctx.GetClient(), kv.UnSpecified) return predicates, ds @@ -532,6 +541,28 @@ func Conds2TableDual(p LogicalPlan, conds []expression.Expression) LogicalPlan { return nil } +// DeleteTrueExprs deletes the surely true expressions +func DeleteTrueExprs(p LogicalPlan, conds []expression.Expression) []expression.Expression { + newConds := make([]expression.Expression, 0, len(conds)) + for _, cond := range conds { + con, ok := cond.(*expression.Constant) + if !ok { + newConds = append(newConds, cond) + continue + } + if expression.ContainMutableConst(p.SCtx(), []expression.Expression{con}) { + newConds = append(newConds, cond) + continue + } + sc := p.SCtx().GetSessionVars().StmtCtx + if isTrue, err := con.Value.ToBool(sc); err == nil && isTrue == 1 { + continue + } + newConds = append(newConds, cond) + } + return newConds +} + // outerJoinPropConst propagates constant equal and column equal conditions over outer join. func (p *LogicalJoin) outerJoinPropConst(predicates []expression.Expression) []expression.Expression { outerTable := p.children[0] diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 3234652e5d000..34e50df03661b 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -300,5 +300,11 @@ "cases": [ "select (2) in (select b from t) from (select t.a < (select t.a from t t1 limit 1) from t) t" ] + }, + { + "name": "TestMergeContinuousSelections", + "cases": [ + "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;" + ] } ] diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 8b67310894d4b..1f25f899a68d9 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -2608,5 +2608,33 @@ ] } ] + }, + { + "Name": "TestMergeContinuousSelections", + "Cases": [ + { + "SQL": "desc format = 'brief' SELECT table2 . `col_char_64` AS field1 FROM `ts` AS table2 INNER JOIN (SELECT DISTINCT SUBQUERY3_t1 . * FROM `ts` AS SUBQUERY3_t1 LEFT OUTER JOIN `ts` AS SUBQUERY3_t2 ON SUBQUERY3_t2 . `col_varchar_64_not_null` = SUBQUERY3_t1 . `col_varchar_key`) AS table3 ON (table3 . `col_varchar_key` = table2 . `col_varchar_64`) WHERE table3 . `col_char_64_not_null` >= SOME (SELECT SUBQUERY4_t1 . `col_varchar_64` AS SUBQUERY4_field1 FROM `ts` AS SUBQUERY4_t1) GROUP BY field1 ;", + "Plan": [ + "HashAgg 7992.00 root group by:test.ts.col_char_64, funcs:firstrow(test.ts.col_char_64)->test.ts.col_char_64", + "└─HashJoin 9990.00 root CARTESIAN inner join, other cond:or(ge(test.ts.col_char_64_not_null, Column#25), if(ne(Column#26, 0), NULL, 0))", + " ├─Selection(Build) 0.80 root ne(Column#27, 0)", + " │ └─HashAgg 1.00 root funcs:min(Column#33)->Column#25, funcs:sum(Column#34)->Column#26, funcs:count(Column#35)->Column#27", + " │ └─TableReader 1.00 root data:ExchangeSender", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#39)->Column#33, funcs:sum(Column#40)->Column#34, funcs:count(1)->Column#35", + " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(22,0) BINARY)->Column#40", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 12487.50 root data:ExchangeSender", + " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.ts.col_varchar_64, test.ts.col_varchar_key)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_64))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:table2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.ts.col_varchar_key))", + " └─TableFullScan 10000.00 cop[tiflash] table:SUBQUERY3_t1 keep order:false, stats:pseudo" + ] + } + ] } ] diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index 5593ab52bee74..962894b5c35c9 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -714,13 +714,13 @@ "SQL": "select * from t1 join t2 on true where t1.a=5 and t2.a in (6,7,8) and t1.a-t2.a=1 and t2.b = 6", "Result": null, "Plan": [ - "Projection 80.00 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.a, test_partition.t2.id, test_partition.t2.b", - "└─HashJoin 80.00 root CARTESIAN inner join", - " ├─TableReader(Build) 8.00 root partition:p1 data:Selection", - " │ └─Selection 8.00 cop[tikv] 1, eq(minus(5, test_partition.t2.a), 1), eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", + "Projection 0.24 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.a, test_partition.t2.id, test_partition.t2.b", + "└─HashJoin 0.24 root CARTESIAN inner join", + " ├─TableReader(Build) 0.02 root partition:p1 data:Selection", + " │ └─Selection 0.02 cop[tikv] eq(minus(5, test_partition.t2.a), 1), eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 10.00 root partition:p0 data:Selection", - " └─Selection 10.00 cop[tikv] 1, eq(test_partition.t1.a, 5)", + " └─Selection 10.00 cop[tikv] eq(test_partition.t1.a, 5)", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ] }, @@ -1129,25 +1129,24 @@ "3 3 3 7 7 7" ], "Plan": [ - "Sort 80.16 root test_partition.t1.id, test_partition.t1.a", - "└─Projection 80.16 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", - " └─HashJoin 80.16 root CARTESIAN inner join", - " ├─TableReader(Build) 8.00 root partition:p1 data:Selection", - " │ └─Selection 8.00 cop[tikv] 1, eq(test_partition.t2.b, 7), eq(test_partition.t2.id, 7), in(test_partition.t2.a, 6, 7, 8)", + "Sort 0.00 root test_partition.t1.id, test_partition.t1.a", + "└─Projection 0.00 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", + " └─HashJoin 0.00 root CARTESIAN inner join", + " ├─TableReader(Build) 0.00 root partition:p1 data:Selection", + " │ └─Selection 0.00 cop[tikv] eq(test_partition.t2.b, 7), eq(test_partition.t2.id, 7), in(test_partition.t2.a, 6, 7, 8)", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 10.02 root partition:p0 data:Selection", - " └─Selection 10.02 cop[tikv] 1, or(eq(test_partition.t1.a, 1), and(eq(test_partition.t1.a, 3), in(test_partition.t1.b, 3, 5)))", + " └─Selection 10.02 cop[tikv] or(eq(test_partition.t1.a, 1), and(eq(test_partition.t1.a, 3), in(test_partition.t1.b, 3, 5)))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "Sort 0.05 root test_partition_1.t1.id, test_partition_1.t1.a", - "└─HashJoin 0.05 root CARTESIAN inner join", - " ├─IndexReader(Build) 0.02 root partition:p0 index:Selection", - " │ └─Selection 0.02 cop[tikv] or(eq(test_partition_1.t1.a, 1), and(eq(test_partition_1.t1.a, 3), in(test_partition_1.t1.b, 3, 5)))", - " │ └─IndexRangeScan 20.00 cop[tikv] table:t1, index:a(a, b, id) range:[1,1], [3,3], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 2.40 root partition:p1 index:Selection", - " └─Selection 2.40 cop[tikv] 1", - " └─IndexRangeScan 3.00 cop[tikv] table:t2, index:a(a, b, id) range:[6 7 7,6 7 7], [7 7 7,7 7 7], [8 7 7,8 7 7], keep order:false, stats:pseudo" + "Sort 30.60 root test_partition_1.t1.id, test_partition_1.t1.a", + "└─Projection 30.60 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", + " └─HashJoin 30.60 root CARTESIAN inner join", + " ├─IndexReader(Build) 3.00 root partition:p1 index:IndexRangeScan", + " │ └─IndexRangeScan 3.00 cop[tikv] table:t2, index:a(a, b, id) range:[6 7 7,6 7 7], [7 7 7,7 7 7], [8 7 7,8 7 7], keep order:false, stats:pseudo", + " └─IndexReader(Probe) 10.20 root partition:p0 index:IndexRangeScan", + " └─IndexRangeScan 10.20 cop[tikv] table:t1, index:a(a, b, id) range:[1,1], [3 3,3 3], [3 5,3 5], keep order:false, stats:pseudo" ] }, { @@ -1734,22 +1733,22 @@ "5 5 5 6 6 6" ], "Plan": [ - "Projection 80.00 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", - "└─HashJoin 80.00 root CARTESIAN inner join", - " ├─TableReader(Build) 8.00 root partition:p1 data:Selection", - " │ └─Selection 8.00 cop[tikv] 1, eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", + "Projection 0.30 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", + "└─HashJoin 0.30 root CARTESIAN inner join", + " ├─TableReader(Build) 0.03 root partition:p1 data:Selection", + " │ └─Selection 0.03 cop[tikv] eq(test_partition.t2.b, 6), in(test_partition.t2.a, 6, 7, 8)", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 10.00 root partition:p0 data:Selection", - " └─Selection 10.00 cop[tikv] 1, eq(test_partition.t1.a, 5)", + " └─Selection 10.00 cop[tikv] eq(test_partition.t1.a, 5)", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "Projection 300.00 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", - "└─HashJoin 300.00 root CARTESIAN inner join", - " ├─IndexReader(Build) 3.00 root partition:p1 index:IndexRangeScan", - " │ └─IndexRangeScan 3.00 cop[tikv] table:t2, index:a(a, b, id) range:[6 6 NULL,6 6 +inf], [7 6 NULL,7 6 +inf], [8 6 NULL,8 6 +inf], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 100.00 root partition:p0 index:IndexRangeScan", - " └─IndexRangeScan 100.00 cop[tikv] table:t1, index:a(a, b, id) range:[5 NULL,5 +inf], keep order:false, stats:pseudo" + "Projection 3.00 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", + "└─HashJoin 3.00 root CARTESIAN inner join", + " ├─IndexReader(Build) 0.30 root partition:p1 index:IndexRangeScan", + " │ └─IndexRangeScan 0.30 cop[tikv] table:t2, index:a(a, b, id) range:[6 6,6 6], [7 6,7 6], [8 6,8 6], keep order:false, stats:pseudo", + " └─IndexReader(Probe) 10.00 root partition:p0 index:IndexRangeScan", + " └─IndexRangeScan 10.00 cop[tikv] table:t1, index:a(a, b, id) range:[5,5], keep order:false, stats:pseudo" ] }, { @@ -2127,25 +2126,25 @@ "3 3 3 7 7 7" ], "Plan": [ - "Sort 675761.06 root test_partition.t1.id, test_partition.t1.a", - "└─Projection 675761.06 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", - " └─HashJoin 675761.06 root CARTESIAN inner join", - " ├─TableReader(Build) 200.00 root partition:p1 data:Selection", - " │ └─Selection 200.00 cop[tikv] 1, ge(test_partition.t2.a, 6), ge(test_partition.t2.b, 7), ge(test_partition.t2.id, 7), le(test_partition.t2.a, 8)", + "Sort 93855.70 root test_partition.t1.id, test_partition.t1.a", + "└─Projection 93855.70 root test_partition.t1.id, test_partition.t1.a, test_partition.t1.b, test_partition.t2.id, test_partition.t2.a, test_partition.t2.b", + " └─HashJoin 93855.70 root CARTESIAN inner join", + " ├─TableReader(Build) 27.78 root partition:p1 data:Selection", + " │ └─Selection 27.78 cop[tikv] ge(test_partition.t2.a, 6), ge(test_partition.t2.b, 7), ge(test_partition.t2.id, 7), le(test_partition.t2.a, 8)", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 3378.81 root partition:p0 data:Selection", - " └─Selection 3378.81 cop[tikv] 1, or(le(test_partition.t1.a, 1), and(le(test_partition.t1.a, 3), and(ge(test_partition.t1.b, 3), le(test_partition.t1.b, 5))))", + " └─Selection 3378.81 cop[tikv] or(le(test_partition.t1.a, 1), and(le(test_partition.t1.a, 3), and(ge(test_partition.t1.b, 3), le(test_partition.t1.b, 5))))", " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "IndexPlan": [ - "Sort 224577.93 root test_partition_1.t1.id, test_partition_1.t1.a", - "└─Projection 224577.93 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", - " └─HashJoin 224577.93 root CARTESIAN inner join", - " ├─IndexReader(Build) 200.00 root partition:p1 index:Selection", - " │ └─Selection 200.00 cop[tikv] ge(test_partition_1.t2.b, 7), ge(test_partition_1.t2.id, 7)", + "Sort 73851.85 root test_partition_1.t1.id, test_partition_1.t1.a", + "└─Projection 73851.85 root test_partition_1.t1.id, test_partition_1.t1.a, test_partition_1.t1.b, test_partition_1.t2.id, test_partition_1.t2.a, test_partition_1.t2.b", + " └─HashJoin 73851.85 root CARTESIAN inner join", + " ├─IndexReader(Build) 27.78 root partition:p1 index:Selection", + " │ └─Selection 27.78 cop[tikv] ge(test_partition_1.t2.b, 7), ge(test_partition_1.t2.id, 7)", " │ └─IndexRangeScan 250.00 cop[tikv] table:t2, index:a(a, b, id) range:[6,8], keep order:false, stats:pseudo", - " └─IndexReader(Probe) 1122.89 root partition:p0 index:Selection", - " └─Selection 1122.89 cop[tikv] or(le(test_partition_1.t1.a, 1), and(le(test_partition_1.t1.a, 3), and(ge(test_partition_1.t1.b, 3), le(test_partition_1.t1.b, 5))))", + " └─IndexReader(Probe) 2658.67 root partition:p0 index:Selection", + " └─Selection 2658.67 cop[tikv] or(le(test_partition_1.t1.a, 1), and(le(test_partition_1.t1.a, 3), and(ge(test_partition_1.t1.b, 3), le(test_partition_1.t1.b, 5))))", " └─IndexRangeScan 3323.33 cop[tikv] table:t1, index:a(a, b, id) range:[-inf,3], keep order:false, stats:pseudo" ] }, @@ -2761,20 +2760,20 @@ "5 5 5 8 8 8" ], "Plan": [ - "HashJoin 2000.00 root CARTESIAN inner join", + "HashJoin 833.33 root CARTESIAN inner join", "├─TableReader(Build) 10.00 root partition:p0 data:Selection", - "│ └─Selection 10.00 cop[tikv] 1, eq(test_partition.t1.a, 5)", + "│ └─Selection 10.00 cop[tikv] eq(test_partition.t1.a, 5)", "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - "└─TableReader(Probe) 200.00 root partition:p1 data:Selection", - " └─Selection 200.00 cop[tikv] 1, ge(test_partition.t2.a, 6), ge(test_partition.t2.b, 6), le(test_partition.t2.a, 8)", + "└─TableReader(Probe) 83.33 root partition:p1 data:Selection", + " └─Selection 83.33 cop[tikv] ge(test_partition.t2.a, 6), ge(test_partition.t2.b, 6), le(test_partition.t2.a, 8)", " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "IndexPlan": [ - "HashJoin 20000.00 root CARTESIAN inner join", - "├─IndexReader(Build) 100.00 root partition:p0 index:IndexRangeScan", - "│ └─IndexRangeScan 100.00 cop[tikv] table:t1, index:a(a, b, id) range:[5 NULL,5 +inf], keep order:false, stats:pseudo", - "└─IndexReader(Probe) 200.00 root partition:p1 index:Selection", - " └─Selection 200.00 cop[tikv] ge(test_partition_1.t2.b, 6)", + "HashJoin 833.33 root CARTESIAN inner join", + "├─IndexReader(Build) 10.00 root partition:p0 index:IndexRangeScan", + "│ └─IndexRangeScan 10.00 cop[tikv] table:t1, index:a(a, b, id) range:[5,5], keep order:false, stats:pseudo", + "└─IndexReader(Probe) 83.33 root partition:p1 index:Selection", + " └─Selection 83.33 cop[tikv] ge(test_partition_1.t2.b, 6)", " └─IndexRangeScan 250.00 cop[tikv] table:t2, index:a(a, b, id) range:[6,8], keep order:false, stats:pseudo" ] }, diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index a0125e601c94a..ce33bec905757 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -818,7 +818,7 @@ }, { "SQL": "select (select count(1) k from t s where s.a = t.a having k != 0) from t", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]]->Sel([1]))->Projection}(test.t.a,test.t.a)->Projection" + "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->IndexReader(Index(t.f)[[NULL,+inf]])->Projection}(test.t.a,test.t.a)->Projection" }, { "SQL": "select sum(to_base64(e)) from t group by e,d,c order by c", @@ -1233,12 +1233,12 @@ }, { "SQL": "select /*+ HASH_AGG() */ t1.a from t t1 where t1.a < any(select t2.b from t t2)", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]]->Sel([1]))->TableReader(Table(t)->Sel([1])->HashAgg)->HashAgg->Sel([ne(Column#27, 0) 1])}", + "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->TableReader(Table(t)->HashAgg)->HashAgg->Sel([ne(Column#27, 0)])}", "Warning": "" }, { "SQL": "select /*+ hash_agg() */ t1.a from t t1 where t1.a != any(select t2.b from t t2)", - "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]]->Sel([1]))->TableReader(Table(t)->Sel([1]))->HashAgg->Sel([ne(Column#28, 0) 1])}", + "Best": "LeftHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->TableReader(Table(t))->HashAgg->Sel([ne(Column#28, 0)])}", "Warning": "" }, { diff --git a/planner/core/testdata/point_get_plan_out.json b/planner/core/testdata/point_get_plan_out.json index b0053fde6044d..9f9ac9e27deb9 100644 --- a/planner/core/testdata/point_get_plan_out.json +++ b/planner/core/testdata/point_get_plan_out.json @@ -15,8 +15,7 @@ { "SQL": "select b, c from t where t.b = 2 and t.c = 2 and t.b+1=3", "Plan": [ - "Selection 0.80 root 1", - "└─Point_Get 1.00 root table:t, index:b(b, c) " + "Point_Get 1.00 root table:t, index:b(b, c) " ], "Res": [ "2 2" From 3dd2e546f841fa3e1f32812fce8e560cef66e897 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 10:15:39 +0800 Subject: [PATCH 007/343] store/tikv: make tikv.ErrTokenLimit as a normal error instead of terror (#24484) --- store/driver/txn/error.go | 6 ++++++ store/tikv/error/errcode.go | 3 --- store/tikv/error/error.go | 10 +++++++++- store/tikv/region_request.go | 3 +-- store/tikv/region_request_test.go | 5 ++++- 5 files changed, 20 insertions(+), 7 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index b5e69b4e522a9..c026b8899ae72 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -38,6 +38,8 @@ import ( // tikv error instance var ( + // ErrTokenLimit is the error that token is up to the limit. + ErrTokenLimit = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStoreLimit) // ErrTiKVServerTimeout is the error when tikv server is timeout. ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerTimeout) @@ -253,6 +255,10 @@ func ToTiDBErr(err error) error { return ErrRegionUnavailable } + if e, ok := err.(*tikverr.ErrTokenLimit); ok { + return ErrTokenLimit.GenWithStackByArgs(e.StoreID) + } + return errors.Trace(originErr) } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index bf0e3c7c4b91d..01e8db4d12473 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -20,7 +20,4 @@ const ( CodeLockWaitTimeout = 1205 CodeQueryInterrupted = 1317 CodeLockAcquireFailAndNoWaitSet = 3572 - - // TiKV/PD/TiFlash errors. - CodeTiKVStoreLimit = 9008 ) diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 8ed21d8db192f..70ec995e88489 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -60,7 +60,6 @@ var ( ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(CodeLockWaitTimeout) - ErrTokenLimit = dbterror.ClassTiKV.NewStd(CodeTiKVStoreLimit) ErrUnknown = dbterror.ClassTiKV.NewStd(CodeUnknown) ) @@ -190,3 +189,12 @@ type ErrGCTooEarly struct { func (e *ErrGCTooEarly) Error() string { return fmt.Sprintf("GC life time is shorter than transaction duration, transaction starts at %v, GC safe point is %v", e.TxnStartTS, e.GCSafePoint) } + +// ErrTokenLimit is the error that token is up to the limit. +type ErrTokenLimit struct { + StoreID uint64 +} + +func (e *ErrTokenLimit) Error() string { + return fmt.Sprintf("Store token is up to the limit, store id = %d.", e.StoreID) +} diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index cad0ed0379e96..74ecdd7ce72b8 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -530,8 +530,7 @@ func (s *RegionRequestSender) getStoreToken(st *Store, limit int64) error { return nil } metrics.TiKVStoreLimitErrorCounter.WithLabelValues(st.addr, strconv.FormatUint(st.storeID, 10)).Inc() - return tikverr.ErrTokenLimit.GenWithStackByArgs(st.storeID) - + return &tikverr.ErrTokenLimit{StoreID: st.storeID} } func (s *RegionRequestSender) releaseStoreToken(st *Store) { diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 81e9cc4498a07..9c5172e52f372 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/kvproto/pkg/tikvpb" + tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/config" @@ -177,7 +178,9 @@ func (s *testRegionRequestToThreeStoresSuite) TestStoreTokenLimit(c *C) { resp, err := s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) c.Assert(err, NotNil) c.Assert(resp, IsNil) - c.Assert(err.Error(), Equals, "[tikv:9008]Store token is up to the limit, store id = 1") + e, ok := errors.Cause(err).(*tikverr.ErrTokenLimit) + c.Assert(ok, IsTrue) + c.Assert(e.StoreID, Equals, uint64(1)) kv.StoreLimit.Store(oldStoreLimit) } From 9e13287d9db85860c1adeed51b419e5cba2362fa Mon Sep 17 00:00:00 2001 From: xhe Date: Tue, 11 May 2021 10:31:39 +0800 Subject: [PATCH 008/343] *: remove SchemaVersion in TransactionContext (#24236) --- executor/ddl.go | 1 - executor/simple.go | 6 ++-- session/session.go | 47 ++++++++++++++--------------- sessionctx/binloginfo/binloginfo.go | 2 +- sessionctx/variable/session.go | 34 ++++++++++++--------- 5 files changed, 46 insertions(+), 44 deletions(-) diff --git a/executor/ddl.go b/executor/ddl.go index 64a597e2eb024..81f7221d1e60e 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -140,7 +140,6 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { is := dom.InfoSchema() txnCtx := e.ctx.GetSessionVars().TxnCtx txnCtx.InfoSchema = is - txnCtx.SchemaVersion = is.SchemaMetaVersion() // DDL will force commit old transaction, after DDL, in transaction status should be false. e.ctx.GetSessionVars().SetInTxn(false) return nil diff --git a/executor/simple.go b/executor/simple.go index 65df5ca43117f..24cb857aec3d5 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -686,13 +686,13 @@ func (e *SimpleExec) executeStartTransactionReadOnlyWithTimestampBound(ctx conte } failpoint.Inject("mockStalenessTxnSchemaVer", func(val failpoint.Value) { if val.(bool) { - staleVer = e.ctx.GetSessionVars().TxnCtx.SchemaVersion - 1 + staleVer = e.ctx.GetSessionVars().GetInfoSchema().SchemaMetaVersion() - 1 } else { - staleVer = e.ctx.GetSessionVars().TxnCtx.SchemaVersion + staleVer = e.ctx.GetSessionVars().GetInfoSchema().SchemaMetaVersion() } }) // TODO: currently we directly check the schema version. In future, we can cache the stale infoschema instead. - if e.ctx.GetSessionVars().TxnCtx.SchemaVersion > staleVer { + if e.ctx.GetSessionVars().GetInfoSchema().SchemaMetaVersion() > staleVer { return errors.New("schema version changed after the staleness startTS") } diff --git a/session/session.go b/session/session.go index 94582fbcb6886..7c499d83afda1 100644 --- a/session/session.go +++ b/session/session.go @@ -491,7 +491,7 @@ func (s *session) doCommit(ctx context.Context) error { physicalTableIDs = append(physicalTableIDs, id) } // Set this option for 2 phase commit to validate schema lease. - s.txn.SetOption(tikvstore.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.sessionVars.TxnCtx.SchemaVersion, physicalTableIDs)) + s.txn.SetOption(tikvstore.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.sessionVars.GetInfoSchema().SchemaMetaVersion(), physicalTableIDs)) s.txn.SetOption(tikvstore.InfoSchema, s.sessionVars.TxnCtx.InfoSchema) s.txn.SetOption(tikvstore.CommitHook, func(info string, _ error) { s.sessionVars.LastTxnInfo = info }) if s.GetSessionVars().EnableAmendPessimisticTxn { @@ -1485,7 +1485,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex if err != nil { if !kv.ErrKeyExists.Equal(err) { logutil.Logger(ctx).Warn("run statement failed", - zap.Int64("schemaVersion", s.sessionVars.TxnCtx.SchemaVersion), + zap.Int64("schemaVersion", s.sessionVars.GetInfoSchema().SchemaMetaVersion()), zap.Error(err), zap.String("session", s.String())) } @@ -1926,7 +1926,7 @@ func (s *session) NewTxn(ctx context.Context) error { } vars := s.GetSessionVars() logutil.Logger(ctx).Info("NewTxn() inside a transaction auto commit", - zap.Int64("schemaVersion", vars.TxnCtx.SchemaVersion), + zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), zap.Uint64("txnStartTS", txnID), zap.String("txnScope", txnScope)) } @@ -1942,13 +1942,12 @@ func (s *session) NewTxn(ctx context.Context) error { s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() s.sessionVars.TxnCtx = &variable.TransactionContext{ - InfoSchema: is, - SchemaVersion: is.SchemaMetaVersion(), - CreateTime: time.Now(), - StartTS: txn.StartTS(), - ShardStep: int(s.sessionVars.ShardAllocateStep), - IsStaleness: false, - TxnScope: s.sessionVars.CheckAndGetTxnScope(), + InfoSchema: is, + CreateTime: time.Now(), + StartTS: txn.StartTS(), + ShardStep: int(s.sessionVars.ShardAllocateStep), + IsStaleness: false, + TxnScope: s.sessionVars.CheckAndGetTxnScope(), } return nil } @@ -2678,11 +2677,10 @@ func (s *session) PrepareTxnCtx(ctx context.Context) { is := domain.GetDomain(s).InfoSchema() s.sessionVars.TxnCtx = &variable.TransactionContext{ - InfoSchema: is, - SchemaVersion: is.SchemaMetaVersion(), - CreateTime: time.Now(), - ShardStep: int(s.sessionVars.ShardAllocateStep), - TxnScope: s.GetSessionVars().CheckAndGetTxnScope(), + InfoSchema: is, + CreateTime: time.Now(), + ShardStep: int(s.sessionVars.ShardAllocateStep), + TxnScope: s.GetSessionVars().CheckAndGetTxnScope(), } if !s.sessionVars.IsAutocommit() || s.sessionVars.RetryInfo.Retrying { if s.sessionVars.TxnMode == ast.Pessimistic { @@ -2754,7 +2752,7 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc } vars := s.GetSessionVars() logutil.Logger(ctx).Info("InitTxnWithExactStaleness() inside a transaction auto commit", - zap.Int64("schemaVersion", vars.TxnCtx.SchemaVersion), + zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), zap.Uint64("txnStartTS", txnID), zap.String("txnScope", txnScope)) } @@ -2792,13 +2790,12 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() s.sessionVars.TxnCtx = &variable.TransactionContext{ - InfoSchema: is, - SchemaVersion: is.SchemaMetaVersion(), - CreateTime: time.Now(), - StartTS: txn.StartTS(), - ShardStep: int(s.sessionVars.ShardAllocateStep), - IsStaleness: true, - TxnScope: txnScope, + InfoSchema: is, + CreateTime: time.Now(), + StartTS: txn.StartTS(), + ShardStep: int(s.sessionVars.ShardAllocateStep), + IsStaleness: true, + TxnScope: txnScope, } return nil } @@ -2825,7 +2822,7 @@ func logStmt(execStmt *executor.ExecStmt, vars *variable.SessionVars) { *ast.RevokeStmt, *ast.AlterTableStmt, *ast.CreateDatabaseStmt, *ast.CreateIndexStmt, *ast.CreateTableStmt, *ast.DropDatabaseStmt, *ast.DropIndexStmt, *ast.DropTableStmt, *ast.RenameTableStmt, *ast.TruncateTableStmt: user := vars.User - schemaVersion := vars.TxnCtx.SchemaVersion + schemaVersion := vars.GetInfoSchema().SchemaMetaVersion() if ss, ok := execStmt.StmtNode.(ast.SensitiveStmtNode); ok { logutil.BgLogger().Info("CRUCIAL OPERATION", zap.Uint64("conn", vars.ConnectionID), @@ -2854,7 +2851,7 @@ func logQuery(query string, vars *variable.SessionVars) { logutil.BgLogger().Info("GENERAL_LOG", zap.Uint64("conn", vars.ConnectionID), zap.Stringer("user", vars.User), - zap.Int64("schemaVersion", vars.TxnCtx.SchemaVersion), + zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), zap.Uint64("txnStartTS", vars.TxnCtx.StartTS), zap.Uint64("forUpdateTS", vars.TxnCtx.GetForUpdateTS()), zap.Bool("isReadConsistency", vars.IsIsolation(ast.ReadCommitted)), diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 044e6cdc11df9..58313505e1c8e 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -101,7 +101,7 @@ func GetPrewriteValue(ctx sessionctx.Context, createIfNotExists bool) *binlog.Pr vars := ctx.GetSessionVars() v, ok := vars.TxnCtx.Binlog.(*binlog.PrewriteValue) if !ok && createIfNotExists { - schemaVer := ctx.GetSessionVars().TxnCtx.SchemaVersion + schemaVer := ctx.GetSessionVars().GetInfoSchema().SchemaMetaVersion() v = &binlog.PrewriteValue{SchemaVersion: schemaVer} vars.TxnCtx.Binlog = v } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 89b31da2e8bbd..7db9de383ba55 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -129,13 +129,12 @@ func (r *retryInfoAutoIDs) getCurrent() (int64, bool) { // TransactionContext is used to store variables that has transaction scope. type TransactionContext struct { - forUpdateTS uint64 - stmtFuture oracle.Future - Binlog interface{} - InfoSchema interface{} - History interface{} - SchemaVersion int64 - StartTS uint64 + forUpdateTS uint64 + stmtFuture oracle.Future + Binlog interface{} + InfoSchema interface{} + History interface{} + StartTS uint64 // ShardStep indicates the max size of continuous rowid shard in one transaction. ShardStep int @@ -870,19 +869,26 @@ func (s *SessionVars) BuildParserConfig() parser.ParserConfig { } } +// FIXME: remove this interface +// infoschemaMetaVersion is a workaround. Due to circular dependency, +// can not return the complete interface. But SchemaMetaVersion is widely used for logging. +// So we give a convenience for that +type infoschemaMetaVersion interface { + SchemaMetaVersion() int64 +} + // GetInfoSchema returns snapshotInfoSchema if snapshot schema is set. // Otherwise, transaction infoschema is returned. // Nil if there is no available infoschema. -func (s *SessionVars) GetInfoSchema() interface{} { - type IS interface { - SchemaMetaVersion() int64 - } - if snap, ok := s.SnapshotInfoschema.(IS); ok { +func (s *SessionVars) GetInfoSchema() infoschemaMetaVersion { + if snap, ok := s.SnapshotInfoschema.(infoschemaMetaVersion); ok { logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", s.ConnectionID), zap.Int64("schemaVersion", snap.SchemaMetaVersion())) return snap } - if s.TxnCtx != nil && s.TxnCtx.InfoSchema != nil { - return s.TxnCtx.InfoSchema + if s.TxnCtx != nil { + if is, ok := s.TxnCtx.InfoSchema.(infoschemaMetaVersion); ok { + return is + } } return nil } From f13f696400be61ae43dd28aeaf61920817732b17 Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Tue, 11 May 2021 10:57:38 +0800 Subject: [PATCH 009/343] executor: add correctness tests about PointGet and BatchGet (#24467) --- executor/partition_table_test.go | 57 ++++++++++++++++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 55b8fa51786d7..555e152265349 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -227,6 +227,63 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows()) } +func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_batchget_pointget") + tk.MustExec("use test_batchget_pointget") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash partition table + tk.MustExec("create table thash(a int, unique key(a)) partition by hash(a) partitions 4;") + + // regular partition table + tk.MustExec("create table tregular(a int, unique key(a));") + + vals := make([]string, 0, 100) + // insert data into range partition table and hash partition table + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v)", i+1)) + } + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular values " + strings.Join(vals, ",")) + + // test PointGet + for i := 0; i < 100; i++ { + // explain select a from t where a = {x}; // x >= 1 and x <= 100 Check if PointGet is used + // select a from t where a={x}; // the result is {x} + x := rand.Intn(100) + 1 + queryHash := fmt.Sprintf("select a from thash where a=%v", x) + queryRegular := fmt.Sprintf("select a from thash where a=%v", x) + c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryHash).Check(tk.MustQuery(queryRegular).Rows()) + } + + // test empty PointGet + queryHash := fmt.Sprintf("select a from thash where a=200") + c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryHash).Check(testkit.Rows()) + + // test BatchGet + for i := 0; i < 100; i++ { + // explain select a from t where a in ({x1}, {x2}, ... {x10}); // BatchGet is used + // select a from t where where a in ({x1}, {x2}, ... {x10}); + points := make([]string, 0, 10) + for i := 0; i < 10; i++ { + x := rand.Intn(100) + 1 + points = append(points, fmt.Sprintf("%v", x)) + } + + queryHash := fmt.Sprintf("select a from thash where a in (%v)", strings.Join(points, ",")) + queryRegular := fmt.Sprintf("select a from tregular where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } +} + func (s *partitionTableSuite) TestView(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 7c1997517c76fda1b27bf64e4e653a33ddfe88e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Tue, 11 May 2021 11:17:38 +0800 Subject: [PATCH 010/343] txn: Use TransactionOption in store/tikv module (#23255) --- kv/kv.go | 7 ++ session/session.go | 12 +-- session/txn.go | 4 +- store/driver/tikv_driver.go | 20 +--- store/mockstore/mockstorage/storage.go | 16 +--- store/tikv/extract_start_ts_test.go | 122 +++++++++++++++++++++++++ store/tikv/kv.go | 70 +------------- store/tikv/tests/2pc_test.go | 5 +- store/tikv/txn.go | 106 ++++++++++++++------- 9 files changed, 220 insertions(+), 142 deletions(-) create mode 100644 store/tikv/extract_start_ts_test.go diff --git a/kv/kv.go b/kv/kv.go index 1b1e1a5f46a4a..a6a23a88df01d 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -341,6 +341,8 @@ type Driver interface { } // TransactionOption indicates the option when beginning a transaction +// `TxnScope` must be set for each object +// Every other fields are optional, but currently at most one of them can be set type TransactionOption struct { TxnScope string StartTS *uint64 @@ -349,6 +351,11 @@ type TransactionOption struct { MaxPrevSec *uint64 } +// DefaultTransactionOption creates a default TransactionOption, ie. Work in GlobalTxnScope and get start ts when got used +func DefaultTransactionOption() TransactionOption { + return TransactionOption{TxnScope: oracle.GlobalTxnScope} +} + // SetMaxPrevSec set maxPrevSec func (to TransactionOption) SetMaxPrevSec(maxPrevSec uint64) TransactionOption { to.MaxPrevSec = &maxPrevSec diff --git a/session/session.go b/session/session.go index 7c499d83afda1..2f842f92e183a 100644 --- a/session/session.go +++ b/session/session.go @@ -1931,7 +1931,7 @@ func (s *session) NewTxn(ctx context.Context) error { zap.String("txnScope", txnScope)) } - txn, err := s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(s.sessionVars.CheckAndGetTxnScope())) + txn, err := s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(s.sessionVars.CheckAndGetTxnScope())) if err != nil { return err } @@ -2728,7 +2728,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { } // no need to get txn from txnFutureCh since txn should init with startTs - txn, err := s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTs(startTS)) + txn, err := s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTs(startTS)) if err != nil { return err } @@ -2761,22 +2761,22 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc txnScope := s.GetSessionVars().CheckAndGetTxnScope() switch option.Mode { case ast.TimestampBoundReadTimestamp: - txn, err = s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(txnScope).SetStartTs(option.StartTS)) + txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetStartTs(option.StartTS)) if err != nil { return err } case ast.TimestampBoundExactStaleness: - txn, err = s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(txnScope).SetPrevSec(option.PrevSec)) + txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetPrevSec(option.PrevSec)) if err != nil { return err } case ast.TimestampBoundMaxStaleness: - txn, err = s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(txnScope).SetMaxPrevSec(option.PrevSec)) + txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetMaxPrevSec(option.PrevSec)) if err != nil { return err } case ast.TimestampBoundMinReadTimestamp: - txn, err = s.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(txnScope).SetMinStartTS(option.StartTS)) + txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetMinStartTS(option.StartTS)) if err != nil { return err } diff --git a/session/txn.go b/session/txn.go index eed4698f60a65..aebed7ed920b2 100644 --- a/session/txn.go +++ b/session/txn.go @@ -353,14 +353,14 @@ type txnFuture struct { func (tf *txnFuture) wait() (kv.Transaction, error) { startTS, err := tf.future.Wait() if err == nil { - return tf.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(tf.txnScope).SetStartTs(startTS)) + return tf.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(tf.txnScope).SetStartTs(startTS)) } else if config.GetGlobalConfig().Store == "unistore" { return nil, err } logutil.BgLogger().Warn("wait tso failed", zap.Error(err)) // It would retry get timestamp. - return tf.store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(tf.txnScope)) + return tf.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(tf.txnScope)) } func (s *session) getTxnFuture(ctx context.Context) *txnFuture { diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index 17308d33c6be3..cb14736844e68 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/util/logutil" pd "github.com/tikv/pd/client" @@ -306,27 +305,10 @@ func (s *tikvStore) Begin() (kv.Transaction, error) { // BeginWithOption begins a transaction with given option func (s *tikvStore) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { - txnScope := option.TxnScope - if txnScope == "" { - txnScope = oracle.GlobalTxnScope - } - var txn *tikv.KVTxn - var err error - if option.StartTS != nil { - txn, err = s.BeginWithStartTS(txnScope, *option.StartTS) - } else if option.PrevSec != nil { - txn, err = s.BeginWithExactStaleness(txnScope, *option.PrevSec) - } else if option.MaxPrevSec != nil { - txn, err = s.BeginWithMaxPrevSec(txnScope, *option.MaxPrevSec) - } else if option.MinStartTS != nil { - txn, err = s.BeginWithMinStartTS(txnScope, *option.MinStartTS) - } else { - txn, err = s.BeginWithTxnScope(txnScope) - } + txn, err := s.KVStore.BeginWithOption(option) if err != nil { return nil, txn_driver.ToTiDBErr(err) } - return txn_driver.NewTiKVTxn(txn), err } diff --git a/store/mockstore/mockstorage/storage.go b/store/mockstore/mockstorage/storage.go index 05ece29c57a6a..36ded5e434817 100644 --- a/store/mockstore/mockstorage/storage.go +++ b/store/mockstore/mockstorage/storage.go @@ -22,7 +22,6 @@ import ( driver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/oracle" ) // Wraps tikv.KVStore and make it compatible with kv.Storage. @@ -85,20 +84,7 @@ func (s *mockStorage) ShowStatus(ctx context.Context, key string) (interface{}, // BeginWithOption begins a transaction with given option func (s *mockStorage) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { - txnScope := option.TxnScope - if txnScope == "" { - txnScope = oracle.GlobalTxnScope - } - if option.StartTS != nil { - return newTiKVTxn(s.BeginWithStartTS(txnScope, *option.StartTS)) - } else if option.PrevSec != nil { - return newTiKVTxn(s.BeginWithExactStaleness(txnScope, *option.PrevSec)) - } else if option.MaxPrevSec != nil { - return newTiKVTxn(s.BeginWithMaxPrevSec(txnScope, *option.MaxPrevSec)) - } else if option.MinStartTS != nil { - return newTiKVTxn(s.BeginWithMinStartTS(txnScope, *option.MinStartTS)) - } - return newTiKVTxn(s.BeginWithTxnScope(txnScope)) + return newTiKVTxn(s.KVStore.BeginWithOption(option)) } // GetSnapshot gets a snapshot that is able to read any data which data is <= ver. diff --git a/store/tikv/extract_start_ts_test.go b/store/tikv/extract_start_ts_test.go new file mode 100644 index 0000000000000..1422e387bfda5 --- /dev/null +++ b/store/tikv/extract_start_ts_test.go @@ -0,0 +1,122 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tikv + +import ( + "context" + + . "github.com/pingcap/check" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/mockstore/unistore" + "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/store/tikv/tikvrpc" +) + +type extractStartTsSuite struct { + store *KVStore +} + +var _ = Suite(&extractStartTsSuite{}) + +func (s *extractStartTsSuite) SetUpTest(c *C) { + client, pdClient, cluster, err := unistore.New("") + c.Assert(err, IsNil) + unistore.BootstrapWithSingleStore(cluster) + store, err := NewTestTiKVStore(client, pdClient, nil, nil, 0) + c.Assert(err, IsNil) + store.regionCache.storeMu.stores[2] = &Store{ + storeID: 2, + storeType: tikvrpc.TiKV, + state: uint64(resolved), + labels: []*metapb.StoreLabel{ + { + Key: DCLabelKey, + Value: oracle.LocalTxnScope, + }, + }, + } + store.regionCache.storeMu.stores[3] = &Store{ + storeID: 3, + storeType: tikvrpc.TiKV, + state: uint64(resolved), + labels: []*metapb.StoreLabel{{ + Key: DCLabelKey, + Value: "Some Random Label", + }}, + } + store.resolveTSMu.resolveTS[2] = 102 + store.resolveTSMu.resolveTS[3] = 101 + s.store = store +} + +func (s *extractStartTsSuite) TestExtractStartTs(c *C) { + i := uint64(100) + cases := []kv.TransactionOption{ + // StartTS setted + {TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}, + // PrevSec setted + {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}, + // MinStartTS setted, global + {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}, + // MinStartTS setted, local + {TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}, + // MaxPrevSec setted + // however we need to add more cases to check the behavior when it fall backs to MinStartTS setted + // see `TestMaxPrevSecFallback` + {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + // nothing setted + {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}, + } + bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + stalenessTimestamp, _ := s.store.getStalenessTimestamp(bo, oracle.GlobalTxnScope, 100) + expectedTs := []uint64{ + 100, + stalenessTimestamp, + + 101, + 102, + + stalenessTimestamp, + // it's too hard to figure out the value `getTimestampWithRetry` returns + // so we just check whether it is greater than stalenessTimestamp + 0, + } + for i, cs := range cases { + expected := expectedTs[i] + result, _ := extractStartTs(s.store, cs) + if expected == 0 { + c.Assert(result, Greater, stalenessTimestamp) + } else { + c.Assert(result, Equals, expected) + } + } +} + +func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { + s.store.resolveTSMu.resolveTS[2] = 0x8000000000000002 + s.store.resolveTSMu.resolveTS[3] = 0x8000000000000001 + + i := uint64(100) + cases := []kv.TransactionOption{ + {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + {TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + } + expectedTs := []uint64{0x8000000000000001, 0x8000000000000002} + for i, cs := range cases { + expected := expectedTs[i] + result, _ := extractStartTs(s.store, cs) + c.Assert(result, Equals, expected) + } +} diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 0f4824a785ecb..5ddca52726a04 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" + tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" @@ -184,72 +184,12 @@ func (s *KVStore) runSafePointChecker() { // Begin a global transaction. func (s *KVStore) Begin() (*KVTxn, error) { - return s.BeginWithTxnScope(oracle.GlobalTxnScope) + return s.BeginWithOption(tidbkv.DefaultTransactionOption()) } -// BeginWithTxnScope begins a transaction with the given txnScope (local or global) -func (s *KVStore) BeginWithTxnScope(txnScope string) (*KVTxn, error) { - txn, err := newTiKVTxn(s, txnScope) - if err != nil { - return nil, errors.Trace(err) - } - return txn, nil -} - -// BeginWithStartTS begins a transaction with startTS. -func (s *KVStore) BeginWithStartTS(txnScope string, startTS uint64) (*KVTxn, error) { - txn, err := newTiKVTxnWithStartTS(s, txnScope, startTS, s.nextReplicaReadSeed()) - if err != nil { - return nil, errors.Trace(err) - } - return txn, nil -} - -// BeginWithExactStaleness begins transaction with given staleness -func (s *KVStore) BeginWithExactStaleness(txnScope string, prevSec uint64) (*KVTxn, error) { - txn, err := newTiKVTxnWithExactStaleness(s, txnScope, prevSec) - if err != nil { - return nil, errors.Trace(err) - } - return txn, nil -} - -// BeginWithMinStartTS begins transaction with the least startTS -func (s *KVStore) BeginWithMinStartTS(txnScope string, minStartTS uint64) (*KVTxn, error) { - stores := make([]*Store, 0) - allStores := s.regionCache.getStoresByType(tikvrpc.TiKV) - if txnScope != oracle.GlobalTxnScope { - for _, store := range allStores { - if store.IsLabelsMatch([]*metapb.StoreLabel{ - { - Key: DCLabelKey, - Value: txnScope, - }, - }) { - stores = append(stores, store) - } - } - } else { - stores = allStores - } - resolveTS := s.getMinResolveTSByStores(stores) - startTS := minStartTS - // If the resolveTS is larger than the minStartTS, we will use resolveTS as StartTS, otherwise we will use - // minStartTS directly. - if oracle.CompareTS(startTS, resolveTS) < 0 { - startTS = resolveTS - } - return s.BeginWithStartTS(txnScope, startTS) -} - -// BeginWithMaxPrevSec begins transaction with given max previous seconds for startTS -func (s *KVStore) BeginWithMaxPrevSec(txnScope string, maxPrevSec uint64) (*KVTxn, error) { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - minStartTS, err := s.getStalenessTimestamp(bo, txnScope, maxPrevSec) - if err != nil { - return nil, errors.Trace(err) - } - return s.BeginWithMinStartTS(txnScope, minStartTS) +// BeginWithOption begins a transaction with the given TransactionOption +func (s *KVStore) BeginWithOption(options tidbkv.TransactionOption) (*KVTxn, error) { + return newTiKVTxnWithOptions(s, options) } // GetSnapshot gets a snapshot that is able to read any data which data is <= ver. diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 6d7d7e89d1a8e..8043ccdab89f0 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + tidbkv "github.com/pingcap/tidb/kv" drivertxn "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" @@ -602,12 +603,12 @@ func (s *testCommitterSuite) TestRejectCommitTS(c *C) { // Use max.Uint64 to read the data and success. // That means the final commitTS > startTS+2, it's not the one we provide. // So we cover the rety commitTS logic. - txn1, err := s.store.BeginWithStartTS(oracle.GlobalTxnScope, committer.GetStartTS()+2) + txn1, err := s.store.BeginWithOption(tidbkv.DefaultTransactionOption().SetStartTs(committer.GetStartTS() + 2)) c.Assert(err, IsNil) _, err = txn1.Get(bo.GetCtx(), []byte("x")) c.Assert(tikverr.IsErrNotFound(err), IsTrue) - txn2, err := s.store.BeginWithStartTS(oracle.GlobalTxnScope, math.MaxUint64) + txn2, err := s.store.BeginWithOption(tidbkv.DefaultTransactionOption().SetStartTs(math.MaxUint64)) c.Assert(err, IsNil) val, err := txn2.Get(bo.GetCtx(), []byte("x")) c.Assert(err, IsNil) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 4e462653c415c..0ae2df13c12e9 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -30,10 +30,14 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/kv" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" + tikv "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" "go.uber.org/zap" @@ -60,7 +64,7 @@ type KVTxn struct { commitTS uint64 mu sync.Mutex // For thread-safe LockKeys function. setCnt int64 - vars *kv.Variables + vars *tikv.Variables committer *twoPhaseCommitter lockedCnt int @@ -83,44 +87,80 @@ type KVTxn struct { kvFilter KVFilter } -func newTiKVTxn(store *KVStore, txnScope string) (*KVTxn, error) { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTS, err := store.getTimestampWithRetry(bo, txnScope) - if err != nil { - return nil, errors.Trace(err) +func extractStartTs(store *KVStore, options kv.TransactionOption) (uint64, error) { + var startTs uint64 + var err error + if options.StartTS != nil { + startTs = *options.StartTS + } else if options.PrevSec != nil { + bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + startTs, err = store.getStalenessTimestamp(bo, options.TxnScope, *options.PrevSec) + } else if options.MinStartTS != nil { + stores := make([]*Store, 0) + allStores := store.regionCache.getStoresByType(tikvrpc.TiKV) + if options.TxnScope != oracle.GlobalTxnScope { + for _, store := range allStores { + if store.IsLabelsMatch([]*metapb.StoreLabel{ + { + Key: DCLabelKey, + Value: options.TxnScope, + }, + }) { + stores = append(stores, store) + } + } + } else { + stores = allStores + } + resolveTS := store.getMinResolveTSByStores(stores) + startTs = *options.MinStartTS + // If the resolveTS is larger than the minStartTS, we will use resolveTS as StartTS, otherwise we will use + // minStartTS directly. + if oracle.CompareTS(startTs, resolveTS) < 0 { + startTs = resolveTS + } + } else if options.MaxPrevSec != nil { + bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + minStartTS, err := store.getStalenessTimestamp(bo, options.TxnScope, *options.MaxPrevSec) + if err != nil { + return 0, errors.Trace(err) + } + options.MinStartTS = &minStartTS + return extractStartTs(store, options) + } else { + bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + startTs, err = store.getTimestampWithRetry(bo, options.TxnScope) } - return newTiKVTxnWithStartTS(store, txnScope, startTS, store.nextReplicaReadSeed()) + return startTs, err } -// newTiKVTxnWithStartTS creates a txn with startTS. -func newTiKVTxnWithStartTS(store *KVStore, txnScope string, startTS uint64, replicaReadSeed uint32) (*KVTxn, error) { - snapshot := newTiKVSnapshot(store, startTS, replicaReadSeed) - return &KVTxn{ +func newTiKVTxnWithOptions(store *KVStore, options kv.TransactionOption) (*KVTxn, error) { + if options.TxnScope == "" { + options.TxnScope = oracle.GlobalTxnScope + } + startTs, err := extractStartTs(store, options) + if err != nil { + return nil, errors.Trace(err) + } + snapshot := newTiKVSnapshot(store, startTs, store.nextReplicaReadSeed()) + newTiKVTxn := &KVTxn{ snapshot: snapshot, us: unionstore.NewUnionStore(snapshot), store: store, - startTS: startTS, + startTS: startTs, startTime: time.Now(), valid: true, - vars: kv.DefaultVars, - scope: txnScope, - }, nil -} - -func newTiKVTxnWithExactStaleness(store *KVStore, txnScope string, prevSec uint64) (*KVTxn, error) { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTS, err := store.getStalenessTimestamp(bo, txnScope, prevSec) - if err != nil { - return nil, errors.Trace(err) + vars: tikv.DefaultVars, + scope: options.TxnScope, } - return newTiKVTxnWithStartTS(store, txnScope, startTS, store.nextReplicaReadSeed()) + return newTiKVTxn, nil } // SetSuccess is used to probe if kv variables are set or not. It is ONLY used in test cases. var SetSuccess = false // SetVars sets variables to the transaction. -func (txn *KVTxn) SetVars(vars *kv.Variables) { +func (txn *KVTxn) SetVars(vars *tikv.Variables) { txn.vars = vars txn.snapshot.vars = vars failpoint.Inject("probeSetVars", func(val failpoint.Value) { @@ -131,7 +171,7 @@ func (txn *KVTxn) SetVars(vars *kv.Variables) { } // GetVars gets variables from the transaction. -func (txn *KVTxn) GetVars() *kv.Variables { +func (txn *KVTxn) GetVars() *tikv.Variables { return txn.vars } @@ -184,7 +224,7 @@ func (txn *KVTxn) SetOption(opt int, val interface{}) { txn.us.SetOption(opt, val) txn.snapshot.SetOption(opt, val) switch opt { - case kv.SchemaAmender: + case tikv.SchemaAmender: txn.schemaAmender = val.(SchemaAmender) } } @@ -442,8 +482,8 @@ func (txn *KVTxn) onCommitted(err error) { } // LockKeys tries to lock the entries with the keys in KV store. -// lockWaitTime in ms, except that tidbkv.LockAlwaysWait(0) means always wait lock, tidbkv.LockNowait(-1) means nowait lock -func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput ...[]byte) error { +// lockWaitTime in ms, except that kv.LockAlwaysWait(0) means always wait lock, kv.LockNowait(-1) means nowait lock +func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput ...[]byte) error { // Exclude keys that are already locked. var err error keys := make([][]byte, 0, len(keysInput)) @@ -494,7 +534,7 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput . if lockCtx.ReturnValues && locked { // An already locked key can not return values, we add an entry to let the caller get the value // in other ways. - lockCtx.Values[string(key)] = kv.ReturnedValue{AlreadyLocked: true} + lockCtx.Values[string(key)] = tikv.ReturnedValue{AlreadyLocked: true} } } if len(keys) == 0 { @@ -574,16 +614,16 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput . } } for _, key := range keys { - valExists := kv.SetKeyLockedValueExists + valExists := tikv.SetKeyLockedValueExists // PointGet and BatchPointGet will return value in pessimistic lock response, the value may not exist. // For other lock modes, the locked key values always exist. if lockCtx.ReturnValues { val, _ := lockCtx.Values[string(key)] if len(val.Value) == 0 { - valExists = kv.SetKeyLockedValueNotExists + valExists = tikv.SetKeyLockedValueNotExists } } - memBuf.UpdateFlags(key, kv.SetKeyLocked, kv.DelNeedCheckExists, valExists) + memBuf.UpdateFlags(key, tikv.SetKeyLocked, tikv.DelNeedCheckExists, valExists) } txn.lockedCnt += len(keys) return nil From d27fda32fb20a6db930ea41a71053eb1a96a25d2 Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Tue, 11 May 2021 11:59:39 +0800 Subject: [PATCH 011/343] executor: add correctness tests about direct reading with aggregations (#24491) --- executor/partition_table_test.go | 144 +++++++++++++++++++++++++++++++ 1 file changed, 144 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 555e152265349..8ce126972ec73 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -434,6 +434,150 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { tk.MustIndexLookup("select * from tlist where a<1") } +func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_dr_agg") + tk.MustExec("use test_dr_agg") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // list partition table + tk.MustExec(`create table tlist(a int, b int, index idx_a(a), index idx_b(b)) partition by list(a)( + partition p0 values in (1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8), + partition p2 values in (9, 10, 11, 12));`) + + // range partition table + tk.MustExec(`create table trange(a int, b int, index idx_a(a), index idx_b(b)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table + tk.MustExec(`create table thash(a int, b int) partition by hash(a) partitions 4;`) + + // regular table + tk.MustExec("create table tregular1(a int, b int, index idx_a(a))") + tk.MustExec("create table tregular2(a int, b int, index idx_a(a))") + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000))) + } + + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular1 values " + strings.Join(vals, ",")) + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(12)+1, rand.Intn(20))) + } + + tk.MustExec("insert into tlist values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular2 values " + strings.Join(vals, ",")) + + // test range partition + for i := 0; i < 2000; i++ { + // select /*+ stream_agg() */ a from t where a > ? group by a; + // select /*+ hash_agg() */ a from t where a > ? group by a; + // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; + // select /*+ hash_agg() */ a from t where a in (?, ?, ?) group by a; + x := rand.Intn(1099) + + queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from trange where a > %v group by a;", x) + queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from trange where a > %v group by a;", x) + queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + + y := rand.Intn(1099) + z := rand.Intn(1099) + + queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from trange where a in(%v, %v, %v) group by a;", x, y, z) + queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a in(%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) + + queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from trange where a in (%v, %v, %v) group by a;", x, y, z) + queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a in (%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) + } + + // test hash partition + for i := 0; i < 2000; i++ { + // select /*+ stream_agg() */ a from t where a > ? group by a; + // select /*+ hash_agg() */ a from t where a > ? group by a; + // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; + // select /*+ hash_agg() */ a from t where a in (?, ?, ?) group by a; + x := rand.Intn(1099) + + queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from thash where a > %v group by a;", x) + queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from thash where a > %v group by a;", x) + queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + + y := rand.Intn(1099) + z := rand.Intn(1099) + + queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from thash where a in(%v, %v, %v) group by a;", x, y, z) + queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular1 where a in(%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) + + queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from thash where a in (%v, %v, %v) group by a;", x, y, z) + queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular1 where a in (%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) + } + + // test list partition + for i := 0; i < 2000; i++ { + // select /*+ stream_agg() */ a from t where a > ? group by a; + // select /*+ hash_agg() */ a from t where a > ? group by a; + // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; + // select /*+ hash_agg() */ a from t where a in (?, ?, ?) group by a; + x := rand.Intn(12) + 1 + + queryPartition1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tlist where a > %v group by a;", x) + queryRegular1 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular2 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition1, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tlist where a > %v group by a;", x) + queryRegular2 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular2 where a > %v group by a;", x) + c.Assert(tk.HasPlan(queryPartition2, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + + y := rand.Intn(12) + 1 + z := rand.Intn(12) + 1 + + queryPartition3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tlist where a in(%v, %v, %v) group by a;", x, y, z) + queryRegular3 := fmt.Sprintf("select /*+ stream_agg() */ count(*), sum(b), max(b), a from tregular2 where a in(%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition3, "StreamAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition3).Sort().Check(tk.MustQuery(queryRegular3).Sort().Rows()) + + queryPartition4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tlist where a in (%v, %v, %v) group by a;", x, y, z) + queryRegular4 := fmt.Sprintf("select /*+ hash_agg() */ count(*), sum(b), max(b), a from tregular2 where a in (%v, %v, %v) group by a;", x, y, z) + c.Assert(tk.HasPlan(queryPartition4, "HashAgg"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition4).Sort().Check(tk.MustQuery(queryRegular4).Sort().Rows()) + } +} + func (s *globalIndexSuite) TestGlobalIndexScan(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists p") From daf133cf81bff352a2c0732cc3e8b20a0e6fe111 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 12:13:39 +0800 Subject: [PATCH 012/343] store/copr: removes the unused fields in batchCopIterator (#24446) --- store/copr/batch_coprocessor.go | 18 +++++------------- 1 file changed, 5 insertions(+), 13 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 5506c9d497ac2..3231f95bbc824 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -33,8 +33,6 @@ import ( "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - "github.com/pingcap/tidb/util/memory" "go.uber.org/zap" ) @@ -184,13 +182,11 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V return copErrorResponse{err} } it := &batchCopIterator{ - store: c.store.kvStore, - req: req, - finishCh: make(chan struct{}), - vars: vars, - memTracker: req.MemTracker, - ClientHelper: tikv.NewClientHelper(c.store.kvStore.store, util.NewTSSet(5)), - rpcCancel: tikv.NewRPCanceller(), + store: c.store.kvStore, + req: req, + finishCh: make(chan struct{}), + vars: vars, + rpcCancel: tikv.NewRPCanceller(), } ctx = context.WithValue(ctx, tikv.RPCCancellerCtxKey{}, it.rpcCancel) it.tasks = tasks @@ -200,8 +196,6 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V } type batchCopIterator struct { - *tikv.ClientHelper - store *kvStore req *kv.Request finishCh chan struct{} @@ -213,8 +207,6 @@ type batchCopIterator struct { vars *tikv.Variables - memTracker *memory.Tracker - rpcCancel *tikv.RPCCanceller wg sync.WaitGroup From 9b86513a4dc9cf22f4fe92a6b4faabad1233160b Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 12:25:38 +0800 Subject: [PATCH 013/343] =?UTF-8?q?store/tikv:=20make=20tikv.ErrLockWaitTi?= =?UTF-8?q?meout=20as=20a=20normal=20error=20instead=20of=E2=80=A6=20(#244?= =?UTF-8?q?24)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- session/pessimistic_test.go | 19 ++++++++++--------- store/driver/txn/error.go | 6 ++++++ store/tikv/error/error.go | 3 ++- store/tikv/tests/2pc_test.go | 2 +- 4 files changed, 19 insertions(+), 11 deletions(-) diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 05a93e3aee30d..8fdd635b51bc1 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -31,6 +31,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" + txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/oracle" @@ -733,10 +734,10 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { timeoutErr := <-timeoutErrCh c.Assert(timeoutErr, NotNil) - c.Assert(timeoutErr.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Assert(timeoutErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) timeoutErr = <-timeoutErrCh c.Assert(timeoutErr, NotNil) - c.Assert(timeoutErr.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Assert(timeoutErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) // tk4 lock c1 = 2 tk4.MustExec("begin pessimistic") @@ -749,7 +750,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { _, err := tk2.Exec("delete from tk where c1 = 2") c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big - c.Check(err.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(err.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) tk4.MustExec("commit") @@ -767,7 +768,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { _, err = tk2.Exec("delete from tk where c1 = 3") // tk2 tries to lock c1 = 3 fail, this delete should be rollback, but previous update should be keeped c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big - c.Check(err.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(err.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) tk2.MustExec("commit") tk3.MustExec("commit") @@ -841,7 +842,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeoutWaitStart(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/PessimisticLockErrWriteConflict"), IsNil) waitErr := <-done c.Assert(waitErr, NotNil) - c.Check(waitErr.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(waitErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) c.Check(duration, GreaterEqual, 1000*time.Millisecond) c.Check(duration, LessEqual, 3000*time.Millisecond) tk2.MustExec("rollback") @@ -1279,7 +1280,7 @@ func (s *testPessimisticSuite) TestBatchPointGetLockIndex(c *C) { tk2.MustExec("begin pessimistic") err := tk2.ExecToErr("insert into t1 values(2, 2, 2)") c.Assert(err, NotNil) - c.Assert(tikverr.ErrLockWaitTimeout.Equal(err), IsTrue) + c.Assert(txndriver.ErrLockWaitTimeout.Equal(err), IsTrue) err = tk2.ExecToErr("select * from t1 where c2 = 3 for update nowait") c.Assert(err, NotNil) c.Assert(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) @@ -1996,11 +1997,11 @@ func (s *testPessimisticSuite) TestSelectForUpdateWaitSeconds(c *C) { waitErr2 := <-errCh waitErr3 := <-errCh c.Assert(waitErr, NotNil) - c.Check(waitErr.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(waitErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) c.Assert(waitErr2, NotNil) - c.Check(waitErr2.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(waitErr2.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) c.Assert(waitErr3, NotNil) - c.Check(waitErr3.Error(), Equals, tikverr.ErrLockWaitTimeout.Error()) + c.Check(waitErr3.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) c.Assert(time.Since(start).Seconds(), Less, 45.0) tk2.MustExec("commit") tk3.MustExec("rollback") diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index c026b8899ae72..5bbdc01024947 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -50,6 +50,8 @@ var ( // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) + // ErrLockWaitTimeout is the error that wait for the lock is timeout. + ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(errno.ErrLockWaitTimeout) // ErrTiKVServerBusy is the error when tikv server is busy. ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) // ErrTiFlashServerBusy is the error that tiflash server is busy. @@ -251,6 +253,10 @@ func ToTiDBErr(err error) error { return ErrResolveLockTimeout } + if errors.ErrorEqual(err, tikverr.ErrLockWaitTimeout) { + return ErrLockWaitTimeout + } + if errors.ErrorEqual(err, tikverr.ErrRegionUnavailable) { return ErrRegionUnavailable } diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index 70ec995e88489..bde97e8b2e5d4 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -44,6 +44,8 @@ var ( ErrTiKVMaxTimestampNotSynced = errors.New("tikv max timestamp not synced") // ErrResolveLockTimeout is the error that resolve lock timeout. ErrResolveLockTimeout = errors.New("resolve lock timeout") + // ErrLockWaitTimeout is the error that wait for the lock is timeout. + ErrLockWaitTimeout = errors.New("lock wait timeout") // ErrTiKVServerBusy is the error when tikv server is busy. ErrTiKVServerBusy = errors.New("tikv server busy") // ErrTiFlashServerBusy is the error that tiflash server is busy. @@ -59,7 +61,6 @@ const MismatchClusterID = "mismatch cluster id" var ( ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) - ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(CodeLockWaitTimeout) ErrUnknown = dbterror.ClassTiKV.NewStd(CodeUnknown) ) diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 8043ccdab89f0..d1e635f205efa 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -1025,7 +1025,7 @@ func (s *testCommitterSuite) TestPessimisticLockPrimary(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/txnNotFoundRetTTL"), IsNil) c.Assert(err, IsNil) waitErr := <-doneCh - c.Assert(tikverr.ErrLockWaitTimeout.Equal(waitErr), IsTrue) + c.Assert(tikverr.ErrLockWaitTimeout, Equals, waitErr) } func (s *testCommitterSuite) TestResolvePessimisticLock(c *C) { From 3fed33fc17354038cebc121669b67e3dcecd21ad Mon Sep 17 00:00:00 2001 From: rebelice Date: Tue, 11 May 2021 12:47:38 +0800 Subject: [PATCH 014/343] statistics: dump FMSketch to KV only for partition table with dynamic prune mode (#24453) --- executor/analyze.go | 7 +-- statistics/handle/ddl.go | 6 ++- statistics/handle/dump.go | 6 ++- statistics/handle/dump_test.go | 2 +- statistics/handle/handle.go | 4 +- statistics/handle/handle_test.go | 85 +++++++++++++++++++++++--------- statistics/handle/update.go | 11 +++-- 7 files changed, 83 insertions(+), 38 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index b7e9e51fec7da..8345382939865 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -160,9 +160,9 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } var err1 error if result.StatsVer == statistics.Version3 { - err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, nil, result.TopNs[i], result.Fms[i], result.StatsVer, 1) + err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, nil, result.TopNs[i], result.Fms[i], result.StatsVer, 1, result.TableID.IsPartitionTable() && needGlobalStats) } else { - err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, result.Cms[i], result.TopNs[i], result.Fms[i], result.StatsVer, 1) + err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, result.Cms[i], result.TopNs[i], result.Fms[i], result.StatsVer, 1, result.TableID.IsPartitionTable() && needGlobalStats) } if err1 != nil { err = err1 @@ -198,7 +198,8 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } for i := 0; i < globalStats.Num; i++ { hg, cms, topN, fms := globalStats.Hg[i], globalStats.Cms[i], globalStats.TopN[i], globalStats.Fms[i] - err = statsHandle.SaveStatsToStorage(globalStatsID.tableID, globalStats.Count, info.isIndex, hg, cms, topN, fms, info.statsVersion, 1) + // fms for global stats doesn't need to dump to kv. + err = statsHandle.SaveStatsToStorage(globalStatsID.tableID, globalStats.Count, info.isIndex, hg, cms, topN, fms, info.statsVersion, 1, false) if err != nil { logutil.Logger(ctx).Error("save global-level stats to storage failed", zap.Error(err)) } diff --git a/statistics/handle/ddl.go b/statistics/handle/ddl.go index 34f30c1241e1c..2ddc460ff690b 100644 --- a/statistics/handle/ddl.go +++ b/statistics/handle/ddl.go @@ -112,7 +112,8 @@ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error { } for i := 0; i < newColGlobalStats.Num; i++ { hg, cms, topN, fms := newColGlobalStats.Hg[i], newColGlobalStats.Cms[i], newColGlobalStats.TopN[i], newColGlobalStats.Fms[i] - err = h.SaveStatsToStorage(tableID, newColGlobalStats.Count, 0, hg, cms, topN, fms, 2, 1) + // fms for global stats doesn't need to dump to kv. + err = h.SaveStatsToStorage(tableID, newColGlobalStats.Count, 0, hg, cms, topN, fms, 2, 1, false) if err != nil { return err } @@ -141,7 +142,8 @@ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error { } for i := 0; i < newIndexGlobalStats.Num; i++ { hg, cms, topN, fms := newIndexGlobalStats.Hg[i], newIndexGlobalStats.Cms[i], newIndexGlobalStats.TopN[i], newIndexGlobalStats.Fms[i] - err = h.SaveStatsToStorage(tableID, newIndexGlobalStats.Count, 1, hg, cms, topN, fms, 2, 1) + // fms for global stats doesn't need to dump to kv. + err = h.SaveStatsToStorage(tableID, newIndexGlobalStats.Count, 1, hg, cms, topN, fms, 2, 1, false) if err != nil { return err } diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 1f20855742a76..36971076b4644 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -230,13 +230,15 @@ func (h *Handle) loadStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, } for _, col := range tbl.Columns { - err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, &col.Histogram, col.CMSketch, col.TopN, col.FMSketch, int(col.StatsVer), 1) + // loadStatsFromJSON doesn't support partition table now. + err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, &col.Histogram, col.CMSketch, col.TopN, col.FMSketch, int(col.StatsVer), 1, false) if err != nil { return errors.Trace(err) } } for _, idx := range tbl.Indices { - err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 1, &idx.Histogram, idx.CMSketch, idx.TopN, nil, int(idx.StatsVer), 1) + // loadStatsFromJSON doesn't support partition table now. + err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 1, &idx.Histogram, idx.CMSketch, idx.TopN, nil, int(idx.StatsVer), 1, false) if err != nil { return errors.Trace(err) } diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index ef02403c48ac5..f0c7c2fb09110 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -215,7 +215,7 @@ func (s *testStatsSuite) TestDumpCMSketchWithTopN(c *C) { cms, _, _, _ := statistics.NewCMSketchAndTopN(5, 2048, fakeData, 20, 100) stat := h.GetTableStats(tableInfo) - err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, nil, nil, statistics.Version2, 1) + err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, nil, nil, statistics.Version2, 1, false) c.Assert(err, IsNil) c.Assert(h.Update(is), IsNil) diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 7036242b4e3a7..a8f51f8924e81 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -952,7 +952,7 @@ func (h *Handle) extendedStatsFromStorage(reader *statsReader, table *statistics } // SaveStatsToStorage saves the stats to storage. -func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, fms *statistics.FMSketch, statsVersion int, isAnalyzed int64) (err error) { +func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, fms *statistics.FMSketch, statsVersion int, isAnalyzed int64, needDumpFMS bool) (err error) { h.mu.Lock() defer h.mu.Unlock() ctx := context.TODO() @@ -1001,7 +1001,7 @@ func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg if _, err := exec.ExecuteInternal(ctx, "delete from mysql.stats_fm_sketch where table_id = %? and is_index = %? and hist_id = %?", tableID, isIndex, hg.ID); err != nil { return err } - if fmSketch != nil { + if fmSketch != nil && needDumpFMS { if _, err = exec.ExecuteInternal(ctx, "insert into mysql.stats_fm_sketch (table_id, is_index, hist_id, value) values (%?, %?, %?, %?)", tableID, isIndex, hg.ID, fmSketch); err != nil { return err } diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 3bb95e8219478..2bf1d93ff4246 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2332,16 +2332,18 @@ func (s *testStatsSuite) TestDuplicateFMSketch(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.MustExec("create table t(a int, b int, c int)") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + defer tk.MustExec("set @@tidb_partition_prune_mode='static'") + tk.MustExec("create table t(a int, b int, c int) partition by hash(a) partitions 3") tk.MustExec("insert into t values (1, 1, 1)") tk.MustExec("analyze table t") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("3")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("9")) tk.MustExec("analyze table t") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("3")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("9")) - tk.MustExec("alter table t drop column a") + tk.MustExec("alter table t drop column b") c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), time.Duration(0)), IsNil) - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("2")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) } func (s *testStatsSuite) TestIndexFMSketch(c *C) { @@ -2349,24 +2351,26 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int, c int, index ia(a), index ibc(b, c))") + tk.MustExec("create table t(a int, b int, c int, index ia(a), index ibc(b, c)) partition by hash(a) partitions 3") tk.MustExec("insert into t values (1, 1, 1)") + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + defer tk.MustExec("set @@tidb_partition_prune_mode='static'") tk.MustExec("analyze table t index ia") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("1")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("3")) tk.MustExec("analyze table t index ibc") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("2")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) tk.MustExec("analyze table t") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("5")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("15")) tk.MustExec("drop table if exists t") c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), 0), IsNil) // clustered index tk.MustExec("drop table if exists t") tk.MustExec("set @@tidb_enable_clustered_index=ON") - tk.MustExec("create table t (a datetime, b datetime, primary key (a))") + tk.MustExec("create table t (a datetime, b datetime, primary key (a)) partition by hash(year(a)) partitions 3") tk.MustExec("insert into t values ('2000-01-01', '2000-01-01')") tk.MustExec("analyze table t") - tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("2")) + tk.MustQuery("select count(*) from mysql.stats_fm_sketch").Check(testkit.Rows("6")) tk.MustExec("drop table if exists t") c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), 0), IsNil) @@ -2383,25 +2387,23 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { } tk.MustExec("set @@tidb_enable_clustered_index=OFF") - tk.MustExec("create table t(a int, key(a))") + tk.MustExec("create table t(a int, key(a)) partition by hash(a) partitions 3") tk.MustExec("insert into t values (1), (2), (2), (3)") - checkNDV(2, 3) - tk.MustExec("insert into t values (4), (5)") - checkNDV(2, 5) + checkNDV(6, 1) + tk.MustExec("insert into t values (4), (5), (6)") + checkNDV(6, 2) tk.MustExec("insert into t values (2), (5)") - checkNDV(2, 5) + checkNDV(6, 2) tk.MustExec("drop table if exists t") c.Assert(s.do.StatsHandle().GCStats(s.do.InfoSchema(), 0), IsNil) // clustered index tk.MustExec("set @@tidb_enable_clustered_index=ON") - tk.MustExec("create table t (a datetime, b datetime, primary key (a))") - tk.MustExec("insert into t values ('2000-01-01', '2000-01-01')") - checkNDV(2, 1) - tk.MustExec("insert into t values ('2020-01-01', '2020-01-01')") - checkNDV(2, 2) - tk.MustExec("insert into t values ('1999-01-01', '1999-01-01'), ('1999-01-02', '1999-01-02'), ('1999-01-03', '1999-01-03')") - checkNDV(2, 5) + tk.MustExec("create table t (a datetime, b datetime, primary key (a)) partition by hash(year(a)) partitions 3") + tk.MustExec("insert into t values ('2000-01-01', '2001-01-01'), ('2001-01-01', '2001-01-01'), ('2002-01-01', '2001-01-01')") + checkNDV(6, 1) + tk.MustExec("insert into t values ('1999-01-01', '1998-01-01'), ('1997-01-02', '1999-01-02'), ('1998-01-03', '1999-01-03')") + checkNDV(6, 2) } func (s *testStatsSuite) TestShowExtendedStats4DropColumn(c *C) { @@ -2802,3 +2804,40 @@ func (s *testSerialStatsSuite) TestIssues24349(c *C) { "test t global b 0 1 10 1 4 4 0", )) } + +func (s *testStatsSuite) TestIssues24401(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + + // normal table with static prune mode + testKit.MustExec("set @@tidb_partition_prune_mode='static'") + testKit.MustExec("create table t(a int, index(a))") + testKit.MustExec("insert into t values (1), (2), (3)") + testKit.MustExec("analyze table t") + testKit.MustQuery("select * from mysql.stats_fm_sketch").Check(testkit.Rows()) + + // partition table with static prune mode + testKit.MustExec("create table tp(a int, index(a)) partition by hash(a) partitions 3") + testKit.MustExec("insert into tp values (1), (2), (3)") + testKit.MustExec("analyze table tp") + testKit.MustQuery("select * from mysql.stats_fm_sketch").Check(testkit.Rows()) + + // normal table with dynamic prune mode + testKit.MustExec("set @@tidb_partition_prune_mode='dynamic'") + defer testKit.MustExec("set @@tidb_partition_prune_mode='static'") + testKit.MustExec("analyze table t") + testKit.MustQuery("select * from mysql.stats_fm_sketch").Check(testkit.Rows()) + + // partition table with dynamic prune mode + testKit.MustExec("analyze table tp") + rows := testKit.MustQuery("select * from mysql.stats_fm_sketch").Rows() + lenRows := len(rows) + c.Assert(lenRows, Equals, 6) + + // check fm-sketch won't increase infinitely + testKit.MustExec("insert into t values (10), (20), (30), (12), (23), (23), (4344)") + testKit.MustExec("analyze table tp") + rows = testKit.MustQuery("select * from mysql.stats_fm_sketch").Rows() + c.Assert(len(rows), Equals, lenRows) +} diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 6f472fc61fdc7..21438adc5b29d 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -750,11 +750,11 @@ func (h *Handle) handleSingleHistogramUpdate(is infoschema.InfoSchema, rows []ch return nil } var tbl *statistics.Table - if table.Meta().GetPartitionInfo() == nil || h.CurrentPruneMode() == variable.Dynamic { - tbl = h.GetTableStats(table.Meta()) - } else { - tbl = h.GetPartitionStats(table.Meta(), physicalTableID) + // feedback for partition is not ready + if table.Meta().GetPartitionInfo() != nil { + return nil } + tbl = h.GetTableStats(table.Meta()) var cms *statistics.CMSketch var hist *statistics.Histogram var topN *statistics.TopN @@ -822,7 +822,8 @@ func (h *Handle) deleteOutdatedFeedback(tableID, histID, isIndex int64) error { func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.QueryFeedback, hist *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, fms *statistics.FMSketch, statsVersion int64) error { hist = statistics.UpdateHistogram(hist, q, int(statsVersion)) - err := h.SaveStatsToStorage(tableID, -1, int(isIndex), hist, cms, topN, fms, int(statsVersion), 0) + // feedback for partition is not ready. + err := h.SaveStatsToStorage(tableID, -1, int(isIndex), hist, cms, topN, fms, int(statsVersion), 0, false) metrics.UpdateStatsCounter.WithLabelValues(metrics.RetLabel(err)).Inc() return errors.Trace(err) } From 24e9c1342b1b637c8cc56fa20eded5ea2f6c8510 Mon Sep 17 00:00:00 2001 From: jianzhiyao <739319867@qq.com> Date: Tue, 11 May 2021 12:59:38 +0800 Subject: [PATCH 015/343] ddl: speedup test case TestModifyColumnTime (#24469) --- ddl/db_test.go | 188 +++++++++++++++++++++++++++++++++++++------------ 1 file changed, 143 insertions(+), 45 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index e865de39d3248..eddad6d0d635d 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -4801,52 +4801,9 @@ func (s *testSerialDBSuite) TestModifyColumnCharset(c *C) { } -func (s *testDBSuite1) TestModifyColumnTime(c *C) { - limit := variable.GetDDLErrorCountLimit() - variable.SetDDLErrorCountLimit(3) - - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") - enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType - tk.Se.GetSessionVars().EnableChangeColumnType = true - - // Set time zone to UTC. - originalTz := tk.Se.GetSessionVars().TimeZone - tk.Se.GetSessionVars().TimeZone = time.UTC - defer func() { - variable.SetDDLErrorCountLimit(limit) - tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType - tk.Se.GetSessionVars().TimeZone = originalTz - }() - - now := time.Now().UTC() - now = time.Date(now.Year(), now.Month(), now.Day(), 0, 0, 0, 0, time.UTC) - timeToDate1 := now.Format("2006-01-02") - timeToDate2 := now.AddDate(0, 0, 30).Format("2006-01-02") - - timeToDatetime1 := now.Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToDatetime2 := now.Add(20 * time.Hour).Format("2006-01-02 15:04:05") - timeToDatetime3 := now.Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToDatetime4 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToDatetime5 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Format("2006-01-02 15:04:05") - - timeToTimestamp1 := now.Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToTimestamp2 := now.Add(20 * time.Hour).Format("2006-01-02 15:04:05") - timeToTimestamp3 := now.Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToTimestamp4 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") - timeToTimestamp5 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Format("2006-01-02 15:04:05") +func (s *testDBSuite1) TestModifyColumnTime_TimeToYear(c *C) { currentYear := strconv.Itoa(time.Now().Year()) - - // 1. In conversion between date/time, fraction parts are taken into account - // Refer to doc: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-type-conversion.html - // 2. Failed tests are commentd to pass unit-test - tests := []struct { - from string - value string - to string - expect string - err uint16 - }{ + tests := []testModifyColumnTimeCase{ // time to year, it's reasonable to return current year and discard the time (even if MySQL may get data out of range error). {"time", `"30 20:00:12"`, "year", currentYear, 0}, {"time", `"30 20:00"`, "year", currentYear, 0}, @@ -4862,7 +4819,16 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"time", `"20:00:12.498"`, "year", currentYear, 0}, {"time", `"200012.498"`, "year", currentYear, 0}, {"time", `200012.498`, "year", currentYear, 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimeToDate(c *C) { + now := time.Now().UTC() + now = time.Date(now.Year(), now.Month(), now.Day(), 0, 0, 0, 0, time.UTC) + timeToDate1 := now.Format("2006-01-02") + timeToDate2 := now.AddDate(0, 0, 30).Format("2006-01-02") + tests := []testModifyColumnTimeCase{ // time to date {"time", `"30 20:00:12"`, "date", timeToDate2, 0}, {"time", `"30 20:00"`, "date", timeToDate2, 0}, @@ -4878,7 +4844,19 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"time", `"20:00:12.498"`, "date", timeToDate1, 0}, {"time", `"200012.498"`, "date", timeToDate1, 0}, {"time", `200012.498`, "date", timeToDate1, 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimeToDatetime(c *C) { + now := time.Now().UTC() + now = time.Date(now.Year(), now.Month(), now.Day(), 0, 0, 0, 0, time.UTC) + timeToDatetime1 := now.Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToDatetime2 := now.Add(20 * time.Hour).Format("2006-01-02 15:04:05") + timeToDatetime3 := now.Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToDatetime4 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToDatetime5 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Format("2006-01-02 15:04:05") + tests := []testModifyColumnTimeCase{ // time to datetime {"time", `"30 20:00:12"`, "datetime", timeToDatetime4, 0}, {"time", `"30 20:00"`, "datetime", timeToDatetime5, 0}, @@ -4894,7 +4872,19 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"time", `"20:00:12.498"`, "datetime", timeToDatetime1, 0}, {"time", `"200012.498"`, "datetime", timeToDatetime1, 0}, {"time", `200012.498`, "datetime", timeToDatetime1, 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimeToTimestamp(c *C) { + now := time.Now().UTC() + now = time.Date(now.Year(), now.Month(), now.Day(), 0, 0, 0, 0, time.UTC) + timeToTimestamp1 := now.Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToTimestamp2 := now.Add(20 * time.Hour).Format("2006-01-02 15:04:05") + timeToTimestamp3 := now.Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToTimestamp4 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Add(12 * time.Second).Format("2006-01-02 15:04:05") + timeToTimestamp5 := now.AddDate(0, 0, 30).Add(20 * time.Hour).Format("2006-01-02 15:04:05") + tests := []testModifyColumnTimeCase{ // time to timestamp {"time", `"30 20:00:12"`, "timestamp", timeToTimestamp4, 0}, {"time", `"30 20:00"`, "timestamp", timeToTimestamp5, 0}, @@ -4910,7 +4900,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"time", `"20:00:12.498"`, "timestamp", timeToTimestamp1, 0}, {"time", `"200012.498"`, "timestamp", timeToTimestamp1, 0}, {"time", `200012.498`, "timestamp", timeToTimestamp1, 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DateToTime(c *C) { + tests := []testModifyColumnTimeCase{ // date to time {"date", `"2019-01-02"`, "time", "00:00:00", 0}, {"date", `"19-01-02"`, "time", "00:00:00", 0}, @@ -4918,7 +4913,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"date", `"190102"`, "time", "00:00:00", 0}, {"date", `20190102`, "time", "00:00:00", 0}, {"date", `190102`, "time", "00:00:00", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DateToYear(c *C) { + tests := []testModifyColumnTimeCase{ // date to year {"date", `"2019-01-02"`, "year", "2019", 0}, {"date", `"19-01-02"`, "year", "2019", 0}, @@ -4926,7 +4926,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"date", `"190102"`, "year", "2019", 0}, {"date", `20190102`, "year", "2019", 0}, {"date", `190102`, "year", "2019", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DateToDatetime(c *C) { + tests := []testModifyColumnTimeCase{ // date to datetime {"date", `"2019-01-02"`, "datetime", "2019-01-02 00:00:00", 0}, {"date", `"19-01-02"`, "datetime", "2019-01-02 00:00:00", 0}, @@ -4934,7 +4939,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"date", `"190102"`, "datetime", "2019-01-02 00:00:00", 0}, {"date", `20190102`, "datetime", "2019-01-02 00:00:00", 0}, {"date", `190102`, "datetime", "2019-01-02 00:00:00", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DateToTimestamp(c *C) { + tests := []testModifyColumnTimeCase{ // date to timestamp {"date", `"2019-01-02"`, "timestamp", "2019-01-02 00:00:00", 0}, {"date", `"19-01-02"`, "timestamp", "2019-01-02 00:00:00", 0}, @@ -4942,7 +4952,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"date", `"190102"`, "timestamp", "2019-01-02 00:00:00", 0}, {"date", `20190102`, "timestamp", "2019-01-02 00:00:00", 0}, {"date", `190102`, "timestamp", "2019-01-02 00:00:00", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimestampToYear(c *C) { + tests := []testModifyColumnTimeCase{ // timestamp to year {"timestamp", `"2006-01-02 15:04:05"`, "year", "2006", 0}, {"timestamp", `"06-01-02 15:04:05"`, "year", "2006", 0}, @@ -4951,7 +4966,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"timestamp", `20060102150405`, "year", "2006", 0}, {"timestamp", `060102150405`, "year", "2006", 0}, {"timestamp", `"2006-01-02 23:59:59.506"`, "year", "2006", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimestampToTime(c *C) { + tests := []testModifyColumnTimeCase{ // timestamp to time {"timestamp", `"2006-01-02 15:04:05"`, "time", "15:04:05", 0}, {"timestamp", `"06-01-02 15:04:05"`, "time", "15:04:05", 0}, @@ -4960,7 +4980,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"timestamp", `20060102150405`, "time", "15:04:05", 0}, {"timestamp", `060102150405`, "time", "15:04:05", 0}, {"timestamp", `"2006-01-02 23:59:59.506"`, "time", "00:00:00", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimestampToDate(c *C) { + tests := []testModifyColumnTimeCase{ // timestamp to date {"timestamp", `"2006-01-02 15:04:05"`, "date", "2006-01-02", 0}, {"timestamp", `"06-01-02 15:04:05"`, "date", "2006-01-02", 0}, @@ -4969,7 +4994,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"timestamp", `20060102150405`, "date", "2006-01-02", 0}, {"timestamp", `060102150405`, "date", "2006-01-02", 0}, {"timestamp", `"2006-01-02 23:59:59.506"`, "date", "2006-01-03", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_TimestampToDatetime(c *C) { + tests := []testModifyColumnTimeCase{ // timestamp to datetime {"timestamp", `"2006-01-02 15:04:05"`, "datetime", "2006-01-02 15:04:05", 0}, {"timestamp", `"06-01-02 15:04:05"`, "datetime", "2006-01-02 15:04:05", 0}, @@ -4978,7 +5008,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"timestamp", `20060102150405`, "datetime", "2006-01-02 15:04:05", 0}, {"timestamp", `060102150405`, "datetime", "2006-01-02 15:04:05", 0}, {"timestamp", `"2006-01-02 23:59:59.506"`, "datetime", "2006-01-03 00:00:00", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DatetimeToYear(c *C) { + tests := []testModifyColumnTimeCase{ // datetime to year {"datetime", `"2006-01-02 15:04:05"`, "year", "2006", 0}, {"datetime", `"06-01-02 15:04:05"`, "year", "2006", 0}, @@ -4990,7 +5025,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { // MySQL will get "Data truncation: Out of range value for column 'a' at row 1. {"datetime", `"1000-01-02 23:59:59"`, "year", "", errno.ErrInvalidYear}, {"datetime", `"9999-01-02 23:59:59"`, "year", "", errno.ErrInvalidYear}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DatetimeToTime(c *C) { + tests := []testModifyColumnTimeCase{ // datetime to time {"datetime", `"2006-01-02 15:04:05"`, "time", "15:04:05", 0}, {"datetime", `"06-01-02 15:04:05"`, "time", "15:04:05", 0}, @@ -5001,7 +5041,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"datetime", `"2006-01-02 23:59:59.506"`, "time", "00:00:00", 0}, {"datetime", `"1000-01-02 23:59:59"`, "time", "23:59:59", 0}, {"datetime", `"9999-01-02 23:59:59"`, "time", "23:59:59", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DatetimeToDate(c *C) { + tests := []testModifyColumnTimeCase{ // datetime to date {"datetime", `"2006-01-02 15:04:05"`, "date", "2006-01-02", 0}, {"datetime", `"06-01-02 15:04:05"`, "date", "2006-01-02", 0}, @@ -5012,7 +5057,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"datetime", `"2006-01-02 23:59:59.506"`, "date", "2006-01-03", 0}, {"datetime", `"1000-01-02 23:59:59"`, "date", "1000-01-02", 0}, {"datetime", `"9999-01-02 23:59:59"`, "date", "9999-01-02", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_DatetimeToTimestamp(c *C) { + tests := []testModifyColumnTimeCase{ // datetime to timestamp {"datetime", `"2006-01-02 15:04:05"`, "timestamp", "2006-01-02 15:04:05", 0}, {"datetime", `"06-01-02 15:04:05"`, "timestamp", "2006-01-02 15:04:05", 0}, @@ -5023,7 +5073,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"datetime", `"2006-01-02 23:59:59.506"`, "timestamp", "2006-01-03 00:00:00", 0}, {"datetime", `"1000-01-02 23:59:59"`, "timestamp", "1000-01-02 23:59:59", 0}, {"datetime", `"9999-01-02 23:59:59"`, "timestamp", "9999-01-02 23:59:59", 0}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_YearToTime(c *C) { + tests := []testModifyColumnTimeCase{ // year to time // failed cases are not handled by TiDB {"year", `"2019"`, "time", "00:20:19", 0}, @@ -5036,7 +5091,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"year", `69`, "time", "", errno.ErrTruncatedWrongValue}, {"year", `70`, "time", "", errno.ErrTruncatedWrongValue}, {"year", `99`, "time", "", errno.ErrTruncatedWrongValue}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_YearToDate(c *C) { + tests := []testModifyColumnTimeCase{ // year to date {"year", `"2019"`, "date", "", errno.ErrTruncatedWrongValue}, {"year", `2019`, "date", "", errno.ErrTruncatedWrongValue}, @@ -5049,7 +5109,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"year", `69`, "date", "", errno.ErrTruncatedWrongValue}, {"year", `70`, "date", "", errno.ErrTruncatedWrongValue}, {"year", `99`, "date", "", errno.ErrTruncatedWrongValue}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_YearToDatetime(c *C) { + tests := []testModifyColumnTimeCase{ // year to datetime {"year", `"2019"`, "datetime", "", errno.ErrTruncatedWrongValue}, {"year", `2019`, "datetime", "", errno.ErrTruncatedWrongValue}, @@ -5062,7 +5127,12 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"year", `69`, "datetime", "", errno.ErrTruncatedWrongValue}, {"year", `70`, "datetime", "", errno.ErrTruncatedWrongValue}, {"year", `99`, "datetime", "", errno.ErrTruncatedWrongValue}, + } + testModifyColumnTime(c, s.store, tests) +} +func (s *testDBSuite1) TestModifyColumnTime_YearToTimestamp(c *C) { + tests := []testModifyColumnTimeCase{ // year to timestamp {"year", `"2019"`, "timestamp", "", errno.ErrTruncatedWrongValue}, {"year", `2019`, "timestamp", "", errno.ErrTruncatedWrongValue}, @@ -5076,6 +5146,34 @@ func (s *testDBSuite1) TestModifyColumnTime(c *C) { {"year", `70`, "timestamp", "", errno.ErrTruncatedWrongValue}, {"year", `99`, "timestamp", "", errno.ErrTruncatedWrongValue}, } + testModifyColumnTime(c, s.store, tests) +} + +type testModifyColumnTimeCase struct { + from string + value string + to string + expect string + err uint16 +} + +func testModifyColumnTime(c *C, store kv.Storage, tests []testModifyColumnTimeCase) { + limit := variable.GetDDLErrorCountLimit() + variable.SetDDLErrorCountLimit(3) + + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test_db") + enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType + tk.Se.GetSessionVars().EnableChangeColumnType = true + + // Set time zone to UTC. + originalTz := tk.Se.GetSessionVars().TimeZone + tk.Se.GetSessionVars().TimeZone = time.UTC + defer func() { + variable.SetDDLErrorCountLimit(limit) + tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType + tk.Se.GetSessionVars().TimeZone = originalTz + }() for _, t := range tests { tk.MustExec("drop table if exists t_mc") From e26b8470c7f8b2b6e319e5d2c7b72e6293992720 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Tue, 11 May 2021 13:11:38 +0800 Subject: [PATCH 016/343] variable: use the defined variable (#24509) --- sessionctx/variable/noop.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index a3382fc1159d1..05553c6f7eec1 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -156,7 +156,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "myisam_mmap_size", Value: "18446744073709551615"}, {Scope: ScopeNone, Name: "innodb_buffer_pool_instances", Value: "8"}, {Scope: ScopeGlobal | ScopeSession, Name: "max_length_for_sort_data", Value: "1024", IsHintUpdatable: true}, - {Scope: ScopeNone, Name: "character_set_system", Value: "utf8"}, + {Scope: ScopeNone, Name: CharacterSetSystem, Value: "utf8"}, {Scope: ScopeGlobal, Name: InnodbOptimizeFullTextOnly, Value: "0"}, {Scope: ScopeNone, Name: "character_sets_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/charsets/"}, {Scope: ScopeGlobal | ScopeSession, Name: QueryCacheType, Value: Off, Type: TypeEnum, PossibleValues: []string{Off, On, "DEMAND"}}, From c8bc701170165e82cc7893ab68523f9a2059a7e6 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 10 May 2021 23:23:38 -0600 Subject: [PATCH 017/343] *: misc cleanups to variable package (#24507) --- ddl/util/util.go | 4 +- executor/aggfuncs/builder.go | 2 +- executor/analyze.go | 2 +- executor/checksum.go | 2 +- executor/infoschema_reader.go | 2 +- executor/set_test.go | 2 +- executor/show.go | 2 +- expression/aggregation/descriptor.go | 2 +- expression/helper.go | 2 +- planner/core/expression_rewriter.go | 36 ++++++++------ sessionctx/variable/noop.go | 2 +- sessionctx/variable/session.go | 31 +----------- sessionctx/variable/sysvar.go | 23 +++++++-- sessionctx/variable/sysvar_test.go | 54 ++++++++++++--------- sessionctx/variable/varsutil.go | 29 +++-------- sessionctx/variable/varsutil_test.go | 72 ++++++++++++++-------------- 16 files changed, 127 insertions(+), 140 deletions(-) diff --git a/ddl/util/util.go b/ddl/util/util.go index 0c3ec2608b9eb..0e5eb8fe2051d 100644 --- a/ddl/util/util.go +++ b/ddl/util/util.go @@ -186,7 +186,9 @@ func LoadGlobalVars(ctx sessionctx.Context, varNames []string) error { for _, row := range rows { varName := row.GetString(0) varValue := row.GetString(1) - variable.SetLocalSystemVar(varName, varValue) + if err = ctx.GetSessionVars().SetSystemVar(varName, varValue); err != nil { + return err + } } } return nil diff --git a/executor/aggfuncs/builder.go b/executor/aggfuncs/builder.go index 666743e4c137c..c914ea4838f2d 100644 --- a/executor/aggfuncs/builder.go +++ b/executor/aggfuncs/builder.go @@ -479,7 +479,7 @@ func buildGroupConcat(ctx sessionctx.Context, aggFuncDesc *aggregation.AggFuncDe panic(fmt.Sprintf("Error happened when buildGroupConcat: %s", err.Error())) } var s string - s, err = variable.GetSessionSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen) + s, err = variable.GetSessionOrGlobalSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen) if err != nil { panic(fmt.Sprintf("Error happened when buildGroupConcat: no system variable named '%s'", variable.GroupConcatMaxLen)) } diff --git a/executor/analyze.go b/executor/analyze.go index 8345382939865..9cf9c75b1261c 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -211,7 +211,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) { sessionVars := ctx.GetSessionVars() - concurrency, err := variable.GetSessionSystemVar(sessionVars, variable.TiDBBuildStatsConcurrency) + concurrency, err := variable.GetSessionOrGlobalSystemVar(sessionVars, variable.TiDBBuildStatsConcurrency) if err != nil { return 0, err } diff --git a/executor/checksum.go b/executor/checksum.go index 419c76042ad15..63f622d2f8140 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -269,7 +269,7 @@ func (c *checksumContext) HandleResponse(update *tipb.ChecksumResponse) { func getChecksumTableConcurrency(ctx sessionctx.Context) (int, error) { sessionVars := ctx.GetSessionVars() - concurrency, err := variable.GetSessionSystemVar(sessionVars, variable.TiDBChecksumTableConcurrency) + concurrency, err := variable.GetSessionOrGlobalSystemVar(sessionVars, variable.TiDBChecksumTableConcurrency) if err != nil { return 0, err } diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index a39992799b27a..0ec0c48885ecf 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1675,7 +1675,7 @@ func (e *memtableRetriever) setDataFromSessionVar(ctx sessionctx.Context) error sessionVars := ctx.GetSessionVars() for _, v := range variable.GetSysVars() { var value string - value, err = variable.GetSessionSystemVar(sessionVars, v.Name) + value, err = variable.GetSessionOrGlobalSystemVar(sessionVars, v.Name) if err != nil { return err } diff --git a/executor/set_test.go b/executor/set_test.go index ec97914cc12a5..4f24ca83519de 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -570,7 +570,7 @@ func (s *testSuite5) TestSetCharset(c *C) { check := func(args ...string) { for i, v := range characterSetVariables { - sVar, err := variable.GetSessionSystemVar(sessionVars, v) + sVar, err := variable.GetSessionOrGlobalSystemVar(sessionVars, v) c.Assert(err, IsNil) c.Assert(sVar, Equals, args[i], Commentf("%d: %s", i, characterSetVariables[i])) } diff --git a/executor/show.go b/executor/show.go index 725c0e0e28ec2..c5df3bb290268 100644 --- a/executor/show.go +++ b/executor/show.go @@ -693,7 +693,7 @@ func (e *ShowExec) fetchShowVariables() (err error) { if v.Hidden { continue } - value, err = variable.GetSessionSystemVar(sessionVars, v.Name) + value, err = variable.GetSessionOrGlobalSystemVar(sessionVars, v.Name) if err != nil { return errors.Trace(err) } diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index af16d26b1f81a..4415b0688ce09 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -210,7 +210,7 @@ func (a *AggFuncDesc) GetAggFunc(ctx sessionctx.Context) Aggregation { var s string var err error var maxLen uint64 - s, err = variable.GetSessionSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen) + s, err = variable.GetSessionOrGlobalSystemVar(ctx.GetSessionVars(), variable.GroupConcatMaxLen) if err != nil { panic(fmt.Sprintf("Error happened when GetAggFunc: no system variable named '%s'", variable.GroupConcatMaxLen)) } diff --git a/expression/helper.go b/expression/helper.go index 1e89c86e705f2..c5f91dbd090b5 100644 --- a/expression/helper.go +++ b/expression/helper.go @@ -139,7 +139,7 @@ func getStmtTimestamp(ctx sessionctx.Context) (time.Time, error) { } sessionVars := ctx.GetSessionVars() - timestampStr, err := variable.GetSessionSystemVar(sessionVars, "timestamp") + timestampStr, err := variable.GetSessionOrGlobalSystemVar(sessionVars, "timestamp") if err != nil { return now, err } diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 25ba4a21460fe..8b52318a260a1 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1215,25 +1215,33 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { er.ctxStackAppend(f, types.EmptyName) return } - var val string - var err error - if v.ExplicitScope { - err = variable.ValidateGetSystemVar(name, v.IsGlobal) - if err != nil { - er.err = err - return - } - } sysVar := variable.GetSysVar(name) if sysVar == nil { er.err = variable.ErrUnknownSystemVar.GenWithStackByArgs(name) return } - // Variable is @@gobal.variable_name or variable is only global scope variable. - if v.IsGlobal || sysVar.Scope == variable.ScopeGlobal { + if v.ExplicitScope && !sysVar.HasNoneScope() { + if v.IsGlobal && !sysVar.HasGlobalScope() { + er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "GLOBAL") + return + } + if !v.IsGlobal && !sysVar.HasSessionScope() { + er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "SESSION") + return + } + } + var val string + var err error + if sysVar.HasNoneScope() { + val = sysVar.Value + } else if v.IsGlobal || !sysVar.HasSessionScope() { + // The condition "|| !sysVar.HasSessionScope()" is a workaround + // for issue https://github.com/pingcap/tidb/issues/24368 + // Where global values are cached incorrectly. When this issue closes, + // the if statement here can be simplified. val, err = variable.GetGlobalSystemVar(sessionVars, name) } else { - val, err = variable.GetSessionSystemVar(sessionVars, name) + val, err = variable.GetSessionOrGlobalSystemVar(sessionVars, name) } if err != nil { er.err = err @@ -1241,8 +1249,8 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { } nativeVal, nativeType, nativeFlag := sysVar.GetNativeValType(val) e := expression.DatumToConstant(nativeVal, nativeType, nativeFlag) - e.GetType().Charset, _ = er.sctx.GetSessionVars().GetSystemVar(variable.CharacterSetConnection) - e.GetType().Collate, _ = er.sctx.GetSessionVars().GetSystemVar(variable.CollationConnection) + e.GetType().Charset, _ = sessionVars.GetSystemVar(variable.CharacterSetConnection) + e.GetType().Collate, _ = sessionVars.GetSystemVar(variable.CollationConnection) er.ctxStackAppend(e, types.EmptyName) } diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 05553c6f7eec1..c510d9c73ce3a 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -38,7 +38,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: SuperReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, - {Scope: ScopeGlobal, Name: serverReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal, Name: ReadOnly, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkReadOnly(vars, normalizedValue, originalValue, scope, false) }}, {Scope: ScopeGlobal, Name: ConnectTimeout, Value: "10", Type: TypeUnsigned, MinValue: 2, MaxValue: secondsPerYear, AutoConvertOutOfRange: true}, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 7db9de383ba55..618120b5da6e6 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1445,20 +1445,6 @@ func (s *SessionVars) LazyCheckKeyNotExists() bool { return s.PresumeKeyNotExists || (s.TxnCtx.IsPessimistic && !s.StmtCtx.DupKeyAsWarning) } -// SetLocalSystemVar sets values of the local variables which in "server" scope. -func SetLocalSystemVar(name string, val string) { - switch name { - case TiDBDDLReorgWorkerCount: - SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount))) - case TiDBDDLReorgBatchSize: - SetDDLReorgBatchSize(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgBatchSize))) - case TiDBDDLErrorCountLimit: - SetDDLErrorCountLimit(tidbOptInt64(val, DefTiDBDDLErrorCountLimit)) - case TiDBRowFormatVersion: - SetDDLReorgRowFormat(tidbOptInt64(val, DefTiDBRowFormatV2)) - } -} - // special session variables. const ( SQLModeVar = "sql_mode" @@ -1469,22 +1455,7 @@ const ( TransactionIsolation = "transaction_isolation" TxnIsolationOneShot = "tx_isolation_one_shot" MaxExecutionTime = "max_execution_time" -) - -// these variables are useless for TiDB, but still need to validate their values for some compatible issues. -// TODO: some more variables need to be added here. -const ( - serverReadOnly = "read_only" -) - -var ( - // TxIsolationNames are the valid values of the variable "tx_isolation" or "transaction_isolation". - TxIsolationNames = map[string]struct{}{ - "READ-UNCOMMITTED": {}, - "READ-COMMITTED": {}, - "REPEATABLE-READ": {}, - "SERIALIZABLE": {}, - } + ReadOnly = "read_only" ) // TableDelta stands for the changed count for one table or partition. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index c17238c9ae9c5..9a01c19470722 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -171,6 +171,11 @@ func (sv *SysVar) SetGlobalFromHook(s *SessionVars, val string, skipAliases bool return nil } +// HasNoneScope returns true if the scope for the sysVar is None. +func (sv *SysVar) HasNoneScope() bool { + return sv.Scope == ScopeNone +} + // HasSessionScope returns true if the scope for the sysVar includes session. func (sv *SysVar) HasSessionScope() bool { return sv.Scope&ScopeSession != 0 @@ -1060,6 +1065,7 @@ var defaultSysVars = []*SysVar{ } else if formatVersion == DefTiDBRowFormatV2 { s.RowEncoder.Enable = true } + SetDDLReorgRowFormat(tidbOptInt64(val, DefTiDBRowFormatV2)) return nil }}, {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { @@ -1138,9 +1144,18 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeSession, Name: TiDBConfig, Value: "", ReadOnly: true}, - {Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: uint64(maxDDLReorgWorkerCount)}, - {Scope: ScopeGlobal, Name: TiDBDDLReorgBatchSize, Value: strconv.Itoa(DefTiDBDDLReorgBatchSize), Type: TypeUnsigned, MinValue: int64(MinDDLReorgBatchSize), MaxValue: uint64(MaxDDLReorgBatchSize), AutoConvertOutOfRange: true}, - {Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: uint64(maxDDLReorgWorkerCount), SetSession: func(s *SessionVars, val string) error { + SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount))) + return nil + }}, + {Scope: ScopeGlobal, Name: TiDBDDLReorgBatchSize, Value: strconv.Itoa(DefTiDBDDLReorgBatchSize), Type: TypeUnsigned, MinValue: int64(MinDDLReorgBatchSize), MaxValue: uint64(MaxDDLReorgBatchSize), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { + SetDDLReorgBatchSize(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgBatchSize))) + return nil + }}, + {Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { + SetDDLErrorCountLimit(tidbOptInt64(val, DefTiDBDDLErrorCountLimit)) + return nil + }}, {Scope: ScopeSession, Name: TiDBDDLReorgPriority, Value: "PRIORITY_LOW", SetSession: func(s *SessionVars, val string) error { s.setDDLReorgPriority(val) return nil @@ -1210,7 +1225,7 @@ var defaultSysVars = []*SysVar{ // To prevent this strange position, prevent setting to OFF when any of these sysVars are ON of the same scope. if normalizedValue == Off { - for _, potentialIncompatibleSysVar := range []string{TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, serverReadOnly} { + for _, potentialIncompatibleSysVar := range []string{TxReadOnly, TransactionReadOnly, OfflineMode, SuperReadOnly, ReadOnly} { val, _ := vars.GetSystemVar(potentialIncompatibleSysVar) // session scope if scope == ScopeGlobal { // global scope var err error diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index cc765854a5c0b..71979a57b7eef 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -223,14 +223,22 @@ func (*testSysVarSuite) TestScope(c *C) { sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} c.Assert(sv.HasSessionScope(), IsTrue) c.Assert(sv.HasGlobalScope(), IsTrue) + c.Assert(sv.HasNoneScope(), IsFalse) sv = SysVar{Scope: ScopeGlobal, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} c.Assert(sv.HasSessionScope(), IsFalse) c.Assert(sv.HasGlobalScope(), IsTrue) + c.Assert(sv.HasNoneScope(), IsFalse) + + sv = SysVar{Scope: ScopeSession, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} + c.Assert(sv.HasSessionScope(), IsTrue) + c.Assert(sv.HasGlobalScope(), IsFalse) + c.Assert(sv.HasNoneScope(), IsFalse) sv = SysVar{Scope: ScopeNone, Name: "mynewsysvar", Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "AUTO"}} c.Assert(sv.HasSessionScope(), IsFalse) c.Assert(sv.HasGlobalScope(), IsFalse) + c.Assert(sv.HasNoneScope(), IsTrue) } func (*testSysVarSuite) TestBuiltInCase(c *C) { @@ -444,31 +452,31 @@ func (*testSysVarSuite) TestGetScopeNoneSystemVar(c *C) { } func (*testSysVarSuite) TestInstanceScopedVars(c *C) { - // This tests instance scoped variables through GetSessionSystemVar(). + // This tests instance scoped variables through GetSessionOrGlobalSystemVar(). // Eventually these should be changed to use getters so that the switch // statement in GetSessionOnlySysVars can be removed. vars := NewSessionVars() - val, err := GetSessionSystemVar(vars, TiDBCurrentTS) + val, err := GetSessionOrGlobalSystemVar(vars, TiDBCurrentTS) c.Assert(err, IsNil) c.Assert(val, Equals, fmt.Sprintf("%d", vars.TxnCtx.StartTS)) - val, err = GetSessionSystemVar(vars, TiDBLastTxnInfo) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBLastTxnInfo) c.Assert(err, IsNil) c.Assert(val, Equals, vars.LastTxnInfo) - val, err = GetSessionSystemVar(vars, TiDBLastQueryInfo) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBLastQueryInfo) c.Assert(err, IsNil) info, err := json.Marshal(vars.LastQueryInfo) c.Assert(err, IsNil) c.Assert(val, Equals, string(info)) - val, err = GetSessionSystemVar(vars, TiDBGeneralLog) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBGeneralLog) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(ProcessGeneralLog.Load())) - val, err = GetSessionSystemVar(vars, TiDBPProfSQLCPU) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBPProfSQLCPU) c.Assert(err, IsNil) expected := "0" if EnablePProfSQLCPU.Load() { @@ -476,74 +484,74 @@ func (*testSysVarSuite) TestInstanceScopedVars(c *C) { } c.Assert(val, Equals, expected) - val, err = GetSessionSystemVar(vars, TiDBExpensiveQueryTimeThreshold) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBExpensiveQueryTimeThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveQueryTimeThreshold))) - val, err = GetSessionSystemVar(vars, TiDBMemoryUsageAlarmRatio) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBMemoryUsageAlarmRatio) c.Assert(err, IsNil) c.Assert(val, Equals, fmt.Sprintf("%g", MemoryUsageAlarmRatio.Load())) - val, err = GetSessionSystemVar(vars, TiDBConfig) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBConfig) c.Assert(err, IsNil) conf := config.GetGlobalConfig() j, err := json.MarshalIndent(conf, "", "\t") c.Assert(err, IsNil) c.Assert(val, Equals, config.HideConfig(string(j))) - val, err = GetSessionSystemVar(vars, TiDBForcePriority) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBForcePriority) c.Assert(err, IsNil) c.Assert(val, Equals, mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))]) - val, err = GetSessionSystemVar(vars, TiDBDDLSlowOprThreshold) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBDDLSlowOprThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, strconv.FormatUint(uint64(atomic.LoadUint32(&DDLSlowOprThreshold)), 10)) - val, err = GetSessionSystemVar(vars, PluginDir) + val, err = GetSessionOrGlobalSystemVar(vars, PluginDir) c.Assert(err, IsNil) c.Assert(val, Equals, config.GetGlobalConfig().Plugin.Dir) - val, err = GetSessionSystemVar(vars, PluginLoad) + val, err = GetSessionOrGlobalSystemVar(vars, PluginLoad) c.Assert(err, IsNil) c.Assert(val, Equals, config.GetGlobalConfig().Plugin.Load) - val, err = GetSessionSystemVar(vars, TiDBSlowLogThreshold) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBSlowLogThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10)) - val, err = GetSessionSystemVar(vars, TiDBRecordPlanInSlowLog) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBRecordPlanInSlowLog) c.Assert(err, IsNil) c.Assert(val, Equals, strconv.FormatUint(uint64(atomic.LoadUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog)), 10)) - val, err = GetSessionSystemVar(vars, TiDBEnableSlowLog) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBEnableSlowLog) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().Log.EnableSlowLog)) - val, err = GetSessionSystemVar(vars, TiDBQueryLogMaxLen) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBQueryLogMaxLen) c.Assert(err, IsNil) c.Assert(val, Equals, strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen), 10)) - val, err = GetSessionSystemVar(vars, TiDBCheckMb4ValueInUTF8) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBCheckMb4ValueInUTF8) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8)) - val, err = GetSessionSystemVar(vars, TiDBCapturePlanBaseline) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBCapturePlanBaseline) c.Assert(err, IsNil) c.Assert(val, Equals, CapturePlanBaseline.GetVal()) - val, err = GetSessionSystemVar(vars, TiDBFoundInPlanCache) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBFoundInPlanCache) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(vars.PrevFoundInPlanCache)) - val, err = GetSessionSystemVar(vars, TiDBFoundInBinding) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBFoundInBinding) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(vars.PrevFoundInBinding)) - val, err = GetSessionSystemVar(vars, TiDBEnableCollectExecutionInfo) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBEnableCollectExecutionInfo) c.Assert(err, IsNil) c.Assert(val, Equals, BoolToOnOff(config.GetGlobalConfig().EnableCollectExecutionInfo)) - val, err = GetSessionSystemVar(vars, TiDBTxnScope) + val, err = GetSessionOrGlobalSystemVar(vars, TiDBTxnScope) c.Assert(err, IsNil) c.Assert(val, Equals, vars.TxnScope.GetVarValue()) } diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index d43a45b1cdbc7..809420ca8ee6c 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -161,10 +161,9 @@ func checkIsolationLevel(vars *SessionVars, normalizedValue string, originalValu return normalizedValue, nil } -// GetSessionSystemVar gets a system variable. -// If it is a session only variable, use the default value defined in code. -// Returns error if there is no such variable. -func GetSessionSystemVar(s *SessionVars, key string) (string, error) { +// GetSessionOrGlobalSystemVar gets a system variable of session or global scope. +// It also respects TIDB's special "instance" scope in GetSessionOnlySysVars. +func GetSessionOrGlobalSystemVar(s *SessionVars, key string) (string, error) { key = strings.ToLower(key) gVal, ok, err := GetSessionOnlySysVars(s, key) if err != nil || ok { @@ -174,6 +173,9 @@ func GetSessionSystemVar(s *SessionVars, key string) (string, error) { if err != nil { return "", err } + // This cache results in incorrect behavior since changes to global + // variables will not be picked up. It should be removed once + // https://github.com/pingcap/tidb/issues/24368 is closed. s.systems[key] = gVal return gVal, nil } @@ -313,25 +315,6 @@ func SetStmtVar(vars *SessionVars, name string, value string) error { return vars.SetStmtVar(name, sVal) } -// ValidateGetSystemVar checks if system variable exists and validates its scope when get system variable. -func ValidateGetSystemVar(name string, isGlobal bool) error { - sysVar := GetSysVar(name) - if sysVar == nil { - return ErrUnknownSystemVar.GenWithStackByArgs(name) - } - switch sysVar.Scope { - case ScopeGlobal: - if !isGlobal { - return ErrIncorrectScope.GenWithStackByArgs(name, "GLOBAL") - } - case ScopeSession: - if isGlobal { - return ErrIncorrectScope.GenWithStackByArgs(name, "SESSION") - } - } - return nil -} - const ( // initChunkSizeUpperBound indicates upper bound value of tidb_init_chunk_size. initChunkSizeUpperBound = 32 diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 5757eaeeac403..a96897140b9b4 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -124,7 +124,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err := SetSessionSystemVar(v, "autocommit", "1") c.Assert(err, IsNil) - val, err := GetSessionSystemVar(v, "autocommit") + val, err := GetSessionOrGlobalSystemVar(v, "autocommit") c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(SetSessionSystemVar(v, "autocommit", ""), NotNil) @@ -132,20 +132,20 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { // 0 converts to OFF err = SetSessionSystemVar(v, "foreign_key_checks", "0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, "foreign_key_checks") + val, err = GetSessionOrGlobalSystemVar(v, "foreign_key_checks") c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") // 1/ON is not supported (generates a warning and sets to OFF) err = SetSessionSystemVar(v, "foreign_key_checks", "1") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, "foreign_key_checks") + val, err = GetSessionOrGlobalSystemVar(v, "foreign_key_checks") c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") err = SetSessionSystemVar(v, "sql_mode", "strict_trans_tables") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, "sql_mode") + val, err = GetSessionOrGlobalSystemVar(v, "sql_mode") c.Assert(err, IsNil) c.Assert(val, Equals, "STRICT_TRANS_TABLES") c.Assert(v.StrictSQLMode, IsTrue) @@ -253,7 +253,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { // Test case for TiDBConfig session variable. err = SetSessionSystemVar(v, TiDBConfig, "abc") c.Assert(terror.ErrorEqual(err, ErrIncorrectScope), IsTrue) - val, err = GetSessionSystemVar(v, TiDBConfig) + val, err = GetSessionOrGlobalSystemVar(v, TiDBConfig) c.Assert(err, IsNil) bVal, err := json.MarshalIndent(config.GetGlobalConfig(), "", "\t") c.Assert(err, IsNil) @@ -261,13 +261,13 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, TiDBEnableStreaming, "1") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableStreaming) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableStreaming) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(v.EnableStreaming, Equals, true) err = SetSessionSystemVar(v, TiDBEnableStreaming, "0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableStreaming) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableStreaming) c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") c.Assert(v.EnableStreaming, Equals, false) @@ -282,7 +282,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, TiDBRetryLimit, "3") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBRetryLimit) + val, err = GetSessionOrGlobalSystemVar(v, TiDBRetryLimit) c.Assert(err, IsNil) c.Assert(val, Equals, "3") c.Assert(v.RetryLimit, Equals, int64(3)) @@ -290,7 +290,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.EnableTablePartition, Equals, "") err = SetSessionSystemVar(v, TiDBEnableTablePartition, "on") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableTablePartition) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableTablePartition) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(v.EnableTablePartition, Equals, "ON") @@ -298,7 +298,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.EnableListTablePartition, Equals, false) err = SetSessionSystemVar(v, TiDBEnableListTablePartition, "on") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableListTablePartition) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableListTablePartition) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(v.EnableListTablePartition, Equals, true) @@ -306,33 +306,33 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) err = SetSessionSystemVar(v, TiDBOptJoinReorderThreshold, "5") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptJoinReorderThreshold) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptJoinReorderThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, "5") c.Assert(v.TiDBOptJoinReorderThreshold, Equals, 5) err = SetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8, "1") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8) + val, err = GetSessionOrGlobalSystemVar(v, TiDBCheckMb4ValueInUTF8) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, true) err = SetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8, "0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8) + val, err = GetSessionOrGlobalSystemVar(v, TiDBCheckMb4ValueInUTF8) c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, false) err = SetSessionSystemVar(v, TiDBLowResolutionTSO, "1") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBLowResolutionTSO) + val, err = GetSessionOrGlobalSystemVar(v, TiDBLowResolutionTSO) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(v.LowResolutionTSO, Equals, true) err = SetSessionSystemVar(v, TiDBLowResolutionTSO, "0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBLowResolutionTSO) + val, err = GetSessionOrGlobalSystemVar(v, TiDBLowResolutionTSO) c.Assert(err, IsNil) c.Assert(val, Equals, "OFF") c.Assert(v.LowResolutionTSO, Equals, false) @@ -340,7 +340,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.CorrelationThreshold, Equals, 0.9) err = SetSessionSystemVar(v, TiDBOptCorrelationThreshold, "0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptCorrelationThreshold) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptCorrelationThreshold) c.Assert(err, IsNil) c.Assert(val, Equals, "0") c.Assert(v.CorrelationThreshold, Equals, float64(0)) @@ -348,7 +348,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.CPUFactor, Equals, 3.0) err = SetSessionSystemVar(v, TiDBOptCPUFactor, "5.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptCPUFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptCPUFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") c.Assert(v.CPUFactor, Equals, 5.0) @@ -356,7 +356,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.CopCPUFactor, Equals, 3.0) err = SetSessionSystemVar(v, TiDBOptCopCPUFactor, "5.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptCopCPUFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptCopCPUFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") c.Assert(v.CopCPUFactor, Equals, 5.0) @@ -364,7 +364,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.CopTiFlashConcurrencyFactor, Equals, 24.0) err = SetSessionSystemVar(v, TiDBOptTiFlashConcurrencyFactor, "5.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptTiFlashConcurrencyFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptTiFlashConcurrencyFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") c.Assert(v.CopCPUFactor, Equals, 5.0) @@ -372,7 +372,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.NetworkFactor, Equals, 1.0) err = SetSessionSystemVar(v, TiDBOptNetworkFactor, "3.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptNetworkFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptNetworkFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "3.0") c.Assert(v.NetworkFactor, Equals, 3.0) @@ -380,7 +380,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.ScanFactor, Equals, 1.5) err = SetSessionSystemVar(v, TiDBOptScanFactor, "3.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptScanFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptScanFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "3.0") c.Assert(v.ScanFactor, Equals, 3.0) @@ -388,7 +388,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.DescScanFactor, Equals, 3.0) err = SetSessionSystemVar(v, TiDBOptDescScanFactor, "5.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptDescScanFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptDescScanFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") c.Assert(v.DescScanFactor, Equals, 5.0) @@ -396,7 +396,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.SeekFactor, Equals, 20.0) err = SetSessionSystemVar(v, TiDBOptSeekFactor, "50.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptSeekFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptSeekFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "50.0") c.Assert(v.SeekFactor, Equals, 50.0) @@ -404,7 +404,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.MemoryFactor, Equals, 0.001) err = SetSessionSystemVar(v, TiDBOptMemoryFactor, "1.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptMemoryFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptMemoryFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "1.0") c.Assert(v.MemoryFactor, Equals, 1.0) @@ -412,7 +412,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.DiskFactor, Equals, 1.5) err = SetSessionSystemVar(v, TiDBOptDiskFactor, "1.1") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptDiskFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptDiskFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "1.1") c.Assert(v.DiskFactor, Equals, 1.1) @@ -420,57 +420,57 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(v.ConcurrencyFactor, Equals, 3.0) err = SetSessionSystemVar(v, TiDBOptConcurrencyFactor, "5.0") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBOptConcurrencyFactor) + val, err = GetSessionOrGlobalSystemVar(v, TiDBOptConcurrencyFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") c.Assert(v.ConcurrencyFactor, Equals, 5.0) err = SetSessionSystemVar(v, TiDBReplicaRead, "follower") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBReplicaRead) + val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "follower") c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) err = SetSessionSystemVar(v, TiDBReplicaRead, "leader") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBReplicaRead) + val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "leader") c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) err = SetSessionSystemVar(v, TiDBReplicaRead, "leader-and-follower") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBReplicaRead) + val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "leader-and-follower") c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadMixed) err = SetSessionSystemVar(v, TiDBEnableStmtSummary, "ON") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableStmtSummary) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableStmtSummary) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") err = SetSessionSystemVar(v, TiDBRedactLog, "ON") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBRedactLog) + val, err = GetSessionOrGlobalSystemVar(v, TiDBRedactLog) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") err = SetSessionSystemVar(v, TiDBStmtSummaryRefreshInterval, "10") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBStmtSummaryRefreshInterval) + val, err = GetSessionOrGlobalSystemVar(v, TiDBStmtSummaryRefreshInterval) c.Assert(err, IsNil) c.Assert(val, Equals, "10") err = SetSessionSystemVar(v, TiDBStmtSummaryHistorySize, "10") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBStmtSummaryHistorySize) + val, err = GetSessionOrGlobalSystemVar(v, TiDBStmtSummaryHistorySize) c.Assert(err, IsNil) c.Assert(val, Equals, "10") err = SetSessionSystemVar(v, TiDBStmtSummaryMaxStmtCount, "10") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBStmtSummaryMaxStmtCount) + val, err = GetSessionOrGlobalSystemVar(v, TiDBStmtSummaryMaxStmtCount) c.Assert(err, IsNil) c.Assert(val, Equals, "10") err = SetSessionSystemVar(v, TiDBStmtSummaryMaxStmtCount, "a") @@ -478,7 +478,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, TiDBStmtSummaryMaxSQLLength, "10") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBStmtSummaryMaxSQLLength) + val, err = GetSessionOrGlobalSystemVar(v, TiDBStmtSummaryMaxSQLLength) c.Assert(err, IsNil) c.Assert(val, Equals, "10") err = SetSessionSystemVar(v, TiDBStmtSummaryMaxSQLLength, "a") @@ -492,7 +492,7 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, TiDBEnableChangeColumnType, "ON") c.Assert(err, IsNil) - val, err = GetSessionSystemVar(v, TiDBEnableChangeColumnType) + val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableChangeColumnType) c.Assert(err, IsNil) c.Assert(val, Equals, "ON") c.Assert(v.systems[TiDBEnableChangeColumnType], Equals, "ON") From 81ce996c413abf926375e99d1a8bd8ef7696aa84 Mon Sep 17 00:00:00 2001 From: mmyj Date: Tue, 11 May 2021 13:35:38 +0800 Subject: [PATCH 018/343] session: add a test case to cover batch point get for temporary table (#24519) --- session/session_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/session/session_test.go b/session/session_test.go index 84442a8a16956..9ed2f9759243b 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4296,6 +4296,8 @@ func (s *testSessionSuite3) TestGlobalTemporaryTable(c *C) { tk.MustQuery("select c from g_tmp where b = 3").Check(testkit.Rows("3")) // Cover point get. tk.MustQuery("select * from g_tmp where a = 3").Check(testkit.Rows("3 3 3")) + // Cover batch point get. + tk.MustQuery("select * from g_tmp where a in (2,3,4)").Check(testkit.Rows("3 3 3", "4 7 9")) tk.MustExec("commit") // The global temporary table data is discard after the transaction commit. From 5460b5c77f0fd13a56f6bee965508a42d9c36d31 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Tue, 11 May 2021 14:03:38 +0800 Subject: [PATCH 019/343] store, executor: rename resolveTS to safeTS to be consistent with TiKV (#24538) --- executor/stale_txn_test.go | 44 ++++++++++++++--------------- store/tikv/extract_start_ts_test.go | 8 +++--- store/tikv/kv.go | 39 ++++++++++++++----------- store/tikv/txn.go | 8 +++--- 4 files changed, 53 insertions(+), 46 deletions(-) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index c68b8a5bfa511..ce5202ae58a75 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -256,65 +256,65 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { defer tk.MustExec(`drop table if exists t`) tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`) testcases := []struct { - name string - sql string - injectResolveTS uint64 - useResolveTS bool + name string + sql string + injectSafeTS uint64 + useSafeTS bool }{ { - name: "max 20 seconds ago, resolveTS 10 secs ago", + name: "max 20 seconds ago, safeTS 10 secs ago", sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:20'`, - injectResolveTS: func() uint64 { + injectSafeTS: func() uint64 { phy := time.Now().Add(-10*time.Second).Unix() * 1000 return oracle.ComposeTS(phy, 0) }(), - useResolveTS: true, + useSafeTS: true, }, { - name: "max 10 seconds ago, resolveTS 20 secs ago", + name: "max 10 seconds ago, safeTS 20 secs ago", sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`, - injectResolveTS: func() uint64 { + injectSafeTS: func() uint64 { phy := time.Now().Add(-20*time.Second).Unix() * 1000 return oracle.ComposeTS(phy, 0) }(), - useResolveTS: false, + useSafeTS: false, }, { - name: "max 20 seconds ago, resolveTS 10 secs ago", + name: "max 20 seconds ago, safeTS 10 secs ago", sql: func() string { return fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MIN READ TIMESTAMP '%v'`, time.Now().Add(-20*time.Second).Format("2006-01-02 15:04:05")) }(), - injectResolveTS: func() uint64 { + injectSafeTS: func() uint64 { phy := time.Now().Add(-10*time.Second).Unix() * 1000 return oracle.ComposeTS(phy, 0) }(), - useResolveTS: true, + useSafeTS: true, }, { - name: "max 10 seconds ago, resolveTS 20 secs ago", + name: "max 10 seconds ago, safeTS 20 secs ago", sql: func() string { return fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MIN READ TIMESTAMP '%v'`, time.Now().Add(-10*time.Second).Format("2006-01-02 15:04:05")) }(), - injectResolveTS: func() uint64 { + injectSafeTS: func() uint64 { phy := time.Now().Add(-20*time.Second).Unix() * 1000 return oracle.ComposeTS(phy, 0) }(), - useResolveTS: false, + useSafeTS: false, }, } for _, testcase := range testcases { c.Log(testcase.name) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/injectResolveTS", - fmt.Sprintf("return(%v)", testcase.injectResolveTS)), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/injectSafeTS", + fmt.Sprintf("return(%v)", testcase.injectSafeTS)), IsNil) tk.MustExec(testcase.sql) - if testcase.useResolveTS { - c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.injectResolveTS) + if testcase.useSafeTS { + c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.injectSafeTS) } else { - c.Assert(oracle.CompareTS(tk.Se.GetSessionVars().TxnCtx.StartTS, testcase.injectResolveTS), Equals, 1) + c.Assert(oracle.CompareTS(tk.Se.GetSessionVars().TxnCtx.StartTS, testcase.injectSafeTS), Equals, 1) } tk.MustExec("commit") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/injectResolveTS") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/injectSafeTS") } } diff --git a/store/tikv/extract_start_ts_test.go b/store/tikv/extract_start_ts_test.go index 1422e387bfda5..a108a0f7e41cb 100644 --- a/store/tikv/extract_start_ts_test.go +++ b/store/tikv/extract_start_ts_test.go @@ -56,8 +56,8 @@ func (s *extractStartTsSuite) SetUpTest(c *C) { Value: "Some Random Label", }}, } - store.resolveTSMu.resolveTS[2] = 102 - store.resolveTSMu.resolveTS[3] = 101 + store.setSafeTS(2, 102) + store.setSafeTS(3, 101) s.store = store } @@ -105,8 +105,8 @@ func (s *extractStartTsSuite) TestExtractStartTs(c *C) { } func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { - s.store.resolveTSMu.resolveTS[2] = 0x8000000000000002 - s.store.resolveTSMu.resolveTS[3] = 0x8000000000000001 + s.store.setSafeTS(2, 0x8000000000000002) + s.store.setSafeTS(3, 0x8000000000000001) i := uint64(100) cases := []kv.TransactionOption{ diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 5ddca52726a04..a487b0024e3e9 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -82,10 +82,10 @@ type KVStore struct { spMutex sync.RWMutex // this is used to update safePoint and spTime closed chan struct{} // this is used to nofity when the store is closed - resolveTSMu struct { - sync.RWMutex - resolveTS map[uint64]uint64 // storeID -> resolveTS - } + // storeID -> safeTS, stored as map[uint64]uint64 + // safeTS here will be used during the Stale Read process, + // it indicates the safe timestamp point that can be used to read consistent but may not the latest data. + safeTSMap sync.Map replicaReadSeed uint32 // this is used to load balance followers / learners when replica read is enabled } @@ -142,7 +142,6 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client replicaReadSeed: rand.Uint32(), } store.lockResolver = newLockResolver(store) - store.resolveTSMu.resolveTS = make(map[uint64]uint64) go store.runSafePointChecker() go store.safeTSUpdater() @@ -337,20 +336,30 @@ func (s *KVStore) GetTiKVClient() (client Client) { return s.client } -func (s *KVStore) getMinResolveTSByStores(stores []*Store) uint64 { - failpoint.Inject("injectResolveTS", func(val failpoint.Value) { +func (s *KVStore) getSafeTS(storeID uint64) uint64 { + safeTS, ok := s.safeTSMap.Load(storeID) + if !ok { + return 0 + } + return safeTS.(uint64) +} + +func (s *KVStore) setSafeTS(storeID, safeTS uint64) { + s.safeTSMap.Store(storeID, safeTS) +} + +func (s *KVStore) getMinSafeTSByStores(stores []*Store) uint64 { + failpoint.Inject("injectSafeTS", func(val failpoint.Value) { injectTS := val.(int) failpoint.Return(uint64(injectTS)) }) minSafeTS := uint64(math.MaxUint64) - s.resolveTSMu.RLock() - defer s.resolveTSMu.RUnlock() // when there is no store, return 0 in order to let minStartTS become startTS directly if len(stores) < 1 { return 0 } for _, store := range stores { - safeTS := s.resolveTSMu.resolveTS[store.storeID] + safeTS := s.getSafeTS(store.storeID) if safeTS < minSafeTS { minSafeTS = safeTS } @@ -368,12 +377,12 @@ func (s *KVStore) safeTSUpdater() { case <-s.Closed(): return case <-t.C: - s.updateResolveTS(ctx) + s.updateSafeTS(ctx) } } } -func (s *KVStore) updateResolveTS(ctx context.Context) { +func (s *KVStore) updateSafeTS(ctx context.Context) { stores := s.regionCache.getStoresByType(tikvrpc.TiKV) tikvClient := s.GetTiKVClient() wg := &sync.WaitGroup{} @@ -389,13 +398,11 @@ func (s *KVStore) updateResolveTS(ctx context.Context) { EndKey: []byte(""), }}), ReadTimeoutShort) if err != nil { - logutil.BgLogger().Debug("update resolveTS failed", zap.Error(err), zap.Uint64("store-id", storeID)) + logutil.BgLogger().Debug("update safeTS failed", zap.Error(err), zap.Uint64("store-id", storeID)) return } safeTSResp := resp.Resp.(*kvrpcpb.StoreSafeTSResponse) - s.resolveTSMu.Lock() - s.resolveTSMu.resolveTS[storeID] = safeTSResp.GetSafeTs() - s.resolveTSMu.Unlock() + s.setSafeTS(storeID, safeTSResp.GetSafeTs()) }(ctx, wg, storeID, storeAddr) } wg.Wait() diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 0ae2df13c12e9..1d678d010957e 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -112,12 +112,12 @@ func extractStartTs(store *KVStore, options kv.TransactionOption) (uint64, error } else { stores = allStores } - resolveTS := store.getMinResolveTSByStores(stores) + safeTS := store.getMinSafeTSByStores(stores) startTs = *options.MinStartTS - // If the resolveTS is larger than the minStartTS, we will use resolveTS as StartTS, otherwise we will use + // If the safeTS is larger than the minStartTS, we will use safeTS as StartTS, otherwise we will use // minStartTS directly. - if oracle.CompareTS(startTs, resolveTS) < 0 { - startTs = resolveTS + if oracle.CompareTS(startTs, safeTS) < 0 { + startTs = safeTS } } else if options.MaxPrevSec != nil { bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) From 6e348f44d2b5dc68a9aa13fa72d8847218909337 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Tue, 11 May 2021 14:41:39 +0800 Subject: [PATCH 020/343] parser: update parser to fix Can't recognize numeric literals when set 'ANSI_QUOTES' sql_mode (#24522) --- expression/integration_test.go | 9 +++++++++ go.mod | 2 +- go.sum | 4 ++-- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index d4b6a031087e4..f15dc5822be15 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9156,6 +9156,15 @@ func (s *testIntegrationSerialSuite) TestIssue23805(c *C) { tk.MustExec("insert ignore into tbl_5 set col_28 = 'ZmZIdSnq' , col_25 = '18:50:52.00' on duplicate key update col_26 = 'y';\n") } +func (s *testIntegrationSuite) TestIssue24429(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("set @@sql_mode = ANSI_QUOTES;") + tk.MustExec("use test") + tk.MustExec("create table t (a int);") + tk.MustQuery(`select t."a"=10 from t;`).Check(testkit.Rows()) +} + func (s *testIntegrationSuite) TestVitessHash(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) diff --git a/go.mod b/go.mod index bf927f9cc55ce..cd8becf5a757c 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 - github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde + github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 diff --git a/go.sum b/go.sum index a3ebad580db64..4e1030039a04f 100644 --- a/go.sum +++ b/go.sum @@ -443,8 +443,8 @@ github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 h1:ERrF0fTuIOnwfGbt71Ji3DKbOEaP189tjym50u8gpC8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde h1:CcGOCE3kr8aYBy6rRcWWldidL1X5smQxV79nlnzOk+o= -github.com/pingcap/parser v0.0.0-20210427084954-8e8ed7927bde/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= +github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c h1:GLFd+wBN7EsV6ad/tVGFCD37taOyzIMVs3SdiWZF18I= +github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 h1:A9KL9R+lWSVPH8IqUuH1QSTRJ5FGoY1bT2IcfPKsWD8= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= From c1a1550ba0c80726ec51b226a2f68e572a835901 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 14:59:38 +0800 Subject: [PATCH 021/343] store/tikv: make tikv.ErrUnknown as a normal error instead of terror (#24486) --- store/driver/txn/error.go | 6 ++++++ store/tikv/error/errcode.go | 1 - store/tikv/error/error.go | 3 ++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 5bbdc01024947..2a69a6567a489 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -60,6 +60,8 @@ var ( ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) // ErrRegionUnavailable is the error when region is not available. ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) + // ErrUnknown is the unknow error. + ErrUnknown = dbterror.ClassTiKV.NewStd(errno.ErrUnknown) ) // Registers error returned from TiKV. @@ -265,6 +267,10 @@ func ToTiDBErr(err error) error { return ErrTokenLimit.GenWithStackByArgs(e.StoreID) } + if errors.ErrorEqual(err, tikverr.ErrUnknown) { + return ErrUnknown + } + return errors.Trace(originErr) } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 01e8db4d12473..17ee9a3ccac2c 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -16,7 +16,6 @@ package error // MySQL error code. // This value is numeric. It is not portable to other database systems. const ( - CodeUnknown = 1105 CodeLockWaitTimeout = 1205 CodeQueryInterrupted = 1317 CodeLockAcquireFailAndNoWaitSet = 3572 diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index bde97e8b2e5d4..d83482c679449 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -52,6 +52,8 @@ var ( ErrTiFlashServerBusy = errors.New("tiflash server busy") // ErrRegionUnavailable is the error when region is not available. ErrRegionUnavailable = errors.New("region unavailable") + // ErrUnknown is the unknow error. + ErrUnknown = errors.New("unknow") ) // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. @@ -61,7 +63,6 @@ const MismatchClusterID = "mismatch cluster id" var ( ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) - ErrUnknown = dbterror.ClassTiKV.NewStd(CodeUnknown) ) // IsErrNotFound checks if err is a kind of NotFound error. From b639ff55fdb3e534ecaa0f52bb52e257eb4d934e Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 15:19:40 +0800 Subject: [PATCH 022/343] store/tikv: make tikv.ErrQueryInterrupted as a normal error (#24474) --- session/pessimistic_test.go | 2 +- store/copr/batch_coprocessor.go | 3 +-- store/copr/coprocessor.go | 3 +-- store/copr/mpp.go | 3 +-- store/driver/txn/error.go | 6 ++++++ store/tikv/error/errcode.go | 2 -- store/tikv/error/error.go | 3 ++- 7 files changed, 12 insertions(+), 10 deletions(-) diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 8fdd635b51bc1..fa79cfae8ff02 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -612,7 +612,7 @@ func (s *testPessimisticSuite) TestWaitLockKill(c *C) { _, err := tk2.Exec("update test_kill set c = c + 1 where id = 1") wg.Done() c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, tikverr.ErrQueryInterrupted), IsTrue) + c.Assert(terror.ErrorEqual(err, txndriver.ErrQueryInterrupted), IsTrue) tk.MustExec("rollback") } diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 3231f95bbc824..af224c59fc38a 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/kv" txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" - tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -262,7 +261,7 @@ func (b *batchCopIterator) recvFromRespCh(ctx context.Context) (resp *batchCopRe return case <-ticker.C: if atomic.LoadUint32(b.vars.Killed) == 1 { - resp = &batchCopResponse{err: tikverr.ErrQueryInterrupted} + resp = &batchCopResponse{err: txndriver.ErrQueryInterrupted} ok = true return } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index cb799edb16f70..5e7eab303e84f 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -37,7 +37,6 @@ import ( tidbmetrics "github.com/pingcap/tidb/metrics" txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" - tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -477,7 +476,7 @@ func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan *copRes return case <-ticker.C: if atomic.LoadUint32(it.vars.Killed) == 1 { - resp = &copResponse{err: tikverr.ErrQueryInterrupted} + resp = &copResponse{err: txndriver.ErrQueryInterrupted} ok = true return } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 3ea07e744f9b9..10784912faa9b 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/kv" txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" - tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/tikvrpc" "go.uber.org/zap" @@ -406,7 +405,7 @@ func (m *mppIterator) nextImpl(ctx context.Context) (resp *mppResponse, ok bool, return case <-ticker.C: if m.vars != nil && m.vars.Killed != nil && atomic.LoadUint32(m.vars.Killed) == 1 { - err = tikverr.ErrQueryInterrupted + err = txndriver.ErrQueryInterrupted exit = true return } diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 2a69a6567a489..456a9f118c9cd 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -47,6 +47,8 @@ var ( ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) // ErrTiKVStaleCommand is the error that the command is stale in tikv. ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) + // ErrQueryInterrupted is the error when the query is interrupted. + ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(errno.ErrQueryInterrupted) // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) @@ -231,6 +233,10 @@ func ToTiDBErr(err error) error { return ErrTiFlashServerTimeout } + if errors.ErrorEqual(err, tikverr.ErrQueryInterrupted) { + return ErrQueryInterrupted + } + if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { return ErrTiKVServerBusy } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go index 17ee9a3ccac2c..5455c75da2ed4 100644 --- a/store/tikv/error/errcode.go +++ b/store/tikv/error/errcode.go @@ -16,7 +16,5 @@ package error // MySQL error code. // This value is numeric. It is not portable to other database systems. const ( - CodeLockWaitTimeout = 1205 - CodeQueryInterrupted = 1317 CodeLockAcquireFailAndNoWaitSet = 3572 ) diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index d83482c679449..d8d5122fad774 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -38,6 +38,8 @@ var ( ErrTiKVServerTimeout = errors.New("tikv server timeout") // ErrTiFlashServerTimeout is the error when tiflash server is timeout. ErrTiFlashServerTimeout = errors.New("tiflash server timeout") + // ErrQueryInterrupted is the error when the query is interrupted. + ErrQueryInterrupted = errors.New("query interruppted") // ErrTiKVStaleCommand is the error that the command is stale in tikv. ErrTiKVStaleCommand = errors.New("tikv stale command") // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. @@ -61,7 +63,6 @@ const MismatchClusterID = "mismatch cluster id" // error instances. var ( - ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(CodeQueryInterrupted) ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) ) From 263a47e85ce04f74ec80d1d35b426618bc89b5a3 Mon Sep 17 00:00:00 2001 From: rebelice Date: Tue, 11 May 2021 15:51:39 +0800 Subject: [PATCH 023/343] executor: add a test for dynamic partition prune mode with equal expression (#24520) --- executor/partition_table_test.go | 66 ++++++++++++++++++++++++++++++++ util/testkit/testkit.go | 11 ++++++ 2 files changed, 77 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 8ce126972ec73..0c9e63129ebf1 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -434,6 +434,72 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { tk.MustIndexLookup("select * from tlist where a<1") } +func createTable4DynamicPruneModeTestWithExpression(tk *testkit.TestKit) { + tk.MustExec("create table trange(a int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11));") + tk.MustExec("create table thash(a int) partition by hash(a) partitions 4;") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into trange values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") + tk.MustExec("insert into thash values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") + tk.MustExec("insert into t values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") + tk.MustExec("set session tidb_partition_prune_mode='dynamic'") + tk.MustExec("analyze table trange") + tk.MustExec("analyze table thash") + tk.MustExec("analyze table t") +} + +type testData4Expression struct { + sql string + partitions []string +} + +func (s *partitionTableSuite) TestDynamicPruneModeWithEqualExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_equal_expression") + tk.MustExec("create database db_equal_expression") + tk.MustExec("use db_equal_expression") + createTable4DynamicPruneModeTestWithExpression(tk) + + tables := []string{"trange", "thash"} + tests := []testData4Expression{ + { + sql: "select * from %s where a = 2", + partitions: []string{ + "p0", + "p2", + }, + }, + { + sql: "select * from %s where a = 4 or a = 1", + partitions: []string{ + "p0,p1", + "p0,p1", + }, + }, + { + sql: "select * from %s where a = -1", + partitions: []string{ + "p0", + "p1", + }, + }, + { + sql: "select * from %s where a is NULL", + partitions: []string{ + "p0", + "p0", + }, + }, + } + + for _, t := range tests { + for i := range t.partitions { + sql := fmt.Sprintf(t.sql, tables[i]) + c.Assert(tk.MustPartition(sql, t.partitions[i]), IsTrue) + tk.MustQuery(sql).Sort().Check(tk.MustQuery(fmt.Sprintf(t.sql, "t")).Sort().Rows()) + } + } +} + func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 4992e28663b1a..06eb826c56b78 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -255,6 +255,17 @@ func (tk *TestKit) MustNoGlobalStats(table string) bool { return true } +// MustPartition checks if the result execution plan must read specific partitions. +func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) bool { + rs := tk.MustQuery("explain "+sql, args...) + for i := range rs.rows { + if strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 { + return true + } + } + return false +} + // MustUseIndex checks if the result execution plan contains specific index(es). func (tk *TestKit) MustUseIndex(sql string, index string, args ...interface{}) bool { rs := tk.MustQuery("explain "+sql, args...) From 7d41c868128d07bcbc2359533d39e7736b180620 Mon Sep 17 00:00:00 2001 From: Mingcong Han Date: Tue, 11 May 2021 16:21:39 +0800 Subject: [PATCH 024/343] planner: create new column slice in PreparePossibleProperties (#24342) --- planner/core/integration_test.go | 17 +++++++++++++++++ planner/core/property_cols_prune.go | 15 ++++++++------- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 4ae7342a4f7d7..7f58d2db6d466 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3597,6 +3597,23 @@ func (s *testIntegrationSuite) TestIssue24095(c *C) { } } +func (s *testIntegrationSuite) TestIssue24281(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists member, agent, deposit, view_member_agents") + tk.MustExec("create table member(login varchar(50) NOT NULL, agent_login varchar(100) DEFAULT NULL, PRIMARY KEY(login))") + tk.MustExec("create table agent(login varchar(50) NOT NULL, data varchar(100) DEFAULT NULL, share_login varchar(50) NOT NULL, PRIMARY KEY(login))") + tk.MustExec("create table deposit(id varchar(50) NOT NULL, member_login varchar(50) NOT NULL, transfer_amount int NOT NULL, PRIMARY KEY(id), KEY midx(member_login, transfer_amount))") + tk.MustExec("create definer='root'@'localhost' view view_member_agents (member, share_login) as select m.login as member, a.share_login AS share_login from member as m join agent as a on m.agent_login = a.login") + + tk.MustExec(" select s.member_login as v1, SUM(s.transfer_amount) AS v2 " + + "FROM deposit AS s " + + "JOIN view_member_agents AS v ON s.member_login = v.member " + + "WHERE 1 = 1 AND v.share_login = 'somevalue' " + + "GROUP BY s.member_login " + + "UNION select 1 as v1, 2 as v2") +} + func (s *testIntegrationSuite) TestConflictReadFromStorage(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/property_cols_prune.go b/planner/core/property_cols_prune.go index 91a9f34fb9017..9cd83adc412de 100644 --- a/planner/core/property_cols_prune.go +++ b/planner/core/property_cols_prune.go @@ -148,21 +148,22 @@ func (p *LogicalProjection) PreparePossibleProperties(schema *expression.Schema, } } tmpSchema := expression.NewSchema(oldCols...) - for i := len(childProperties) - 1; i >= 0; i-- { - for j, col := range childProperties[i] { + newProperties := make([][]*expression.Column, 0, len(childProperties)) + for _, childProperty := range childProperties { + newChildProperty := make([]*expression.Column, 0, len(childProperty)) + for _, col := range childProperty { pos := tmpSchema.ColumnIndex(col) if pos >= 0 { - childProperties[i][j] = newCols[pos] + newChildProperty = append(newChildProperty, newCols[pos]) } else { - childProperties[i] = childProperties[i][:j] break } } - if len(childProperties[i]) == 0 { - childProperties = append(childProperties[:i], childProperties[i+1:]...) + if len(newChildProperty) != 0 { + newProperties = append(newProperties, newChildProperty) } } - return childProperties + return newProperties } func clonePossibleProperties(props [][]*expression.Column) [][]*expression.Column { From 7611952a09f84637456e180274606d803ed69868 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 11 May 2021 17:11:39 +0800 Subject: [PATCH 025/343] store/tikv: make tikv.ErrLockAcquireFailAndNoWaitSet as a normal error (#24475) --- session/pessimistic_test.go | 21 ++++++++++----------- store/driver/txn/error.go | 8 +++++++- store/tikv/error/errcode.go | 20 -------------------- store/tikv/error/error.go | 8 ++------ 4 files changed, 19 insertions(+), 38 deletions(-) delete mode 100644 store/tikv/error/errcode.go diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index fa79cfae8ff02..83f0057384aea 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" txndriver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" - tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -1132,11 +1131,11 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { tk1.MustExec("begin pessimistic") err := tk1.ExecToErr("select * from t where k = 2 for update nowait") - c.Check(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 4 for update nowait") - c.Check(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 7 for update nowait") - c.Check(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk1.MustExec("rollback") @@ -1148,9 +1147,9 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { tk1.MustExec("begin pessimistic") err = tk1.ExecToErr("select * from t where k = 2 for update nowait") - c.Check(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 6 for update nowait") - c.Check(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk1.MustExec("rollback") } @@ -1283,7 +1282,7 @@ func (s *testPessimisticSuite) TestBatchPointGetLockIndex(c *C) { c.Assert(txndriver.ErrLockWaitTimeout.Equal(err), IsTrue) err = tk2.ExecToErr("select * from t1 where c2 = 3 for update nowait") c.Assert(err, NotNil) - c.Assert(tikverr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Assert(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk2.MustExec("rollback") } @@ -1430,12 +1429,12 @@ func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { tk2.MustExec("begin pessimistic") err := tk2.ExecToErr("select * from tu where z = 3 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("begin pessimistic") tk.MustExec("insert into tu(x, y) values(2, 2);") err = tk2.ExecToErr("select * from tu where z = 4 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) // test batch point get lock tk.MustExec("begin pessimistic") @@ -1444,12 +1443,12 @@ func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { tk2.MustExec("begin pessimistic") err = tk2.ExecToErr("select x from tu where z in (3, 7, 9) for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("begin pessimistic") tk.MustExec("insert into tu(x, y) values(5, 6);") err = tk2.ExecToErr("select * from tu where z = 11 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("commit") tk2.MustExec("commit") diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 456a9f118c9cd..4c8e770c44ff7 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -51,7 +51,9 @@ var ( ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(errno.ErrQueryInterrupted) // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) - ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) + // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. + ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(errno.ErrLockAcquireFailAndNoWaitSet) + ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) // ErrLockWaitTimeout is the error that wait for the lock is timeout. ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(errno.ErrLockWaitTimeout) // ErrTiKVServerBusy is the error when tikv server is busy. @@ -257,6 +259,10 @@ func ToTiDBErr(err error) error { return ErrTiKVMaxTimestampNotSynced } + if errors.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet) { + return ErrLockAcquireFailAndNoWaitSet + } + if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { return ErrResolveLockTimeout } diff --git a/store/tikv/error/errcode.go b/store/tikv/error/errcode.go deleted file mode 100644 index 5455c75da2ed4..0000000000000 --- a/store/tikv/error/errcode.go +++ /dev/null @@ -1,20 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package error - -// MySQL error code. -// This value is numeric. It is not portable to other database systems. -const ( - CodeLockAcquireFailAndNoWaitSet = 3572 -) diff --git a/store/tikv/error/error.go b/store/tikv/error/error.go index d8d5122fad774..898354cc11a2d 100644 --- a/store/tikv/error/error.go +++ b/store/tikv/error/error.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/pingcap/tidb/util/dbterror" ) var ( @@ -44,6 +43,8 @@ var ( ErrTiKVStaleCommand = errors.New("tikv stale command") // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. ErrTiKVMaxTimestampNotSynced = errors.New("tikv max timestamp not synced") + // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. + ErrLockAcquireFailAndNoWaitSet = errors.New("lock acquired failed and no wait is setted") // ErrResolveLockTimeout is the error that resolve lock timeout. ErrResolveLockTimeout = errors.New("resolve lock timeout") // ErrLockWaitTimeout is the error that wait for the lock is timeout. @@ -61,11 +62,6 @@ var ( // MismatchClusterID represents the message that the cluster ID of the PD client does not match the PD. const MismatchClusterID = "mismatch cluster id" -// error instances. -var ( - ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(CodeLockAcquireFailAndNoWaitSet) -) - // IsErrNotFound checks if err is a kind of NotFound error. func IsErrNotFound(err error) bool { return errors.ErrorEqual(err, ErrNotExist) From c5e20dd1a2b5bf662cb42e240dac97da050c9729 Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Tue, 11 May 2021 19:21:40 +0800 Subject: [PATCH 026/343] variable: fix the TiDBDistSQLScanConcurrency comment (#24451) --- sessionctx/variable/tidb_vars.go | 1 - 1 file changed, 1 deletion(-) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 30d52ac54f386..54616f902db5e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -216,7 +216,6 @@ const ( // A distsql scan task can be a table scan or a index scan, which may be distributed to many TiKV nodes. // Higher concurrency may reduce latency, but with the cost of higher memory usage and system performance impact. // If the query has a LIMIT clause, high concurrency makes the system do much more work than needed. - // tidb_distsql_scan_concurrency is deprecated, use tidb_executor_concurrency instead. TiDBDistSQLScanConcurrency = "tidb_distsql_scan_concurrency" // tidb_opt_insubquery_to_join_and_agg is used to enable/disable the optimizer rule of rewriting IN subquery. From 3eedd404b852287a899e3952ed776a2db03d80af Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 11 May 2021 19:59:40 +0800 Subject: [PATCH 027/343] executor: add some test cases about partition table dynamic-mode with clustered-index (#24523) --- session/clustered_index_test.go | 49 +++++++++++++++++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/session/clustered_index_test.go b/session/clustered_index_test.go index fd40cfd567f11..0f79b1b13fc2e 100644 --- a/session/clustered_index_test.go +++ b/session/clustered_index_test.go @@ -14,11 +14,16 @@ package session_test import ( + "fmt" + "math/rand" + "strings" + . "github.com/pingcap/check" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" ) @@ -578,6 +583,50 @@ func (s *testClusteredSerialSuite) TestPrefixClusteredIndexAddIndexAndRecover(c tk1.MustExec("admin check table t") } +func (s *testClusteredSerialSuite) TestPartitionTable(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_view") + tk.MustExec("use test_view") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table thash (a int, b int, c varchar(32), primary key(a, b) clustered) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, c varchar(32), primary key(a, b) clustered) partition by range columns(a) ( + partition p0 values less than (3000), + partition p1 values less than (6000), + partition p2 values less than (9000), + partition p3 values less than (10000))`) + tk.MustExec(`create table tnormal (a int, b int, c varchar(32), primary key(a, b))`) + + vals := make([]string, 0, 4000) + existedPK := make(map[string]struct{}, 4000) + for i := 0; i < 4000; { + a := rand.Intn(10000) + b := rand.Intn(10000) + pk := fmt.Sprintf("%v, %v", a, b) + if _, ok := existedPK[pk]; ok { + continue + } + existedPK[pk] = struct{}{} + i++ + vals = append(vals, fmt.Sprintf(`(%v, %v, '%v')`, a, b, rand.Intn(10000))) + } + + tk.MustExec("insert into thash values " + strings.Join(vals, ", ")) + tk.MustExec("insert into trange values " + strings.Join(vals, ", ")) + tk.MustExec("insert into tnormal values " + strings.Join(vals, ", ")) + + for i := 0; i < 200; i++ { + cond := fmt.Sprintf("where a in (%v, %v, %v) and b < %v", rand.Intn(10000), rand.Intn(10000), rand.Intn(10000), rand.Intn(10000)) + result := tk.MustQuery("select * from tnormal " + cond).Sort().Rows() + tk.MustQuery("select * from thash use index(primary) " + cond).Sort().Check(result) + tk.MustQuery("select * from trange use index(primary) " + cond).Sort().Check(result) + } +} + // https://github.com/pingcap/tidb/issues/23106 func (s *testClusteredSerialSuite) TestClusteredIndexDecodeRestoredDataV5(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) From e40f8c0881aad4438b8f738ee35ff54dd64205c5 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 12 May 2021 16:40:58 +0800 Subject: [PATCH 028/343] *: fix revoke statement for CURRENT_USER() and refine error message (#24052) * *: fix revoke statement for CURRENT_USER() and refine error message --- errno/errname.go | 2 +- errors.toml | 2 +- executor/revoke.go | 6 ++++++ planner/core/optimizer.go | 4 ++-- planner/core/point_get_plan.go | 2 +- privilege/privileges/privileges_test.go | 16 +++++++++++----- session/session_test.go | 2 +- 7 files changed, 23 insertions(+), 11 deletions(-) diff --git a/errno/errname.go b/errno/errname.go index 5afdbbb91c4c0..62662ce5ac934 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1020,7 +1020,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrBuildExecutor: mysql.Message("Failed to build executor", nil), ErrBatchInsertFail: mysql.Message("Batch insert failed, please clean the table and try again.", nil), ErrGetStartTS: mysql.Message("Can not get start ts", nil), - ErrPrivilegeCheckFail: mysql.Message("privilege check fail", nil), // this error message should begin lowercased to be compatible with the test + ErrPrivilegeCheckFail: mysql.Message("privilege check for '%s' fail", nil), // this error message should begin lowercased to be compatible with the test 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), diff --git a/errors.toml b/errors.toml index 458af951629d8..0ce61654373fb 100644 --- a/errors.toml +++ b/errors.toml @@ -1133,7 +1133,7 @@ Schema has changed ["planner:8121"] error = ''' -privilege check fail +privilege check for '%s' fail ''' ["planner:8122"] diff --git a/executor/revoke.go b/executor/revoke.go index 1477534962fbe..b090f048c62a7 100644 --- a/executor/revoke.go +++ b/executor/revoke.go @@ -88,8 +88,14 @@ func (e *RevokeExec) Next(ctx context.Context, req *chunk.Chunk) error { return err } + sessVars := e.ctx.GetSessionVars() // Revoke for each user. for _, user := range e.Users { + if user.User.CurrentUser { + user.User.Username = sessVars.User.AuthUsername + user.User.Hostname = sessVars.User.AuthHostname + } + // Check if user exists. exists, err := userExists(e.ctx, user.User.Username, user.User.Hostname) if err != nil { diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index d79d83331723b..59c228767171a 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -103,13 +103,13 @@ func CheckPrivilege(activeRoles []*auth.RoleIdentity, pm privilege.Manager, vs [ if v.privilege == mysql.ExtendedPriv { if !pm.RequestDynamicVerification(activeRoles, v.dynamicPriv, v.dynamicWithGrant) { if v.err == nil { - return ErrPrivilegeCheckFail + return ErrPrivilegeCheckFail.GenWithStackByArgs(v.dynamicPriv) } return v.err } } else if !pm.RequestVerification(activeRoles, v.db, v.table, v.column, v.privilege) { if v.err == nil { - return ErrPrivilegeCheckFail + return ErrPrivilegeCheckFail.GenWithStackByArgs(v.privilege.String()) } return v.err } diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 77a853a4568b6..f7edbc1648819 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -994,7 +994,7 @@ func checkFastPlanPrivilege(ctx sessionctx.Context, dbName, tableName string, ch var visitInfos []visitInfo for _, checkType := range checkTypes { if pm != nil && !pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, dbName, tableName, "", checkType) { - return errors.New("privilege check fail") + return ErrPrivilegeCheckFail.GenWithStackByArgs(checkType.String()) } // This visitInfo is only for table lock check, so we do not need column field, // just fill it empty string. diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 2efb565b3ed2b..3038aad397076 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -843,6 +843,12 @@ func (s *testPrivilegeSuite) TestRevokePrivileges(c *C) { c.Assert(se.Auth(&auth.UserIdentity{Username: "hasgrant", Hostname: "localhost", AuthUsername: "hasgrant", AuthHostname: "%"}, nil, nil), IsTrue) mustExec(c, se, "REVOKE SELECT ON mysql.* FROM 'withoutgrant'") mustExec(c, se, "REVOKE ALL ON mysql.* FROM withoutgrant") + + // For issue https://github.com/pingcap/tidb/issues/23850 + mustExec(c, se, "CREATE USER u4") + mustExec(c, se, "GRANT ALL ON *.* TO u4 WITH GRANT OPTION") + c.Assert(se.Auth(&auth.UserIdentity{Username: "u4", Hostname: "localhost", AuthUsername: "u4", AuthHostname: "%"}, nil, nil), IsTrue) + mustExec(c, se, "REVOKE ALL ON *.* FROM CURRENT_USER()") } func (s *testPrivilegeSuite) TestSetGlobal(c *C) { @@ -1006,14 +1012,14 @@ func (s *testPrivilegeSuite) TestSystemSchema(c *C) { _, err = se.ExecuteInternal(context.Background(), "drop table information_schema.tables") c.Assert(strings.Contains(err.Error(), "denied to user"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "update information_schema.tables set table_name = 'tst' where table_name = 'mysql'") - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) // Test performance_schema. mustExec(c, se, `select * from performance_schema.events_statements_summary_by_digest`) _, err = se.ExecuteInternal(context.Background(), "drop table performance_schema.events_statements_summary_by_digest") c.Assert(strings.Contains(err.Error(), "denied to user"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "update performance_schema.events_statements_summary_by_digest set schema_name = 'tst'") - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "delete from performance_schema.events_statements_summary_by_digest") c.Assert(strings.Contains(err.Error(), "DELETE command denied to user"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "create table performance_schema.t(a int)") @@ -1025,7 +1031,7 @@ func (s *testPrivilegeSuite) TestSystemSchema(c *C) { _, err = se.ExecuteInternal(context.Background(), "drop table metrics_schema.tidb_query_duration") c.Assert(strings.Contains(err.Error(), "denied to user"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "update metrics_schema.tidb_query_duration set instance = 'tst'") - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "delete from metrics_schema.tidb_query_duration") c.Assert(strings.Contains(err.Error(), "DELETE command denied to user"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "create table metric_schema.t(a int)") @@ -1041,9 +1047,9 @@ func (s *testPrivilegeSuite) TestAdminCommand(c *C) { c.Assert(se.Auth(&auth.UserIdentity{Username: "test_admin", Hostname: "localhost"}, nil, nil), IsTrue) _, err := se.ExecuteInternal(context.Background(), "ADMIN SHOW DDL JOBS") - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) _, err = se.ExecuteInternal(context.Background(), "ADMIN CHECK TABLE t") - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil), IsTrue) _, err = se.ExecuteInternal(context.Background(), "ADMIN SHOW DDL JOBS") diff --git a/session/session_test.go b/session/session_test.go index 9ed2f9759243b..5fa7779fc65c3 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2892,7 +2892,7 @@ func (s *testSessionSuite2) TestUpdatePrivilege(c *C) { _, err := tk1.Exec("update t2 set id = 666 where id = 1;") c.Assert(err, NotNil) - c.Assert(strings.Contains(err.Error(), "privilege check fail"), IsTrue) + c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) // Cover a bug that t1 and t2 both require update privilege. // In fact, the privlege check for t1 should be update, and for t2 should be select. From ec618f1619bdf473a22861f6f87c486fd0baaa6c Mon Sep 17 00:00:00 2001 From: Zhi Qi <30543181+LittleFall@users.noreply.github.com> Date: Wed, 12 May 2021 18:05:40 +0800 Subject: [PATCH 029/343] planner: support set tidb_allow_mpp to `2` or `ENFORCE` to enforce use mpp mode. (#24516) --- executor/mpp_gather.go | 2 +- planner/core/exhaust_physical_plans.go | 6 +- planner/core/integration_test.go | 126 +++++++++++++++++++++++++ planner/core/task.go | 8 +- sessionctx/variable/session.go | 19 +++- sessionctx/variable/sysvar.go | 6 +- sessionctx/variable/tidb_vars.go | 4 +- 7 files changed, 157 insertions(+), 14 deletions(-) diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 64236558af94e..7cfeb613c40f6 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -30,7 +30,7 @@ import ( ) func useMPPExecution(ctx sessionctx.Context, tr *plannercore.PhysicalTableReader) bool { - if !ctx.GetSessionVars().AllowMPPExecution { + if !ctx.GetSessionVars().IsMPPAllowed() { return false } _, ok := tr.GetTablePlan().(*plannercore.PhysicalExchangeSender) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index d4f2923b6220a..cd227657a75d9 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1662,7 +1662,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } joins := make([]PhysicalPlan, 0, 8) canPushToTiFlash := p.canPushToCop(kv.TiFlash) - if p.ctx.GetSessionVars().AllowMPPExecution && canPushToTiFlash { + if p.ctx.GetSessionVars().IsMPPAllowed() && canPushToTiFlash { if p.shouldUseMPPBCJ() { mppJoins := p.tryToGetMppHashJoin(prop, true) if (p.preferJoinType & preferBCJoin) > 0 { @@ -1965,7 +1965,7 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl if !lt.limitHints.preferLimitToCop { allTaskTypes = append(allTaskTypes, property.RootTaskType) } - if lt.ctx.GetSessionVars().AllowMPPExecution { + if lt.ctx.GetSessionVars().IsMPPAllowed() { allTaskTypes = append(allTaskTypes, property.MppTaskType) } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) @@ -2355,7 +2355,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy taskTypes = append(taskTypes, property.CopTiFlashLocalReadTaskType) } canPushDownToTiFlash := la.canPushToCop(kv.TiFlash) - canPushDownToMPP := la.ctx.GetSessionVars().AllowMPPExecution && la.checkCanPushDownToMPP() && canPushDownToTiFlash + canPushDownToMPP := la.ctx.GetSessionVars().IsMPPAllowed() && la.checkCanPushDownToMPP() && canPushDownToTiFlash if la.HasDistinct() { // TODO: remove after the cost estimation of distinct pushdown is implemented. if !la.ctx.GetSessionVars().AllowDistinctAggPushDown { diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 7f58d2db6d466..98ab7b7898370 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3706,3 +3706,129 @@ func (s *testIntegrationSerialSuite) TestMergeContinuousSelections(c *C) { res.Check(testkit.Rows(output[i].Plan...)) } } + +func (s *testIntegrationSerialSuite) TestEnforceMPP(c *C) { + tk := testkit.NewTestKit(c, s.store) + + // test value limit of tidb_opt_tiflash_concurrency_factor + err := tk.ExecToErr("set @@tidb_opt_tiflash_concurrency_factor = 0") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, `[variable:1231]Variable 'tidb_opt_tiflash_concurrency_factor' can't be set to the value of '0'`) + + tk.MustExec("set @@tidb_opt_tiflash_concurrency_factor = 1") + tk.MustQuery("select @@tidb_opt_tiflash_concurrency_factor").Check(testkit.Rows("1")) + tk.MustExec("set @@tidb_opt_tiflash_concurrency_factor = 24") + tk.MustQuery("select @@tidb_opt_tiflash_concurrency_factor").Check(testkit.Rows("24")) + + // test set tidb_allow_mpp + tk.MustExec("set @@session.tidb_allow_mpp = 0") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("OFF")) + tk.MustExec("set @@session.tidb_allow_mpp = 1") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ON")) + tk.MustExec("set @@session.tidb_allow_mpp = 2") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ENFORCE")) + + tk.MustExec("set @@session.tidb_allow_mpp = off") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("OFF")) + tk.MustExec("set @@session.tidb_allow_mpp = oN") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ON")) + tk.MustExec("set @@session.tidb_allow_mpp = enForcE") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ENFORCE")) + + tk.MustExec("set @@global.tidb_allow_mpp = faLsE") + tk.MustQuery("select @@global.tidb_allow_mpp").Check(testkit.Rows("OFF")) + tk.MustExec("set @@global.tidb_allow_mpp = True") + tk.MustQuery("select @@global.tidb_allow_mpp").Check(testkit.Rows("ON")) + + err = tk.ExecToErr("set @@global.tidb_allow_mpp = enforceWithTypo") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, `[variable:1231]Variable 'tidb_allow_mpp' can't be set to the value of 'enforceWithTypo'`) + + // test query + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("create index idx on t(a)") + + // 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 == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + // ban mpp + tk.MustExec("set @@session.tidb_allow_mpp = 0") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("OFF")) + + // read from tiflash, batch cop. + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1").Check(testkit.Rows( + "StreamAgg_20 1.00 285050.00 root funcs:count(Column#5)->Column#3", + "└─TableReader_21 1.00 19003.88 root data:StreamAgg_9", + " └─StreamAgg_9 1.00 19006.88 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_19 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_18 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + // open mpp + tk.MustExec("set @@session.tidb_allow_mpp = 1") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ON")) + + // should use tikv to index read + tk.MustQuery("explain format='verbose' select count(*) from t where a=1;").Check(testkit.Rows( + "StreamAgg_30 1.00 485.00 root funcs:count(Column#6)->Column#3", + "└─IndexReader_31 1.00 32.88 root index:StreamAgg_10", + " └─StreamAgg_10 1.00 35.88 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_29 10.00 455.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo")) + + // read from tikv, indexRead + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1;").Check(testkit.Rows( + "StreamAgg_18 1.00 485.00 root funcs:count(Column#5)->Column#3", + "└─IndexReader_19 1.00 32.88 root index:StreamAgg_10", + " └─StreamAgg_10 1.00 35.88 cop[tikv] funcs:count(1)->Column#5", + " └─IndexRangeScan_17 10.00 455.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo")) + + // read from tiflash, mpp with large cost + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1").Check(testkit.Rows( + "HashAgg_21 1.00 11910.68 root funcs:count(Column#5)->Column#3", + "└─TableReader_23 1.00 11877.08 root data:ExchangeSender_22", + " └─ExchangeSender_22 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + // enforce mpp + tk.MustExec("set @@session.tidb_allow_mpp = 2") + tk.MustQuery("select @@session.tidb_allow_mpp").Check(testkit.Rows("ENFORCE")) + + // should use mpp + tk.MustQuery("explain format='verbose' select count(*) from t where a=1;").Check(testkit.Rows( + "HashAgg_24 1.00 33.60 root funcs:count(Column#5)->Column#3", + "└─TableReader_26 1.00 0.00 root data:ExchangeSender_25", + " └─ExchangeSender_25 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_23 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_22 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) + + // read from tikv, indexRead + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1;").Check(testkit.Rows( + "StreamAgg_18 1.00 485.00 root funcs:count(Column#5)->Column#3", + "└─IndexReader_19 1.00 32.88 root index:StreamAgg_10", + " └─StreamAgg_10 1.00 35.88 cop[tikv] funcs:count(1)->Column#5", + " └─IndexRangeScan_17 10.00 455.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo")) + + // read from tiflash, mpp with little cost + tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1").Check(testkit.Rows( + "HashAgg_21 1.00 33.60 root funcs:count(Column#5)->Column#3", + "└─TableReader_23 1.00 0.00 root data:ExchangeSender_22", + " └─ExchangeSender_22 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#5", + " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) +} diff --git a/planner/core/task.go b/planner/core/task.go index 205f5eb77b08a..fa6855503dd0e 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -2026,11 +2026,15 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { StoreType: kv.TiFlash, }.Init(ctx, t.p.SelectBlockOffset()) p.stats = t.p.statsInfo() + + p.cost = t.cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor + if p.ctx.GetSessionVars().IsMPPEnforced() { + p.cost = 0 + } rt := &rootTask{ p: p, - cst: t.cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor, + cst: p.cost, } - p.cost = rt.cost() return rt } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 618120b5da6e6..d6bb5763e67d8 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -492,11 +492,12 @@ type SessionVars struct { AllowWriteRowID bool // AllowBatchCop means if we should send batch coprocessor to TiFlash. Default value is 1, means to use batch cop in case of aggregation and join. - // If value is set to 2 , which means to force to send batch cop for any query. Value is set to 0 means never use batch cop. + // Value set to 2 means to force to send batch cop for any query. Value set to 0 means never use batch cop. AllowBatchCop int - // AllowMPPExecution will prefer using mpp way to execute a query. - AllowMPPExecution bool + // AllowMPPExecution means if we should use mpp way to execute query. Default value is "ON", means to be determined by the optimizer. + // Value set to "ENFORCE" means to use mpp whenever possible. Value set to means never use mpp. + allowMPPExecution string // TiDBAllowAutoRandExplicitInsert indicates whether explicit insertion on auto_random column is allowed. AllowAutoRandExplicitInsert bool @@ -845,6 +846,16 @@ func (s *SessionVars) AllocMPPTaskID(startTS uint64) int64 { return 1 } +// IsMPPAllowed returns whether mpp execution is allowed. +func (s *SessionVars) IsMPPAllowed() bool { + return s.allowMPPExecution != "OFF" +} + +// IsMPPEnforced returns whether mpp execution is enforced. +func (s *SessionVars) IsMPPEnforced() bool { + return s.allowMPPExecution == "ENFORCE" +} + // CheckAndGetTxnScope will return the transaction scope we should use in the current session. func (s *SessionVars) CheckAndGetTxnScope() string { if s.InRestrictedSQL { @@ -1094,7 +1105,7 @@ func NewSessionVars() *SessionVars { terror.Log(vars.SetSystemVar(TiDBEnableStreaming, enableStreaming)) vars.AllowBatchCop = DefTiDBAllowBatchCop - vars.AllowMPPExecution = DefTiDBAllowMPPExecution + vars.allowMPPExecution = DefTiDBAllowMPPExecution var enableChunkRPC string if config.GetGlobalConfig().TiKVClient.EnableChunkRPC { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 9a01c19470722..73a8ca0066450 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -720,8 +720,8 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Type: TypeBool, Value: BoolToOnOff(DefTiDBAllowMPPExecution), SetSession: func(s *SessionVars, val string) error { - s.AllowMPPExecution = TiDBOptOn(val) + {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "ENFORCE"}, SetSession: func(s *SessionVars, val string) error { + s.allowMPPExecution = val return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBBCJThresholdCount, Value: strconv.Itoa(DefBroadcastJoinThresholdCount), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { @@ -793,7 +793,7 @@ var defaultSysVars = []*SysVar{ s.CPUFactor = tidbOptFloat64(val, DefOptCPUFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { s.CopTiFlashConcurrencyFactor = tidbOptFloat64(val, DefOptTiFlashConcurrencyFactor) return nil }}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 54616f902db5e..e416f9a695fc3 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -290,6 +290,8 @@ const ( // The default value is 0 TiDBAllowBatchCop = "tidb_allow_batch_cop" + // TiDBAllowMPPExecution means if we should use mpp way to execute query. Default value is 1 (or 'ON'), means to be determined by the optimizer. + // Value set to 2 (or 'ENFORCE') which means to use mpp whenever possible. Value set to 2 (or 'OFF') means never use mpp. TiDBAllowMPPExecution = "tidb_allow_mpp" // TiDBInitChunkSize is used to control the init chunk size during query execution. @@ -613,7 +615,7 @@ const ( DefBroadcastJoinThresholdCount = 10 * 1024 DefTiDBOptimizerSelectivityLevel = 0 DefTiDBAllowBatchCop = 1 - DefTiDBAllowMPPExecution = true + DefTiDBAllowMPPExecution = "ON" DefTiDBTxnMode = "" DefTiDBRowFormatV1 = 1 DefTiDBRowFormatV2 = 2 From 01798548d973bae07109bd2b72a7087678e78f52 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 12 May 2021 18:27:40 +0800 Subject: [PATCH 030/343] store/tikv: remove use of SchemaAmender option in store/tikv (#24408) --- store/driver/txn/txn_driver.go | 2 ++ store/tikv/txn.go | 9 +++++---- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 50bba80d2b54e..24e6f11c8fef9 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -148,6 +148,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case tikvstore.InfoSchema: txn.SetSchemaVer(val.(tikv.SchemaVer)) + case tikvstore.SchemaAmender: + txn.SetSchemaAmender(val.(tikv.SchemaAmender)) case tikvstore.CommitHook: txn.SetCommitCallback(val.(func(string, error))) case tikvstore.Enable1PC: diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 1d678d010957e..47ccdce12caea 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -223,10 +223,6 @@ func (txn *KVTxn) Delete(k []byte) error { func (txn *KVTxn) SetOption(opt int, val interface{}) { txn.us.SetOption(opt, val) txn.snapshot.SetOption(opt, val) - switch opt { - case tikv.SchemaAmender: - txn.schemaAmender = val.(SchemaAmender) - } } // GetOption returns the option @@ -265,6 +261,11 @@ func (txn *KVTxn) SetPriority(pri Priority) { txn.GetSnapshot().SetPriority(pri) } +// SetSchemaAmender sets an amender to update mutations after schema change. +func (txn *KVTxn) SetSchemaAmender(sa SchemaAmender) { + txn.schemaAmender = sa +} + // SetCommitCallback sets up a function that will be called when the transaction // is finished. func (txn *KVTxn) SetCommitCallback(f func(string, error)) { From 081291b06ef7bc121fb9abbc60648bd7dd5759a9 Mon Sep 17 00:00:00 2001 From: Howie Date: Wed, 12 May 2021 18:51:40 +0800 Subject: [PATCH 031/343] *: the value of tikv-client.store-liveness-timeout should not less than 0 (#24244) --- tidb-server/main.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tidb-server/main.go b/tidb-server/main.go index f070d2eeec48d..3e2351bf7c352 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -572,7 +572,7 @@ func setGlobalVars() { kvcache.GlobalLRUMemUsageTracker.AttachToGlobalTracker(executor.GlobalMemoryUsageTracker) t, err := time.ParseDuration(cfg.TiKVClient.StoreLivenessTimeout) - if err != nil { + if err != nil || t < 0 { logutil.BgLogger().Fatal("invalid duration value for store-liveness-timeout", zap.String("currentValue", cfg.TiKVClient.StoreLivenessTimeout)) } From 1ae648bd61a64aae18ee63e82c57720983968c61 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 12 May 2021 19:11:40 +0800 Subject: [PATCH 032/343] store/tikv: remove use of EnableAsyncCommit option in store/tikv (#24462) --- store/driver/txn/txn_driver.go | 2 ++ store/tikv/2pc.go | 4 +--- store/tikv/tests/2pc_test.go | 2 +- store/tikv/tests/async_commit_test.go | 4 ++-- store/tikv/tests/snapshot_fail_test.go | 5 +++-- store/tikv/txn.go | 6 ++++++ 6 files changed, 15 insertions(+), 8 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 24e6f11c8fef9..739b983fd3d99 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -152,6 +152,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetSchemaAmender(val.(tikv.SchemaAmender)) case tikvstore.CommitHook: txn.SetCommitCallback(val.(func(string, error))) + case tikvstore.EnableAsyncCommit: + txn.SetEnableAsyncCommit(val.(bool)) case tikvstore.Enable1PC: txn.SetEnable1PC(val.(bool)) case tikvstore.TxnScope: diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 8703b1861c65d..ee94eceec166a 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -825,12 +825,10 @@ func (c *twoPhaseCommitter) checkAsyncCommit() bool { return false } - enableAsyncCommitOption := c.txn.us.GetOption(kv.EnableAsyncCommit) - enableAsyncCommit := enableAsyncCommitOption != nil && enableAsyncCommitOption.(bool) asyncCommitCfg := config.GetGlobalConfig().TiKVClient.AsyncCommit // TODO the keys limit need more tests, this value makes the unit test pass by now. // Async commit is not compatible with Binlog because of the non unique timestamp issue. - if c.sessionID > 0 && enableAsyncCommit && + if c.sessionID > 0 && c.txn.enableAsyncCommit && uint(c.mutations.Len()) <= asyncCommitCfg.KeysLimit && !c.shouldWriteBinlog() { totalKeySize := uint64(0) diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index d1e635f205efa..5589752043b2b 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -105,7 +105,7 @@ func (s *testCommitterSuite) begin(c *C) tikv.TxnProbe { func (s *testCommitterSuite) beginAsyncCommit(c *C) tikv.TxnProbe { txn, err := s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, true) + txn.SetEnableAsyncCommit(true) return txn } diff --git a/store/tikv/tests/async_commit_test.go b/store/tikv/tests/async_commit_test.go index 0f4985fa7ab86..381771bfa0836 100644 --- a/store/tikv/tests/async_commit_test.go +++ b/store/tikv/tests/async_commit_test.go @@ -134,7 +134,7 @@ func (s *testAsyncCommitCommon) beginAsyncCommitWithLinearizability(c *C) tikv.T func (s *testAsyncCommitCommon) beginAsyncCommit(c *C) tikv.TxnProbe { txn, err := s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, true) + txn.SetEnableAsyncCommit(true) return tikv.TxnProbe{KVTxn: txn} } @@ -160,7 +160,7 @@ func (s *testAsyncCommitSuite) SetUpTest(c *C) { func (s *testAsyncCommitSuite) lockKeysWithAsyncCommit(c *C, keys, values [][]byte, primaryKey, primaryValue []byte, commitPrimary bool) (uint64, uint64) { txn, err := s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, true) + txn.SetEnableAsyncCommit(true) for i, k := range keys { if len(values[i]) > 0 { err = txn.Set(k, values[i]) diff --git a/store/tikv/tests/snapshot_fail_test.go b/store/tikv/tests/snapshot_fail_test.go index 1360841bd743a..aca3c59099cf7 100644 --- a/store/tikv/tests/snapshot_fail_test.go +++ b/store/tikv/tests/snapshot_fail_test.go @@ -152,6 +152,7 @@ func (s *testSnapshotFailSuite) TestRetryMaxTsPointGetSkipLock(c *C) { err = txn.Set([]byte("k2"), []byte("v2")) c.Assert(err, IsNil) txn.SetOption(kv.EnableAsyncCommit, true) + txn.SetEnableAsyncCommit(true) c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing", "return"), IsNil) c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/twoPCShortLockTTL", "return"), IsNil) @@ -181,7 +182,7 @@ func (s *testSnapshotFailSuite) TestRetryMaxTsPointGetSkipLock(c *C) { // Prewrite k1 and k2 again without committing them txn, err = s.store.Begin() c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, true) + txn.SetEnableAsyncCommit(true) err = txn.Set([]byte("k1"), []byte("v3")) c.Assert(err, IsNil) err = txn.Set([]byte("k2"), []byte("v4")) @@ -210,7 +211,7 @@ func (s *testSnapshotFailSuite) TestRetryPointGetResolveTS(c *C) { c.Assert(txn.Set([]byte("k1"), []byte("v1")), IsNil) err = txn.Set([]byte("k2"), []byte("v2")) c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, false) + txn.SetEnableAsyncCommit(false) txn.SetEnable1PC(false) txn.SetOption(kv.GuaranteeLinearizability, false) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 47ccdce12caea..a8c0f70f8da8d 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -82,6 +82,7 @@ type KVTxn struct { syncLog bool priority Priority isPessimistic bool + enableAsyncCommit bool enable1PC bool scope string kvFilter KVFilter @@ -272,6 +273,11 @@ func (txn *KVTxn) SetCommitCallback(f func(string, error)) { txn.commitCallback = f } +// SetEnableAsyncCommit indicates if the transaction will try to use async commit. +func (txn *KVTxn) SetEnableAsyncCommit(b bool) { + txn.enableAsyncCommit = b +} + // SetEnable1PC indicates if the transaction will try to use 1 phase commit. func (txn *KVTxn) SetEnable1PC(b bool) { txn.enable1PC = b From b1d134d7d9264a72df902087866b6de23d4efb93 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Wed, 12 May 2021 19:25:40 +0800 Subject: [PATCH 033/343] txn: Add txn state's view (#22908) --- bindinfo/bind_test.go | 5 ++ domain/domain_test.go | 5 ++ executor/adapter.go | 3 + executor/builder.go | 4 +- executor/executor_pkg_test.go | 5 ++ executor/explainfor_test.go | 5 ++ executor/infoschema_reader.go | 38 +++++++++ executor/infoschema_reader_test.go | 5 ++ executor/prepared_test.go | 5 ++ executor/seqtest/prepared_test.go | 5 ++ infoschema/cluster.go | 3 + infoschema/infoschema_test.go | 1 + infoschema/tables.go | 38 +++++++-- infoschema/tables_test.go | 36 +++++++- kv/kv.go | 1 + server/server.go | 19 ++++- session/session.go | 24 +++++- session/session_test.go | 102 ++++++++++++++++++++++ session/txn.go | 133 +++++++++++++++++++++++------ session/txninfo/txn_info.go | 96 +++++++++++++++++++++ util/processinfo.go | 2 + 21 files changed, 493 insertions(+), 42 deletions(-) create mode 100644 session/txninfo/txn_info.go diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 1eb7e1478b2f9..22b60187f0a77 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/util" @@ -70,6 +71,10 @@ type mockSessionManager struct { PS []*util.ProcessInfo } +func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) for _, item := range msm.PS { diff --git a/domain/domain_test.go b/domain/domain_test.go index 7c9d9ff633bc5..a4432b0fb1fe6 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" @@ -241,6 +242,10 @@ type mockSessionManager struct { PS []*util.ProcessInfo } +func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) for _, item := range msm.PS { diff --git a/executor/adapter.go b/executor/adapter.go index 5f5229195c3f9..44d00cd1efa1e 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" + "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -377,6 +378,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { if txn.Valid() { txnStartTS = txn.StartTS() } + return &recordSet{ executor: e, stmt: a, @@ -590,6 +592,7 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { } e, err = a.handlePessimisticLockError(ctx, err) if err != nil { + // todo: Report deadlock if ErrDeadlock.Equal(err) { metrics.StatementDeadlockDetectDuration.Observe(time.Since(startLocking).Seconds()) } diff --git a/executor/builder.go b/executor/builder.go index 40282d1030b2c..3324e52f894ff 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1531,7 +1531,9 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TablePlacementPolicy), strings.ToLower(infoschema.TableClientErrorsSummaryGlobal), strings.ToLower(infoschema.TableClientErrorsSummaryByUser), - strings.ToLower(infoschema.TableClientErrorsSummaryByHost): + strings.ToLower(infoschema.TableClientErrorsSummaryByHost), + strings.ToLower(infoschema.TableTiDBTrx), + strings.ToLower(infoschema.ClusterTableTiDBTrx): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), table: v.Table, diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 7cc5a8a69d66e..5591dcefde54d 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" plannerutil "github.com/pingcap/tidb/planner/util" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" @@ -60,6 +61,10 @@ type mockSessionManager struct { serverID uint64 } +func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + // ShowProcessList implements the SessionManager.ShowProcessList interface. func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index a113200a925d8..e29a7a3e24cee 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/israce" @@ -38,6 +39,10 @@ type mockSessionManager1 struct { PS []*util.ProcessInfo } +func (msm *mockSessionManager1) ShowTxnList() []*txninfo.TxnInfo { + return nil +} + // ShowProcessList implements the SessionManager.ShowProcessList interface. func (msm *mockSessionManager1) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 0ec0c48885ecf..ae338bdd644d2 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -149,6 +149,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex infoschema.TableClientErrorsSummaryByUser, infoschema.TableClientErrorsSummaryByHost: err = e.setDataForClientErrorsSummary(sctx, e.table.Name.O) + case infoschema.TableTiDBTrx: + e.setDataForTiDBTrx(sctx) + case infoschema.ClusterTableTiDBTrx: + err = e.setDataForClusterTiDBTrx(sctx) } if err != nil { return nil, err @@ -2011,6 +2015,40 @@ func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context return nil } +func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) { + sm := ctx.GetSessionManager() + if sm == nil { + return + } + + loginUser := ctx.GetSessionVars().User + var hasProcessPriv bool + if pm := privilege.GetPrivilegeManager(ctx); pm != nil { + if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) { + hasProcessPriv = true + } + } + infoList := sm.ShowTxnList() + for _, info := range infoList { + // If you have the PROCESS privilege, you can see all running transactions. + // Otherwise, you can see only your own transactions. + if !hasProcessPriv && loginUser != nil && info.Username != loginUser.Username { + continue + } + e.rows = append(e.rows, info.ToDatum()) + } +} + +func (e *memtableRetriever) setDataForClusterTiDBTrx(ctx sessionctx.Context) error { + e.setDataForTiDBTrx(ctx) + rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) + if err != nil { + return err + } + e.rows = rows + return nil +} + type hugeMemTableRetriever struct { dummyCloser table *model.TableInfo diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index c3e125824873d..e19eb9d9b3064 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" @@ -728,6 +729,10 @@ type mockSessionManager struct { serverID uint64 } +func (sm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { return sm.processInfoMap } diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 1f8edf79d942e..e0e2c19ee0f22 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/domain" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/israce" @@ -135,6 +136,10 @@ type mockSessionManager2 struct { killed bool } +func (sm *mockSessionManager2) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + func (sm *mockSessionManager2) ShowProcessList() map[uint64]*util.ProcessInfo { pl := make(map[uint64]*util.ProcessInfo) if pi, ok := sm.GetProcessInfo(0); ok { diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index 916f218db1f9d..bb8f05e5eff54 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/kvcache" @@ -796,6 +797,10 @@ type mockSessionManager1 struct { Se session.Session } +func (msm *mockSessionManager1) ShowTxnList() []*txninfo.TxnInfo { + panic("unimplemented!") +} + // ShowProcessList implements the SessionManager.ShowProcessList interface. func (msm *mockSessionManager1) ShowProcessList() map[uint64]*util.ProcessInfo { ret := make(map[uint64]*util.ProcessInfo) diff --git a/infoschema/cluster.go b/infoschema/cluster.go index f113e90a0f587..2d196fe5b0023 100644 --- a/infoschema/cluster.go +++ b/infoschema/cluster.go @@ -37,6 +37,8 @@ const ( ClusterTableStatementsSummary = "CLUSTER_STATEMENTS_SUMMARY" // ClusterTableStatementsSummaryHistory is the string constant of cluster statement summary history table. ClusterTableStatementsSummaryHistory = "CLUSTER_STATEMENTS_SUMMARY_HISTORY" + // ClusterTableTiDBTrx is the string constant of cluster transaction running table. + ClusterTableTiDBTrx = "CLUSTER_TIDB_TRX" ) // memTableToClusterTables means add memory table to cluster table. @@ -45,6 +47,7 @@ var memTableToClusterTables = map[string]string{ TableProcesslist: ClusterTableProcesslist, TableStatementsSummary: ClusterTableStatementsSummary, TableStatementsSummaryHistory: ClusterTableStatementsSummaryHistory, + TableTiDBTrx: ClusterTableTiDBTrx, } func init() { diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index c3892e6527962..6aa0c5526f467 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -332,6 +332,7 @@ func (*testSuite) TestInfoTables(c *C) { "TABLESPACES", "COLLATION_CHARACTER_SET_APPLICABILITY", "PROCESSLIST", + "TIDB_TRX", } for _, t := range infoTables { tb, err1 := is.TableByName(util.InformationSchemaName, model.NewCIStr(t)) diff --git a/infoschema/tables.go b/infoschema/tables.go index bfca649e89fdd..2d5112ada05c0 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -31,11 +31,13 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" @@ -161,6 +163,8 @@ const ( TableClientErrorsSummaryByUser = "CLIENT_ERRORS_SUMMARY_BY_USER" // TableClientErrorsSummaryByHost is the string constant of client errors table. TableClientErrorsSummaryByHost = "CLIENT_ERRORS_SUMMARY_BY_HOST" + // TableTiDBTrx is current running transaction status table. + TableTiDBTrx = "TIDB_TRX" ) var tableIDMap = map[string]int64{ @@ -233,22 +237,25 @@ var tableIDMap = map[string]int64{ TableClientErrorsSummaryGlobal: autoid.InformationSchemaDBID + 67, TableClientErrorsSummaryByUser: autoid.InformationSchemaDBID + 68, TableClientErrorsSummaryByHost: autoid.InformationSchemaDBID + 69, + TableTiDBTrx: autoid.InformationSchemaDBID + 70, + ClusterTableTiDBTrx: autoid.InformationSchemaDBID + 71, } type columnInfo struct { - name string - tp byte - size int - decimal int - flag uint - deflt interface{} - comment string + name string + tp byte + size int + decimal int + flag uint + deflt interface{} + comment string + enumElems []string } func buildColumnInfo(col columnInfo) *model.ColumnInfo { mCharset := charset.CharsetBin mCollation := charset.CharsetBin - if col.tp == mysql.TypeVarchar || col.tp == mysql.TypeBlob || col.tp == mysql.TypeLongBlob { + if col.tp == mysql.TypeVarchar || col.tp == mysql.TypeBlob || col.tp == mysql.TypeLongBlob || col.tp == mysql.TypeEnum { mCharset = charset.CharsetUTF8MB4 mCollation = charset.CollationUTF8MB4 } @@ -259,6 +266,7 @@ func buildColumnInfo(col columnInfo) *model.ColumnInfo { Flen: col.size, Decimal: col.decimal, Flag: col.flag, + Elems: col.enumElems, } return &model.ColumnInfo{ Name: model.NewCIStr(col.name), @@ -1332,6 +1340,19 @@ var tableClientErrorsSummaryByHostCols = []columnInfo{ {name: "LAST_SEEN", tp: mysql.TypeTimestamp, size: 26}, } +var tableTiDBTrxCols = []columnInfo{ + {name: "ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.PriKeyFlag | mysql.NotNullFlag | mysql.UnsignedFlag}, + {name: "START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Start time of the transaction"}, + {name: "DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, + {name: "STATE", tp: mysql.TypeEnum, enumElems: txninfo.TxnRunningStateStrs, comment: "Current running state of the transaction"}, + {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current lock waiting's start time"}, + {name: "LEN", tp: mysql.TypeLonglong, size: 64, comment: "How many entries are in MemDB"}, + {name: "SIZE", tp: mysql.TypeLonglong, size: 64, comment: "MemDB used memory"}, + {name: "SESSION_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.UnsignedFlag, comment: "Which session this transaction belongs to"}, + {name: "USER", tp: mysql.TypeVarchar, size: 16, comment: "The user who open this session"}, + {name: "DB", tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"}, +} + // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. // The returned description string may be: // - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. @@ -1701,6 +1722,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableClientErrorsSummaryGlobal: tableClientErrorsSummaryGlobalCols, TableClientErrorsSummaryByUser: tableClientErrorsSummaryByUserCols, TableClientErrorsSummaryByHost: tableClientErrorsSummaryByHostCols, + TableTiDBTrx: tableTiDBTrxCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index f30f25ba6abfa..6cc24300c1be4 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -28,6 +28,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/fn" + "github.com/pingcap/parser" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -42,6 +43,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util" @@ -121,7 +123,7 @@ func (s *testClusterTableSuite) setUpRPCService(c *C, addr string) (*grpc.Server lis, err := net.Listen("tcp", addr) c.Assert(err, IsNil) // Fix issue 9836 - sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1)} + sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "root", @@ -276,7 +278,7 @@ func (s *testTableSuite) TestInfoschemaFieldValue(c *C) { tk1.MustQuery("select distinct(table_schema) from information_schema.tables").Check(testkit.Rows("INFORMATION_SCHEMA")) // Fix issue 9836 - sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1)} + sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 1), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "root", @@ -433,6 +435,11 @@ func (s *testTableSuite) TestCurrentTimestampAsDefault(c *C) { type mockSessionManager struct { processInfoMap map[uint64]*util.ProcessInfo + txnInfo []*txninfo.TxnInfo +} + +func (sm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo { + return sm.txnInfo } func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo { @@ -459,7 +466,7 @@ func (s *testTableSuite) TestSomeTables(c *C) { c.Assert(err, IsNil) tk := testkit.NewTestKit(c, s.store) tk.Se = se - sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2)} + sm := &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "user-1", @@ -516,7 +523,7 @@ func (s *testTableSuite) TestSomeTables(c *C) { fmt.Sprintf("3 user-3 127.0.0.1:12345 test Init DB 9223372036 %s %s", "in transaction", "check port"), )) - sm = &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2)} + sm = &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2), nil} sm.processInfoMap[1] = &util.ProcessInfo{ ID: 1, User: "user-1", @@ -1509,3 +1516,24 @@ func (s *testTableSuite) TestInfoschemaClientErrors(c *C) { err = tk.ExecToErr("FLUSH CLIENT_ERRORS_SUMMARY") c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RELOAD privilege(s) for this operation") } + +func (s *testTableSuite) TestTrx(c *C) { + tk := s.newTestKitWithRoot(c) + _, digest := parser.NormalizeDigest("select * from trx for update;") + sm := &mockSessionManager{nil, make([]*txninfo.TxnInfo, 1)} + sm.txnInfo[0] = &txninfo.TxnInfo{ + StartTS: 424768545227014155, + CurrentSQLDigest: digest, + State: txninfo.TxnRunningNormal, + BlockStartTime: nil, + EntriesCount: 1, + EntriesSize: 19, + ConnectionID: 2, + Username: "root", + CurrentDB: "test", + } + tk.Se.SetSessionManager(sm) + tk.MustQuery("select * from information_schema.TIDB_TRX;").Check( + testkit.Rows("424768545227014155 2021-05-07 12:56:48 " + digest + " Normal 1 19 2 root test"), + ) +} diff --git a/kv/kv.go b/kv/kv.go index a6a23a88df01d..1fad79d641009 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -154,6 +154,7 @@ type Transaction interface { // String implements fmt.Stringer interface. String() string // LockKeys tries to lock the entries with the keys in KV store. + // Will block until all keys are locked successfully or an error occurs. LockKeys(ctx context.Context, lockCtx *LockCtx, keys ...Key) error // SetOption sets an option with a value, when val is nil, uses the default // value of this option. diff --git a/server/server.go b/server/server.go index f7a6021a11221..29f5307895cc2 100644 --- a/server/server.go +++ b/server/server.go @@ -37,7 +37,6 @@ import ( "math/rand" "net" "net/http" - "unsafe" // For pprof _ "net/http/pprof" @@ -46,6 +45,7 @@ import ( "sync" "sync/atomic" "time" + "unsafe" "github.com/blacktear23/go-proxyprotocol" "github.com/pingcap/errors" @@ -56,6 +56,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/plugin" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util" @@ -557,6 +558,22 @@ func (s *Server) ShowProcessList() map[uint64]*util.ProcessInfo { return rs } +// ShowTxnList shows all txn info for displaying in `TIDB_TRX` +func (s *Server) ShowTxnList() []*txninfo.TxnInfo { + s.rwlock.RLock() + defer s.rwlock.RUnlock() + rs := make([]*txninfo.TxnInfo, 0, len(s.clients)) + for _, client := range s.clients { + if client.ctx.Session != nil { + info := client.ctx.Session.TxnInfo() + if info != nil { + rs = append(rs, info) + } + } + } + return rs +} + // GetProcessInfo implements the SessionManager interface. func (s *Server) GetProcessInfo(id uint64) (*util.ProcessInfo, bool) { s.rwlock.RLock() diff --git a/session/session.go b/session/session.go index 2f842f92e183a..0b4cb309f434b 100644 --- a/session/session.go +++ b/session/session.go @@ -41,6 +41,9 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tipb/go-binlog" + "go.uber.org/zap" + "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" @@ -58,6 +61,7 @@ import ( "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/privilege/privileges" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -81,8 +85,6 @@ import ( "github.com/pingcap/tidb/util/sli" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/timeutil" - "github.com/pingcap/tipb/go-binlog" - "go.uber.org/zap" ) var ( @@ -145,6 +147,8 @@ type Session interface { Auth(user *auth.UserIdentity, auth []byte, salt []byte) bool AuthWithoutVerification(user *auth.UserIdentity) bool ShowProcess() *util.ProcessInfo + // Return the information of the txn current running + TxnInfo() *txninfo.TxnInfo // PrepareTxnCtx is exported for test. PrepareTxnCtx(context.Context) // FieldList returns fields list of a table. @@ -183,7 +187,7 @@ func (h *StmtHistory) Count() int { type session struct { // processInfo is used by ShowProcess(), and should be modified atomically. processInfo atomic.Value - txn TxnState + txn LazyTxn mu struct { sync.RWMutex @@ -442,6 +446,19 @@ func (s *session) FieldList(tableName string) ([]*ast.ResultField, error) { return fields, nil } +func (s *session) TxnInfo() *txninfo.TxnInfo { + txnInfo := s.txn.Info() + if txnInfo == nil { + return nil + } + processInfo := s.ShowProcess() + txnInfo.CurrentSQLDigest = processInfo.Digest + txnInfo.ConnectionID = processInfo.ID + txnInfo.Username = processInfo.User + txnInfo.CurrentDB = processInfo.DB + return txnInfo +} + func (s *session) doCommit(ctx context.Context) error { if !s.txn.Valid() { return nil @@ -524,6 +541,7 @@ func (s *session) doCommit(ctx context.Context) error { if err = memBuffer.Delete(iter.Key()); err != nil { return errors.Trace(err) } + s.txn.UpdateEntriesCountAndSize() if err = iter.Next(); err != nil { return errors.Trace(err) } diff --git a/session/session_test.go b/session/session_test.go index 5fa7779fc65c3..3baee4f0ef6f1 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -42,6 +42,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" + txninfo "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" @@ -83,6 +84,7 @@ var _ = SerialSuites(&testSessionSerialSuite{}) var _ = SerialSuites(&testBackupRestoreSuite{}) var _ = Suite(&testClusteredSuite{}) var _ = SerialSuites(&testClusteredSerialSuite{}) +var _ = SerialSuites(&testTxnStateSuite{}) type testSessionSuiteBase struct { cluster cluster.Cluster @@ -4303,3 +4305,103 @@ func (s *testSessionSuite3) TestGlobalTemporaryTable(c *C) { // The global temporary table data is discard after the transaction commit. tk.MustQuery("select * from g_tmp").Check(testkit.Rows()) } + +type testTxnStateSuite struct { + testSessionSuiteBase +} + +func (s *testTxnStateSuite) TestBasic(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1);") + info := tk.Se.TxnInfo() + c.Assert(info, IsNil) + tk.MustExec("begin pessimistic;") + tk.MustExec("select * from t for update;") + info = tk.Se.TxnInfo() + _, expectedDigest := parser.NormalizeDigest("select * from t for update;") + c.Assert(info.CurrentSQLDigest, Equals, expectedDigest) + c.Assert(info.State, Equals, txninfo.TxnRunningNormal) + c.Assert(info.BlockStartTime, IsNil) + // len and size will be covered in TestLenAndSize + c.Assert(info.ConnectionID, Equals, tk.Se.GetSessionVars().ConnectionID) + c.Assert(info.Username, Equals, "") + c.Assert(info.CurrentDB, Equals, "test") + tk.MustExec("commit;") + info = tk.Se.TxnInfo() + c.Assert(info, IsNil) +} + +func (s *testTxnStateSuite) TestEntriesCountAndSize(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("begin pessimistic;") + tk.MustExec("insert into t(a) values (1);") + info := tk.Se.TxnInfo() + c.Assert(info.EntriesCount, Equals, uint64(1)) + c.Assert(info.EntriesSize, Equals, uint64(29)) + tk.MustExec("insert into t(a) values (2);") + info = tk.Se.TxnInfo() + c.Assert(info.EntriesCount, Equals, uint64(2)) + c.Assert(info.EntriesSize, Equals, uint64(58)) + tk.MustExec("commit;") +} + +func (s *testTxnStateSuite) TestBlocked(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1);") + tk.MustExec("begin pessimistic;") + tk.MustExec("select * from t where a = 1 for update;") + go func() { + tk2.MustExec("begin pessimistic") + tk2.MustExec("select * from t where a = 1 for update;") + tk2.MustExec("commit;") + }() + time.Sleep(100 * time.Millisecond) + c.Assert(tk2.Se.TxnInfo().State, Equals, txninfo.TxnLockWaiting) + c.Assert(tk2.Se.TxnInfo().BlockStartTime, NotNil) + tk.MustExec("commit;") +} + +func (s *testTxnStateSuite) TestCommitting(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1), (2);") + tk.MustExec("begin pessimistic;") + tk.MustExec("select * from t where a = 1 for update;") + ch := make(chan struct{}) + go func() { + tk2.MustExec("begin pessimistic") + c.Assert(tk2.Se.TxnInfo(), NotNil) + tk2.MustExec("select * from t where a = 2 for update;") + failpoint.Enable("github.com/pingcap/tidb/session/mockSlowCommit", "sleep(200)") + defer failpoint.Disable("github.com/pingcap/tidb/session/mockSlowCommit") + tk2.MustExec("commit;") + ch <- struct{}{} + }() + time.Sleep(100 * time.Millisecond) + c.Assert(tk2.Se.TxnInfo().State, Equals, txninfo.TxnCommitting) + tk.MustExec("commit;") + <-ch +} + +func (s *testTxnStateSuite) TestRollbacking(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int);") + tk.MustExec("insert into t(a) values (1), (2);") + ch := make(chan struct{}) + go func() { + tk.MustExec("begin pessimistic") + tk.MustExec("insert into t(a) values (3);") + failpoint.Enable("github.com/pingcap/tidb/session/mockSlowRollback", "sleep(200)") + defer failpoint.Disable("github.com/pingcap/tidb/session/mockSlowRollback") + tk.MustExec("rollback;") + ch <- struct{}{} + }() + time.Sleep(100 * time.Millisecond) + c.Assert(tk.Se.TxnInfo().State, Equals, txninfo.TxnRollingBack) + <-ch +} diff --git a/session/txn.go b/session/txn.go index aebed7ed920b2..133cafb976aae 100644 --- a/session/txn.go +++ b/session/txn.go @@ -20,6 +20,8 @@ import ( "runtime/trace" "strings" "sync/atomic" + "time" + "unsafe" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" @@ -28,6 +30,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -39,12 +42,12 @@ import ( "go.uber.org/zap" ) -// TxnState wraps kv.Transaction to provide a new kv.Transaction. +// LazyTxn wraps kv.Transaction to provide a new kv.Transaction. // 1. It holds all statement related modification in the buffer before flush to the txn, // so if execute statement meets error, the txn won't be made dirty. // 2. It's a lazy transaction, that means it's a txnFuture before StartTS() is really need. -type TxnState struct { - // States of a TxnState should be one of the followings: +type LazyTxn struct { + // States of a LazyTxn should be one of the followings: // Invalid: kv.Transaction == nil && txnFuture == nil // Pending: kv.Transaction == nil && txnFuture != nil // Valid: kv.Transaction != nil && txnFuture == nil @@ -55,23 +58,40 @@ type TxnState struct { stagingHandle kv.StagingHandle mutations map[int64]*binlog.TableMutation writeSLI sli.TxnWriteThroughputSLI + + // following atomic fields are used for filling TxnInfo + // we need these fields because kv.Transaction provides no thread safety promise + // but we hope getting TxnInfo is a thread safe op + + infoStartTS uint64 + // current executing state + State txninfo.TxnRunningState + // last trying to block start time + blockStartTime unsafe.Pointer // *time.Time, cannot use atomic.Value here because it is possible to be nil + // how many entries are there in the memBuffer, should be equal to self.(kv.Transaction).Len() + EntriesCount uint64 + // how many memory space do the entries in the memBuffer take, should be equal to self.(kv.Transaction).Size() + EntriesSize uint64 } // GetTableInfo returns the cached index name. -func (txn *TxnState) GetTableInfo(id int64) *model.TableInfo { +func (txn *LazyTxn) GetTableInfo(id int64) *model.TableInfo { return txn.Transaction.GetTableInfo(id) } // CacheTableInfo caches the index name. -func (txn *TxnState) CacheTableInfo(id int64, info *model.TableInfo) { +func (txn *LazyTxn) CacheTableInfo(id int64, info *model.TableInfo) { txn.Transaction.CacheTableInfo(id, info) } -func (txn *TxnState) init() { +func (txn *LazyTxn) init() { txn.mutations = make(map[int64]*binlog.TableMutation) + atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) + atomic.StoreUint64(&txn.EntriesCount, 0) + atomic.StoreUint64(&txn.EntriesSize, 0) } -func (txn *TxnState) initStmtBuf() { +func (txn *LazyTxn) initStmtBuf() { if txn.Transaction == nil { return } @@ -81,14 +101,14 @@ func (txn *TxnState) initStmtBuf() { } // countHint is estimated count of mutations. -func (txn *TxnState) countHint() int { +func (txn *LazyTxn) countHint() int { if txn.stagingHandle == kv.InvalidStagingHandle { return 0 } return txn.Transaction.GetMemBuffer().Len() - txn.initCnt } -func (txn *TxnState) flushStmtBuf() { +func (txn *LazyTxn) flushStmtBuf() { if txn.stagingHandle == kv.InvalidStagingHandle { return } @@ -97,17 +117,19 @@ func (txn *TxnState) flushStmtBuf() { txn.initCnt = buf.Len() } -func (txn *TxnState) cleanupStmtBuf() { +func (txn *LazyTxn) cleanupStmtBuf() { if txn.stagingHandle == kv.InvalidStagingHandle { return } buf := txn.Transaction.GetMemBuffer() buf.Cleanup(txn.stagingHandle) txn.initCnt = buf.Len() + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) } // Size implements the MemBuffer interface. -func (txn *TxnState) Size() int { +func (txn *LazyTxn) Size() int { if txn.Transaction == nil { return 0 } @@ -115,19 +137,19 @@ func (txn *TxnState) Size() int { } // Valid implements the kv.Transaction interface. -func (txn *TxnState) Valid() bool { +func (txn *LazyTxn) Valid() bool { return txn.Transaction != nil && txn.Transaction.Valid() } -func (txn *TxnState) pending() bool { +func (txn *LazyTxn) pending() bool { return txn.Transaction == nil && txn.txnFuture != nil } -func (txn *TxnState) validOrPending() bool { +func (txn *LazyTxn) validOrPending() bool { return txn.txnFuture != nil || txn.Valid() } -func (txn *TxnState) String() string { +func (txn *LazyTxn) String() string { if txn.Transaction != nil { return txn.Transaction.String() } @@ -138,7 +160,7 @@ func (txn *TxnState) String() string { } // GoString implements the "%#v" format for fmt.Printf. -func (txn *TxnState) GoString() string { +func (txn *LazyTxn) GoString() string { var s strings.Builder s.WriteString("Txn{") if txn.pending() { @@ -157,18 +179,25 @@ func (txn *TxnState) GoString() string { return s.String() } -func (txn *TxnState) changeInvalidToValid(kvTxn kv.Transaction) { +func (txn *LazyTxn) changeInvalidToValid(kvTxn kv.Transaction) { txn.Transaction = kvTxn + atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) + atomic.StoreUint64(&txn.infoStartTS, kvTxn.StartTS()) txn.initStmtBuf() + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) txn.txnFuture = nil } -func (txn *TxnState) changeInvalidToPending(future *txnFuture) { +func (txn *LazyTxn) changeInvalidToPending(future *txnFuture) { txn.Transaction = nil txn.txnFuture = future + atomic.StoreUint64(&txn.infoStartTS, 0) + atomic.StoreUint64(&txn.EntriesCount, uint64(0)) + atomic.StoreUint64(&txn.EntriesSize, uint64(0)) } -func (txn *TxnState) changePendingToValid(ctx context.Context) error { +func (txn *LazyTxn) changePendingToValid(ctx context.Context) error { if txn.txnFuture == nil { return errors.New("transaction future is not set") } @@ -183,17 +212,24 @@ func (txn *TxnState) changePendingToValid(ctx context.Context) error { return err } txn.Transaction = t + atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) + atomic.StoreUint64(&txn.infoStartTS, t.StartTS()) txn.initStmtBuf() + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) return nil } -func (txn *TxnState) changeToInvalid() { +func (txn *LazyTxn) changeToInvalid() { if txn.stagingHandle != kv.InvalidStagingHandle { txn.Transaction.GetMemBuffer().Cleanup(txn.stagingHandle) } txn.stagingHandle = kv.InvalidStagingHandle txn.Transaction = nil txn.txnFuture = nil + atomic.StoreUint64(&txn.infoStartTS, 0) + atomic.StoreUint64(&txn.EntriesCount, 0) + atomic.StoreUint64(&txn.EntriesSize, 0) } var hasMockAutoIncIDRetry = int64(0) @@ -223,7 +259,7 @@ func ResetMockAutoRandIDRetryCount(failTimes int64) { } // Commit overrides the Transaction interface. -func (txn *TxnState) Commit(ctx context.Context) error { +func (txn *LazyTxn) Commit(ctx context.Context) error { defer txn.reset() if len(txn.mutations) != 0 || txn.countHint() != 0 { logutil.BgLogger().Error("the code should never run here", @@ -233,6 +269,10 @@ func (txn *TxnState) Commit(ctx context.Context) error { return errors.Trace(kv.ErrInvalidTxn) } + atomic.StoreInt32(&txn.State, txninfo.TxnCommitting) + + failpoint.Inject("mockSlowCommit", func(_ failpoint.Value) {}) + // mockCommitError8942 is used for PR #8942. failpoint.Inject("mockCommitError8942", func(val failpoint.Value) { if val.(bool) { @@ -259,17 +299,34 @@ func (txn *TxnState) Commit(ctx context.Context) error { } // Rollback overrides the Transaction interface. -func (txn *TxnState) Rollback() error { +func (txn *LazyTxn) Rollback() error { defer txn.reset() + atomic.StoreInt32(&txn.State, txninfo.TxnRollingBack) + // mockSlowRollback is used to mock a rollback which takes a long time + failpoint.Inject("mockSlowRollback", func(_ failpoint.Value) {}) return txn.Transaction.Rollback() } -func (txn *TxnState) reset() { +// LockKeys Wrap the inner transaction's `LockKeys` to record the status +func (txn *LazyTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { + originState := atomic.LoadInt32(&txn.State) + atomic.StoreInt32(&txn.State, txninfo.TxnLockWaiting) + t := time.Now() + atomic.StorePointer(&txn.blockStartTime, unsafe.Pointer(&t)) + err := txn.Transaction.LockKeys(ctx, lockCtx, keys...) + atomic.StorePointer(&txn.blockStartTime, unsafe.Pointer(nil)) + atomic.StoreInt32(&txn.State, originState) + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) + return err +} + +func (txn *LazyTxn) reset() { txn.cleanup() txn.changeToInvalid() } -func (txn *TxnState) cleanup() { +func (txn *LazyTxn) cleanup() { txn.cleanupStmtBuf() txn.initStmtBuf() for key := range txn.mutations { @@ -278,7 +335,7 @@ func (txn *TxnState) cleanup() { } // KeysNeedToLock returns the keys need to be locked. -func (txn *TxnState) KeysNeedToLock() ([]kv.Key, error) { +func (txn *LazyTxn) KeysNeedToLock() ([]kv.Key, error) { if txn.stagingHandle == kv.InvalidStagingHandle { return nil, nil } @@ -316,6 +373,32 @@ func keyNeedToLock(k, v []byte, flags tikvstore.KeyFlags) bool { return !isNonUniqueIndex } +// Info dump the TxnState to Datum for displaying in `TIDB_TRX` +// This function is supposed to be thread safe +func (txn *LazyTxn) Info() *txninfo.TxnInfo { + startTs := atomic.LoadUint64(&txn.infoStartTS) + if startTs == 0 { + return nil + } + return &txninfo.TxnInfo{ + StartTS: startTs, + State: atomic.LoadInt32(&txn.State), + BlockStartTime: (*time.Time)(atomic.LoadPointer(&txn.blockStartTime)), + EntriesCount: atomic.LoadUint64(&txn.EntriesCount), + EntriesSize: atomic.LoadUint64(&txn.EntriesSize), + } +} + +// UpdateEntriesCountAndSize updates the EntriesCount and EntriesSize +// Note this function is not thread safe, because +// txn.Transaction can be changed during this function's execution if running parallel. +func (txn *LazyTxn) UpdateEntriesCountAndSize() { + if txn.Valid() { + atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) + } +} + func getBinlogMutation(ctx sessionctx.Context, tableID int64) *binlog.TableMutation { bin := binloginfo.GetPrewriteValue(ctx, true) for i := range bin.Mutations { diff --git a/session/txninfo/txn_info.go b/session/txninfo/txn_info.go new file mode 100644 index 0000000000000..77a2d8c90cd05 --- /dev/null +++ b/session/txninfo/txn_info.go @@ -0,0 +1,96 @@ +// Copyright 2021 PingCAP, Inc. + +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package txninfo + +import ( + "time" + + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/types" +) + +// TxnRunningState is the current state of a transaction +type TxnRunningState = int32 + +const ( + // TxnRunningNormal means the transaction is running normally + TxnRunningNormal TxnRunningState = iota + // TxnLockWaiting means the transaction is blocked on a lock + TxnLockWaiting + // TxnCommitting means the transaction is (at least trying to) committing + TxnCommitting + // TxnRollingBack means the transaction is rolling back + TxnRollingBack +) + +// TxnRunningStateStrs is the names of the TxnRunningStates +var TxnRunningStateStrs = []string{ + "Normal", "LockWaiting", "Committing", "RollingBack", +} + +// TxnInfo is information about a running transaction +// This is supposed to be the datasource of `TIDB_TRX` in infoschema +type TxnInfo struct { + StartTS uint64 + // digest of SQL current running + CurrentSQLDigest string + // current executing State + State TxnRunningState + // last trying to block start time + BlockStartTime *time.Time + // How many entries are in MemDB + EntriesCount uint64 + // MemDB used memory + EntriesSize uint64 + + // the following fields will be filled in `session` instead of `LazyTxn` + + // Which session this transaction belongs to + ConnectionID uint64 + // The user who open this session + Username string + // The schema this transaction works on + CurrentDB string +} + +// ToDatum Converts the `TxnInfo` to `Datum` to show in the `TIDB_TRX` table +func (info *TxnInfo) ToDatum() []types.Datum { + humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(info.StartTS)*1e6) + var blockStartTime interface{} + if info.BlockStartTime == nil { + blockStartTime = nil + } else { + blockStartTime = types.NewTime(types.FromGoTime(*info.BlockStartTime), mysql.TypeTimestamp, 0) + } + e, err := types.ParseEnumValue(TxnRunningStateStrs, uint64(info.State+1)) + if err != nil { + panic("this should never happen") + } + state := types.NewMysqlEnumDatum(e) + datums := types.MakeDatums( + info.StartTS, + types.NewTime(types.FromGoTime(humanReadableStartTime), mysql.TypeTimestamp, 0), + info.CurrentSQLDigest, + ) + datums = append(datums, state) + datums = append(datums, types.MakeDatums( + blockStartTime, + info.EntriesCount, + info.EntriesSize, + info.ConnectionID, + info.Username, + info.CurrentDB)...) + return datums +} diff --git a/util/processinfo.go b/util/processinfo.go index 29716d914c3de..ebbf17094b80d 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -22,6 +22,7 @@ import ( "time" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/execdetails" @@ -161,6 +162,7 @@ func serverStatus2Str(state uint16) string { // kill statement rely on this interface. type SessionManager interface { ShowProcessList() map[uint64]*ProcessInfo + ShowTxnList() []*txninfo.TxnInfo GetProcessInfo(id uint64) (*ProcessInfo, bool) Kill(connectionID uint64, query bool) KillAllConnections() From e7db5338101b44ba589fcd264c5247d40ca13882 Mon Sep 17 00:00:00 2001 From: mmyj Date: Wed, 12 May 2021 19:49:40 +0800 Subject: [PATCH 034/343] planner: ignore lock for temporary table of PointGet and BatchPointGet (#24540) --- planner/core/integration_test.go | 26 +++++++++++++++ planner/core/optimizer.go | 25 ++++++++++++++ planner/core/point_get_plan.go | 10 ++++-- .../core/testdata/integration_suite_in.json | 9 +++++ .../core/testdata/integration_suite_out.json | 33 +++++++++++++++++++ 5 files changed, 101 insertions(+), 2 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 98ab7b7898370..9e8eaa9204af9 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3832,3 +3832,29 @@ func (s *testIntegrationSerialSuite) TestEnforceMPP(c *C) { " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) } + +func (s *testIntegrationSuite) TestEliminateLockForTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test;") + tk.MustExec("create global temporary table t1 (a int primary key, b int, c int, index i_b(b)) on commit delete rows;") + defer func() { + tk.MustExec("drop global temporary table if exists t1;") + }() + tk.MustExec("begin;") + tk.MustExec("insert t1 values (8,8,9);") + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) + }) + tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 59c228767171a..0c1c4a668d3c8 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" @@ -186,6 +187,7 @@ func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan { plan = InjectExtraProjection(plan) mergeContinuousSelections(plan) plan = eliminateUnionScanAndLock(sctx, plan) + plan = eliminateLockForTemporaryTable(plan) plan = enableParallelApply(sctx, plan) return plan } @@ -322,6 +324,29 @@ func eliminateUnionScanAndLock(sctx sessionctx.Context, p PhysicalPlan) Physical }) } +// eliminateLockForTemporaryTable eliminates lock for the temporary table. +func eliminateLockForTemporaryTable(p PhysicalPlan) PhysicalPlan { + iteratePhysicalPlan(p, func(p PhysicalPlan) bool { + if len(p.Children()) > 1 { + return false + } + switch x := p.(type) { + case *PointGetPlan: + if x.TblInfo.TempTableType != model.TempTableNone { + x.Lock = false + x.LockWaitTime = 0 + } + case *BatchPointGetPlan: + if x.TblInfo.TempTableType != model.TempTableNone { + x.Lock = false + x.LockWaitTime = 0 + } + } + return true + }) + return p +} + func iteratePhysicalPlan(p PhysicalPlan, f func(p PhysicalPlan) bool) { if !f(p) { return diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index f7edbc1648819..fbc0bf9333a29 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -462,7 +462,10 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) { if tidbutil.IsMemDB(fp.dbName) { return nil } - fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) + // ignore lock for temporary table. + if fp.TblInfo.TempTableType == model.TempTableNone { + fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) + } p = fp return } @@ -480,7 +483,10 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) { p = tableDual.Init(ctx, &property.StatsInfo{}, 0) return } - fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) + // ignore lock for temporary table. + if fp.TblInfo.TempTableType == model.TempTableNone { + fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) + } p = fp return } diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 087b32110e18f..f386f8d7f24e8 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -294,5 +294,14 @@ "select sum(1) from s1", "select count(1) as cnt from s1 union select count(1) as cnt from s2" ] + }, + { + "name": "TestEliminateLockForTemporaryTable", + "cases": [ + "select * from t1 where a = 2 for update", + "select * from t1 where a in (1,2) for update", + "select c + 1 from t1 where a = 2 and c = 2 for update", + "select c + 1 from t1 where a in (1,2) and c = 2 for update" + ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 7c735fcb5657c..902ff19276cc0 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1564,5 +1564,38 @@ ] } ] + }, + { + "Name": "TestEliminateLockForTemporaryTable", + "Cases": [ + { + "SQL": "select * from t1 where a = 2 for update", + "Plan": [ + "Point_Get 1.00 root table:t1 handle:2" + ] + }, + { + "SQL": "select * from t1 where a in (1,2) for update", + "Plan": [ + "Batch_Point_Get 2.00 root table:t1 handle:[1 2], keep order:false, desc:false" + ] + }, + { + "SQL": "select c + 1 from t1 where a = 2 and c = 2 for update", + "Plan": [ + "Projection 0.00 root plus(test.t1.c, 1)->Column#4", + "└─Selection 0.00 root eq(test.t1.c, 2)", + " └─Point_Get 1.00 root table:t1 handle:2" + ] + }, + { + "SQL": "select c + 1 from t1 where a in (1,2) and c = 2 for update", + "Plan": [ + "Projection 0.00 root plus(test.t1.c, 1)->Column#4", + "└─Selection 0.00 root eq(test.t1.c, 2)", + " └─Batch_Point_Get 2.00 root table:t1 handle:[1 2], keep order:false, desc:false" + ] + } + ] } ] From 9527fa38dbc5f11d43bce462dbdab219ff9749e2 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 12 May 2021 20:01:40 +0800 Subject: [PATCH 035/343] store/tikv: remove use of ReplicaRead transaction option in store/tikv (#24409) --- store/driver/txn/snapshot.go | 2 ++ store/driver/txn/txn_driver.go | 2 ++ store/tikv/snapshot.go | 15 +++++++-------- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 6064db02cdeba..ff8f4b79cc918 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -73,6 +73,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetNotFillCache(val.(bool)) case tikvstore.SnapshotTS: s.KVSnapshot.SetSnapshotTS(val.(uint64)) + case tikvstore.ReplicaRead: + s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) case tikvstore.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) default: diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 739b983fd3d99..10eef3dbfd7d8 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -144,6 +144,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetPessimistic(val.(bool)) case tikvstore.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) + case tikvstore.ReplicaRead: + txn.KVTxn.GetSnapshot().SetReplicaRead(val.(tikvstore.ReplicaReadType)) case tikvstore.TaskID: txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case tikvstore.InfoSchema: diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index a37e1d8343c5e..d77be2c21ff35 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -565,10 +565,6 @@ func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) { // value of this option. Only ReplicaRead is supported for snapshot func (s *KVSnapshot) SetOption(opt int, val interface{}) { switch opt { - case kv.ReplicaRead: - s.mu.Lock() - s.mu.replicaRead = val.(kv.ReplicaReadType) - s.mu.Unlock() case kv.CollectRuntimeStats: s.mu.Lock() s.mu.stats = val.(*SnapshotRuntimeStats) @@ -589,10 +585,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { // DelOption deletes an option. func (s *KVSnapshot) DelOption(opt int) { switch opt { - case kv.ReplicaRead: - s.mu.Lock() - s.mu.replicaRead = kv.ReplicaReadLeader - s.mu.Unlock() case kv.CollectRuntimeStats: s.mu.Lock() s.mu.stats = nil @@ -611,6 +603,13 @@ func (s *KVSnapshot) SetKeyOnly(b bool) { s.keyOnly = b } +// SetReplicaRead sets up the replica read type. +func (s *KVSnapshot) SetReplicaRead(readType kv.ReplicaReadType) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.replicaRead = readType +} + // SetIsolationLevel sets the isolation level used to scan data from tikv. func (s *KVSnapshot) SetIsolationLevel(level IsoLevel) { s.isolationLevel = level From 6a69ee70ca00fc43b07c3fe33f4b216c5d931606 Mon Sep 17 00:00:00 2001 From: Shirly Date: Wed, 12 May 2021 20:41:40 +0800 Subject: [PATCH 036/343] store/driver: move error to single package (#24549) --- executor/point_get_test.go | 12 +- server/conn.go | 8 +- server/conn_stmt.go | 4 +- session/pessimistic_test.go | 42 +++---- store/copr/batch_coprocessor.go | 6 +- store/copr/coprocessor.go | 12 +- store/copr/mpp.go | 14 +-- store/copr/store.go | 12 +- store/driver/error/error.go | 158 ++++++++++++++++++++++++++ store/driver/tikv_driver.go | 11 +- store/driver/txn/error.go | 141 +---------------------- store/driver/txn/snapshot.go | 5 +- store/driver/txn/txn_driver.go | 11 +- store/driver/txn/unionstore_driver.go | 23 ++-- 14 files changed, 242 insertions(+), 217 deletions(-) create mode 100644 store/driver/error/error.go diff --git a/executor/point_get_test.go b/executor/point_get_test.go index 846b6f1628fe1..f66446a6bef83 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/tablecodec" @@ -536,15 +536,15 @@ func (s *testPointGetSuite) TestSelectCheckVisibility(c *C) { c.Assert(expectErr.Equal(err), IsTrue) } // Test point get. - checkSelectResultError("select * from t where a='1'", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t where a='1'", storeerr.ErrGCTooEarly) // Test batch point get. - checkSelectResultError("select * from t where a in ('1','2')", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t where a in ('1','2')", storeerr.ErrGCTooEarly) // Test Index look up read. - checkSelectResultError("select * from t where b > 0 ", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t where b > 0 ", storeerr.ErrGCTooEarly) // Test Index read. - checkSelectResultError("select b from t where b > 0 ", txndriver.ErrGCTooEarly) + checkSelectResultError("select b from t where b > 0 ", storeerr.ErrGCTooEarly) // Test table read. - checkSelectResultError("select * from t", txndriver.ErrGCTooEarly) + checkSelectResultError("select * from t", storeerr.ErrGCTooEarly) } func (s *testPointGetSuite) TestReturnValues(c *C) { diff --git a/server/conn.go b/server/conn.go index 29c87bd0dfd86..78cdd1a46c12d 100644 --- a/server/conn.go +++ b/server/conn.go @@ -76,7 +76,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/arena" @@ -1569,7 +1569,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { retryable, err = cc.handleStmt(ctx, stmt, parserWarns, i == len(stmts)-1) if err != nil { _, allowTiFlashFallback := cc.ctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] - if allowTiFlashFallback && errors.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) && retryable { + if allowTiFlashFallback && errors.ErrorEqual(err, storeerr.ErrTiFlashServerTimeout) && retryable { // When the TiFlash server seems down, we append a warning to remind the user to check the status of the TiFlash // server and fallback to TiKV. warns := append(parserWarns, stmtctx.SQLWarn{Level: stmtctx.WarnLevelError, Err: err}) @@ -1870,10 +1870,10 @@ func (cc *clientConn) writeChunks(ctx context.Context, rs ResultSet, binary bool failpoint.Inject("fetchNextErr", func(value failpoint.Value) { switch value.(string) { case "firstNext": - failpoint.Return(firstNext, txndriver.ErrTiFlashServerTimeout) + failpoint.Return(firstNext, storeerr.ErrTiFlashServerTimeout) case "secondNext": if !firstNext { - failpoint.Return(firstNext, txndriver.ErrTiFlashServerTimeout) + failpoint.Return(firstNext, storeerr.ErrTiFlashServerTimeout) } } }) diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 242b0df80fc83..e9f56306d9800 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -50,7 +50,7 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" @@ -198,7 +198,7 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e ctx = context.WithValue(ctx, util.ExecDetailsKey, &util.ExecDetails{}) retryable, err := cc.executePreparedStmtAndWriteResult(ctx, stmt, args, useCursor) _, allowTiFlashFallback := cc.ctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash] - if allowTiFlashFallback && err != nil && errors.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) && retryable { + if allowTiFlashFallback && err != nil && errors.ErrorEqual(err, storeerr.ErrTiFlashServerTimeout) && retryable { // When the TiFlash server seems down, we append a warning to remind the user to check the status of the TiFlash // server and fallback to TiKV. prevErr := err diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 83f0057384aea..72853d86208a9 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -31,7 +31,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" - txndriver "github.com/pingcap/tidb/store/driver/txn" + storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" @@ -611,7 +611,7 @@ func (s *testPessimisticSuite) TestWaitLockKill(c *C) { _, err := tk2.Exec("update test_kill set c = c + 1 where id = 1") wg.Done() c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrQueryInterrupted), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrQueryInterrupted), IsTrue) tk.MustExec("rollback") } @@ -733,10 +733,10 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { timeoutErr := <-timeoutErrCh c.Assert(timeoutErr, NotNil) - c.Assert(timeoutErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Assert(timeoutErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) timeoutErr = <-timeoutErrCh c.Assert(timeoutErr, NotNil) - c.Assert(timeoutErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Assert(timeoutErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) // tk4 lock c1 = 2 tk4.MustExec("begin pessimistic") @@ -749,7 +749,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { _, err := tk2.Exec("delete from tk where c1 = 2") c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big - c.Check(err.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(err.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) tk4.MustExec("commit") @@ -767,7 +767,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeout(c *C) { _, err = tk2.Exec("delete from tk where c1 = 3") // tk2 tries to lock c1 = 3 fail, this delete should be rollback, but previous update should be keeped c.Check(time.Since(start), GreaterEqual, 1000*time.Millisecond) c.Check(time.Since(start), Less, 3000*time.Millisecond) // unit test diff should not be too big - c.Check(err.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(err.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) tk2.MustExec("commit") tk3.MustExec("commit") @@ -841,7 +841,7 @@ func (s *testPessimisticSuite) TestInnodbLockWaitTimeoutWaitStart(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/PessimisticLockErrWriteConflict"), IsNil) waitErr := <-done c.Assert(waitErr, NotNil) - c.Check(waitErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Check(duration, GreaterEqual, 1000*time.Millisecond) c.Check(duration, LessEqual, 3000*time.Millisecond) tk2.MustExec("rollback") @@ -1131,11 +1131,11 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { tk1.MustExec("begin pessimistic") err := tk1.ExecToErr("select * from t where k = 2 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 4 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 7 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk1.MustExec("rollback") @@ -1147,9 +1147,9 @@ func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { tk1.MustExec("begin pessimistic") err = tk1.ExecToErr("select * from t where k = 2 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) err = tk1.ExecToErr("select * from t where k = 6 for update nowait") - c.Check(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Check(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk1.MustExec("rollback") } @@ -1279,10 +1279,10 @@ func (s *testPessimisticSuite) TestBatchPointGetLockIndex(c *C) { tk2.MustExec("begin pessimistic") err := tk2.ExecToErr("insert into t1 values(2, 2, 2)") c.Assert(err, NotNil) - c.Assert(txndriver.ErrLockWaitTimeout.Equal(err), IsTrue) + c.Assert(storeerr.ErrLockWaitTimeout.Equal(err), IsTrue) err = tk2.ExecToErr("select * from t1 where c2 = 3 for update nowait") c.Assert(err, NotNil) - c.Assert(txndriver.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + c.Assert(storeerr.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) tk.MustExec("rollback") tk2.MustExec("rollback") } @@ -1429,12 +1429,12 @@ func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { tk2.MustExec("begin pessimistic") err := tk2.ExecToErr("select * from tu where z = 3 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("begin pessimistic") tk.MustExec("insert into tu(x, y) values(2, 2);") err = tk2.ExecToErr("select * from tu where z = 4 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) // test batch point get lock tk.MustExec("begin pessimistic") @@ -1443,12 +1443,12 @@ func (s *testPessimisticSuite) TestGenerateColPointGet(c *C) { tk2.MustExec("begin pessimistic") err = tk2.ExecToErr("select x from tu where z in (3, 7, 9) for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("begin pessimistic") tk.MustExec("insert into tu(x, y) values(5, 6);") err = tk2.ExecToErr("select * from tu where z = 11 for update nowait") c.Assert(err, NotNil) - c.Assert(terror.ErrorEqual(err, txndriver.ErrLockAcquireFailAndNoWaitSet), IsTrue) + c.Assert(terror.ErrorEqual(err, storeerr.ErrLockAcquireFailAndNoWaitSet), IsTrue) tk.MustExec("commit") tk2.MustExec("commit") @@ -1996,11 +1996,11 @@ func (s *testPessimisticSuite) TestSelectForUpdateWaitSeconds(c *C) { waitErr2 := <-errCh waitErr3 := <-errCh c.Assert(waitErr, NotNil) - c.Check(waitErr.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Assert(waitErr2, NotNil) - c.Check(waitErr2.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr2.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Assert(waitErr3, NotNil) - c.Check(waitErr3.Error(), Equals, txndriver.ErrLockWaitTimeout.Error()) + c.Check(waitErr3.Error(), Equals, storeerr.ErrLockWaitTimeout.Error()) c.Assert(time.Since(start).Seconds(), Less, 45.0) tk2.MustExec("commit") tk3.MustExec("rollback") diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index af224c59fc38a..1e6cdc3cf09bb 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -261,7 +261,7 @@ func (b *batchCopIterator) recvFromRespCh(ctx context.Context) (resp *batchCopRe return case <-ticker.C: if atomic.LoadUint32(b.vars.Killed) == 1 { - resp = &batchCopResponse{err: txndriver.ErrQueryInterrupted} + resp = &batchCopResponse{err: derr.ErrQueryInterrupted} ok = true return } @@ -387,7 +387,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b } else { logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } - return txndriver.ErrTiFlashServerTimeout + return derr.ErrTiFlashServerTimeout } } } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 5e7eab303e84f..2c1e2d361af76 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -35,7 +35,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" tidbmetrics "github.com/pingcap/tidb/metrics" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -476,7 +476,7 @@ func (it *copIterator) recvFromRespCh(ctx context.Context, respCh <-chan *copRes return case <-ticker.C: if atomic.LoadUint32(it.vars.Killed) == 1 { - resp = &copResponse{err: txndriver.ErrQueryInterrupted} + resp = &copResponse{err: derr.ErrQueryInterrupted} ok = true return } @@ -717,7 +717,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *backoffer, task *copTask, ch ops = append(ops, tikv.WithMatchLabels(worker.req.MatchStoreLabels)) } resp, rpcCtx, storeAddr, err := worker.kvclient.SendReqCtx(bo.TiKVBackoffer(), req, task.region, tikv.ReadTimeoutMedium, getEndPointType(task.storeType), task.storeAddr, ops...) - err = txndriver.ToTiDBErr(err) + err = derr.ToTiDBErr(err) if err != nil { if task.storeType == kv.TiDB { err = worker.handleTiDBSendReqErr(err, task, ch) @@ -874,7 +874,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *backoffer, rpcCtx *tikv.R logutil.BgLogger().Debug("coprocessor encounters", zap.Stringer("lock", lockErr)) msBeforeExpired, err1 := worker.kvclient.ResolveLocks(bo.TiKVBackoffer(), worker.req.StartTs, []*tikv.Lock{tikv.NewLock(lockErr)}) - err1 = txndriver.ToTiDBErr(err1) + err1 = derr.ToTiDBErr(err1) if err1 != nil { return nil, errors.Trace(err1) } @@ -982,11 +982,11 @@ type CopRuntimeStats struct { func (worker *copIteratorWorker) handleTiDBSendReqErr(err error, task *copTask, ch chan<- *copResponse) error { errCode := errno.ErrUnknown errMsg := err.Error() - if terror.ErrorEqual(err, txndriver.ErrTiKVServerTimeout) { + if terror.ErrorEqual(err, derr.ErrTiKVServerTimeout) { errCode = errno.ErrTiKVServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } - if terror.ErrorEqual(err, txndriver.ErrTiFlashServerTimeout) { + if terror.ErrorEqual(err, derr.ErrTiFlashServerTimeout) { errCode = errno.ErrTiFlashServerTimeout errMsg = "TiDB server timeout, address is " + task.storeAddr } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 10784912faa9b..377e439a9392c 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/tidb/kv" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -225,7 +225,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req if sender.GetRPCError() != nil { logutil.BgLogger().Error("mpp dispatch meet io error", zap.String("error", sender.GetRPCError().Error())) // we return timeout to trigger tikv's fallback - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } } else { @@ -235,7 +235,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req if err != nil { logutil.BgLogger().Error("mpp dispatch meet error", zap.String("error", err.Error())) // we return timeout to trigger tikv's fallback - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } @@ -255,7 +255,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req failpoint.Inject("mppNonRootTaskError", func(val failpoint.Value) { if val.(bool) && !req.IsRoot { time.Sleep(1 * time.Second) - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } }) @@ -318,7 +318,7 @@ func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchReques if err != nil { logutil.BgLogger().Error("establish mpp connection meet error", zap.String("error", err.Error())) // we return timeout to trigger tikv's fallback - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } @@ -350,7 +350,7 @@ func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchReques logutil.BgLogger().Info("stream unknown error", zap.Error(err)) } } - m.sendError(txndriver.ErrTiFlashServerTimeout) + m.sendError(derr.ErrTiFlashServerTimeout) return } } @@ -405,7 +405,7 @@ func (m *mppIterator) nextImpl(ctx context.Context) (resp *mppResponse, ok bool, return case <-ticker.C: if m.vars != nil && m.vars.Killed != nil && atomic.LoadUint32(m.vars.Killed) == 1 { - err = txndriver.ErrQueryInterrupted + err = derr.ErrQueryInterrupted exit = true return } diff --git a/store/copr/store.go b/store/copr/store.go index 2cc10ee7bad38..d3f132f85238f 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" - txndriver "github.com/pingcap/tidb/store/driver/txn" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -39,7 +39,7 @@ func (s *kvStore) GetRegionCache() *tikv.RegionCache { // CheckVisibility checks if it is safe to read using given ts. func (s *kvStore) CheckVisibility(startTime uint64) error { err := s.store.CheckVisibility(startTime) - return txndriver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } // GetTiKVClient gets the client instance. @@ -54,13 +54,13 @@ type tikvClient struct { func (c *tikvClient) Close() error { err := c.c.Close() - return txndriver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } // SendRequest sends Request. func (c *tikvClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { res, err := c.c.SendRequest(ctx, addr, req, timeout) - return res, txndriver.ToTiDBErr(err) + return res, derr.ToTiDBErr(err) } // Store wraps tikv.KVStore and provides coprocessor utilities. @@ -147,14 +147,14 @@ func (b *backoffer) TiKVBackoffer() *tikv.Backoffer { // It returns a retryable error if total sleep time exceeds maxSleep. func (b *backoffer) Backoff(typ tikv.BackoffType, err error) error { e := b.b.Backoff(typ, err) - return txndriver.ToTiDBErr(e) + return derr.ToTiDBErr(e) } // BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message // and never sleep more than maxSleepMs for each sleep. func (b *backoffer) BackoffWithMaxSleep(typ tikv.BackoffType, maxSleepMs int, err error) error { e := b.b.BackoffWithMaxSleep(typ, maxSleepMs, err) - return txndriver.ToTiDBErr(e) + return derr.ToTiDBErr(e) } // GetBackoffTimes returns a map contains backoff time count by type. diff --git a/store/driver/error/error.go b/store/driver/error/error.go new file mode 100644 index 0000000000000..17da8f7ef2fa3 --- /dev/null +++ b/store/driver/error/error.go @@ -0,0 +1,158 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package error + +import ( + "github.com/pingcap/errors" + "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/kv" + tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/util/dbterror" +) + +// tikv error instance +var ( + // ErrTokenLimit is the error that token is up to the limit. + ErrTokenLimit = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStoreLimit) + // ErrTiKVServerTimeout is the error when tikv server is timeout. + ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) + ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerTimeout) + // ErrGCTooEarly is the error that GC life time is shorter than transaction duration + ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) + // ErrTiKVStaleCommand is the error that the command is stale in tikv. + ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) + // ErrQueryInterrupted is the error when the query is interrupted. + ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(errno.ErrQueryInterrupted) + // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. + ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) + // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. + ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(errno.ErrLockAcquireFailAndNoWaitSet) + ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) + // ErrLockWaitTimeout is the error that wait for the lock is timeout. + ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(errno.ErrLockWaitTimeout) + // ErrTiKVServerBusy is the error when tikv server is busy. + ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) + // ErrTiFlashServerBusy is the error that tiflash server is busy. + ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerBusy) + // ErrPDServerTimeout is the error when pd server is timeout. + ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) + // ErrRegionUnavailable is the error when region is not available. + ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) + // ErrUnknown is the unknow error. + ErrUnknown = dbterror.ClassTiKV.NewStd(errno.ErrUnknown) +) + +// Registers error returned from TiKV. +var ( + _ = dbterror.ClassTiKV.NewStd(errno.ErrDataOutOfRange) + _ = dbterror.ClassTiKV.NewStd(errno.ErrTruncatedWrongValue) + _ = dbterror.ClassTiKV.NewStd(errno.ErrDivisionByZero) +) + +// ToTiDBErr checks and converts a tikv error to a tidb error. +func ToTiDBErr(err error) error { + originErr := err + if err == nil { + return nil + } + err = errors.Cause(err) + if tikverr.IsErrNotFound(err) { + return kv.ErrNotExist + } + + if e, ok := err.(*tikverr.ErrWriteConflictInLatch); ok { + return kv.ErrWriteConflictInTiDB.FastGenByArgs(e.StartTS) + } + + if e, ok := err.(*tikverr.ErrTxnTooLarge); ok { + return kv.ErrTxnTooLarge.GenWithStackByArgs(e.Size) + } + + if errors.ErrorEqual(err, tikverr.ErrCannotSetNilValue) { + return kv.ErrCannotSetNilValue + } + + if e, ok := err.(*tikverr.ErrEntryTooLarge); ok { + return kv.ErrEntryTooLarge.GenWithStackByArgs(e.Limit, e.Size) + } + + if errors.ErrorEqual(err, tikverr.ErrInvalidTxn) { + return kv.ErrInvalidTxn + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVServerTimeout) { + return ErrTiKVServerTimeout + } + + if e, ok := err.(*tikverr.ErrPDServerTimeout); ok { + if len(e.Error()) == 0 { + return ErrPDServerTimeout + } + return ErrPDServerTimeout.GenWithStackByArgs(e.Error()) + } + + if errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) { + return ErrTiFlashServerTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrQueryInterrupted) { + return ErrQueryInterrupted + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { + return ErrTiKVServerBusy + } + + if errors.ErrorEqual(err, tikverr.ErrTiFlashServerBusy) { + return ErrTiFlashServerBusy + } + + if e, ok := err.(*tikverr.ErrGCTooEarly); ok { + return ErrGCTooEarly.GenWithStackByArgs(e.TxnStartTS, e.GCSafePoint) + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVStaleCommand) { + return ErrTiKVStaleCommand + } + + if errors.ErrorEqual(err, tikverr.ErrTiKVMaxTimestampNotSynced) { + return ErrTiKVMaxTimestampNotSynced + } + + if errors.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet) { + return ErrLockAcquireFailAndNoWaitSet + } + + if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { + return ErrResolveLockTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrLockWaitTimeout) { + return ErrLockWaitTimeout + } + + if errors.ErrorEqual(err, tikverr.ErrRegionUnavailable) { + return ErrRegionUnavailable + } + + if e, ok := err.(*tikverr.ErrTokenLimit); ok { + return ErrTokenLimit.GenWithStackByArgs(e.StoreID) + } + + if errors.ErrorEqual(err, tikverr.ErrUnknown) { + return ErrUnknown + } + + return errors.Trace(originErr) +} diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index cb14736844e68..398be99520aa6 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/copr" + derr "github.com/pingcap/tidb/store/driver/error" txn_driver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/store/tikv" @@ -261,7 +262,7 @@ func (s *tikvStore) StartGCWorker() error { gcWorker, err := gcworker.NewGCWorker(s, s.pdClient) if err != nil { - return txn_driver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } gcWorker.Start() s.gcWorker = gcWorker @@ -286,7 +287,7 @@ func (s *tikvStore) Close() error { } s.coprStore.Close() err := s.KVStore.Close() - return txn_driver.ToTiDBErr(err) + return derr.ToTiDBErr(err) } // GetMemCache return memory manager of the storage @@ -298,7 +299,7 @@ func (s *tikvStore) GetMemCache() kv.MemManager { func (s *tikvStore) Begin() (kv.Transaction, error) { txn, err := s.KVStore.Begin() if err != nil { - return nil, txn_driver.ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return txn_driver.NewTiKVTxn(txn), err } @@ -307,7 +308,7 @@ func (s *tikvStore) Begin() (kv.Transaction, error) { func (s *tikvStore) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { txn, err := s.KVStore.BeginWithOption(option) if err != nil { - return nil, txn_driver.ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return txn_driver.NewTiKVTxn(txn), err } @@ -321,7 +322,7 @@ func (s *tikvStore) GetSnapshot(ver kv.Version) kv.Snapshot { // CurrentVersion returns current max committed version with the given txnScope (local or global). func (s *tikvStore) CurrentVersion(txnScope string) (kv.Version, error) { ver, err := s.KVStore.CurrentTimestamp(txnScope) - return kv.NewVersion(ver), txn_driver.ToTiDBErr(err) + return kv.NewVersion(ver), derr.ToTiDBErr(err) } // ShowStatus returns the specified status of the storage diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 4c8e770c44ff7..39931357567be 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -25,56 +25,16 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/dbterror" "go.uber.org/zap" ) -// tikv error instance -var ( - // ErrTokenLimit is the error that token is up to the limit. - ErrTokenLimit = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStoreLimit) - // ErrTiKVServerTimeout is the error when tikv server is timeout. - ErrTiKVServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerTimeout) - ErrTiFlashServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerTimeout) - // ErrGCTooEarly is the error that GC life time is shorter than transaction duration - ErrGCTooEarly = dbterror.ClassTiKV.NewStd(errno.ErrGCTooEarly) - // ErrTiKVStaleCommand is the error that the command is stale in tikv. - ErrTiKVStaleCommand = dbterror.ClassTiKV.NewStd(errno.ErrTiKVStaleCommand) - // ErrQueryInterrupted is the error when the query is interrupted. - ErrQueryInterrupted = dbterror.ClassTiKV.NewStd(errno.ErrQueryInterrupted) - // ErrTiKVMaxTimestampNotSynced is the error that tikv's max timestamp is not synced. - ErrTiKVMaxTimestampNotSynced = dbterror.ClassTiKV.NewStd(errno.ErrTiKVMaxTimestampNotSynced) - // ErrLockAcquireFailAndNoWaitSet is the error that acquire the lock failed while no wait is setted. - ErrLockAcquireFailAndNoWaitSet = dbterror.ClassTiKV.NewStd(errno.ErrLockAcquireFailAndNoWaitSet) - ErrResolveLockTimeout = dbterror.ClassTiKV.NewStd(errno.ErrResolveLockTimeout) - // ErrLockWaitTimeout is the error that wait for the lock is timeout. - ErrLockWaitTimeout = dbterror.ClassTiKV.NewStd(errno.ErrLockWaitTimeout) - // ErrTiKVServerBusy is the error when tikv server is busy. - ErrTiKVServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiKVServerBusy) - // ErrTiFlashServerBusy is the error that tiflash server is busy. - ErrTiFlashServerBusy = dbterror.ClassTiKV.NewStd(errno.ErrTiFlashServerBusy) - // ErrPDServerTimeout is the error when pd server is timeout. - ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) - // ErrRegionUnavailable is the error when region is not available. - ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) - // ErrUnknown is the unknow error. - ErrUnknown = dbterror.ClassTiKV.NewStd(errno.ErrUnknown) -) - -// Registers error returned from TiKV. -var ( - _ = dbterror.ClassTiKV.NewStd(errno.ErrDataOutOfRange) - _ = dbterror.ClassTiKV.NewStd(errno.ErrTruncatedWrongValue) - _ = dbterror.ClassTiKV.NewStd(errno.ErrDivisionByZero) -) - func genKeyExistsError(name string, value string, err error) error { if err != nil { logutil.BgLogger().Info("extractKeyExistsErr meets error", zap.Error(err)) @@ -186,104 +146,7 @@ func extractKeyErr(err error) error { notFoundDetail := prettyLockNotFoundKey(e.Retryable) return kv.ErrTxnRetryable.GenWithStackByArgs(e.Retryable + " " + notFoundDetail) } - return ToTiDBErr(err) -} - -// ToTiDBErr checks and converts a tikv error to a tidb error. -func ToTiDBErr(err error) error { - originErr := err - if err == nil { - return nil - } - err = errors.Cause(err) - if tikverr.IsErrNotFound(err) { - return kv.ErrNotExist - } - - if e, ok := err.(*tikverr.ErrWriteConflictInLatch); ok { - return kv.ErrWriteConflictInTiDB.FastGenByArgs(e.StartTS) - } - - if e, ok := err.(*tikverr.ErrTxnTooLarge); ok { - return kv.ErrTxnTooLarge.GenWithStackByArgs(e.Size) - } - - if errors.ErrorEqual(err, tikverr.ErrCannotSetNilValue) { - return kv.ErrCannotSetNilValue - } - - if e, ok := err.(*tikverr.ErrEntryTooLarge); ok { - return kv.ErrEntryTooLarge.GenWithStackByArgs(e.Limit, e.Size) - } - - if errors.ErrorEqual(err, tikverr.ErrInvalidTxn) { - return kv.ErrInvalidTxn - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVServerTimeout) { - return ErrTiKVServerTimeout - } - - if e, ok := err.(*tikverr.ErrPDServerTimeout); ok { - if len(e.Error()) == 0 { - return ErrPDServerTimeout - } - return ErrPDServerTimeout.GenWithStackByArgs(e.Error()) - } - - if errors.ErrorEqual(err, tikverr.ErrTiFlashServerTimeout) { - return ErrTiFlashServerTimeout - } - - if errors.ErrorEqual(err, tikverr.ErrQueryInterrupted) { - return ErrQueryInterrupted - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVServerBusy) { - return ErrTiKVServerBusy - } - - if errors.ErrorEqual(err, tikverr.ErrTiFlashServerBusy) { - return ErrTiFlashServerBusy - } - - if e, ok := err.(*tikverr.ErrGCTooEarly); ok { - return ErrGCTooEarly.GenWithStackByArgs(e.TxnStartTS, e.GCSafePoint) - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVStaleCommand) { - return ErrTiKVStaleCommand - } - - if errors.ErrorEqual(err, tikverr.ErrTiKVMaxTimestampNotSynced) { - return ErrTiKVMaxTimestampNotSynced - } - - if errors.ErrorEqual(err, tikverr.ErrLockAcquireFailAndNoWaitSet) { - return ErrLockAcquireFailAndNoWaitSet - } - - if errors.ErrorEqual(err, tikverr.ErrResolveLockTimeout) { - return ErrResolveLockTimeout - } - - if errors.ErrorEqual(err, tikverr.ErrLockWaitTimeout) { - return ErrLockWaitTimeout - } - - if errors.ErrorEqual(err, tikverr.ErrRegionUnavailable) { - return ErrRegionUnavailable - } - - if e, ok := err.(*tikverr.ErrTokenLimit); ok { - return ErrTokenLimit.GenWithStackByArgs(e.StoreID) - } - - if errors.ErrorEqual(err, tikverr.ErrUnknown) { - return ErrUnknown - } - - return errors.Trace(originErr) + return derr.ToTiDBErr(err) } func newWriteConflictError(conflict *kvrpcpb.WriteConflict) error { diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index ff8f4b79cc918..fde6e93662fa8 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -18,6 +18,7 @@ import ( "unsafe" "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" ) @@ -48,7 +49,7 @@ func (s *tikvSnapshot) Get(ctx context.Context, k kv.Key) ([]byte, error) { func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { scanner, err := s.KVSnapshot.Iter(k, upperBound) if err != nil { - return nil, ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return &tikvScanner{scanner.(*tikv.Scanner)}, err } @@ -57,7 +58,7 @@ func (s *tikvSnapshot) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { func (s *tikvSnapshot) IterReverse(k kv.Key) (kv.Iterator, error) { scanner, err := s.KVSnapshot.IterReverse(k) if err != nil { - return nil, ToTiDBErr(err) + return nil, derr.ToTiDBErr(err) } return &tikvScanner{scanner.(*tikv.Scanner)}, err } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 10eef3dbfd7d8..10186d45756bb 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/binloginfo" + derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -75,7 +76,7 @@ func (txn *tikvTxn) GetSnapshot() kv.Snapshot { // The Iterator must be Closed after use. func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := txn.KVTxn.Iter(k, upperBound) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -84,7 +85,7 @@ func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (txn *tikvTxn) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := txn.KVTxn.IterReverse(k) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } // BatchGet gets kv from the memory buffer of statement and transaction, and the kv storage. @@ -101,17 +102,17 @@ func (txn *tikvTxn) BatchGet(ctx context.Context, keys []kv.Key) (map[string][]b func (txn *tikvTxn) Delete(k kv.Key) error { err := txn.KVTxn.Delete(k) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (txn *tikvTxn) Get(ctx context.Context, k kv.Key) ([]byte, error) { data, err := txn.KVTxn.Get(ctx, k) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (txn *tikvTxn) Set(k kv.Key, v []byte) error { err := txn.KVTxn.Set(k, v) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (txn *tikvTxn) GetMemBuffer() kv.MemBuffer { diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index 9db2325a0148f..5a2f56bfe4233 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -17,6 +17,7 @@ import ( "context" "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/unionstore" ) @@ -39,17 +40,17 @@ func (m *memBuffer) Delete(k kv.Key) error { func (m *memBuffer) DeleteWithFlags(k kv.Key, ops ...tikvstore.FlagsOp) error { err := m.MemDB.DeleteWithFlags(k, ops...) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (m *memBuffer) Get(_ context.Context, key kv.Key) ([]byte, error) { data, err := m.MemDB.Get(key) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (m *memBuffer) GetFlags(key kv.Key) (tikvstore.KeyFlags, error) { data, err := m.MemDB.GetFlags(key) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (m *memBuffer) Staging() kv.StagingHandle { @@ -73,12 +74,12 @@ func (m *memBuffer) InspectStage(handle kv.StagingHandle, f func(kv.Key, tikvsto func (m *memBuffer) Set(key kv.Key, value []byte) error { err := m.MemDB.Set(key, value) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) error { err := m.MemDB.SetWithFlags(key, value, ops...) - return ToTiDBErr(err) + return derr.ToTiDBErr(err) } // Iter creates an Iterator positioned on the first entry that k <= entry's key. @@ -87,7 +88,7 @@ func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) er // The Iterator must be Closed after use. func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := m.MemDB.Iter(k, upperBound) - return &tikvIterator{Iterator: it}, ToTiDBErr(err) + return &tikvIterator{Iterator: it}, derr.ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -96,7 +97,7 @@ func (m *memBuffer) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { // TODO: Add lower bound limit func (m *memBuffer) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := m.MemDB.IterReverse(k) - return &tikvIterator{Iterator: it}, ToTiDBErr(err) + return &tikvIterator{Iterator: it}, derr.ToTiDBErr(err) } // SnapshotIter returns a Iterator for a snapshot of MemBuffer. @@ -121,7 +122,7 @@ func (u *tikvUnionStore) GetMemBuffer() kv.MemBuffer { func (u *tikvUnionStore) Get(ctx context.Context, k kv.Key) ([]byte, error) { data, err := u.KVUnionStore.Get(ctx, k) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } func (u *tikvUnionStore) HasPresumeKeyNotExists(k kv.Key) bool { @@ -134,7 +135,7 @@ func (u *tikvUnionStore) UnmarkPresumeKeyNotExists(k kv.Key) { func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { it, err := u.KVUnionStore.Iter(k, upperBound) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. @@ -143,7 +144,7 @@ func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) // TODO: Add lower bound limit func (u *tikvUnionStore) IterReverse(k kv.Key) (kv.Iterator, error) { it, err := u.KVUnionStore.IterReverse(k) - return newKVIterator(it), ToTiDBErr(err) + return newKVIterator(it), derr.ToTiDBErr(err) } type tikvGetter struct { @@ -156,7 +157,7 @@ func newKVGetter(getter unionstore.Getter) kv.Getter { func (g *tikvGetter) Get(_ context.Context, k kv.Key) ([]byte, error) { data, err := g.Getter.Get(k) - return data, ToTiDBErr(err) + return data, derr.ToTiDBErr(err) } // tikvIterator wraps unionstore.Iterator as kv.Iterator From 89c0e6c91a780b894648e042147aafa0b0c1a5e5 Mon Sep 17 00:00:00 2001 From: Howie Date: Wed, 12 May 2021 21:49:40 +0800 Subject: [PATCH 037/343] ddl: add check table compatibility for temporary table (#24501) --- errors.toml | 5 +++++ executor/admin_test.go | 16 ++++++++++++++++ executor/ddl.go | 8 ++++++-- expression/integration_test.go | 2 ++ infoschema/error.go | 2 ++ planner/core/preprocess.go | 27 +++++++++++++++++++++++++++ 6 files changed, 58 insertions(+), 2 deletions(-) diff --git a/errors.toml b/errors.toml index 0ce61654373fb..a54913fa1bd2c 100644 --- a/errors.toml +++ b/errors.toml @@ -1281,6 +1281,11 @@ error = ''' Unknown SEQUENCE: '%-.300s' ''' +["schema:8003"] +error = ''' +TiDB admin check table failed. +''' + ["schema:8020"] error = ''' Table '%s' was locked in %s by %v diff --git a/executor/admin_test.go b/executor/admin_test.go index c9cda897a4745..20095eb59a0ba 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -76,6 +76,22 @@ func (s *testSuite5) TestAdminCheckIndex(c *C) { check() } +func (s *testSuite5) TestAdminCheckIndexInTemporaryMode(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists temporary_admin_test;") + tk.MustExec("create global temporary table temporary_admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c1), unique key(c2)) ON COMMIT DELETE ROWS;") + tk.MustExec("insert temporary_admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);") + tk.MustGetErrCode("admin check table temporary_admin_test;", mysql.ErrAdminCheckTable) + tk.MustExec("drop table if exists temporary_admin_test;") + + tk.MustExec("drop table if exists non_temporary_admin_test;") + tk.MustExec("create table non_temporary_admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c1), unique key(c2));") + tk.MustExec("insert non_temporary_admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);") + tk.MustExec("admin check table non_temporary_admin_test;") + tk.MustExec("drop table if exists non_temporary_admin_test;") +} + func (s *testSuite5) TestAdminRecoverIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/ddl.go b/executor/ddl.go index 81f7221d1e60e..2f10555d21e1e 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -311,8 +311,12 @@ func (e *DDLExec) dropTableObject(objects []*ast.TableName, obt objectType, ifEx if isSystemTable(tn.Schema.L, tn.Name.L) { return errors.Errorf("Drop tidb system table '%s.%s' is forbidden", tn.Schema.L, tn.Name.L) } - - if obt == tableObject && config.CheckTableBeforeDrop { + tableInfo, err := e.is.TableByName(tn.Schema, tn.Name) + if err != nil { + return err + } + tempTableType := tableInfo.Meta().TempTableType + if obt == tableObject && config.CheckTableBeforeDrop && tempTableType == model.TempTableNone { logutil.BgLogger().Warn("admin check table before drop", zap.String("database", fullti.Schema.O), zap.String("table", fullti.Name.O), diff --git a/expression/integration_test.go b/expression/integration_test.go index f15dc5822be15..a3d983069cce9 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9161,8 +9161,10 @@ func (s *testIntegrationSuite) TestIssue24429(c *C) { tk.MustExec("set @@sql_mode = ANSI_QUOTES;") tk.MustExec("use test") + tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int);") tk.MustQuery(`select t."a"=10 from t;`).Check(testkit.Rows()) + tk.MustExec("drop table if exists t;") } func (s *testIntegrationSuite) TestVitessHash(c *C) { diff --git a/infoschema/error.go b/infoschema/error.go index a0ef7ab9c8760..cb49e48419dec 100644 --- a/infoschema/error.go +++ b/infoschema/error.go @@ -69,4 +69,6 @@ var ( ErrTableLocked = dbterror.ClassSchema.NewStd(mysql.ErrTableLocked) // ErrWrongObject returns when the table/view/sequence is not the expected object. ErrWrongObject = dbterror.ClassSchema.NewStd(mysql.ErrWrongObject) + // ErrAdminCheckTable returns when the check table in temporary mode. + ErrAdminCheckTable = dbterror.ClassSchema.NewStd(mysql.ErrAdminCheckTable) ) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index a3719fe4c4b0b..b5caf55e8de03 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -125,6 +125,8 @@ type preprocessor struct { func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { switch node := in.(type) { + case *ast.AdminStmt: + p.checkAdminCheckTableGrammar(node) case *ast.DeleteStmt: p.stmtTp = TypeDelete case *ast.SelectStmt: @@ -557,6 +559,31 @@ func (p *preprocessor) checkDropDatabaseGrammar(stmt *ast.DropDatabaseStmt) { } } +func (p *preprocessor) checkAdminCheckTableGrammar(stmt *ast.AdminStmt) { + for _, table := range stmt.Tables { + currentDB := p.ctx.GetSessionVars().CurrentDB + if table.Schema.String() != "" { + currentDB = table.Schema.L + } + if currentDB == "" { + p.err = errors.Trace(ErrNoDB) + return + } + sName := model.NewCIStr(currentDB) + tName := table.Name + tableInfo, err := p.is.TableByName(sName, tName) + if err != nil { + p.err = err + return + } + tempTableType := tableInfo.Meta().TempTableType + if stmt.Tp == ast.AdminCheckTable && tempTableType != model.TempTableNone { + p.err = infoschema.ErrAdminCheckTable + return + } + } +} + func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { tName := stmt.Table.Name.String() if isIncorrectName(tName) { From 6228e22467d3233f14e490a890c427db1759d683 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 12 May 2021 22:13:40 +0800 Subject: [PATCH 038/343] store/tikv: remove use of IsStatenessReadOnly option in store/tikv (#24464) --- store/driver/txn/snapshot.go | 2 ++ store/driver/txn/txn_driver.go | 2 ++ store/tikv/snapshot.go | 11 +++++++---- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index fde6e93662fa8..95029978ed11e 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -78,6 +78,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) case tikvstore.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) + case tikvstore.IsStalenessReadOnly: + s.KVSnapshot.SetIsStatenessReadOnly(val.(bool)) default: s.KVSnapshot.SetOption(opt, val) } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 10186d45756bb..200f9e4e18a58 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -161,6 +161,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetEnable1PC(val.(bool)) case tikvstore.TxnScope: txn.SetScope(val.(string)) + case tikvstore.IsStalenessReadOnly: + txn.KVTxn.GetSnapshot().SetIsStatenessReadOnly(val.(bool)) default: txn.KVTxn.SetOption(opt, val) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index d77be2c21ff35..4e41aa609aad8 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -571,10 +571,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.mu.Unlock() case kv.SampleStep: s.sampleStep = val.(uint32) - case kv.IsStalenessReadOnly: - s.mu.Lock() - s.mu.isStaleness = val.(bool) - s.mu.Unlock() case kv.MatchStoreLabels: s.mu.Lock() s.mu.matchStoreLabels = val.([]*metapb.StoreLabel) @@ -628,6 +624,13 @@ func (s *KVSnapshot) SetTaskID(id uint64) { s.mu.taskID = id } +// SetIsStatenessReadOnly indicates whether the transaction is staleness read only transaction +func (s *KVSnapshot) SetIsStatenessReadOnly(b bool) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.isStaleness = b +} + // SnapCacheHitCount gets the snapshot cache hit count. Only for test. func (s *KVSnapshot) SnapCacheHitCount() int { return int(atomic.LoadInt64(&s.mu.hitCnt)) From b6b352cc57c11bea1daae414efbd110db3d0fe64 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 12 May 2021 22:39:40 +0800 Subject: [PATCH 039/343] store/tikv: change backoff type for missed tiflash peer. (#24577) --- store/copr/batch_coprocessor.go | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 1e6cdc3cf09bb..4bec370a9a4d5 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -126,8 +126,10 @@ func buildBatchCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.Key if err != nil { return nil, errors.Trace(err) } - // If the region is not found in cache, it must be out - // of date and already be cleaned up. We should retry and generate new tasks. + // When rpcCtx is nil, it's not only attributed to the miss region, but also + // some TiFlash stores crash and can't be recovered. + // That is not an error that can be easily recovered, so we regard this error + // same as rpc error. if rpcCtx == nil { needRetry = true logutil.BgLogger().Info("retry for TiFlash peer with region missing", zap.Uint64("region id", task.region.GetID())) @@ -147,8 +149,10 @@ func buildBatchCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.Key } } if needRetry { - // Backoff once for each retry. - err = bo.Backoff(tikv.BoRegionMiss, errors.New("Cannot find region with TiFlash peer")) + // As mentioned above, nil rpcCtx is always attributed to failed stores. + // It's equal to long poll the store but get no response. Here we'd better use + // TiFlash error to trigger the TiKV fallback mechanism. + err = bo.Backoff(tikv.BoTiFlashRPC, errors.New("Cannot find region with TiFlash peer")) if err != nil { return nil, errors.Trace(err) } From e5840626e5d5ce115030630917916c2901789781 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 12 May 2021 23:07:40 +0800 Subject: [PATCH 040/343] store/tikv: remove use of MatchStoreLabels transaction option in store/tikv (#24465) --- store/driver/txn/snapshot.go | 3 +++ store/driver/txn/txn_driver.go | 3 +++ store/tikv/snapshot.go | 11 +++++++---- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 95029978ed11e..6dc2e5c18465c 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -17,6 +17,7 @@ import ( "context" "unsafe" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" @@ -80,6 +81,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetTaskID(val.(uint64)) case tikvstore.IsStalenessReadOnly: s.KVSnapshot.SetIsStatenessReadOnly(val.(bool)) + case tikvstore.MatchStoreLabels: + s.KVSnapshot.SetMatchStoreLabels(val.([]*metapb.StoreLabel)) default: s.KVSnapshot.SetOption(opt, val) } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 200f9e4e18a58..d12a70dc2e2b5 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -19,6 +19,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/binloginfo" @@ -163,6 +164,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetScope(val.(string)) case tikvstore.IsStalenessReadOnly: txn.KVTxn.GetSnapshot().SetIsStatenessReadOnly(val.(bool)) + case tikvstore.MatchStoreLabels: + txn.KVTxn.GetSnapshot().SetMatchStoreLabels(val.([]*metapb.StoreLabel)) default: txn.KVTxn.SetOption(opt, val) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 4e41aa609aad8..d28ee7ed1fd6b 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -571,10 +571,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.mu.Unlock() case kv.SampleStep: s.sampleStep = val.(uint32) - case kv.MatchStoreLabels: - s.mu.Lock() - s.mu.matchStoreLabels = val.([]*metapb.StoreLabel) - s.mu.Unlock() } } @@ -631,6 +627,13 @@ func (s *KVSnapshot) SetIsStatenessReadOnly(b bool) { s.mu.isStaleness = b } +// SetMatchStoreLabels sets up labels to filter target stores. +func (s *KVSnapshot) SetMatchStoreLabels(labels []*metapb.StoreLabel) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.matchStoreLabels = labels +} + // SnapCacheHitCount gets the snapshot cache hit count. Only for test. func (s *KVSnapshot) SnapCacheHitCount() int { return int(atomic.LoadInt64(&s.mu.hitCnt)) From 771255e0100494e5ff34d8b5c5fcb6c92e0c4ad7 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Wed, 12 May 2021 23:45:40 +0800 Subject: [PATCH 041/343] executor, meta: Allocate auto id for global temporary tables (#24506) --- executor/insert_test.go | 117 +++++++++++++++++++++++++++++++++ sessionctx/variable/session.go | 24 ++++++- table/tables/tables.go | 73 ++++++++++++++++---- util/tableutil/tableutil.go | 40 +++++++++++ 4 files changed, 240 insertions(+), 14 deletions(-) create mode 100644 util/tableutil/tableutil.go diff --git a/executor/insert_test.go b/executor/insert_test.go index bee38e51c0fea..ffcfdc214bdb9 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -1590,3 +1590,120 @@ func (s *testSuite10) TestBinaryLiteralInsertToSet(c *C) { tk.MustExec("insert into bintest(h) values(0x61)") tk.MustQuery("select * from bintest").Check(testkit.Rows("a")) } + +var _ = SerialSuites(&testSuite13{&baseTestSuite{}}) + +type testSuite13 struct { + *baseTestSuite +} + +func (s *testSuite13) TestGlobalTempTableAutoInc(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec("drop table if exists temp_test") + tk.MustExec("create global temporary table temp_test(id int primary key auto_increment) on commit delete rows") + defer tk.MustExec("drop table if exists temp_test") + + // Data is cleared after transaction auto commits. + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select * from temp_test").Check(testkit.Rows()) + + // Data is not cleared inside a transaction. + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select * from temp_test").Check(testkit.Rows("1")) + tk.MustExec("commit") + + // AutoID allocator is cleared. + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select * from temp_test").Check(testkit.Rows("1")) + // Test whether auto-inc is incremental + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select id from temp_test order by id").Check(testkit.Rows("1", "2")) + tk.MustExec("commit") + + // multi-value insert + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0), (0)") + tk.MustQuery("select id from temp_test order by id").Check(testkit.Rows("1", "2")) + tk.MustExec("insert into temp_test(id) values(0), (0)") + tk.MustQuery("select id from temp_test order by id").Check(testkit.Rows("1", "2", "3", "4")) + tk.MustExec("commit") + + // rebase + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(10)") + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select id from temp_test order by id").Check(testkit.Rows("10", "11")) + tk.MustExec("insert into temp_test(id) values(20), (30)") + tk.MustExec("insert into temp_test(id) values(0), (0)") + tk.MustQuery("select id from temp_test order by id").Check(testkit.Rows("10", "11", "20", "30", "31", "32")) + tk.MustExec("commit") +} + +func (s *testSuite13) TestGlobalTempTableRowID(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec("drop table if exists temp_test") + tk.MustExec("create global temporary table temp_test(id int) on commit delete rows") + defer tk.MustExec("drop table if exists temp_test") + + // Data is cleared after transaction auto commits. + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select _tidb_rowid from temp_test").Check(testkit.Rows()) + + // Data is not cleared inside a transaction. + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select _tidb_rowid from temp_test").Check(testkit.Rows("1")) + tk.MustExec("commit") + + // AutoID allocator is cleared. + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select _tidb_rowid from temp_test").Check(testkit.Rows("1")) + // Test whether row id is incremental + tk.MustExec("insert into temp_test(id) values(0)") + tk.MustQuery("select _tidb_rowid from temp_test order by _tidb_rowid").Check(testkit.Rows("1", "2")) + tk.MustExec("commit") + + // multi-value insert + tk.MustExec("begin") + tk.MustExec("insert into temp_test(id) values(0), (0)") + tk.MustQuery("select _tidb_rowid from temp_test order by _tidb_rowid").Check(testkit.Rows("1", "2")) + tk.MustExec("insert into temp_test(id) values(0), (0)") + tk.MustQuery("select _tidb_rowid from temp_test order by _tidb_rowid").Check(testkit.Rows("1", "2", "3", "4")) + tk.MustExec("commit") +} + +func (s *testSuite13) TestGlobalTempTableParallel(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`use test`) + tk.MustExec("drop table if exists temp_test") + tk.MustExec("create global temporary table temp_test(id int primary key auto_increment) on commit delete rows") + defer tk.MustExec("drop table if exists temp_test") + + threads := 8 + loops := 1 + wg := sync.WaitGroup{} + wg.Add(threads) + + insertFunc := func() { + defer wg.Done() + newTk := testkit.NewTestKitWithInit(c, s.store) + newTk.MustExec("begin") + for i := 0; i < loops; i++ { + newTk.MustExec("insert temp_test value(0)") + newTk.MustExec("insert temp_test value(0), (0)") + } + maxID := strconv.Itoa(loops * 3) + newTk.MustQuery("select max(id) from temp_test").Check(testkit.Rows(maxID)) + newTk.MustExec("commit") + } + + for i := 0; i < threads; i++ { + go insertFunc() + } + wg.Wait() +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index d6bb5763e67d8..815f42cc03ef3 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" + "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" @@ -49,6 +50,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/stringutil" + "github.com/pingcap/tidb/util/tableutil" "github.com/pingcap/tidb/util/timeutil" "github.com/twmb/murmur3" atomic2 "go.uber.org/atomic" @@ -174,7 +176,9 @@ type TransactionContext struct { // TableDeltaMap lock to prevent potential data race tdmLock sync.Mutex - GlobalTemporaryTables map[int64]struct{} + // GlobalTemporaryTables is used to store transaction-specific information for global temporary tables. + // It can also be stored in sessionCtx with local temporary tables, but it's easier to clean this data after transaction ends. + GlobalTemporaryTables map[int64]tableutil.TempTable } // GetShard returns the shard prefix for the next `count` rowids. @@ -1456,6 +1460,24 @@ func (s *SessionVars) LazyCheckKeyNotExists() bool { return s.PresumeKeyNotExists || (s.TxnCtx.IsPessimistic && !s.StmtCtx.DupKeyAsWarning) } +// GetTemporaryTable returns a TempTable by tableInfo. +func (s *SessionVars) GetTemporaryTable(tblInfo *model.TableInfo) tableutil.TempTable { + if tblInfo.TempTableType == model.TempTableGlobal { + if s.TxnCtx.GlobalTemporaryTables == nil { + s.TxnCtx.GlobalTemporaryTables = make(map[int64]tableutil.TempTable) + } + globalTempTables := s.TxnCtx.GlobalTemporaryTables + globalTempTable, ok := globalTempTables[tblInfo.ID] + if !ok { + globalTempTable = tableutil.TempTableFromMeta(tblInfo) + globalTempTables[tblInfo.ID] = globalTempTable + } + return globalTempTable + } + // TODO: check local temporary tables + return nil +} + // special session variables. const ( SQLModeVar = "sql_mode" diff --git a/table/tables/tables.go b/table/tables/tables.go index 8fd3cca9e2657..74fd2d82f3ef9 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" @@ -46,6 +47,7 @@ import ( "github.com/pingcap/tidb/util/generatedexpr" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/stringutil" + "github.com/pingcap/tidb/util/tableutil" "github.com/pingcap/tipb/go-binlog" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" @@ -322,8 +324,8 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context, sh := memBuffer.Staging() defer memBuffer.Cleanup(sh) - if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { - addTemporaryTableID(sctx, meta.ID) + if m := t.Meta(); m.TempTableType == model.TempTableGlobal { + addTemporaryTable(sctx, m) } var colIDs, binlogColIDs []int64 @@ -588,12 +590,9 @@ func TryGetCommonPkColumns(tbl table.Table) []*table.Column { return pkCols } -func addTemporaryTableID(sctx sessionctx.Context, id int64) { - txnCtx := sctx.GetSessionVars().TxnCtx - if txnCtx.GlobalTemporaryTables == nil { - txnCtx.GlobalTemporaryTables = make(map[int64]struct{}) - } - txnCtx.GlobalTemporaryTables[id] = struct{}{} +func addTemporaryTable(sctx sessionctx.Context, tblInfo *model.TableInfo) { + tempTable := sctx.GetSessionVars().GetTemporaryTable(tblInfo) + tempTable.SetModified(true) } // AddRecord implements table.Table AddRecord interface. @@ -608,8 +607,8 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts . fn.ApplyOn(&opt) } - if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { - addTemporaryTableID(sctx, meta.ID) + if m := t.Meta(); m.TempTableType == model.TempTableGlobal { + addTemporaryTable(sctx, m) } var ctx context.Context @@ -1010,8 +1009,8 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type return err } - if meta := t.Meta(); meta.TempTableType == model.TempTableGlobal { - addTemporaryTableID(ctx, meta.ID) + if m := t.Meta(); m.TempTableType == model.TempTableGlobal { + addTemporaryTable(ctx, m) } // The table has non-public column and this column is doing the operation of "modify/change column". @@ -1370,7 +1369,14 @@ func OverflowShardBits(recordID int64, shardRowIDBits uint64, typeBitsLength uin // Allocators implements table.Table Allocators interface. func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { - if ctx == nil || ctx.GetSessionVars().IDAllocator == nil { + if ctx == nil { + return t.allocs + } else if ctx.GetSessionVars().IDAllocator == nil { + // Use an independent allocator for global temporary tables. + if t.meta.TempTableType == model.TempTableGlobal { + alloc := ctx.GetSessionVars().GetTemporaryTable(t.meta).GetAutoIDAllocator() + return autoid.Allocators{alloc} + } return t.allocs } @@ -1498,6 +1504,7 @@ func getDuplicateErrorHandleString(t table.Table, handle kv.Handle, row []types. func init() { table.TableFromMeta = TableFromMeta table.MockTableFromMeta = MockTableFromMeta + tableutil.TempTableFromMeta = TempTableFromMeta } // sequenceCommon cache the sequence value. @@ -1763,3 +1770,43 @@ func BuildTableScanFromInfos(tableInfo *model.TableInfo, columnInfos []*model.Co } return tsExec } + +// TemporaryTable is used to store transaction-specific or session-specific information for global / local temporary tables. +// For example, stats and autoID should have their own copies of data, instead of being shared by all sessions. +type TemporaryTable struct { + // Whether it's modified in this transaction. + modified bool + // The stats of this table. So far it's always pseudo stats. + stats *statistics.Table + // The autoID allocator of this table. + autoIDAllocator autoid.Allocator +} + +// TempTableFromMeta builds a TempTable from model.TableInfo. +func TempTableFromMeta(tblInfo *model.TableInfo) tableutil.TempTable { + return &TemporaryTable{ + modified: false, + stats: statistics.PseudoTable(tblInfo), + autoIDAllocator: autoid.NewAllocatorFromTempTblInfo(tblInfo), + } +} + +// GetAutoIDAllocator is implemented from TempTable.GetAutoIDAllocator. +func (t *TemporaryTable) GetAutoIDAllocator() autoid.Allocator { + return t.autoIDAllocator +} + +// SetModified is implemented from TempTable.SetModified. +func (t *TemporaryTable) SetModified(modified bool) { + t.modified = modified +} + +// GetModified is implemented from TempTable.GetModified. +func (t *TemporaryTable) GetModified() bool { + return t.modified +} + +// GetStats is implemented from TempTable.GetStats. +func (t *TemporaryTable) GetStats() interface{} { + return t.stats +} diff --git a/util/tableutil/tableutil.go b/util/tableutil/tableutil.go new file mode 100644 index 0000000000000..11cbe626dcc56 --- /dev/null +++ b/util/tableutil/tableutil.go @@ -0,0 +1,40 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tableutil + +import ( + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/meta/autoid" +) + +// TempTable is used to store transaction-specific or session-specific information for global / local temporary tables. +// For example, stats and autoID should have their own copies of data, instead of being shared by all sessions. +type TempTable interface { + // GetAutoIDAllocator gets the autoID allocator of this table. + GetAutoIDAllocator() autoid.Allocator + + // SetModified sets that the table is modified. + SetModified(bool) + + // GetModified queries whether the table is modified. + GetModified() bool + + // The stats of this table (*statistics.Table). + // Define the return type as interface{} here to avoid cycle imports. + GetStats() interface{} +} + +// TempTableFromMeta builds a TempTable from *model.TableInfo. +// Currently, it is assigned to tables.TempTableFromMeta in tidb package's init function. +var TempTableFromMeta func(tblInfo *model.TableInfo) TempTable From d54e8841a130a6c1d2b0f5ad13d13d4590f1d55b Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 00:27:40 +0800 Subject: [PATCH 042/343] store/tikv: remove use of SampleStep option in store/tikv (#24461) --- store/driver/txn/snapshot.go | 2 ++ store/driver/txn/txn_driver.go | 2 ++ store/tikv/snapshot.go | 7 +++++-- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 6dc2e5c18465c..ee1d1eeee29d8 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -77,6 +77,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetSnapshotTS(val.(uint64)) case tikvstore.ReplicaRead: s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) + case tikvstore.SampleStep: + s.KVSnapshot.SetSampleStep(val.(uint32)) case tikvstore.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) case tikvstore.IsStalenessReadOnly: diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index d12a70dc2e2b5..12f2c8233ccb1 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -154,6 +154,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetSchemaVer(val.(tikv.SchemaVer)) case tikvstore.SchemaAmender: txn.SetSchemaAmender(val.(tikv.SchemaAmender)) + case tikvstore.SampleStep: + txn.KVTxn.GetSnapshot().SetSampleStep(val.(uint32)) case tikvstore.CommitHook: txn.SetCommitCallback(val.(func(string, error))) case tikvstore.EnableAsyncCommit: diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index d28ee7ed1fd6b..ae65f15dc18e6 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -569,8 +569,6 @@ func (s *KVSnapshot) SetOption(opt int, val interface{}) { s.mu.Lock() s.mu.stats = val.(*SnapshotRuntimeStats) s.mu.Unlock() - case kv.SampleStep: - s.sampleStep = val.(uint32) } } @@ -607,6 +605,11 @@ func (s *KVSnapshot) SetIsolationLevel(level IsoLevel) { s.isolationLevel = level } +// SetSampleStep skips 'step - 1' number of keys after each returned key. +func (s *KVSnapshot) SetSampleStep(step uint32) { + s.sampleStep = step +} + // SetPriority sets the priority for tikv to execute commands. func (s *KVSnapshot) SetPriority(pri Priority) { s.priority = pri From 61131c6f1df18b1e696025f9f034940df0596142 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Thu, 13 May 2021 00:47:39 +0800 Subject: [PATCH 043/343] executor: add partition pruning tests for adding and dropping partition operations (#24573) --- executor/partition_table_test.go | 43 ++++++++++++++++++++++++++++---- util/testkit/testkit.go | 8 +++--- 2 files changed, 43 insertions(+), 8 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 0c9e63129ebf1..5be39c3a04d54 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -494,12 +494,45 @@ func (s *partitionTableSuite) TestDynamicPruneModeWithEqualExpression(c *C) { for _, t := range tests { for i := range t.partitions { sql := fmt.Sprintf(t.sql, tables[i]) - c.Assert(tk.MustPartition(sql, t.partitions[i]), IsTrue) - tk.MustQuery(sql).Sort().Check(tk.MustQuery(fmt.Sprintf(t.sql, "t")).Sort().Rows()) + tk.MustPartition(sql, t.partitions[i]).Sort().Check(tk.MustQuery(fmt.Sprintf(t.sql, "t")).Sort().Rows()) } } } +func (s *partitionTableSuite) TestAddDropPartitions(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_add_drop_partition") + tk.MustExec("use test_add_drop_partition") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table t(a int) partition by range(a) ( + partition p0 values less than (5), + partition p1 values less than (10), + partition p2 values less than (15))`) + tk.MustExec(`insert into t values (2), (7), (12)`) + tk.MustPartition(`select * from t where a < 3`, "p0").Sort().Check(testkit.Rows("2")) + tk.MustPartition(`select * from t where a < 8`, "p0,p1").Sort().Check(testkit.Rows("2", "7")) + tk.MustPartition(`select * from t where a < 20`, "all").Sort().Check(testkit.Rows("12", "2", "7")) + + // remove p0 + tk.MustExec(`alter table t drop partition p0`) + tk.MustPartition(`select * from t where a < 3`, "p1").Sort().Check(testkit.Rows()) + tk.MustPartition(`select * from t where a < 8`, "p1").Sort().Check(testkit.Rows("7")) + tk.MustPartition(`select * from t where a < 20`, "all").Sort().Check(testkit.Rows("12", "7")) + + // add 2 more partitions + tk.MustExec(`alter table t add partition (partition p3 values less than (20))`) + tk.MustExec(`alter table t add partition (partition p4 values less than (40))`) + tk.MustExec(`insert into t values (15), (25)`) + tk.MustPartition(`select * from t where a < 3`, "p1").Sort().Check(testkit.Rows()) + tk.MustPartition(`select * from t where a < 8`, "p1").Sort().Check(testkit.Rows("7")) + tk.MustPartition(`select * from t where a < 20`, "p1,p2,p3").Sort().Check(testkit.Rows("12", "15", "7")) +} + func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") @@ -511,15 +544,15 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") // list partition table - tk.MustExec(`create table tlist(a int, b int, index idx_a(a), index idx_b(b)) partition by list(a)( + tk.MustExec(`create table tlist(a int, b int, index idx_a(a), index idx_b(b)) partition by list(a)( partition p0 values in (1, 2, 3, 4), partition p1 values in (5, 6, 7, 8), partition p2 values in (9, 10, 11, 12));`) // range partition table tk.MustExec(`create table trange(a int, b int, index idx_a(a), index idx_b(b)) partition by range(a) ( - partition p0 values less than(300), - partition p1 values less than (500), + partition p0 values less than(300), + partition p1 values less than (500), partition p2 values less than(1100));`) // hash partition table diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 06eb826c56b78..d6d6e41bb6e9f 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -256,14 +256,16 @@ func (tk *TestKit) MustNoGlobalStats(table string) bool { } // MustPartition checks if the result execution plan must read specific partitions. -func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) bool { +func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) *Result { rs := tk.MustQuery("explain "+sql, args...) + ok := false for i := range rs.rows { if strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 { - return true + ok = true } } - return false + tk.c.Assert(ok, check.IsTrue) + return tk.MustQuery(sql, args...) } // MustUseIndex checks if the result execution plan contains specific index(es). From 75fcd6586ba6e26da1a63f62d9aff007b7a25284 Mon Sep 17 00:00:00 2001 From: Howie Date: Thu, 13 May 2021 02:19:39 +0800 Subject: [PATCH 044/343] ddl: forbid partition on temporary mode before put into queue (#24565) --- ddl/ddl_api.go | 5 ++++- ddl/partition.go | 7 +++++++ ddl/table.go | 5 ----- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b03b4ca66f536..e6d77c9e674e9 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1617,7 +1617,10 @@ func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.T return errors.Trace(err) } if err = checkAddPartitionTooManyPartitions(uint64(len(tbInfo.Partition.Definitions))); err != nil { - return errors.Trace(err) + return err + } + if err = checkAddPartitionOnTemporaryMode(tbInfo); err != nil { + return err } switch tbInfo.Partition.Type { diff --git a/ddl/partition.go b/ddl/partition.go index 4cc71eb1c8d74..0cafa9d2ff525 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -1476,6 +1476,13 @@ func checkAddPartitionTooManyPartitions(piDefs uint64) error { return nil } +func checkAddPartitionOnTemporaryMode(tbInfo *model.TableInfo) error { + if tbInfo.Partition != nil && tbInfo.TempTableType != model.TempTableNone { + return ErrPartitionNoTemporary + } + return nil +} + func checkNoHashPartitions(ctx sessionctx.Context, partitionNum uint64) error { if partitionNum == 0 { return ast.ErrNoParts.GenWithStackByArgs("partitions") diff --git a/ddl/table.go b/ddl/table.go index acd209a7bb0da..668de3ac41c05 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -56,11 +56,6 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) job.State = model.JobStateCancelled return ver, errors.Trace(err) } - if tbInfo.Partition != nil && (tbInfo.TempTableType == model.TempTableGlobal || tbInfo.TempTableType == model.TempTableLocal) { - // unsupported ddl, cancel this job. - job.State = model.JobStateCancelled - return ver, errors.Trace(ErrPartitionNoTemporary) - } tbInfo.State = model.StateNone err := checkTableNotExists(d, t, schemaID, tbInfo.Name.L) From f92df3c5b17cae2784c14863dca996c841ef5cd7 Mon Sep 17 00:00:00 2001 From: jianzhiyao <739319867@qq.com> Date: Thu, 13 May 2021 10:35:40 +0800 Subject: [PATCH 045/343] ddl: speedup test case TestIndexOnMultipleGeneratedColumn (#24487) --- ddl/db_integration_test.go | 42 ++++++++++++++++++++++++++++++++------ 1 file changed, 36 insertions(+), 6 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index fe6bca7dc4563..405ada57f15ec 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -969,9 +969,9 @@ func (s *testIntegrationSuite5) TestModifyColumnOption(c *C) { func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database if not exists test") tk.MustExec("use test") + tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int as (a + 1), c int as (b + 1))") tk.MustExec("insert into t (a) values (1)") @@ -980,42 +980,72 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { res := tk.MustQuery("select * from t use index(idx) where c > 1") tk.MustQuery("select * from t ignore index(idx) where c > 1").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn1(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int as (a + 1), c int as (b + 1), d int as (c + 1))") tk.MustExec("insert into t (a) values (1)") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 3 4")) - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn2(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a bigint, b decimal as (a+1), c varchar(20) as (b*2), d float as (a*23+b-1+length(c)))") tk.MustExec("insert into t (a) values (1)") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 4 25")) - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn3(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a varchar(10), b float as (length(a)+123), c varchar(20) as (right(a, 2)), d float as (b+b-7+1-3+3*ASCII(c)))") tk.MustExec("insert into t (a) values ('adorable')") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("adorable 131 le 577")) // 131+131-7+1-3+3*108 - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn4(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a bigint, b decimal as (a), c int(10) as (a+b), d float as (a+b+c), e decimal as (a+b+c+d))") tk.MustExec("insert into t (a) values (1)") tk.MustExec("create index idx on t (d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 1 2 4 8")) - res = tk.MustQuery("select * from t use index(idx) where d > 2") + res := tk.MustQuery("select * from t use index(idx) where d > 2") tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") +} + +func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn5(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test") + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a bigint, b bigint as (a+1) virtual, c bigint as (b+1) virtual)") @@ -1025,7 +1055,7 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { tk.MustExec("alter table t add column(d bigint as (c+1) virtual)") tk.MustExec("alter table t add index idx_d(d)") tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 3 4")) - res = tk.MustQuery("select * from t use index(idx_d) where d > 2") + res := tk.MustQuery("select * from t use index(idx_d) where d > 2") tk.MustQuery("select * from t ignore index(idx_d) where d > 2").Check(res.Rows()) tk.MustExec("admin check table t") } From dc5c2264234f6135cbdae32a6f8c6ea54331c2ba Mon Sep 17 00:00:00 2001 From: sylzd Date: Thu, 13 May 2021 11:15:39 +0800 Subject: [PATCH 046/343] execution: Fix issue 24439 Inconsistent error with MySQL for GRANT CREATE USER ON .* (#24485) --- errors.toml | 5 +++++ executor/errors.go | 1 + executor/grant.go | 6 ++++++ executor/grant_test.go | 4 ++++ go.mod | 2 +- go.sum | 4 ++-- 6 files changed, 19 insertions(+), 3 deletions(-) diff --git a/errors.toml b/errors.toml index a54913fa1bd2c..926823909f96e 100644 --- a/errors.toml +++ b/errors.toml @@ -501,6 +501,11 @@ error = ''' Deadlock found when trying to get lock; try restarting transaction ''' +["executor:1221"] +error = ''' +Incorrect usage of %s and %s +''' + ["executor:1242"] error = ''' Subquery returns more than 1 row diff --git a/executor/errors.go b/executor/errors.go index 7f3345659e4f9..ad8104a96e7ee 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -42,6 +42,7 @@ var ( ErrTableaccessDenied = dbterror.ClassExecutor.NewStd(mysql.ErrTableaccessDenied) ErrBadDB = dbterror.ClassExecutor.NewStd(mysql.ErrBadDB) ErrWrongObject = dbterror.ClassExecutor.NewStd(mysql.ErrWrongObject) + ErrWrongUsage = dbterror.ClassExecutor.NewStd(mysql.ErrWrongUsage) ErrRoleNotGranted = dbterror.ClassPrivilege.NewStd(mysql.ErrRoleNotGranted) ErrDeadlock = dbterror.ClassExecutor.NewStd(mysql.ErrLockDeadlock) ErrQueryInterrupted = dbterror.ClassExecutor.NewStd(mysql.ErrQueryInterrupted) diff --git a/executor/grant.go b/executor/grant.go index 6c715758b6c7f..86620cc0124b1 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -473,6 +473,12 @@ func (e *GrantExec) grantDBLevel(priv *ast.PrivElem, user *ast.UserSpec, interna if priv.Priv == mysql.UsagePriv { return nil } + for _, v := range mysql.StaticGlobalOnlyPrivs { + if v == priv.Priv { + return ErrWrongUsage.GenWithStackByArgs("DB GRANT", "GLOBAL PRIVILEGES") + } + } + dbName := e.Level.DBName if len(dbName) == 0 { dbName = e.ctx.GetSessionVars().CurrentDB diff --git a/executor/grant_test.go b/executor/grant_test.go index bb720b48b730b..13686494feddc 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -89,6 +89,10 @@ func (s *testSuite3) TestGrantDBScope(c *C) { sql := fmt.Sprintf("SELECT %s FROM mysql.DB WHERE User=\"testDB1\" and host=\"localhost\" and db=\"test\";", mysql.Priv2UserCol[v]) tk.MustQuery(sql).Check(testkit.Rows("Y")) } + + // Grant in wrong scope. + _, err := tk.Exec(` grant create user on test.* to 'testDB1'@'localhost';`) + c.Assert(terror.ErrorEqual(err, executor.ErrWrongUsage.GenWithStackByArgs("DB GRANT", "GLOBAL PRIVILEGES")), IsTrue) } func (s *testSuite3) TestWithGrantOption(c *C) { diff --git a/go.mod b/go.mod index cd8becf5a757c..b2bd664da13f3 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 - github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c + github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 diff --git a/go.sum b/go.sum index 4e1030039a04f..c42d129ae5b8b 100644 --- a/go.sum +++ b/go.sum @@ -443,8 +443,8 @@ github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 h1:ERrF0fTuIOnwfGbt71Ji3DKbOEaP189tjym50u8gpC8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c h1:GLFd+wBN7EsV6ad/tVGFCD37taOyzIMVs3SdiWZF18I= -github.com/pingcap/parser v0.0.0-20210508071014-cd9cd78e230c/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= +github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b h1:eLuDQ6eJCEKCbGwhGrkjzagwev1GJGU2Y2kFkAsBzV0= +github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 h1:A9KL9R+lWSVPH8IqUuH1QSTRJ5FGoY1bT2IcfPKsWD8= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= From d5a0e6a1d32c9e1a270f8b168c65db527c9643b4 Mon Sep 17 00:00:00 2001 From: jianzhiyao <739319867@qq.com> Date: Thu, 13 May 2021 11:31:39 +0800 Subject: [PATCH 047/343] *: fix errcheck (#24463) --- store/tikv/config/config_test.go | 13 ++++++------- types/json/binary_functions_test.go | 2 +- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/store/tikv/config/config_test.go b/store/tikv/config/config_test.go index f79f2d09c22c8..a47e1e7e5030b 100644 --- a/store/tikv/config/config_test.go +++ b/store/tikv/config/config_test.go @@ -34,20 +34,19 @@ func (s *testConfigSuite) TestParsePath(c *C) { } func (s *testConfigSuite) TestTxnScopeValue(c *C) { - - failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("bj")`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("bj")`), IsNil) isGlobal, v := GetTxnScopeFromConfig() c.Assert(isGlobal, IsFalse) c.Assert(v, Equals, "bj") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") - failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("")`) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("")`), IsNil) isGlobal, v = GetTxnScopeFromConfig() c.Assert(isGlobal, IsTrue) c.Assert(v, Equals, "global") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") - failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("global")`) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("global")`), IsNil) isGlobal, v = GetTxnScopeFromConfig() c.Assert(isGlobal, IsFalse) c.Assert(v, Equals, "global") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope"), IsNil) } diff --git a/types/json/binary_functions_test.go b/types/json/binary_functions_test.go index 8191638f7b2f6..12a9c8ece8ae2 100644 --- a/types/json/binary_functions_test.go +++ b/types/json/binary_functions_test.go @@ -35,6 +35,6 @@ func (s *testJSONFuncSuite) TestdecodeEscapedUnicode(c *C) { func BenchmarkDecodeEscapedUnicode(b *testing.B) { for i := 0; i < b.N; i++ { in := "597d" - decodeEscapedUnicode([]byte(in)) + _, _, _ = decodeEscapedUnicode([]byte(in)) } } From 68d0a25183f8f13ee0e010b0a07e2d3516232cae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Thu, 13 May 2021 11:55:39 +0800 Subject: [PATCH 048/343] test: make TestExtractStartTs stable (#24585) --- store/tikv/extract_start_ts_test.go | 74 +++++++++++++---------------- store/tikv/kv.go | 14 ++++++ 2 files changed, 47 insertions(+), 41 deletions(-) diff --git a/store/tikv/extract_start_ts_test.go b/store/tikv/extract_start_ts_test.go index a108a0f7e41cb..b392ca365cde8 100644 --- a/store/tikv/extract_start_ts_test.go +++ b/store/tikv/extract_start_ts_test.go @@ -14,9 +14,8 @@ package tikv import ( - "context" - . "github.com/pingcap/check" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore/unistore" @@ -28,7 +27,7 @@ type extractStartTsSuite struct { store *KVStore } -var _ = Suite(&extractStartTsSuite{}) +var _ = SerialSuites(&extractStartTsSuite{}) func (s *extractStartTsSuite) SetUpTest(c *C) { client, pdClient, cluster, err := unistore.New("") @@ -63,60 +62,53 @@ func (s *extractStartTsSuite) SetUpTest(c *C) { func (s *extractStartTsSuite) TestExtractStartTs(c *C) { i := uint64(100) - cases := []kv.TransactionOption{ + // to prevent time change during test case execution + // we use failpoint to make it "fixed" + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockStalenessTimestamp", "return(200)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockCurrentTimestamp", `return(300)`), IsNil) + + cases := []struct { + expectedTS uint64 + option kv.TransactionOption + }{ // StartTS setted - {TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}, + {100, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, // PrevSec setted - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}, + {200, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}}, // MinStartTS setted, global - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}, + {101, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, // MinStartTS setted, local - {TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}, + {102, kv.TransactionOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, // MaxPrevSec setted // however we need to add more cases to check the behavior when it fall backs to MinStartTS setted // see `TestMaxPrevSecFallback` - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + {200, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, // nothing setted - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}, - } - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - stalenessTimestamp, _ := s.store.getStalenessTimestamp(bo, oracle.GlobalTxnScope, 100) - expectedTs := []uint64{ - 100, - stalenessTimestamp, - - 101, - 102, - - stalenessTimestamp, - // it's too hard to figure out the value `getTimestampWithRetry` returns - // so we just check whether it is greater than stalenessTimestamp - 0, + {300, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, } - for i, cs := range cases { - expected := expectedTs[i] - result, _ := extractStartTs(s.store, cs) - if expected == 0 { - c.Assert(result, Greater, stalenessTimestamp) - } else { - c.Assert(result, Equals, expected) - } + for _, cs := range cases { + expected := cs.expectedTS + result, _ := extractStartTs(s.store, cs.option) + c.Assert(result, Equals, expected) } + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockStalenessTimestamp"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockCurrentTimestamp"), IsNil) } func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { s.store.setSafeTS(2, 0x8000000000000002) s.store.setSafeTS(3, 0x8000000000000001) - i := uint64(100) - cases := []kv.TransactionOption{ - {TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, - {TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}, + cases := []struct { + expectedTS uint64 + option kv.TransactionOption + }{ + {0x8000000000000001, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, + {0x8000000000000002, kv.TransactionOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, } - expectedTs := []uint64{0x8000000000000001, 0x8000000000000002} - for i, cs := range cases { - expected := expectedTs[i] - result, _ := extractStartTs(s.store, cs) - c.Assert(result, Equals, expected) + for _, cs := range cases { + result, _ := extractStartTs(s.store, cs.option) + c.Assert(result, Equals, cs.expectedTS) } } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index a487b0024e3e9..981a1b7bc5cab 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -235,6 +235,13 @@ func (s *KVStore) CurrentTimestamp(txnScope string) (uint64, error) { } func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, error) { + failpoint.Inject("MockCurrentTimestamp", func(val failpoint.Value) { + if v, ok := val.(int); ok { + failpoint.Return(uint64(v), nil) + } else { + panic("MockCurrentTimestamp should be a number, try use this failpoint with \"return(ts)\"") + } + }) if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("TiKVStore.getTimestampWithRetry", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -264,6 +271,13 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, } func (s *KVStore) getStalenessTimestamp(bo *Backoffer, txnScope string, prevSec uint64) (uint64, error) { + failpoint.Inject("MockStalenessTimestamp", func(val failpoint.Value) { + if v, ok := val.(int); ok { + failpoint.Return(uint64(v), nil) + } else { + panic("MockStalenessTimestamp should be a number, try use this failpoint with \"return(ts)\"") + } + }) for { startTS, err := s.oracle.GetStaleTimestamp(bo.ctx, txnScope, prevSec) if err == nil { From 5fd39d1ed0e4e5baeb7b759c24e140f7038fcff4 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Thu, 13 May 2021 12:09:39 +0800 Subject: [PATCH 049/343] ddl: forbid recover/flashback temporary tables (#24518) --- executor/ddl.go | 7 ++++ executor/executor_test.go | 71 ++++++++++++++++++++++----------------- 2 files changed, 48 insertions(+), 30 deletions(-) diff --git a/executor/ddl.go b/executor/ddl.go index 2f10555d21e1e..5058704ee2f9e 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -26,12 +26,14 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" @@ -563,6 +565,11 @@ func (e *DDLExec) getRecoverTableByTableName(tableName *ast.TableName) (*model.J if tableInfo == nil || jobInfo == nil { return nil, nil, errors.Errorf("Can't find dropped/truncated table: %v in DDL history jobs", tableName.Name) } + // Dropping local temporary tables won't appear in DDL jobs. + if tableInfo.TempTableType == model.TempTableGlobal { + msg := mysql.Message("Recover/flashback table is not supported on temporary tables", nil) + return nil, nil, dbterror.ClassDDL.NewStdErr(errno.ErrUnsupportedDDLOperation, msg) + } return jobInfo, tableInfo, nil } diff --git a/executor/executor_test.go b/executor/executor_test.go index e69b956f8d82b..3b168636606ed 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5697,24 +5697,15 @@ func (s *testRecoverTable) TearDownSuite(c *C) { s.dom.Close() } -func (s *testRecoverTable) TestRecoverTable(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange") - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("create database if not exists test_recover") - tk.MustExec("use test_recover") - tk.MustExec("drop table if exists t_recover") - tk.MustExec("create table t_recover (a int);") - defer func(originGC bool) { +func (s *testRecoverTable) mockGC(tk *testkit.TestKit) (string, string, string, func()) { + originGC := ddl.IsEmulatorGCEnable() + resetGC := func() { if originGC { ddl.EmulatorGCEnable() } else { ddl.EmulatorGCDisable() } - }(ddl.IsEmulatorGCEnable()) + } // disable emulator GC. // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. @@ -5727,6 +5718,23 @@ func (s *testRecoverTable) TestRecoverTable(c *C) { UPDATE variable_value = '%[1]s'` // clear GC variables first. tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )") + return timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC +} + +func (s *testRecoverTable) TestRecoverTable(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) + defer func() { + err := failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange") + c.Assert(err, IsNil) + }() + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test_recover") + tk.MustExec("use test_recover") + tk.MustExec("drop table if exists t_recover") + tk.MustExec("create table t_recover (a int);") + + timeBeforeDrop, timeAfterDrop, safePointSQL, resetGC := s.mockGC(tk) + defer resetGC() tk.MustExec("insert into t_recover values (1),(2),(3)") tk.MustExec("drop table t_recover") @@ -5819,24 +5827,10 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { tk.MustExec("use test_flashback") tk.MustExec("drop table if exists t_flashback") tk.MustExec("create table t_flashback (a int);") - defer func(originGC bool) { - if originGC { - ddl.EmulatorGCEnable() - } else { - ddl.EmulatorGCDisable() - } - }(ddl.IsEmulatorGCEnable()) - // Disable emulator GC. - // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. - ddl.EmulatorGCDisable() - gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) - safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') - ON DUPLICATE KEY - UPDATE variable_value = '%[1]s'` - // Clear GC variables first. - tk.MustExec("delete from mysql.tidb where variable_name in ( 'tikv_gc_safe_point','tikv_gc_enable' )") + timeBeforeDrop, _, safePointSQL, resetGC := s.mockGC(tk) + defer resetGC() + // Set GC safe point tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) // Set GC enable. @@ -5939,6 +5933,23 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { tk.MustQuery("select a from t order by a").Check(testkit.Rows("1", "2", "3")) } +func (s *testRecoverTable) TestRecoverTempTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database if not exists test_recover") + tk.MustExec("use test_recover") + tk.MustExec("drop table if exists t_recover") + tk.MustExec("create global temporary table t_recover (a int) on commit delete rows;") + + timeBeforeDrop, _, safePointSQL, resetGC := s.mockGC(tk) + defer resetGC() + // Set GC safe point + tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop)) + + tk.MustExec("drop table t_recover") + tk.MustGetErrCode("recover table t_recover;", errno.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("flashback table t_recover;", errno.ErrUnsupportedDDLOperation) +} + func (s *testSuiteP2) TestPointGetPreparedPlan(c *C) { tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("drop database if exists ps_text") From acf2e82b5227cea429347a3c9493214f209205f8 Mon Sep 17 00:00:00 2001 From: lysu Date: Thu, 13 May 2021 12:45:40 +0800 Subject: [PATCH 050/343] executor: fix point_get result on clustered index when new-row-format disabled but new-collation enabled (#24544) --- executor/point_get.go | 3 +++ session/clustered_index_test.go | 15 +++++++++++++++ 2 files changed, 18 insertions(+) diff --git a/executor/point_get.go b/executor/point_get.go index b0cd700c5c920..c34987b7f0c1d 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -531,6 +531,9 @@ func tryDecodeFromHandle(tblInfo *model.TableInfo, schemaColIdx int, col *expres chk.AppendInt64(schemaColIdx, handle.IntValue()) return true, nil } + if types.NeedRestoredData(col.RetType) { + return false, nil + } // Try to decode common handle. if mysql.HasPriKeyFlag(col.RetType.Flag) { for i, hid := range pkCols { diff --git a/session/clustered_index_test.go b/session/clustered_index_test.go index 0f79b1b13fc2e..b7e529f29fe0e 100644 --- a/session/clustered_index_test.go +++ b/session/clustered_index_test.go @@ -665,3 +665,18 @@ func (s *testClusteredSerialSuite) TestPrefixedClusteredIndexUniqueKeyWithNewCol tk.MustExec("admin check table t;") tk.MustExec("drop table t;") } + +func (s *testClusteredSerialSuite) TestClusteredIndexNewCollationWithOldRowFormat(c *C) { + // This case maybe not useful, because newCollation isn't convenience to run on TiKV(it's required serialSuit) + // but unistore doesn't support old row format. + defer collate.SetNewCollationEnabledForTest(false) + collate.SetNewCollationEnabledForTest(true) + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn + tk.Se.GetSessionVars().RowEncoder.Enable = false + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t2(col_1 varchar(132) CHARACTER SET utf8 COLLATE utf8_unicode_ci, primary key(col_1) clustered)") + tk.MustExec("insert into t2 select 'aBc'") + tk.MustQuery("select col_1 from t2 where col_1 = 'aBc'").Check(testkit.Rows("aBc")) +} From 2105317479e90af9282638b18a919262f220095a Mon Sep 17 00:00:00 2001 From: mmyj Date: Thu, 13 May 2021 12:57:39 +0800 Subject: [PATCH 051/343] executor: Improve the performance of appending not fixed columns (#20969) --- util/chunk/chunk.go | 4 ++- util/chunk/chunk_test.go | 77 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 80 insertions(+), 1 deletion(-) diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index a4350bd9628e3..e91cff2559d79 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -505,8 +505,10 @@ func (c *Chunk) Append(other *Chunk, begin, end int) { } else { beginOffset, endOffset := src.offsets[begin], src.offsets[end] dst.data = append(dst.data, src.data[beginOffset:endOffset]...) + lastOffset := dst.offsets[len(dst.offsets)-1] for i := begin; i < end; i++ { - dst.offsets = append(dst.offsets, dst.offsets[len(dst.offsets)-1]+src.offsets[i+1]-src.offsets[i]) + lastOffset += src.offsets[i+1] - src.offsets[i] + dst.offsets = append(dst.offsets, lastOffset) } } for i := begin; i < end; i++ { diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index 67222328794db..22a6ac5b473dc 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -1179,3 +1179,80 @@ func BenchmarkBatchAppendRows(b *testing.B) { }) } } + +func BenchmarkAppendRows(b *testing.B) { + b.ReportAllocs() + rowChk := newChunk(8, 8, 0, 0) + + for i := 0; i < 4096; i++ { + rowChk.AppendNull(0) + rowChk.AppendInt64(1, 1) + rowChk.AppendString(2, "abcd") + rowChk.AppendBytes(3, []byte("abcd")) + } + + type testCaseConf struct { + batchSize int + } + testCaseConfs := []testCaseConf{ + {batchSize: 2}, + {batchSize: 8}, + {batchSize: 16}, + {batchSize: 100}, + {batchSize: 1000}, + {batchSize: 4000}, + } + + chk := newChunk(8, 8, 0, 0) + for _, conf := range testCaseConfs { + b.ResetTimer() + b.Run(fmt.Sprintf("row-%d", conf.batchSize), func(b *testing.B) { + for i := 0; i < b.N; i++ { + chk.Reset() + for j := 0; j < conf.batchSize; j++ { + chk.AppendRow(rowChk.GetRow(j)) + } + } + }) + b.ResetTimer() + b.Run(fmt.Sprintf("column-%d", conf.batchSize), func(b *testing.B) { + for i := 0; i < b.N; i++ { + chk.Reset() + chk.Append(rowChk, 0, conf.batchSize) + } + }) + } +} + +func BenchmarkAppend(b *testing.B) { + b.ReportAllocs() + rowChk := newChunk(0, 0) + + for i := 0; i < 4096; i++ { + rowChk.AppendString(0, "abcd") + rowChk.AppendBytes(1, []byte("abcd")) + } + + type testCaseConf struct { + batchSize int + } + testCaseConfs := []testCaseConf{ + {batchSize: 2}, + {batchSize: 8}, + {batchSize: 16}, + {batchSize: 100}, + {batchSize: 1000}, + {batchSize: 4000}, + } + + chk := newChunk(0, 0) + for _, conf := range testCaseConfs { + b.ResetTimer() + b.Run(fmt.Sprintf("column-%d", conf.batchSize), func(b *testing.B) { + for i := 0; i < b.N; i++ { + chk.Reset() + chk.Append(rowChk, 0, conf.batchSize) + } + }) + } +} From c90f48d80d70d455e93cc606e3ef5162dbf566d6 Mon Sep 17 00:00:00 2001 From: Howie Date: Thu, 13 May 2021 13:07:39 +0800 Subject: [PATCH 052/343] *: typo fix (#24564) --- config/config.go | 6 +++--- tidb-server/main.go | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/config/config.go b/config/config.go index 7d6560b5d6783..664cb5bceaa7d 100644 --- a/config/config.go +++ b/config/config.go @@ -60,9 +60,9 @@ const ( DefHost = "0.0.0.0" // DefStatusHost is the default status host of TiDB DefStatusHost = "0.0.0.0" - // Def TableColumnCountLimit is limit of the number of columns in a table + // DefTableColumnCountLimit is limit of the number of columns in a table DefTableColumnCountLimit = 1017 - // Def TableColumnCountLimit is maximum limitation of the number of columns in a table + // DefMaxOfTableColumnCountLimit is maximum limitation of the number of columns in a table DefMaxOfTableColumnCountLimit = 4096 ) @@ -73,7 +73,7 @@ var ( "tikv": true, "unistore": true, } - // checkTableBeforeDrop enable to execute `admin check table` before `drop table`. + // CheckTableBeforeDrop enable to execute `admin check table` before `drop table`. CheckTableBeforeDrop = false // checkBeforeDropLDFlag is a go build flag. checkBeforeDropLDFlag = "None" diff --git a/tidb-server/main.go b/tidb-server/main.go index 3e2351bf7c352..6429ba960a0cb 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -629,7 +629,7 @@ func setupMetrics() { metrics.TimeJumpBackCounter.Inc() } callBackCount := 0 - sucessCallBack := func() { + successCallBack := func() { callBackCount++ // It is callback by monitor per second, we increase metrics.KeepAliveCounter per 5s. if callBackCount >= 5 { @@ -637,7 +637,7 @@ func setupMetrics() { metrics.KeepAliveCounter.Inc() } } - go systimemon.StartMonitor(time.Now, systimeErrHandler, sucessCallBack) + go systimemon.StartMonitor(time.Now, systimeErrHandler, successCallBack) pushMetric(cfg.Status.MetricsAddr, time.Duration(cfg.Status.MetricsInterval)*time.Second) } From 956149cdbb8359e618240c7f5568ba5af819eb57 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 13 May 2021 13:35:39 +0800 Subject: [PATCH 053/343] planner/core: refresh stale regions in cache for batch cop response (#24457) --- go.mod | 2 +- go.sum | 4 ++-- store/copr/batch_coprocessor.go | 14 ++++++++++++++ store/mockstore/unistore/tikv/server.go | 5 +++++ store/tikv/region_request_test.go | 4 ++++ 5 files changed, 26 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index b2bd664da13f3..c4b668a81dcf1 100644 --- a/go.mod +++ b/go.mod @@ -45,7 +45,7 @@ require ( github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e + github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 diff --git a/go.sum b/go.sum index c42d129ae5b8b..212917e414404 100644 --- a/go.sum +++ b/go.sum @@ -436,8 +436,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17Xtb github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e h1:oUMZ6X/Kpaoxfejh9/jQ+4UZ5xk9MRYcouWJ0oXRKNE= -github.com/pingcap/kvproto v0.0.0-20210429093846-65f54a202d7e/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c h1:cy87vgUJT0U4JuxC7R14PuwBrabI9fDawYhyKTbjOBQ= +github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 4bec370a9a4d5..bf8d1b7c893e0 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -406,6 +406,20 @@ func (b *batchCopIterator) handleBatchCopResponse(bo *backoffer, response *copro return errors.Trace(err) } + if len(response.RetryRegions) > 0 { + logutil.BgLogger().Info("multiple regions are stale and need to be refreshed", zap.Int("region size", len(response.RetryRegions))) + for idx, retry := range response.RetryRegions { + id := tikv.NewRegionVerID(retry.Id, retry.RegionEpoch.ConfVer, retry.RegionEpoch.Version) + logutil.BgLogger().Info("invalid region because tiflash detected stale region", zap.String("region id", id.String())) + b.store.GetRegionCache().InvalidateCachedRegionWithReason(id, tikv.EpochNotMatch) + if idx >= 10 { + logutil.BgLogger().Info("stale regions are too many, so we omit the rest ones") + break + } + } + return + } + resp := batchCopResponse{ pbResp: response, detail: new(CopRuntimeStats), diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index f571ff4fe963f..adf3049330897 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -845,6 +845,11 @@ func (svr *Server) ReadIndex(context.Context, *kvrpcpb.ReadIndexRequest) (*kvrpc return &kvrpcpb.ReadIndexResponse{}, nil } +// GetLockWaitInfo implements implements the tikvpb.TikvServer interface. +func (svr *Server) GetLockWaitInfo(ctx context.Context, _ *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + return &kvrpcpb.GetLockWaitInfoResponse{}, nil +} + // transaction debugger commands. // MvccGetByKey implements implements the tikvpb.TikvServer interface. diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 9c5172e52f372..8d531ee209a78 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -462,6 +462,10 @@ func (s *mockTikvGrpcServer) SplitRegion(context.Context, *kvrpcpb.SplitRegionRe return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + return nil, errors.New("unreachable") +} + func (s *mockTikvGrpcServer) CoprocessorStream(*coprocessor.Request, tikvpb.Tikv_CoprocessorStreamServer) error { return errors.New("unreachable") } From f2c2fbda476f71bd1341c2c35ba727d0256a4914 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Thu, 13 May 2021 14:13:39 +0800 Subject: [PATCH 054/343] binlog: DML on temporary tables do not write binlog (#24570) --- sessionctx/binloginfo/binloginfo_test.go | 33 ++++++++++++++++++++++++ table/tables/tables.go | 3 +++ 2 files changed, 36 insertions(+) diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 2dfca57d73f4c..b0d9ec91888e4 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -698,3 +698,36 @@ func testGetTableByName(c *C, ctx sessionctx.Context, db, table string) table.Ta c.Assert(err, IsNil) return tbl } + +func (s *testBinlogSuite) TestTempTableBinlog(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.Se.GetSessionVars().BinlogClient = s.client + tk.MustExec("begin") + tk.MustExec("drop table if exists temp_table") + ddlQuery := "create global temporary table temp_table(id int) on commit delete rows" + tk.MustExec(ddlQuery) + ok := mustGetDDLBinlog(s, ddlQuery, c) + c.Assert(ok, IsTrue) + + tk.MustExec("insert temp_table value(1)") + tk.MustExec("update temp_table set id=id+1") + tk.MustExec("commit") + prewriteVal := getLatestBinlogPrewriteValue(c, s.pump) + c.Assert(len(prewriteVal.Mutations), Equals, 0) + + tk.MustExec("begin") + tk.MustExec("delete from temp_table") + tk.MustExec("commit") + prewriteVal = getLatestBinlogPrewriteValue(c, s.pump) + c.Assert(len(prewriteVal.Mutations), Equals, 0) + + ddlQuery = "truncate table temp_table" + tk.MustExec(ddlQuery) + ok = mustGetDDLBinlog(s, ddlQuery, c) + c.Assert(ok, IsTrue) + + ddlQuery = "drop table if exists temp_table" + tk.MustExec(ddlQuery) + ok = mustGetDDLBinlog(s, ddlQuery, c) + c.Assert(ok, IsTrue) +} diff --git a/table/tables/tables.go b/table/tables/tables.go index 74fd2d82f3ef9..a6a4180401d6a 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1414,6 +1414,9 @@ func shouldWriteBinlog(ctx sessionctx.Context, tblInfo *model.TableInfo) bool { if ctx.GetSessionVars().BinlogClient == nil { return false } + if tblInfo.TempTableType != model.TempTableNone { + return false + } return !ctx.GetSessionVars().InRestrictedSQL } From cc83cc524f8d3fd661f6e62d129ba043cc74501e Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 14:45:40 +0800 Subject: [PATCH 055/343] store/tikv: remove use of GuaranteeLinearizability option in store/tikv (#24605) --- session/session.go | 2 +- store/driver/txn/txn_driver.go | 4 ++++ store/tikv/2pc.go | 4 +--- store/tikv/tests/async_commit_test.go | 3 +-- store/tikv/tests/snapshot_fail_test.go | 2 +- store/tikv/txn.go | 14 ++++++++++++++ 6 files changed, 22 insertions(+), 7 deletions(-) diff --git a/session/session.go b/session/session.go index 0b4cb309f434b..19312e5fc391e 100644 --- a/session/session.go +++ b/session/session.go @@ -517,7 +517,7 @@ func (s *session) doCommit(ctx context.Context) error { s.txn.SetOption(tikvstore.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) s.txn.SetOption(tikvstore.Enable1PC, s.GetSessionVars().Enable1PC) // priority of the sysvar is lower than `start transaction with causal consistency only` - if s.txn.GetOption(tikvstore.GuaranteeLinearizability) == nil { + if val := s.txn.GetOption(tikvstore.GuaranteeLinearizability); val == nil || val.(bool) { // We needn't ask the TiKV client to guarantee linearizability for auto-commit transactions // because the property is naturally holds: // We guarantee the commitTS of any transaction must not exceed the next timestamp from the TSO. diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 12f2c8233ccb1..8e8d776eea982 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -162,6 +162,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.SetEnableAsyncCommit(val.(bool)) case tikvstore.Enable1PC: txn.SetEnable1PC(val.(bool)) + case tikvstore.GuaranteeLinearizability: + txn.SetCausalConsistency(!val.(bool)) case tikvstore.TxnScope: txn.SetScope(val.(string)) case tikvstore.IsStalenessReadOnly: @@ -175,6 +177,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { func (txn *tikvTxn) GetOption(opt int) interface{} { switch opt { + case tikvstore.GuaranteeLinearizability: + return !txn.KVTxn.IsCasualConsistency() case tikvstore.TxnScope: return txn.KVTxn.GetScope() default: diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index ee94eceec166a..b0fa2018670e1 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -854,9 +854,7 @@ func (c *twoPhaseCommitter) checkOnePC() bool { } func (c *twoPhaseCommitter) needLinearizability() bool { - GuaranteeLinearizabilityOption := c.txn.us.GetOption(kv.GuaranteeLinearizability) - // by default, guarantee - return GuaranteeLinearizabilityOption == nil || GuaranteeLinearizabilityOption.(bool) + return !c.txn.causalConsistency } func (c *twoPhaseCommitter) isAsyncCommit() bool { diff --git a/store/tikv/tests/async_commit_test.go b/store/tikv/tests/async_commit_test.go index 381771bfa0836..f67482e69a44e 100644 --- a/store/tikv/tests/async_commit_test.go +++ b/store/tikv/tests/async_commit_test.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -127,7 +126,7 @@ func (s *testAsyncCommitCommon) mustGetNoneFromSnapshot(c *C, version uint64, ke func (s *testAsyncCommitCommon) beginAsyncCommitWithLinearizability(c *C) tikv.TxnProbe { txn := s.beginAsyncCommit(c) - txn.SetOption(kv.GuaranteeLinearizability, true) + txn.SetCausalConsistency(false) return txn } diff --git a/store/tikv/tests/snapshot_fail_test.go b/store/tikv/tests/snapshot_fail_test.go index aca3c59099cf7..9892061c44b8d 100644 --- a/store/tikv/tests/snapshot_fail_test.go +++ b/store/tikv/tests/snapshot_fail_test.go @@ -213,7 +213,7 @@ func (s *testSnapshotFailSuite) TestRetryPointGetResolveTS(c *C) { c.Assert(err, IsNil) txn.SetEnableAsyncCommit(false) txn.SetEnable1PC(false) - txn.SetOption(kv.GuaranteeLinearizability, false) + txn.SetCausalConsistency(true) // Prewrite the lock without committing it c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforeCommit", `pause`), IsNil) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index a8c0f70f8da8d..baeaa6b7fa508 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -84,6 +84,7 @@ type KVTxn struct { isPessimistic bool enableAsyncCommit bool enable1PC bool + causalConsistency bool scope string kvFilter KVFilter } @@ -283,6 +284,13 @@ func (txn *KVTxn) SetEnable1PC(b bool) { txn.enable1PC = b } +// SetCausalConsistency indicates if the transaction does not need to +// guarantee linearizability. Default value is false which means +// linearizability is guaranteed. +func (txn *KVTxn) SetCausalConsistency(b bool) { + txn.causalConsistency = b +} + // SetScope sets the geographical scope of the transaction. func (txn *KVTxn) SetScope(scope string) { txn.scope = scope @@ -298,6 +306,12 @@ func (txn *KVTxn) IsPessimistic() bool { return txn.isPessimistic } +// IsCasualConsistency returns if the transaction allows linearizability +// inconsistency. +func (txn *KVTxn) IsCasualConsistency() bool { + return txn.causalConsistency +} + // GetScope returns the geographical scope of the transaction. func (txn *KVTxn) GetScope() string { return txn.scope From 9692c1386bbe2caa1dae8a5a3c9aa29c79c49833 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 15:19:40 +0800 Subject: [PATCH 056/343] store/tikv: remove use of CollectRuntimeStats option in store/tikv (#24604) --- store/driver/txn/snapshot.go | 11 +++++++++-- store/driver/txn/txn_driver.go | 11 +++++++++++ store/tikv/snapshot.go | 29 ++++++++--------------------- store/tikv/tests/snapshot_test.go | 3 +-- store/tikv/txn.go | 1 - 5 files changed, 29 insertions(+), 26 deletions(-) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index ee1d1eeee29d8..a6a7d752a72fd 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -81,12 +81,19 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetSampleStep(val.(uint32)) case tikvstore.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) + case tikvstore.CollectRuntimeStats: + s.KVSnapshot.SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) case tikvstore.IsStalenessReadOnly: s.KVSnapshot.SetIsStatenessReadOnly(val.(bool)) case tikvstore.MatchStoreLabels: s.KVSnapshot.SetMatchStoreLabels(val.([]*metapb.StoreLabel)) - default: - s.KVSnapshot.SetOption(opt, val) + } +} + +func (s *tikvSnapshot) DelOption(opt int) { + switch opt { + case tikvstore.CollectRuntimeStats: + s.KVSnapshot.SetRuntimeStats(nil) } } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 8e8d776eea982..0cd51a4480ee1 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -152,6 +152,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case tikvstore.InfoSchema: txn.SetSchemaVer(val.(tikv.SchemaVer)) + case tikvstore.CollectRuntimeStats: + txn.KVTxn.GetSnapshot().SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) case tikvstore.SchemaAmender: txn.SetSchemaAmender(val.(tikv.SchemaAmender)) case tikvstore.SampleStep: @@ -186,6 +188,15 @@ func (txn *tikvTxn) GetOption(opt int) interface{} { } } +func (txn *tikvTxn) DelOption(opt int) { + switch opt { + case tikvstore.CollectRuntimeStats: + txn.KVTxn.GetSnapshot().SetRuntimeStats(nil) + default: + txn.KVTxn.DelOption(opt) + } +} + // SetVars sets variables to the transaction. func (txn *tikvTxn) SetVars(vars interface{}) { if vs, ok := vars.(*tikv.Variables); ok { diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index ae65f15dc18e6..24149fdae44cb 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -561,27 +561,6 @@ func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) { return scanner, errors.Trace(err) } -// SetOption sets an option with a value, when val is nil, uses the default -// value of this option. Only ReplicaRead is supported for snapshot -func (s *KVSnapshot) SetOption(opt int, val interface{}) { - switch opt { - case kv.CollectRuntimeStats: - s.mu.Lock() - s.mu.stats = val.(*SnapshotRuntimeStats) - s.mu.Unlock() - } -} - -// DelOption deletes an option. -func (s *KVSnapshot) DelOption(opt int) { - switch opt { - case kv.CollectRuntimeStats: - s.mu.Lock() - s.mu.stats = nil - s.mu.Unlock() - } -} - // SetNotFillCache indicates whether tikv should skip filling cache when // loading data. func (s *KVSnapshot) SetNotFillCache(b bool) { @@ -623,6 +602,14 @@ func (s *KVSnapshot) SetTaskID(id uint64) { s.mu.taskID = id } +// SetRuntimeStats sets the stats to collect runtime statistics. +// Set it to nil to clear stored stats. +func (s *KVSnapshot) SetRuntimeStats(stats *SnapshotRuntimeStats) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.stats = stats +} + // SetIsStatenessReadOnly indicates whether the transaction is staleness read only transaction func (s *KVSnapshot) SetIsStatenessReadOnly(b bool) { s.mu.Lock() diff --git a/store/tikv/tests/snapshot_test.go b/store/tikv/tests/snapshot_test.go index fa1ccdd5735bd..a126decfc1c7d 100644 --- a/store/tikv/tests/snapshot_test.go +++ b/store/tikv/tests/snapshot_test.go @@ -26,7 +26,6 @@ import ( pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/tikvrpc" "go.uber.org/zap" @@ -270,7 +269,7 @@ func (s *testSnapshotSuite) TestSnapshotRuntimeStats(c *C) { tikv.RecordRegionRequestRuntimeStats(reqStats.Stats, tikvrpc.CmdGet, time.Second) tikv.RecordRegionRequestRuntimeStats(reqStats.Stats, tikvrpc.CmdGet, time.Millisecond) snapshot := s.store.GetSnapshot(0) - snapshot.SetOption(kv.CollectRuntimeStats, &tikv.SnapshotRuntimeStats{}) + snapshot.SetRuntimeStats(&tikv.SnapshotRuntimeStats{}) snapshot.MergeRegionRequestStats(reqStats.Stats) snapshot.MergeRegionRequestStats(reqStats.Stats) bo := tikv.NewBackofferWithVars(context.Background(), 2000, nil) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index baeaa6b7fa508..76891cfa19bdb 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -224,7 +224,6 @@ func (txn *KVTxn) Delete(k []byte) error { // value of this option. func (txn *KVTxn) SetOption(opt int, val interface{}) { txn.us.SetOption(opt, val) - txn.snapshot.SetOption(opt, val) } // GetOption returns the option From 7c8ddd808aad077e8da436d737860626ea7ffc4b Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 13 May 2021 15:47:40 +0800 Subject: [PATCH 057/343] store/tikv: move Backoffer into a single package (#24525) --- store/copr/batch_coprocessor.go | 4 +- store/copr/coprocessor.go | 11 +- store/copr/mpp.go | 4 +- store/driver/tikv_driver.go | 4 +- store/gcworker/gc_worker.go | 14 +- store/gcworker/gc_worker_test.go | 5 +- store/tikv/2pc.go | 84 ++--- store/tikv/backoff.go | 419 ++--------------------- store/tikv/cleanup.go | 3 +- store/tikv/client_batch.go | 5 +- store/tikv/commit.go | 15 +- store/tikv/delete_range.go | 7 +- store/tikv/kv.go | 15 +- store/tikv/lock_resolver.go | 29 +- store/tikv/pessimistic.go | 9 +- store/tikv/prewrite.go | 23 +- store/tikv/range_task.go | 5 +- store/tikv/rawkv.go | 21 +- store/tikv/region_cache.go | 41 +-- store/tikv/region_cache_test.go | 7 +- store/tikv/region_request.go | 52 +-- store/tikv/region_request_test.go | 5 +- store/tikv/retry/backoff.go | 439 +++++++++++++++++++++++++ store/tikv/{ => retry}/backoff_test.go | 2 +- store/tikv/scan.go | 9 +- store/tikv/snapshot.go | 45 +-- store/tikv/split_region.go | 36 +- store/tikv/test_probe.go | 15 +- store/tikv/tests/lock_test.go | 2 +- store/tikv/txn.go | 15 +- 30 files changed, 731 insertions(+), 614 deletions(-) create mode 100644 store/tikv/retry/backoff.go rename store/tikv/{ => retry}/backoff_test.go (98%) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index bf8d1b7c893e0..c070f25a454da 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -177,7 +177,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V if req.KeepOrder || req.Desc { return copErrorResponse{errors.New("batch coprocessor cannot prove keep order or desc property")} } - ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTs) + ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs) bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildBatchCopTasks(bo, c.store.GetRegionCache(), ranges, req.StoreType) @@ -381,7 +381,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b return nil } - if err1 := bo.Backoff(tikv.BoTiKVRPC, errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil { + if err1 := bo.b.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil { return errors.Trace(err) } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 2c1e2d361af76..dd8474fd75c3a 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -72,7 +72,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa logutil.BgLogger().Debug("send batch requests") return c.sendBatch(ctx, req, vars) } - ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTs) + ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs) bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildCopTasks(bo, c.store.GetRegionCache(), ranges, req) @@ -829,11 +829,14 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *backoffer, rpcCtx *ti return nil, nil } - boRPCType := tikv.BoTiKVRPC + err1 := errors.Errorf("recv stream response error: %v, task: %s", err, task) if task.storeType == kv.TiFlash { - boRPCType = tikv.BoTiFlashRPC + err1 = bo.Backoff(tikv.BoTiFlashRPC, err1) + } else { + err1 = bo.b.BackoffTiKVRPC(err1) } - if err1 := bo.Backoff(boRPCType, errors.Errorf("recv stream response error: %v, task: %s", err, task)); err1 != nil { + + if err1 != nil { return nil, errors.Trace(err) } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 377e439a9392c..0d156de69fb20 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -55,7 +55,7 @@ func (c *MPPClient) selectAllTiFlashStore() []kv.MPPTaskMeta { // ConstructMPPTasks receives ScheduleRequest, which are actually collects of kv ranges. We allocates MPPTaskMeta for them and returns. func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest) ([]kv.MPPTaskMeta, error) { - ctx = context.WithValue(ctx, tikv.TxnStartKey, req.StartTS) + ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTS) bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) if req.KeyRanges == nil { return c.selectAllTiFlashStore(), nil @@ -343,7 +343,7 @@ func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchReques return } - if err1 := bo.Backoff(tikv.BoTiKVRPC, errors.Errorf("recv stream response error: %v", err)); err1 != nil { + if err1 := bo.b.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v", err)); err1 != nil { if errors.Cause(err) == context.Canceled { logutil.BgLogger().Info("stream recv timeout", zap.Error(err)) } else { diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index 398be99520aa6..cc0f217280f31 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -206,6 +206,8 @@ var ( ldflagGetEtcdAddrsFromConfig = "0" // 1:Yes, otherwise:No ) +const getAllMembersBackoff = 5000 + // EtcdAddrs returns etcd server addresses. func (s *tikvStore) EtcdAddrs() ([]string, error) { if s.etcdAddrs == nil { @@ -220,7 +222,7 @@ func (s *tikvStore) EtcdAddrs() ([]string, error) { } ctx := context.Background() - bo := tikv.NewBackoffer(ctx, tikv.GetAllMembersBackoff) + bo := tikv.NewBackoffer(ctx, getAllMembersBackoff) etcdAddrs := make([]string, 0) pdClient := s.GetPDClient() if pdClient == nil { diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index 038efa30b92f3..e0aa993558b6c 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -1050,7 +1050,7 @@ func (w *GCWorker) resolveLocksForRange(ctx context.Context, safePoint uint64, s var stat tikv.RangeTaskStat key := startKey - bo := tikv.NewBackofferWithVars(ctx, tikv.GcResolveLockMaxBackoff, nil) + bo := tikv.NewGcResolveLockMaxBackoffer(ctx) failpoint.Inject("setGcResolveMaxBackoff", func(v failpoint.Value) { sleep := v.(int) // cooperate with github.com/pingcap/tidb/store/tikv/invalidCacheAndRetry @@ -1147,7 +1147,7 @@ retryScanAndResolve: if len(key) == 0 || (len(endKey) != 0 && bytes.Compare(key, endKey) >= 0) { break } - bo = tikv.NewBackofferWithVars(ctx, tikv.GcResolveLockMaxBackoff, nil) + bo = tikv.NewGcResolveLockMaxBackoffer(ctx) failpoint.Inject("setGcResolveMaxBackoff", func(v failpoint.Value) { sleep := v.(int) bo = tikv.NewBackofferWithVars(ctx, sleep, nil) @@ -1460,7 +1460,7 @@ func (w *GCWorker) resolveLocksAcrossRegions(ctx context.Context, locks []*tikv. failpoint.Return(errors.New("injectedError")) }) - bo := tikv.NewBackofferWithVars(ctx, tikv.GcResolveLockMaxBackoff, nil) + bo := tikv.NewGcResolveLockMaxBackoffer(ctx) for { if len(locks) == 0 { @@ -1496,18 +1496,20 @@ func (w *GCWorker) resolveLocksAcrossRegions(ctx context.Context, locks []*tikv. } // Recreate backoffer for next region - bo = tikv.NewBackofferWithVars(ctx, tikv.GcResolveLockMaxBackoff, nil) + bo = tikv.NewGcResolveLockMaxBackoffer(ctx) locks = locks[len(locksInRegion):] } return nil } +const gcOneRegionMaxBackoff = 20000 + func (w *GCWorker) uploadSafePointToPD(ctx context.Context, safePoint uint64) error { var newSafePoint uint64 var err error - bo := tikv.NewBackofferWithVars(ctx, tikv.GcOneRegionMaxBackoff, nil) + bo := tikv.NewBackofferWithVars(ctx, gcOneRegionMaxBackoff, nil) for { newSafePoint, err = w.pdClient.UpdateGCSafePoint(ctx, safePoint) if err != nil { @@ -1544,7 +1546,7 @@ func (w *GCWorker) doGCForRange(ctx context.Context, startKey []byte, endKey []b }() key := startKey for { - bo := tikv.NewBackofferWithVars(ctx, tikv.GcOneRegionMaxBackoff, nil) + bo := tikv.NewBackofferWithVars(ctx, gcOneRegionMaxBackoff, nil) loc, err := w.tikvStore.GetRegionCache().LocateKey(bo, key) if err != nil { return stat, errors.Trace(err) diff --git a/store/gcworker/gc_worker_test.go b/store/gcworker/gc_worker_test.go index 3bfd616929aec..bc09651e0d379 100644 --- a/store/gcworker/gc_worker_test.go +++ b/store/gcworker/gc_worker_test.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/oracle/oracles" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" pd "github.com/tikv/pd/client" ) @@ -412,7 +413,7 @@ func (s *testGCWorkerSuite) TestStatusVars(c *C) { func (s *testGCWorkerSuite) TestDoGCForOneRegion(c *C) { ctx := context.Background() - bo := tikv.NewBackofferWithVars(ctx, tikv.GcOneRegionMaxBackoff, nil) + bo := tikv.NewBackofferWithVars(ctx, gcOneRegionMaxBackoff, nil) loc, err := s.tikvStore.GetRegionCache().LocateKey(bo, []byte("")) c.Assert(err, IsNil) var regionErr *errorpb.Error @@ -943,7 +944,7 @@ func (s *testGCWorkerSuite) TestResolveLockRangeMeetRegionEnlargeCausedByRegionM mCluster.Merge(s.initRegion.regionID, region2) regionMeta, _ := mCluster.GetRegion(s.initRegion.regionID) err := s.tikvStore.GetRegionCache().OnRegionEpochNotMatch( - tikv.NewNoopBackoff(context.Background()), + retry.NewNoopBackoff(context.Background()), &tikv.RPCContext{Region: regionID, Store: &tikv.Store{}}, []*metapb.Region{regionMeta}) c.Assert(err, IsNil) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index b0fa2018670e1..9e4c28ad6fd34 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" @@ -542,7 +543,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh switch act := action.(type) { case actionPrewrite: // Do not update regionTxnSize on retries. They are not used when building a PrewriteRequest. - if len(bo.errors) == 0 { + if bo.ErrorsNum() == 0 { for _, group := range groups { c.regionTxnSize[group.region.id] = group.mutations.Len() } @@ -572,7 +573,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh valStr, ok := val.(string) if ok && c.sessionID > 0 { if firstIsPrimary && actionIsPessimiticLock { - logutil.Logger(bo.ctx).Warn("pessimisticLock failpoint", zap.String("valStr", valStr)) + logutil.Logger(bo.GetCtx()).Warn("pessimisticLock failpoint", zap.String("valStr", valStr)) switch valStr { case "pessimisticLockSkipPrimary": err = c.doActionOnBatches(bo, action, batchBuilder.allBatches()) @@ -587,7 +588,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh failpoint.Inject("pessimisticRollbackDoNth", func() { _, actionIsPessimisticRollback := action.(actionPessimisticRollback) if actionIsPessimisticRollback && c.sessionID > 0 { - logutil.Logger(bo.ctx).Warn("pessimisticRollbackDoNth failpoint") + logutil.Logger(bo.GetCtx()).Warn("pessimisticRollbackDoNth failpoint") failpoint.Return(nil) } }) @@ -607,16 +608,16 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh } // Already spawned a goroutine for async commit transaction. if actionIsCommit && !actionCommit.retry && !c.isAsyncCommit() { - secondaryBo := NewBackofferWithVars(context.Background(), int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) + secondaryBo := retry.NewBackofferWithVars(context.Background(), int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) go func() { if c.sessionID > 0 { failpoint.Inject("beforeCommitSecondaries", func(v failpoint.Value) { if s, ok := v.(string); !ok { - logutil.Logger(bo.ctx).Info("[failpoint] sleep 2s before commit secondary keys", + logutil.Logger(bo.GetCtx()).Info("[failpoint] sleep 2s before commit secondary keys", zap.Uint64("sessionID", c.sessionID), zap.Uint64("txnStartTS", c.startTS), zap.Uint64("txnCommitTS", c.commitTS)) time.Sleep(2 * time.Second) } else if s == "skip" { - logutil.Logger(bo.ctx).Info("[failpoint] injected skip committing secondaries", + logutil.Logger(bo.GetCtx()).Info("[failpoint] injected skip committing secondaries", zap.Uint64("sessionID", c.sessionID), zap.Uint64("txnStartTS", c.startTS), zap.Uint64("txnCommitTS", c.commitTS)) failpoint.Return() } @@ -722,6 +723,8 @@ func (tm *ttlManager) close() { close(tm.ch) } +const pessimisticLockMaxBackoff = 20000 + func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { // Ticker is set to 1/2 of the ManagedLockTTL. ticker := time.NewTicker(time.Duration(atomic.LoadUint64(&ManagedLockTTL)) * time.Millisecond / 2) @@ -735,12 +738,12 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { if tm.lockCtx != nil && tm.lockCtx.Killed != nil && atomic.LoadUint32(tm.lockCtx.Killed) != 0 { return } - bo := NewBackofferWithVars(context.Background(), pessimisticLockMaxBackoff, c.txn.vars) - now, err := c.store.GetOracle().GetTimestamp(bo.ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + bo := retry.NewBackofferWithVars(context.Background(), pessimisticLockMaxBackoff, c.txn.vars) + now, err := c.store.GetOracle().GetTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) if err != nil { - err1 := bo.Backoff(BoPDRPC, err) + err1 := bo.Backoff(retry.BoPDRPC, err) if err1 != nil { - logutil.Logger(bo.ctx).Warn("keepAlive get tso fail", + logutil.Logger(bo.GetCtx()).Warn("keepAlive get tso fail", zap.Error(err)) return } @@ -751,7 +754,7 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { if uptime > config.GetGlobalConfig().MaxTxnTTL { // Checks maximum lifetime for the ttlManager, so when something goes wrong // the key will not be locked forever. - logutil.Logger(bo.ctx).Info("ttlManager live up to its lifetime", + logutil.Logger(bo.GetCtx()).Info("ttlManager live up to its lifetime", zap.Uint64("txnStartTS", c.startTS), zap.Uint64("uptime", uptime), zap.Uint64("maxTxnTTL", config.GetGlobalConfig().MaxTxnTTL)) @@ -765,13 +768,13 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { } newTTL := uptime + atomic.LoadUint64(&ManagedLockTTL) - logutil.Logger(bo.ctx).Info("send TxnHeartBeat", + logutil.Logger(bo.GetCtx()).Info("send TxnHeartBeat", zap.Uint64("startTS", c.startTS), zap.Uint64("newTTL", newTTL)) startTime := time.Now() _, err = sendTxnHeartBeat(bo, c.store, c.primary(), c.startTS, newTTL) if err != nil { metrics.TxnHeartBeatHistogramError.Observe(time.Since(startTime).Seconds()) - logutil.Logger(bo.ctx).Warn("send TxnHeartBeat failed", + logutil.Logger(bo.GetCtx()).Warn("send TxnHeartBeat failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) return @@ -801,7 +804,7 @@ func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, tt return 0, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return 0, errors.Trace(err) } @@ -889,6 +892,11 @@ func (c *twoPhaseCommitter) checkOnePCFallBack(action twoPhaseCommitAction, batc } } +const ( + cleanupMaxBackoff = 20000 + tsoMaxBackoff = 15000 +) + func (c *twoPhaseCommitter) cleanup(ctx context.Context) { c.cleanWg.Add(1) go func() { @@ -899,12 +907,12 @@ func (c *twoPhaseCommitter) cleanup(ctx context.Context) { failpoint.Return() }) - cleanupKeysCtx := context.WithValue(context.Background(), TxnStartKey, ctx.Value(TxnStartKey)) + cleanupKeysCtx := context.WithValue(context.Background(), retry.TxnStartKey, ctx.Value(retry.TxnStartKey)) var err error if !c.isOnePC() { - err = c.cleanupMutations(NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations) + err = c.cleanupMutations(retry.NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations) } else if c.isPessimistic { - err = c.pessimisticRollbackMutations(NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations) + err = c.pessimisticRollbackMutations(retry.NewBackofferWithVars(cleanupKeysCtx, cleanupMaxBackoff, c.txn.vars), c.mutations) } if err != nil { @@ -1016,7 +1024,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { if c.shouldWriteBinlog() { binlogChan = c.binlog.Prewrite(ctx, c.primary()) } - prewriteBo := NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars) + prewriteBo := retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars) start := time.Now() err = c.prewriteMutations(prewriteBo, c.mutations) @@ -1035,10 +1043,10 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { commitDetail := c.getDetail() commitDetail.PrewriteTime = time.Since(start) - if prewriteBo.totalSleep > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(prewriteBo.totalSleep)*int64(time.Millisecond)) + if prewriteBo.GetTotalSleep() > 0 { + atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(prewriteBo.GetTotalSleep())*int64(time.Millisecond)) commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, prewriteBo.types...) + commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, prewriteBo.GetTypes()...) commitDetail.Mu.Unlock() } if binlogChan != nil { @@ -1092,7 +1100,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { } else { start = time.Now() logutil.Event(ctx, "start get commit ts") - commitTS, err = c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) + commitTS, err = c.store.getTimestampWithRetry(retry.NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) if err != nil { logutil.Logger(ctx).Warn("2PC get commitTS failed", zap.Error(err), @@ -1175,7 +1183,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { failpoint.Inject("asyncCommitDoNothing", func() { failpoint.Return() }) - commitBo := NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) + commitBo := retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) err := c.commitMutations(commitBo, c.mutations) if err != nil { logutil.Logger(ctx).Warn("2PC async commit failed", zap.Uint64("sessionID", c.sessionID), @@ -1191,13 +1199,13 @@ func (c *twoPhaseCommitter) commitTxn(ctx context.Context, commitDetail *util.Co c.txn.GetMemBuffer().DiscardValues() start := time.Now() - commitBo := NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) + commitBo := retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) err := c.commitMutations(commitBo, c.mutations) commitDetail.CommitTime = time.Since(start) - if commitBo.totalSleep > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(commitBo.totalSleep)*int64(time.Millisecond)) + if commitBo.GetTotalSleep() > 0 { + atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(commitBo.GetTotalSleep())*int64(time.Millisecond)) commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, commitBo.types...) + commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, commitBo.GetTypes()...) commitDetail.Mu.Unlock() } if err != nil { @@ -1282,7 +1290,7 @@ func (c *twoPhaseCommitter) amendPessimisticLock(ctx context.Context, addMutatio retryLimit := config.GetGlobalConfig().PessimisticTxn.MaxRetryCount var err error for tryTimes < retryLimit { - pessimisticLockBo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, c.txn.vars) + pessimisticLockBo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, c.txn.vars) err = c.pessimisticLockMutations(pessimisticLockBo, lCtx, &keysNeedToLock) if err != nil { // KeysNeedToLock won't change, so don't async rollback pessimistic locks here for write conflict. @@ -1328,7 +1336,7 @@ func (c *twoPhaseCommitter) tryAmendTxn(ctx context.Context, startInfoSchema Sch return false, err } if c.prewriteStarted { - prewriteBo := NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars) + prewriteBo := retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars) err = c.prewriteMutations(prewriteBo, addMutations) if err != nil { logutil.Logger(ctx).Warn("amend prewrite has failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) @@ -1361,7 +1369,7 @@ func (c *twoPhaseCommitter) tryAmendTxn(ctx context.Context, startInfoSchema Sch func (c *twoPhaseCommitter) getCommitTS(ctx context.Context, commitDetail *util.CommitDetails) (uint64, error) { start := time.Now() logutil.Event(ctx, "start get commit ts") - commitTS, err := c.store.getTimestampWithRetry(NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) + commitTS, err := c.store.getTimestampWithRetry(retry.NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) if err != nil { logutil.Logger(ctx).Warn("2PC get commitTS failed", zap.Error(err), @@ -1575,20 +1583,20 @@ func (batchExe *batchExecutor) startWorker(exitCh chan struct{}, ch chan error, singleBatchBackoffer, singleBatchCancel = batchExe.backoffer.Fork() defer singleBatchCancel() } - beforeSleep := singleBatchBackoffer.totalSleep + beforeSleep := singleBatchBackoffer.GetTotalSleep() ch <- batchExe.action.handleSingleBatch(batchExe.committer, singleBatchBackoffer, batch) commitDetail := batchExe.committer.getDetail() if commitDetail != nil { // lock operations of pessimistic-txn will let commitDetail be nil - if delta := singleBatchBackoffer.totalSleep - beforeSleep; delta > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(singleBatchBackoffer.totalSleep-beforeSleep)*int64(time.Millisecond)) + if delta := singleBatchBackoffer.GetTotalSleep() - beforeSleep; delta > 0 { + atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(singleBatchBackoffer.GetTotalSleep()-beforeSleep)*int64(time.Millisecond)) commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, singleBatchBackoffer.types...) + commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, singleBatchBackoffer.GetTypes()...) commitDetail.Mu.Unlock() } } }() } else { - logutil.Logger(batchExe.backoffer.ctx).Info("break startWorker", + logutil.Logger(batchExe.backoffer.GetCtx()).Info("break startWorker", zap.Stringer("action", batchExe.action), zap.Int("batch size", len(batches)), zap.Int("index", idx)) break @@ -1601,7 +1609,7 @@ func (batchExe *batchExecutor) process(batches []batchMutations) error { var err error err = batchExe.initUtils() if err != nil { - logutil.Logger(batchExe.backoffer.ctx).Error("batchExecutor initUtils failed", zap.Error(err)) + logutil.Logger(batchExe.backoffer.GetCtx()).Error("batchExecutor initUtils failed", zap.Error(err)) return err } @@ -1618,14 +1626,14 @@ func (batchExe *batchExecutor) process(batches []batchMutations) error { // check results for i := 0; i < len(batches); i++ { if e := <-ch; e != nil { - logutil.Logger(batchExe.backoffer.ctx).Debug("2PC doActionOnBatch failed", + logutil.Logger(batchExe.backoffer.GetCtx()).Debug("2PC doActionOnBatch failed", zap.Uint64("session", batchExe.committer.sessionID), zap.Stringer("action type", batchExe.action), zap.Error(e), zap.Uint64("txnStartTS", batchExe.committer.startTS)) // Cancel other requests and return the first error. if cancel != nil { - logutil.Logger(batchExe.backoffer.ctx).Debug("2PC doActionOnBatch to cancel other actions", + logutil.Logger(batchExe.backoffer.GetCtx()).Debug("2PC doActionOnBatch to cancel other actions", zap.Uint64("session", batchExe.committer.sessionID), zap.Stringer("action type", batchExe.action), zap.Uint64("txnStartTS", batchExe.committer.startTS)) diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index e0115c9e3904a..c622e21d2ee5d 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -1,4 +1,4 @@ -// Copyright 2016 PingCAP, Inc. +// Copyright 2021 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,431 +15,54 @@ package tikv import ( "context" - "fmt" - "math" - "math/rand" - "strings" - "sync/atomic" - "time" - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/log" - tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/util" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" + "github.com/pingcap/tidb/store/tikv/retry" ) -const ( - // NoJitter makes the backoff sequence strict exponential. - NoJitter = 1 + iota - // FullJitter applies random factors to strict exponential. - FullJitter - // EqualJitter is also randomized, but prevents very short sleeps. - EqualJitter - // DecorrJitter increases the maximum jitter based on the last random value. - DecorrJitter -) - -func (t BackoffType) metric() prometheus.Observer { - switch t { - // TODO: distinguish tikv and tiflash in metrics - case BoTiKVRPC, BoTiFlashRPC: - return metrics.BackoffHistogramRPC - case BoTxnLock: - return metrics.BackoffHistogramLock - case BoTxnLockFast: - return metrics.BackoffHistogramLockFast - case BoPDRPC: - return metrics.BackoffHistogramPD - case BoRegionMiss: - return metrics.BackoffHistogramRegionMiss - case boTiKVServerBusy, boTiFlashServerBusy: - return metrics.BackoffHistogramServerBusy - case boStaleCmd: - return metrics.BackoffHistogramStaleCmd - } - return metrics.BackoffHistogramEmpty -} - -// NewBackoffFn creates a backoff func which implements exponential backoff with -// optional jitters. -// See http://www.awsarchitectureblog.com/2015/03/backoff.html -func NewBackoffFn(base, cap, jitter int) func(ctx context.Context, maxSleepMs int) int { - if base < 2 { - // Top prevent panic in 'rand.Intn'. - base = 2 - } - attempts := 0 - lastSleep := base - return func(ctx context.Context, maxSleepMs int) int { - var sleep int - switch jitter { - case NoJitter: - sleep = expo(base, cap, attempts) - case FullJitter: - v := expo(base, cap, attempts) - sleep = rand.Intn(v) - case EqualJitter: - v := expo(base, cap, attempts) - sleep = v/2 + rand.Intn(v/2) - case DecorrJitter: - sleep = int(math.Min(float64(cap), float64(base+rand.Intn(lastSleep*3-base)))) - } - logutil.BgLogger().Debug("backoff", - zap.Int("base", base), - zap.Int("sleep", sleep), - zap.Int("attempts", attempts)) - - realSleep := sleep - // when set maxSleepMs >= 0 in `tikv.BackoffWithMaxSleep` will force sleep maxSleepMs milliseconds. - if maxSleepMs >= 0 && realSleep > maxSleepMs { - realSleep = maxSleepMs - } - select { - case <-time.After(time.Duration(realSleep) * time.Millisecond): - attempts++ - lastSleep = sleep - return realSleep - case <-ctx.Done(): - return 0 - } - } -} - -func expo(base, cap, n int) int { - return int(math.Min(float64(cap), float64(base)*math.Pow(2.0, float64(n)))) -} +// Backoffer is a utility for retrying queries. +type Backoffer = retry.Backoffer // BackoffType defines the backoff type. -type BackoffType int +type BackoffType = retry.BackoffType // Back off types. const ( - BoTiKVRPC BackoffType = iota - BoTiFlashRPC - BoTxnLock - BoTxnLockFast - BoPDRPC - BoRegionMiss - boTiKVServerBusy - boTiFlashServerBusy - boTxnNotFound - boStaleCmd - boMaxTsNotSynced + BoRegionMiss = retry.BoRegionMiss + BoTiFlashRPC = retry.BoTiFlashRPC + BoTxnLockFast = retry.BoTxnLockFast + BoTxnLock = retry.BoTxnLock + BoPDRPC = retry.BoPDRPC ) -func (t BackoffType) createFn(vars *kv.Variables) func(context.Context, int) int { - if vars.Hook != nil { - vars.Hook(t.String(), vars) - } - switch t { - case BoTiKVRPC, BoTiFlashRPC: - return NewBackoffFn(100, 2000, EqualJitter) - case BoTxnLock: - return NewBackoffFn(200, 3000, EqualJitter) - case BoTxnLockFast: - return NewBackoffFn(vars.BackoffLockFast, 3000, EqualJitter) - case BoPDRPC: - return NewBackoffFn(500, 3000, EqualJitter) - case BoRegionMiss: - // change base time to 2ms, because it may recover soon. - return NewBackoffFn(2, 500, NoJitter) - case boTxnNotFound: - return NewBackoffFn(2, 500, NoJitter) - case boTiKVServerBusy, boTiFlashServerBusy: - return NewBackoffFn(2000, 10000, EqualJitter) - case boStaleCmd: - return NewBackoffFn(2, 1000, NoJitter) - case boMaxTsNotSynced: - return NewBackoffFn(2, 500, NoJitter) - } - return nil -} - -func (t BackoffType) String() string { - switch t { - case BoTiKVRPC: - return "tikvRPC" - case BoTiFlashRPC: - return "tiflashRPC" - case BoTxnLock: - return "txnLock" - case BoTxnLockFast: - return "txnLockFast" - case BoPDRPC: - return "pdRPC" - case BoRegionMiss: - return "regionMiss" - case boTiKVServerBusy: - return "tikvServerBusy" - case boTiFlashServerBusy: - return "tiflashServerBusy" - case boStaleCmd: - return "staleCommand" - case boTxnNotFound: - return "txnNotFound" - case boMaxTsNotSynced: - return "maxTsNotSynced" - } - return "" -} - -// TError returns pingcap/error of the backoff type. -func (t BackoffType) TError() error { - switch t { - case BoTiKVRPC: - return tikverr.ErrTiKVServerTimeout - case BoTiFlashRPC: - return tikverr.ErrTiFlashServerTimeout - case BoTxnLock, BoTxnLockFast, boTxnNotFound: - return tikverr.ErrResolveLockTimeout - case BoPDRPC: - return tikverr.NewErrPDServerTimeout("") - case BoRegionMiss: - return tikverr.ErrRegionUnavailable - case boTiKVServerBusy: - return tikverr.ErrTiKVServerBusy - case boTiFlashServerBusy: - return tikverr.ErrTiFlashServerBusy - case boStaleCmd: - return tikverr.ErrTiKVStaleCommand - case boMaxTsNotSynced: - return tikverr.ErrTiKVMaxTimestampNotSynced - } - return tikverr.ErrUnknown -} - // Maximum total sleep time(in ms) for kv/cop commands. const ( - GetAllMembersBackoff = 5000 - tsoMaxBackoff = 15000 - scannerNextMaxBackoff = 20000 - batchGetMaxBackoff = 20000 - getMaxBackoff = 20000 - cleanupMaxBackoff = 20000 - GcOneRegionMaxBackoff = 20000 - GcResolveLockMaxBackoff = 100000 - deleteRangeOneRegionMaxBackoff = 100000 - rawkvMaxBackoff = 20000 - splitRegionBackoff = 20000 - maxSplitRegionsBackoff = 120000 - waitScatterRegionFinishBackoff = 120000 - locateRegionMaxBackoff = 20000 - pessimisticLockMaxBackoff = 20000 - pessimisticRollbackMaxBackoff = 20000 + gcResolveLockMaxBackoff = 100000 ) var ( // CommitMaxBackoff is max sleep time of the 'commit' command CommitMaxBackoff = uint64(41000) - // PrewriteMaxBackoff is max sleep time of the `pre-write` command. PrewriteMaxBackoff = 20000 ) -// Backoffer is a utility for retrying queries. -type Backoffer struct { - ctx context.Context - - fn map[BackoffType]func(context.Context, int) int - maxSleep int - totalSleep int - errors []error - types []fmt.Stringer - vars *kv.Variables - noop bool - - backoffSleepMS map[BackoffType]int - backoffTimes map[BackoffType]int -} - -type txnStartCtxKeyType struct{} - -// TxnStartKey is a key for transaction start_ts info in context.Context. -var TxnStartKey interface{} = txnStartCtxKeyType{} - -// NewBackoffer (Deprecated) creates a Backoffer with maximum sleep time(in ms). -func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { - return &Backoffer{ - ctx: ctx, - maxSleep: maxSleep, - vars: kv.DefaultVars, - } -} - // NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *Backoffer { - return NewBackoffer(ctx, maxSleep).withVars(vars) + return retry.NewBackofferWithVars(ctx, maxSleep, vars) } -// NewNoopBackoff create a Backoffer do nothing just return error directly -func NewNoopBackoff(ctx context.Context) *Backoffer { - return &Backoffer{ctx: ctx, noop: true} -} - -// withVars sets the kv.Variables to the Backoffer and return it. -func (b *Backoffer) withVars(vars *kv.Variables) *Backoffer { - if vars != nil { - b.vars = vars - } - // maxSleep is the max sleep time in millisecond. - // When it is multiplied by BackOffWeight, it should not be greater than MaxInt32. - if math.MaxInt32/b.vars.BackOffWeight >= b.maxSleep { - b.maxSleep *= b.vars.BackOffWeight - } - return b -} - -// Backoff sleeps a while base on the backoffType and records the error message. -// It returns a retryable error if total sleep time exceeds maxSleep. -func (b *Backoffer) Backoff(typ BackoffType, err error) error { - if span := opentracing.SpanFromContext(b.ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan(fmt.Sprintf("tikv.backoff.%s", typ), opentracing.ChildOf(span.Context())) - defer span1.Finish() - opentracing.ContextWithSpan(b.ctx, span1) - } - return b.BackoffWithMaxSleep(typ, -1, err) -} - -// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message -// and never sleep more than maxSleepMs for each sleep. -func (b *Backoffer) BackoffWithMaxSleep(typ BackoffType, maxSleepMs int, err error) error { - if strings.Contains(err.Error(), tikverr.MismatchClusterID) { - logutil.BgLogger().Fatal("critical error", zap.Error(err)) - } - select { - case <-b.ctx.Done(): - return errors.Trace(err) - default: - } - - b.errors = append(b.errors, errors.Errorf("%s at %s", err.Error(), time.Now().Format(time.RFC3339Nano))) - b.types = append(b.types, typ) - if b.noop || (b.maxSleep > 0 && b.totalSleep >= b.maxSleep) { - errMsg := fmt.Sprintf("%s backoffer.maxSleep %dms is exceeded, errors:", typ.String(), b.maxSleep) - for i, err := range b.errors { - // Print only last 3 errors for non-DEBUG log levels. - if log.GetLevel() == zapcore.DebugLevel || i >= len(b.errors)-3 { - errMsg += "\n" + err.Error() - } - } - logutil.BgLogger().Warn(errMsg) - // Use the first backoff type to generate a MySQL error. - return b.types[0].(BackoffType).TError() - } - - // Lazy initialize. - if b.fn == nil { - b.fn = make(map[BackoffType]func(context.Context, int) int) - } - f, ok := b.fn[typ] - if !ok { - f = typ.createFn(b.vars) - b.fn[typ] = f - } - - realSleep := f(b.ctx, maxSleepMs) - typ.metric().Observe(float64(realSleep) / 1000) - b.totalSleep += realSleep - if b.backoffSleepMS == nil { - b.backoffSleepMS = make(map[BackoffType]int) - } - b.backoffSleepMS[typ] += realSleep - if b.backoffTimes == nil { - b.backoffTimes = make(map[BackoffType]int) - } - b.backoffTimes[typ]++ - - stmtExec := b.ctx.Value(util.ExecDetailsKey) - if stmtExec != nil { - detail := stmtExec.(*util.ExecDetails) - atomic.AddInt64(&detail.BackoffDuration, int64(realSleep)*int64(time.Millisecond)) - atomic.AddInt64(&detail.BackoffCount, 1) - } - - if b.vars != nil && b.vars.Killed != nil { - if atomic.LoadUint32(b.vars.Killed) == 1 { - return tikverr.ErrQueryInterrupted - } - } - - var startTs interface{} - if ts := b.ctx.Value(TxnStartKey); ts != nil { - startTs = ts - } - logutil.Logger(b.ctx).Debug("retry later", - zap.Error(err), - zap.Int("totalSleep", b.totalSleep), - zap.Int("maxSleep", b.maxSleep), - zap.Stringer("type", typ), - zap.Reflect("txnStartTS", startTs)) - return nil -} - -func (b *Backoffer) String() string { - if b.totalSleep == 0 { - return "" - } - return fmt.Sprintf(" backoff(%dms %v)", b.totalSleep, b.types) -} - -// Clone creates a new Backoffer which keeps current Backoffer's sleep time and errors, and shares -// current Backoffer's context. -func (b *Backoffer) Clone() *Backoffer { - return &Backoffer{ - ctx: b.ctx, - maxSleep: b.maxSleep, - totalSleep: b.totalSleep, - errors: b.errors, - vars: b.vars, - } -} - -// Fork creates a new Backoffer which keeps current Backoffer's sleep time and errors, and holds -// a child context of current Backoffer's context. -func (b *Backoffer) Fork() (*Backoffer, context.CancelFunc) { - ctx, cancel := context.WithCancel(b.ctx) - return &Backoffer{ - ctx: ctx, - maxSleep: b.maxSleep, - totalSleep: b.totalSleep, - errors: b.errors, - vars: b.vars, - }, cancel -} - -// GetVars returns the binded vars. -func (b *Backoffer) GetVars() *kv.Variables { - return b.vars -} - -// GetTotalSleep returns total sleep time. -func (b *Backoffer) GetTotalSleep() int { - return b.totalSleep -} - -// GetTypes returns type list. -func (b *Backoffer) GetTypes() []fmt.Stringer { - return b.types -} - -// GetCtx returns the binded context. -func (b *Backoffer) GetCtx() context.Context { - return b.ctx +// NewBackoffer creates a Backoffer with maximum sleep time(in ms). +func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { + return retry.NewBackoffer(ctx, maxSleep) } -// GetBackoffTimes returns a map contains backoff time count by type. -func (b *Backoffer) GetBackoffTimes() map[BackoffType]int { - return b.backoffTimes +// TxnStartKey is a key for transaction start_ts info in context.Context. +func TxnStartKey() interface{} { + return retry.TxnStartKey } -// GetBackoffSleepMS returns a map contains backoff sleep time by type. -func (b *Backoffer) GetBackoffSleepMS() map[BackoffType]int { - return b.backoffSleepMS +// NewGcResolveLockMaxBackoffer creates a Backoffer for Gc to resolve lock. +func NewGcResolveLockMaxBackoffer(ctx context.Context) *Backoffer { + return retry.NewBackofferWithVars(ctx, gcResolveLockMaxBackoff, nil) } diff --git a/store/tikv/cleanup.go b/store/tikv/cleanup.go index dc96ed32ab54c..0260d770cdd44 100644 --- a/store/tikv/cleanup.go +++ b/store/tikv/cleanup.go @@ -18,6 +18,7 @@ import ( pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" @@ -49,7 +50,7 @@ func (actionCleanup) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batc return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/client_batch.go b/store/tikv/client_batch.go index 886d20abf0e46..e5ec039fc6911 100644 --- a/store/tikv/client_batch.go +++ b/store/tikv/client_batch.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" @@ -662,7 +663,7 @@ func (c *batchCommandsClient) recreateStreamingClient(err error, streamClient *b *epoch++ c.failPendingRequests(err) // fail all pending requests. - b := NewBackofferWithVars(context.Background(), math.MaxInt32, nil) + b := retry.NewBackofferWithVars(context.Background(), math.MaxInt32, nil) for { // try to re-create the streaming in the loop. if c.isStopped() { return true @@ -672,7 +673,7 @@ func (c *batchCommandsClient) recreateStreamingClient(err error, streamClient *b break } - err2 := b.Backoff(BoTiKVRPC, err1) + err2 := b.BackoffTiKVRPC(err1) // As timeout is set to math.MaxUint32, err2 should always be nil. // This line is added to make the 'make errcheck' pass. terror.Log(err2) diff --git a/store/tikv/commit.go b/store/tikv/commit.go index ce9df6a927355..449081860c029 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -22,6 +22,7 @@ import ( tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" @@ -67,7 +68,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -86,7 +87,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch } if keyErr := commitResp.GetError(); keyErr != nil { if rejected := keyErr.GetCommitTsExpired(); rejected != nil { - logutil.Logger(bo.ctx).Info("2PC commitTS rejected by TiKV, retry with a newer commitTS", + logutil.Logger(bo.GetCtx()).Info("2PC commitTS rejected by TiKV, retry with a newer commitTS", zap.Uint64("txnStartTS", c.startTS), zap.Stringer("info", logutil.Hex(rejected))) @@ -101,7 +102,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch // Update commit ts and retry. commitTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) if err != nil { - logutil.Logger(bo.ctx).Warn("2PC get commitTS failed", + logutil.Logger(bo.GetCtx()).Warn("2PC get commitTS failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) return errors.Trace(err) @@ -126,7 +127,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch } return res } - logutil.Logger(bo.ctx).Error("2PC failed commit key after primary key committed", + logutil.Logger(bo.GetCtx()).Error("2PC failed commit key after primary key committed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS), zap.Uint64("commitTS", c.commitTS), @@ -134,7 +135,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch return errors.Trace(err) } // The transaction maybe rolled back by concurrent transactions. - logutil.Logger(bo.ctx).Debug("2PC failed commit primary key", + logutil.Logger(bo.GetCtx()).Debug("2PC failed commit primary key", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) return err @@ -149,10 +150,10 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch } func (c *twoPhaseCommitter) commitMutations(bo *Backoffer, mutations CommitterMutations) error { - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("twoPhaseCommitter.commitMutations", opentracing.ChildOf(span.Context())) defer span1.Finish() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } return c.doActionOnMutations(bo, actionCommit{}, mutations) diff --git a/store/tikv/delete_range.go b/store/tikv/delete_range.go index 4cbe9fc039749..6a826d718cba1 100644 --- a/store/tikv/delete_range.go +++ b/store/tikv/delete_range.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) @@ -78,6 +79,8 @@ func (t *DeleteRangeTask) Execute(ctx context.Context) error { return err } +const deleteRangeOneRegionMaxBackoff = 100000 + // Execute performs the delete range operation. func (t *DeleteRangeTask) sendReqOnRange(ctx context.Context, r kv.KeyRange) (RangeTaskStat, error) { startKey, rangeEndKey := r.StartKey, r.EndKey @@ -93,7 +96,7 @@ func (t *DeleteRangeTask) sendReqOnRange(ctx context.Context, r kv.KeyRange) (Ra break } - bo := NewBackofferWithVars(ctx, deleteRangeOneRegionMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, deleteRangeOneRegionMaxBackoff, nil) loc, err := t.store.GetRegionCache().LocateKey(bo, startKey) if err != nil { return stat, errors.Trace(err) @@ -121,7 +124,7 @@ func (t *DeleteRangeTask) sendReqOnRange(ctx context.Context, r kv.KeyRange) (Ra return stat, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return stat, errors.Trace(err) } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 981a1b7bc5cab..f61db4168ef7d 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/oracle/oracles" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" @@ -226,7 +227,7 @@ func (s *KVStore) UUID() string { // CurrentTimestamp returns current timestamp with the given txnScope (local or global). func (s *KVStore) CurrentTimestamp(txnScope string) (uint64, error) { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) startTS, err := s.getTimestampWithRetry(bo, txnScope) if err != nil { return 0, errors.Trace(err) @@ -242,14 +243,14 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, panic("MockCurrentTimestamp should be a number, try use this failpoint with \"return(ts)\"") } }) - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("TiKVStore.getTimestampWithRetry", opentracing.ChildOf(span.Context())) defer span1.Finish() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } for { - startTS, err := s.oracle.GetTimestamp(bo.ctx, &oracle.Option{TxnScope: txnScope}) + startTS, err := s.oracle.GetTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: txnScope}) // mockGetTSErrorInRetry should wait MockCommitErrorOnce first, then will run into retry() logic. // Then mockGetTSErrorInRetry will return retryable error when first retry. // Before PR #8743, we don't cleanup txn after meet error such as error like: PD server timeout @@ -263,7 +264,7 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, if err == nil { return startTS, nil } - err = bo.Backoff(BoPDRPC, errors.Errorf("get timestamp failed: %v", err)) + err = bo.Backoff(retry.BoPDRPC, errors.Errorf("get timestamp failed: %v", err)) if err != nil { return 0, errors.Trace(err) } @@ -279,11 +280,11 @@ func (s *KVStore) getStalenessTimestamp(bo *Backoffer, txnScope string, prevSec } }) for { - startTS, err := s.oracle.GetStaleTimestamp(bo.ctx, txnScope, prevSec) + startTS, err := s.oracle.GetStaleTimestamp(bo.GetCtx(), txnScope, prevSec) if err == nil { return startTS, nil } - err = bo.Backoff(BoPDRPC, errors.Errorf("get staleness timestamp failed: %v", err)) + err = bo.Backoff(retry.BoPDRPC, errors.Errorf("get staleness timestamp failed: %v", err)) if err != nil { return 0, errors.Trace(err) } diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index 05feee6d31adb..fe50910a896e6 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/util" pd "github.com/tikv/pd/client" @@ -228,7 +229,7 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc Regi // locks have been cleaned before GC. expiredLocks := locks - callerStartTS, err := lr.store.GetOracle().GetTimestamp(bo.ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + callerStartTS, err := lr.store.GetOracle().GetTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) if err != nil { return false, errors.Trace(err) } @@ -297,7 +298,7 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc Regi } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return false, errors.Trace(err) } @@ -473,8 +474,8 @@ func (t *txnExpireTime) value() int64 { // seconds before calling it after Prewrite. func (lr *LockResolver) GetTxnStatus(txnID uint64, callerStartTS uint64, primary []byte) (TxnStatus, error) { var status TxnStatus - bo := NewBackoffer(context.Background(), cleanupMaxBackoff) - currentTS, err := lr.store.GetOracle().GetLowResolutionTimestamp(bo.ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + bo := retry.NewBackoffer(context.Background(), cleanupMaxBackoff) + currentTS, err := lr.store.GetOracle().GetLowResolutionTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) if err != nil { return status, err } @@ -493,7 +494,7 @@ func (lr *LockResolver) getTxnStatusFromLock(bo *Backoffer, l *Lock, callerStart // Set currentTS to max uint64 to make the lock expired. currentTS = math.MaxUint64 } else { - currentTS, err = lr.store.GetOracle().GetLowResolutionTimestamp(bo.ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + currentTS, err = lr.store.GetOracle().GetLowResolutionTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) if err != nil { return TxnStatus{}, err } @@ -522,12 +523,12 @@ func (lr *LockResolver) getTxnStatusFromLock(bo *Backoffer, l *Lock, callerStart // getTxnStatus() returns it when the secondary locks exist while the primary lock doesn't. // This is likely to happen in the concurrently prewrite when secondary regions // success before the primary region. - if err := bo.Backoff(boTxnNotFound, err); err != nil { - logutil.Logger(bo.ctx).Warn("getTxnStatusFromLock backoff fail", zap.Error(err)) + if err := bo.Backoff(retry.BoTxnNotFound, err); err != nil { + logutil.Logger(bo.GetCtx()).Warn("getTxnStatusFromLock backoff fail", zap.Error(err)) } if lr.store.GetOracle().UntilExpired(l.TxnID, l.TTL, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) <= 0 { - logutil.Logger(bo.ctx).Warn("lock txn not found, lock has expired", + logutil.Logger(bo.GetCtx()).Warn("lock txn not found, lock has expired", zap.Uint64("CallerStartTs", callerStartTS), zap.Stringer("lock str", l)) if l.LockType == kvrpcpb.Op_PessimisticLock { @@ -599,7 +600,7 @@ func (lr *LockResolver) getTxnStatus(bo *Backoffer, txnID uint64, primary []byte return status, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return status, errors.Trace(err) } @@ -735,7 +736,7 @@ func (lr *LockResolver) checkSecondaries(bo *Backoffer, txnID uint64, curKeys [] return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -866,7 +867,7 @@ func (lr *LockResolver) resolveRegionLocks(bo *Backoffer, l *Lock, region Region return errors.Trace(err) } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -934,7 +935,7 @@ func (lr *LockResolver) resolveLock(bo *Backoffer, l *Lock, status TxnStatus, li return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -985,7 +986,7 @@ func (lr *LockResolver) resolvePessimisticLock(bo *Backoffer, l *Lock, cleanRegi return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -997,7 +998,7 @@ func (lr *LockResolver) resolvePessimisticLock(bo *Backoffer, l *Lock, cleanRegi cmdResp := resp.Resp.(*kvrpcpb.PessimisticRollbackResponse) if keyErr := cmdResp.GetErrors(); len(keyErr) > 0 { err = errors.Errorf("unexpected resolve pessimistic lock err: %s, lock: %v", keyErr[0], l) - logutil.Logger(bo.ctx).Error("resolveLock error", zap.Error(err)) + logutil.Logger(bo.GetCtx()).Error("resolveLock error", zap.Error(err)) return err } return nil diff --git a/store/tikv/pessimistic.go b/store/tikv/pessimistic.go index 475efc7ad2a8b..445ced93ff904 100644 --- a/store/tikv/pessimistic.go +++ b/store/tikv/pessimistic.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" @@ -130,7 +131,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -228,7 +229,7 @@ func (actionPessimisticRollback) handleSingleBatch(c *twoPhaseCommitter, bo *Bac return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -247,11 +248,11 @@ func (c *twoPhaseCommitter) pessimisticLockMutations(bo *Backoffer, lockCtx *kv. for _, action := range strings.Split(v, ",") { if action == "delay" { duration := time.Duration(rand.Int63n(int64(time.Second) * 5)) - logutil.Logger(bo.ctx).Info("[failpoint] injected delay at pessimistic lock", + logutil.Logger(bo.GetCtx()).Info("[failpoint] injected delay at pessimistic lock", zap.Uint64("txnStartTS", c.startTS), zap.Duration("duration", duration)) time.Sleep(duration) } else if action == "fail" { - logutil.Logger(bo.ctx).Info("[failpoint] injected failure at pessimistic lock", + logutil.Logger(bo.GetCtx()).Info("[failpoint] injected failure at pessimistic lock", zap.Uint64("txnStartTS", c.startTS)) failpoint.Return(errors.New("injected failure at pessimistic lock")) } diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index 1fea4ba467341..7097ba5dbcd3e 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -27,6 +27,7 @@ import ( tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" @@ -130,7 +131,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff failpoint.Inject("prewritePrimaryFail", func() { // Delay to avoid cancelling other normally ongoing prewrite requests. time.Sleep(time.Millisecond * 50) - logutil.Logger(bo.ctx).Info("[failpoint] injected error on prewriting primary batch", + logutil.Logger(bo.GetCtx()).Info("[failpoint] injected error on prewriting primary batch", zap.Uint64("txnStartTS", c.startTS)) failpoint.Return(errors.New("injected error on prewriting primary batch")) }) @@ -139,7 +140,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff failpoint.Inject("prewriteSecondaryFail", func() { // Delay to avoid cancelling other normally ongoing prewrite requests. time.Sleep(time.Millisecond * 50) - logutil.Logger(bo.ctx).Info("[failpoint] injected error on prewriting secondary batch", + logutil.Logger(bo.GetCtx()).Info("[failpoint] injected error on prewriting secondary batch", zap.Uint64("txnStartTS", c.startTS)) failpoint.Return(errors.New("injected error on prewriting secondary batch")) }) @@ -150,7 +151,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff txnSize := uint64(c.regionTxnSize[batch.region.id]) // When we retry because of a region miss, we don't know the transaction size. We set the transaction size here // to MaxUint64 to avoid unexpected "resolve lock lite". - if len(bo.errors) > 0 { + if bo.ErrorsNum() > 0 { txnSize = math.MaxUint64 } @@ -175,7 +176,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -203,7 +204,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff if prewriteResp.MinCommitTs != 0 { return errors.Trace(errors.New("MinCommitTs must be 0 when 1pc falls back to 2pc")) } - logutil.Logger(bo.ctx).Warn("1pc failed and fallbacks to normal commit procedure", + logutil.Logger(bo.GetCtx()).Warn("1pc failed and fallbacks to normal commit procedure", zap.Uint64("startTS", c.startTS)) metrics.OnePCTxnCounterFallback.Inc() c.setOnePC(false) @@ -212,14 +213,14 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff // For 1PC, there's no racing to access to access `onePCCommmitTS` so it's safe // not to lock the mutex. if c.onePCCommitTS != 0 { - logutil.Logger(bo.ctx).Fatal("one pc happened multiple times", + logutil.Logger(bo.GetCtx()).Fatal("one pc happened multiple times", zap.Uint64("startTS", c.startTS)) } c.onePCCommitTS = prewriteResp.OnePcCommitTs } return nil } else if prewriteResp.OnePcCommitTs != 0 { - logutil.Logger(bo.ctx).Fatal("tikv committed a non-1pc transaction with 1pc protocol", + logutil.Logger(bo.GetCtx()).Fatal("tikv committed a non-1pc transaction with 1pc protocol", zap.Uint64("startTS", c.startTS)) } if c.isAsyncCommit() { @@ -230,7 +231,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff if c.testingKnobs.noFallBack { return nil } - logutil.Logger(bo.ctx).Warn("async commit cannot proceed since the returned minCommitTS is zero, "+ + logutil.Logger(bo.GetCtx()).Warn("async commit cannot proceed since the returned minCommitTS is zero, "+ "fallback to normal path", zap.Uint64("startTS", c.startTS)) c.setAsyncCommit(false) } else { @@ -268,7 +269,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff } atomic.AddInt64(&c.getDetail().ResolveLockTime, int64(time.Since(start))) if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(BoTxnLock, int(msBeforeExpired), errors.Errorf("2PC prewrite lockedKeys: %d", len(locks))) + err = bo.BackoffWithMaxSleep(retry.BoTxnLock, int(msBeforeExpired), errors.Errorf("2PC prewrite lockedKeys: %d", len(locks))) if err != nil { return errors.Trace(err) } @@ -277,10 +278,10 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff } func (c *twoPhaseCommitter) prewriteMutations(bo *Backoffer, mutations CommitterMutations) error { - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("twoPhaseCommitter.prewriteMutations", opentracing.ChildOf(span.Context())) defer span1.Finish() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } // `doActionOnMutations` will unset `useOnePC` if the mutations is splitted into multiple batches. diff --git a/store/tikv/range_task.go b/store/tikv/range_task.go index 1395fac0609a2..bc9b8fa9999c1 100644 --- a/store/tikv/range_task.go +++ b/store/tikv/range_task.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "go.uber.org/zap" ) @@ -91,6 +92,8 @@ func (s *RangeTaskRunner) SetRegionsPerTask(regionsPerTask int) { s.regionsPerTask = regionsPerTask } +const locateRegionMaxBackoff = 20000 + // RunOnRange runs the task on the given range. // Empty startKey or endKey means unbounded. func (s *RangeTaskRunner) RunOnRange(ctx context.Context, startKey, endKey []byte) error { @@ -157,7 +160,7 @@ Loop: default: } - bo := NewBackofferWithVars(ctx, locateRegionMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, locateRegionMaxBackoff, nil) rangeEndKey, err := s.store.GetRegionCache().BatchLoadRegionsFromKey(bo, key, s.regionsPerTask) if err != nil { diff --git a/store/tikv/rawkv.go b/store/tikv/rawkv.go index 2a80d26917a20..b96828df187fd 100644 --- a/store/tikv/rawkv.go +++ b/store/tikv/rawkv.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" pd "github.com/tikv/pd/client" ) @@ -110,6 +111,8 @@ func (c *RawKVClient) Get(key []byte) ([]byte, error) { return cmdResp.Value, nil } +const rawkvMaxBackoff = 20000 + // BatchGet queries values with the keys. func (c *RawKVClient) BatchGet(keys [][]byte) ([][]byte, error) { start := time.Now() @@ -117,7 +120,7 @@ func (c *RawKVClient) BatchGet(keys [][]byte) ([][]byte, error) { metrics.RawkvCmdHistogramWithBatchGet.Observe(time.Since(start).Seconds()) }() - bo := NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) resp, err := c.sendBatchReq(bo, keys, tikvrpc.CmdRawBatchGet) if err != nil { return nil, errors.Trace(err) @@ -184,7 +187,7 @@ func (c *RawKVClient) BatchPut(keys, values [][]byte) error { return errors.New("empty value is not supported") } } - bo := NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) err := c.sendBatchPut(bo, keys, values) return errors.Trace(err) } @@ -218,7 +221,7 @@ func (c *RawKVClient) BatchDelete(keys [][]byte) error { metrics.RawkvCmdHistogramWithBatchDelete.Observe(time.Since(start).Seconds()) }() - bo := NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) resp, err := c.sendBatchReq(bo, keys, tikvrpc.CmdRawBatchDelete) if err != nil { return errors.Trace(err) @@ -350,7 +353,7 @@ func (c *RawKVClient) ReverseScan(startKey, endKey []byte, limit int) (keys [][] } func (c *RawKVClient) sendReq(key []byte, req *tikvrpc.Request, reverse bool) (*tikvrpc.Response, *KeyLocation, error) { - bo := NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) sender := NewRegionRequestSender(c.regionCache, c.rpcClient) for { var loc *KeyLocation @@ -372,7 +375,7 @@ func (c *RawKVClient) sendReq(key []byte, req *tikvrpc.Request, reverse bool) (* return nil, nil, errors.Trace(err) } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return nil, nil, errors.Trace(err) } @@ -456,7 +459,7 @@ func (c *RawKVClient) doBatchReq(bo *Backoffer, batch batch, cmdType tikvrpc.Cmd return batchResp } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { batchResp.err = errors.Trace(err) return batchResp @@ -490,7 +493,7 @@ func (c *RawKVClient) doBatchReq(bo *Backoffer, batch batch, cmdType tikvrpc.Cmd // We can't use sendReq directly, because we need to know the end of the region before we send the request // TODO: Is there any better way to avoid duplicating code with func `sendReq` ? func (c *RawKVClient) sendDeleteRangeReq(startKey []byte, endKey []byte) (*tikvrpc.Response, []byte, error) { - bo := NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), rawkvMaxBackoff, nil) sender := NewRegionRequestSender(c.regionCache, c.rpcClient) for { loc, err := c.regionCache.LocateKey(bo, startKey) @@ -517,7 +520,7 @@ func (c *RawKVClient) sendDeleteRangeReq(startKey []byte, endKey []byte) (*tikvr return nil, nil, errors.Trace(err) } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return nil, nil, errors.Trace(err) } @@ -622,7 +625,7 @@ func (c *RawKVClient) doBatchPut(bo *Backoffer, batch batch) error { return errors.Trace(err) } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index a73684fdf49c5..f6225a2724f8e 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" pd "github.com/tikv/pd/client" atomic2 "go.uber.org/atomic" @@ -194,7 +195,7 @@ func (r *Region) init(c *RegionCache) error { if !exists { store = c.getStoreByStoreID(p.StoreId) } - _, err := store.initResolve(NewNoopBackoff(context.Background()), c) + _, err := store.initResolve(retry.NewNoopBackoff(context.Background()), c) if err != nil { return err } @@ -644,7 +645,7 @@ func (c *RegionCache) findRegionByKey(bo *Backoffer, key []byte, isEndKey bool) // no region data, return error if failure. return nil, err } - logutil.Eventf(bo.ctx, "load region %d from pd, due to cache-miss", lr.GetID()) + logutil.Eventf(bo.GetCtx(), "load region %d from pd, due to cache-miss", lr.GetID()) r = lr c.mu.Lock() c.insertRegionToCache(r) @@ -654,10 +655,10 @@ func (c *RegionCache) findRegionByKey(bo *Backoffer, key []byte, isEndKey bool) lr, err := c.loadRegion(bo, key, isEndKey) if err != nil { // ignore error and use old region info. - logutil.Logger(bo.ctx).Error("load region failure", + logutil.Logger(bo.GetCtx()).Error("load region failure", zap.ByteString("key", key), zap.Error(err)) } else { - logutil.Eventf(bo.ctx, "load region %d from pd, due to need-reload", lr.GetID()) + logutil.Eventf(bo.GetCtx(), "load region %d from pd, due to need-reload", lr.GetID()) r = lr c.mu.Lock() c.insertRegionToCache(r) @@ -674,7 +675,7 @@ func (c *RegionCache) OnSendFail(bo *Backoffer, ctx *RPCContext, scheduleReload if r != nil { peersNum := len(r.meta.Peers) if len(ctx.Meta.Peers) != peersNum { - logutil.Logger(bo.ctx).Info("retry and refresh current ctx after send request fail and up/down stores length changed", + logutil.Logger(bo.GetCtx()).Info("retry and refresh current ctx after send request fail and up/down stores length changed", zap.Stringer("current", ctx), zap.Bool("needReload", scheduleReload), zap.Reflect("oldPeers", ctx.Meta.Peers), @@ -727,20 +728,20 @@ func (c *RegionCache) OnSendFail(bo *Backoffer, ctx *RPCContext, scheduleReload // In case the epoch of the store is increased, try to avoid reloading the current region by also // increasing the epoch stored in `rs`. rs.switchNextProxyStore(r, currentProxyIdx, incEpochStoreIdx) - logutil.Logger(bo.ctx).Info("switch region proxy peer to next due to send request fail", + logutil.Logger(bo.GetCtx()).Info("switch region proxy peer to next due to send request fail", zap.Stringer("current", ctx), zap.Bool("needReload", scheduleReload), zap.Error(err)) } else { rs.switchNextTiKVPeer(r, ctx.AccessIdx) - logutil.Logger(bo.ctx).Info("switch region peer to next due to send request fail", + logutil.Logger(bo.GetCtx()).Info("switch region peer to next due to send request fail", zap.Stringer("current", ctx), zap.Bool("needReload", scheduleReload), zap.Error(err)) } } else { rs.switchNextFlashPeer(r, ctx.AccessIdx) - logutil.Logger(bo.ctx).Info("switch region tiflash peer to next due to send request fail", + logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", zap.Stringer("current", ctx), zap.Bool("needReload", scheduleReload), zap.Error(err)) @@ -763,7 +764,7 @@ func (c *RegionCache) LocateRegionByID(bo *Backoffer, regionID uint64) (*KeyLoca lr, err := c.loadRegionByID(bo, regionID) if err != nil { // ignore error and use old region info. - logutil.Logger(bo.ctx).Error("load region failure", + logutil.Logger(bo.GetCtx()).Error("load region failure", zap.Uint64("regionID", regionID), zap.Error(err)) } else { r = lr @@ -1125,7 +1126,7 @@ func filterUnavailablePeers(region *pd.Region) { // If the given key is the end key of the region that you want, you may set the second argument to true. This is useful // when processing in reverse order. func (c *RegionCache) loadRegion(bo *Backoffer, key []byte, isEndKey bool) (*Region, error) { - ctx := bo.ctx + ctx := bo.GetCtx() if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("loadRegion", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -1136,7 +1137,7 @@ func (c *RegionCache) loadRegion(bo *Backoffer, key []byte, isEndKey bool) (*Reg searchPrev := false for { if backoffErr != nil { - err := bo.Backoff(BoPDRPC, backoffErr) + err := bo.Backoff(retry.BoPDRPC, backoffErr) if err != nil { return nil, errors.Trace(err) } @@ -1183,7 +1184,7 @@ func (c *RegionCache) loadRegion(bo *Backoffer, key []byte, isEndKey bool) (*Reg // loadRegionByID loads region from pd client, and picks the first peer as leader. func (c *RegionCache) loadRegionByID(bo *Backoffer, regionID uint64) (*Region, error) { - ctx := bo.ctx + ctx := bo.GetCtx() if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("loadRegionByID", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -1192,7 +1193,7 @@ func (c *RegionCache) loadRegionByID(bo *Backoffer, regionID uint64) (*Region, e var backoffErr error for { if backoffErr != nil { - err := bo.Backoff(BoPDRPC, backoffErr) + err := bo.Backoff(retry.BoPDRPC, backoffErr) if err != nil { return nil, errors.Trace(err) } @@ -1232,7 +1233,7 @@ func (c *RegionCache) scanRegions(bo *Backoffer, startKey, endKey []byte, limit if limit == 0 { return nil, nil } - ctx := bo.ctx + ctx := bo.GetCtx() if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("scanRegions", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -1242,7 +1243,7 @@ func (c *RegionCache) scanRegions(bo *Backoffer, startKey, endKey []byte, limit var backoffErr error for { if backoffErr != nil { - err := bo.Backoff(BoPDRPC, backoffErr) + err := bo.Backoff(retry.BoPDRPC, backoffErr) if err != nil { return nil, errors.Trace(err) } @@ -1412,7 +1413,7 @@ func (c *RegionCache) OnRegionEpochNotMatch(bo *Backoffer, ctx *RPCContext, curr meta.GetRegionEpoch().GetVersion() < ctx.Region.ver) { err := errors.Errorf("region epoch is ahead of tikv. rpc ctx: %+v, currentRegions: %+v", ctx, currentRegions) logutil.BgLogger().Info("region epoch is ahead of tikv", zap.Error(err)) - return bo.Backoff(BoRegionMiss, err) + return bo.Backoff(retry.BoRegionMiss, err) } } @@ -1775,7 +1776,7 @@ func (s *Store) initResolve(bo *Backoffer, c *RegionCache) (addr string, err err } var store *metapb.Store for { - store, err = c.pdClient.GetStore(bo.ctx, s.storeID) + store, err = c.pdClient.GetStore(bo.GetCtx(), s.storeID) if err != nil { metrics.RegionCacheCounterWithGetStoreError.Inc() } else { @@ -1787,7 +1788,7 @@ func (s *Store) initResolve(bo *Backoffer, c *RegionCache) (addr string, err err return } err = errors.Errorf("loadStore from PD failed, id: %d, err: %v", s.storeID, err) - if err = bo.Backoff(BoPDRPC, err); err != nil { + if err = bo.Backoff(retry.BoPDRPC, err); err != nil { return } continue @@ -1990,7 +1991,7 @@ func (s *Store) checkUntilHealth(c *RegionCache) { } } - bo := NewNoopBackoff(ctx) + bo := retry.NewNoopBackoff(ctx) l := s.requestLiveness(bo, c) if l == reachable { logutil.BgLogger().Info("[health check] store became reachable", zap.Uint64("storeID", s.storeID)) @@ -2020,7 +2021,7 @@ func (s *Store) requestLiveness(bo *Backoffer, c *RegionCache) (l livenessState) }) var ctx context.Context if bo != nil { - ctx = bo.ctx + ctx = bo.GetCtx() } else { ctx = context.Background() } diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index efb2ae9df73ab..7b0e5884abb2a 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" + "github.com/pingcap/tidb/store/tikv/retry" pd "github.com/tikv/pd/client" ) @@ -310,7 +311,7 @@ func (s *testRegionCacheSuite) TestUpdateLeader3(c *C) { ctx, err := s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, seed) c.Assert(err, IsNil) c.Assert(ctx.Addr, Equals, "store2") - s.cache.OnSendFail(NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) + s.cache.OnSendFail(retry.NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) s.cache.checkAndResolve(nil) s.cache.UpdateLeader(loc.Region, s.store2, 0) addr := s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0) @@ -870,7 +871,7 @@ func (s *testRegionCacheSuite) TestRegionEpochAheadOfTiKV(c *C) { c.Assert(err, IsNil) err = cache.OnRegionEpochNotMatch(bo, &RPCContext{Region: region.VerID()}, []*metapb.Region{&r2}) c.Assert(err, IsNil) - c.Assert(len(bo.errors), Equals, 2) + c.Assert(bo.ErrorsNum(), Equals, 2) } func (s *testRegionCacheSuite) TestRegionEpochOnTiFlash(c *C) { @@ -1329,7 +1330,7 @@ func (s *testRegionCacheSuite) TestPeersLenChange(c *C) { s.cache.insertRegionToCache(region) // OnSendFail should not panic - s.cache.OnSendFail(NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) + s.cache.OnSendFail(retry.NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) } func createSampleRegion(startKey, endKey []byte) *Region { diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 74ecdd7ce72b8..ec5b92f81c5ad 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/util" ) @@ -223,12 +224,12 @@ func (s *RegionRequestSender) SendReqCtx( rpcCtx *RPCContext, err error, ) { - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("regionRequest.SendReqCtx", opentracing.ChildOf(span.Context())) defer span1.Finish() // TODO(MyonKeminta): Make sure trace works without cloning the backoffer. // bo = bo.Clone() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } failpoint.Inject("tikvStoreSendReqResult", func(val failpoint.Value) { @@ -248,9 +249,7 @@ func (s *RegionRequestSender) SendReqCtx( }, nil, nil) } case "callBackofferHook": - if bo.vars != nil && bo.vars.Hook != nil { - bo.vars.Hook("callBackofferHook", bo.vars) - } + bo.SetVarsHook("callBackofferHook", bo.GetVars()) case "requestTiDBStoreError": if et == tikvrpc.TiDB { failpoint.Return(nil, nil, tikverr.ErrTiKVServerTimeout) @@ -265,7 +264,7 @@ func (s *RegionRequestSender) SendReqCtx( tryTimes := 0 for { if (tryTimes > 0) && (tryTimes%1000 == 0) { - logutil.Logger(bo.ctx).Warn("retry get ", zap.Uint64("region = ", regionID.GetID()), zap.Int("times = ", tryTimes)) + logutil.Logger(bo.GetCtx()).Warn("retry get ", zap.Uint64("region = ", regionID.GetID()), zap.Int("times = ", tryTimes)) } rpcCtx, err = s.getRPCContext(bo, req, regionID, et, opts...) @@ -275,7 +274,7 @@ func (s *RegionRequestSender) SendReqCtx( failpoint.Inject("invalidCacheAndRetry", func() { // cooperate with github.com/pingcap/tidb/store/gcworker/setGcResolveMaxBackoff - if c := bo.ctx.Value("injectedBackoff"); c != nil { + if c := bo.GetCtx().Value("injectedBackoff"); c != nil { resp, err = tikvrpc.GenRegionErrorResp(req, &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}}) failpoint.Return(resp, nil, err) } @@ -287,12 +286,12 @@ func (s *RegionRequestSender) SendReqCtx( // TODO: Change the returned error to something like "region missing in cache", // and handle this error like EpochNotMatch, which means to re-split the request and retry. - logutil.Logger(bo.ctx).Debug("throwing pseudo region error due to region not found in cache", zap.Stringer("region", ®ionID)) + logutil.Logger(bo.GetCtx()).Debug("throwing pseudo region error due to region not found in cache", zap.Stringer("region", ®ionID)) resp, err = tikvrpc.GenRegionErrorResp(req, &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}}) return resp, nil, err } - logutil.Eventf(bo.ctx, "send %s request to region %d at %s", req.Type, regionID.id, rpcCtx.Addr) + logutil.Eventf(bo.GetCtx(), "send %s request to region %d at %s", req.Type, regionID.id, rpcCtx.Addr) s.storeAddr = rpcCtx.Addr var retry bool resp, retry, err = s.sendReqToRegion(bo, rpcCtx, req, timeout) @@ -301,7 +300,8 @@ func (s *RegionRequestSender) SendReqCtx( } // recheck whether the session/query is killed during the Next() - if bo.vars != nil && bo.vars.Killed != nil && atomic.LoadUint32(bo.vars.Killed) == 1 { + boVars := bo.GetVars() + if boVars != nil && boVars.Killed != nil && atomic.LoadUint32(boVars.Killed) == 1 { return nil, nil, tikverr.ErrQueryInterrupted } failpoint.Inject("mockRetrySendReqToRegion", func(val failpoint.Value) { @@ -392,7 +392,7 @@ func (s *RegionRequestSender) sendReqToRegion(bo *Backoffer, rpcCtx *RPCContext, defer s.releaseStoreToken(rpcCtx.Store) } - ctx := bo.ctx + ctx := bo.GetCtx() if rawHook := ctx.Value(RPCCancellerCtxKey{}); rawHook != nil { var cancel context.CancelFunc ctx, cancel = rawHook.(*RPCCanceller).WithCancel(ctx) @@ -410,7 +410,7 @@ func (s *RegionRequestSender) sendReqToRegion(bo *Backoffer, rpcCtx *RPCContext, } var sessionID uint64 - if v := bo.ctx.Value(util.SessionID); v != nil { + if v := bo.GetCtx().Value(util.SessionID); v != nil { sessionID = v.(uint64) } @@ -443,7 +443,7 @@ func (s *RegionRequestSender) sendReqToRegion(bo *Backoffer, rpcCtx *RPCContext, RecordRegionRequestRuntimeStats(s.Stats, req.Type, time.Since(start)) failpoint.Inject("tikvStoreRespResult", func(val failpoint.Value) { if val.(bool) { - if req.Type == tikvrpc.CmdCop && bo.totalSleep == 0 { + if req.Type == tikvrpc.CmdCop && bo.GetTotalSleep() == 0 { failpoint.Return(&tikvrpc.Response{ Resp: &coprocessor.Response{RegionError: &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}}}, }, false, nil) @@ -544,12 +544,12 @@ func (s *RegionRequestSender) releaseStoreToken(st *Store) { } func (s *RegionRequestSender) onSendFail(bo *Backoffer, ctx *RPCContext, err error) error { - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("regionRequest.onSendFail", opentracing.ChildOf(span.Context())) defer span1.Finish() // TODO(MyonKeminta): Make sure trace works without cloning the backoffer. // bo = bo.Clone() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled { @@ -559,7 +559,7 @@ func (s *RegionRequestSender) onSendFail(bo *Backoffer, ctx *RPCContext, err err } if status.Code(errors.Cause(err)) == codes.Canceled { select { - case <-bo.ctx.Done(): + case <-bo.GetCtx().Done(): return errors.Trace(err) default: // If we don't cancel, but the error code is Canceled, it must be from grpc remote. @@ -578,9 +578,9 @@ func (s *RegionRequestSender) onSendFail(bo *Backoffer, ctx *RPCContext, err err // TODO: the number of retry time should be limited:since region may be unavailable // when some unrecoverable disaster happened. if ctx.Store != nil && ctx.Store.storeType == tikvrpc.TiFlash { - err = bo.Backoff(BoTiFlashRPC, errors.Errorf("send tiflash request error: %v, ctx: %v, try next peer later", err, ctx)) + err = bo.Backoff(retry.BoTiFlashRPC, errors.Errorf("send tiflash request error: %v, ctx: %v, try next peer later", err, ctx)) } else { - err = bo.Backoff(BoTiKVRPC, errors.Errorf("send tikv request error: %v, ctx: %v, try next peer later", err, ctx)) + err = bo.BackoffTiKVRPC(errors.Errorf("send tikv request error: %v, ctx: %v, try next peer later", err, ctx)) } return errors.Trace(err) } @@ -632,13 +632,13 @@ func regionErrorToLabel(e *errorpb.Error) string { return "unknown" } -func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed *uint32, regionErr *errorpb.Error) (retry bool, err error) { - if span := opentracing.SpanFromContext(bo.ctx); span != nil && span.Tracer() != nil { +func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed *uint32, regionErr *errorpb.Error) (shouldRetry bool, err error) { + if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("tikv.onRegionError", opentracing.ChildOf(span.Context())) defer span1.Finish() // TODO(MyonKeminta): Make sure trace works without cloning the backoffer. // bo = bo.Clone() - bo.ctx = opentracing.ContextWithSpan(bo.ctx, span1) + bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } metrics.TiKVRegionErrorCounter.WithLabelValues(regionErrorToLabel(regionErr)).Inc() @@ -654,7 +654,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed // isolated and removed from the Raft group. So it's necessary to reload // the region from PD. s.regionCache.InvalidateCachedRegionWithReason(ctx.Region, NoLeader) - if err = bo.Backoff(BoRegionMiss, errors.Errorf("not leader: %v, ctx: %v", notLeader, ctx)); err != nil { + if err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("not leader: %v, ctx: %v", notLeader, ctx)); err != nil { return false, errors.Trace(err) } } else { @@ -690,9 +690,9 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed zap.String("reason", regionErr.GetServerIsBusy().GetReason()), zap.Stringer("ctx", ctx)) if ctx != nil && ctx.Store != nil && ctx.Store.storeType == tikvrpc.TiFlash { - err = bo.Backoff(boTiFlashServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) + err = bo.Backoff(retry.BoTiFlashServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) } else { - err = bo.Backoff(boTiKVServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) + err = bo.Backoff(retry.BoTiKVServerBusy, errors.Errorf("server is busy, ctx: %v", ctx)) } if err != nil { return false, errors.Trace(err) @@ -701,7 +701,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed } if regionErr.GetStaleCommand() != nil { logutil.BgLogger().Debug("tikv reports `StaleCommand`", zap.Stringer("ctx", ctx)) - err = bo.Backoff(boStaleCmd, errors.Errorf("stale command, ctx: %v", ctx)) + err = bo.Backoff(retry.BoStaleCmd, errors.Errorf("stale command, ctx: %v", ctx)) if err != nil { return false, errors.Trace(err) } @@ -718,7 +718,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed } if regionErr.GetMaxTimestampNotSynced() != nil { logutil.BgLogger().Warn("tikv reports `MaxTimestampNotSynced`", zap.Stringer("ctx", ctx)) - err = bo.Backoff(boMaxTsNotSynced, errors.Errorf("max timestamp not synced, ctx: %v", ctx)) + err = bo.Backoff(retry.BoMaxTsNotSynced, errors.Errorf("max timestamp not synced, ctx: %v", ctx)) if err != nil { return false, errors.Trace(err) } diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 8d531ee209a78..f7fd2a149060d 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/kvproto/pkg/tikvpb" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" @@ -72,7 +73,7 @@ func (s *testRegionRequestToSingleStoreSuite) SetUpTest(c *C) { s.store, s.peer, s.region = mocktikv.BootstrapWithSingleStore(s.cluster) pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster)} s.cache = NewRegionCache(pdCli) - s.bo = NewNoopBackoff(context.Background()) + s.bo = retry.NewNoopBackoff(context.Background()) s.mvccStore = mocktikv.MustNewMVCCStore() client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) s.regionRequestSender = NewRegionRequestSender(s.cache, client) @@ -83,7 +84,7 @@ func (s *testRegionRequestToThreeStoresSuite) SetUpTest(c *C) { s.storeIDs, s.peerIDs, s.regionID, s.leaderPeer = mocktikv.BootstrapWithMultiStores(s.cluster, 3) pdCli := &CodecPDClient{mocktikv.NewPDClient(s.cluster)} s.cache = NewRegionCache(pdCli) - s.bo = NewNoopBackoff(context.Background()) + s.bo = retry.NewNoopBackoff(context.Background()) s.mvccStore = mocktikv.MustNewMVCCStore() client := mocktikv.NewRPCClient(s.cluster, s.mvccStore, nil) s.regionRequestSender = NewRegionRequestSender(s.cache, client) diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go new file mode 100644 index 0000000000000..24dc9174f3fec --- /dev/null +++ b/store/tikv/retry/backoff.go @@ -0,0 +1,439 @@ +// Copyright 2016 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package retry + +import ( + "context" + "fmt" + "math" + "math/rand" + "strings" + "sync/atomic" + "time" + + "github.com/opentracing/opentracing-go" + "github.com/pingcap/errors" + "github.com/pingcap/log" + tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/util" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" +) + +const ( + // NoJitter makes the backoff sequence strict exponential. + NoJitter = 1 + iota + // FullJitter applies random factors to strict exponential. + FullJitter + // EqualJitter is also randomized, but prevents very short sleeps. + EqualJitter + // DecorrJitter increases the maximum jitter based on the last random value. + DecorrJitter +) + +func (t BackoffType) metric() prometheus.Observer { + switch t { + // TODO: distinguish tikv and tiflash in metrics + case boTiKVRPC, BoTiFlashRPC: + return metrics.BackoffHistogramRPC + case BoTxnLock: + return metrics.BackoffHistogramLock + case BoTxnLockFast: + return metrics.BackoffHistogramLockFast + case BoPDRPC: + return metrics.BackoffHistogramPD + case BoRegionMiss: + return metrics.BackoffHistogramRegionMiss + case BoTiKVServerBusy, BoTiFlashServerBusy: + return metrics.BackoffHistogramServerBusy + case BoStaleCmd: + return metrics.BackoffHistogramStaleCmd + } + return metrics.BackoffHistogramEmpty +} + +// NewBackoffFn creates a backoff func which implements exponential backoff with +// optional jitters. +// See http://www.awsarchitectureblog.com/2015/03/backoff.html +func NewBackoffFn(base, cap, jitter int) func(ctx context.Context, maxSleepMs int) int { + if base < 2 { + // Top prevent panic in 'rand.Intn'. + base = 2 + } + attempts := 0 + lastSleep := base + return func(ctx context.Context, maxSleepMs int) int { + var sleep int + switch jitter { + case NoJitter: + sleep = expo(base, cap, attempts) + case FullJitter: + v := expo(base, cap, attempts) + sleep = rand.Intn(v) + case EqualJitter: + v := expo(base, cap, attempts) + sleep = v/2 + rand.Intn(v/2) + case DecorrJitter: + sleep = int(math.Min(float64(cap), float64(base+rand.Intn(lastSleep*3-base)))) + } + logutil.BgLogger().Debug("backoff", + zap.Int("base", base), + zap.Int("sleep", sleep), + zap.Int("attempts", attempts)) + + realSleep := sleep + // when set maxSleepMs >= 0 in `tikv.BackoffWithMaxSleep` will force sleep maxSleepMs milliseconds. + if maxSleepMs >= 0 && realSleep > maxSleepMs { + realSleep = maxSleepMs + } + select { + case <-time.After(time.Duration(realSleep) * time.Millisecond): + attempts++ + lastSleep = sleep + return realSleep + case <-ctx.Done(): + return 0 + } + } +} + +func expo(base, cap, n int) int { + return int(math.Min(float64(cap), float64(base)*math.Pow(2.0, float64(n)))) +} + +// BackoffType defines the backoff type. +type BackoffType int + +// Back off types. +const ( + boTiKVRPC BackoffType = iota + BoTiFlashRPC + BoTxnLock + BoTxnLockFast + BoPDRPC + BoRegionMiss + BoTiKVServerBusy + BoTiFlashServerBusy + BoTxnNotFound + BoStaleCmd + BoMaxTsNotSynced +) + +func (t BackoffType) createFn(vars *kv.Variables) func(context.Context, int) int { + if vars.Hook != nil { + vars.Hook(t.String(), vars) + } + switch t { + case boTiKVRPC, BoTiFlashRPC: + return NewBackoffFn(100, 2000, EqualJitter) + case BoTxnLock: + return NewBackoffFn(200, 3000, EqualJitter) + case BoTxnLockFast: + return NewBackoffFn(vars.BackoffLockFast, 3000, EqualJitter) + case BoPDRPC: + return NewBackoffFn(500, 3000, EqualJitter) + case BoRegionMiss: + // change base time to 2ms, because it may recover soon. + return NewBackoffFn(2, 500, NoJitter) + case BoTxnNotFound: + return NewBackoffFn(2, 500, NoJitter) + case BoTiKVServerBusy, BoTiFlashServerBusy: + return NewBackoffFn(2000, 10000, EqualJitter) + case BoStaleCmd: + return NewBackoffFn(2, 1000, NoJitter) + case BoMaxTsNotSynced: + return NewBackoffFn(2, 500, NoJitter) + } + return nil +} + +func (t BackoffType) String() string { + switch t { + case boTiKVRPC: + return "tikvRPC" + case BoTiFlashRPC: + return "tiflashRPC" + case BoTxnLock: + return "txnLock" + case BoTxnLockFast: + return "txnLockFast" + case BoPDRPC: + return "pdRPC" + case BoRegionMiss: + return "regionMiss" + case BoTiKVServerBusy: + return "tikvServerBusy" + case BoTiFlashServerBusy: + return "tiflashServerBusy" + case BoStaleCmd: + return "staleCommand" + case BoTxnNotFound: + return "txnNotFound" + case BoMaxTsNotSynced: + return "maxTsNotSynced" + } + return "" +} + +// TError returns pingcap/error of the backoff type. +func (t BackoffType) TError() error { + switch t { + case boTiKVRPC: + return tikverr.ErrTiKVServerTimeout + case BoTiFlashRPC: + return tikverr.ErrTiFlashServerTimeout + case BoTxnLock, BoTxnLockFast, BoTxnNotFound: + return tikverr.ErrResolveLockTimeout + case BoPDRPC: + return tikverr.NewErrPDServerTimeout("") + case BoRegionMiss: + return tikverr.ErrRegionUnavailable + case BoTiKVServerBusy: + return tikverr.ErrTiKVServerBusy + case BoTiFlashServerBusy: + return tikverr.ErrTiFlashServerBusy + case BoStaleCmd: + return tikverr.ErrTiKVStaleCommand + case BoMaxTsNotSynced: + return tikverr.ErrTiKVMaxTimestampNotSynced + } + return tikverr.ErrUnknown +} + +// Backoffer is a utility for retrying queries. +type Backoffer struct { + ctx context.Context + + fn map[BackoffType]func(context.Context, int) int + maxSleep int + totalSleep int + errors []error + types []fmt.Stringer + vars *kv.Variables + noop bool + + backoffSleepMS map[BackoffType]int + backoffTimes map[BackoffType]int +} + +type txnStartCtxKeyType struct{} + +// TxnStartKey is a key for transaction start_ts info in context.Context. +var TxnStartKey interface{} = txnStartCtxKeyType{} + +// NewBackoffer (Deprecated) creates a Backoffer with maximum sleep time(in ms). +func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { + return &Backoffer{ + ctx: ctx, + maxSleep: maxSleep, + vars: kv.DefaultVars, + } +} + +// NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. +func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *Backoffer { + return NewBackoffer(ctx, maxSleep).withVars(vars) +} + +// NewNoopBackoff create a Backoffer do nothing just return error directly +func NewNoopBackoff(ctx context.Context) *Backoffer { + return &Backoffer{ctx: ctx, noop: true} +} + +// withVars sets the kv.Variables to the Backoffer and return it. +func (b *Backoffer) withVars(vars *kv.Variables) *Backoffer { + if vars != nil { + b.vars = vars + } + // maxSleep is the max sleep time in millisecond. + // When it is multiplied by BackOffWeight, it should not be greater than MaxInt32. + if math.MaxInt32/b.vars.BackOffWeight >= b.maxSleep { + b.maxSleep *= b.vars.BackOffWeight + } + return b +} + +// Backoff sleeps a while base on the backoffType and records the error message. +// It returns a retryable error if total sleep time exceeds maxSleep. +func (b *Backoffer) Backoff(typ BackoffType, err error) error { + if span := opentracing.SpanFromContext(b.ctx); span != nil && span.Tracer() != nil { + span1 := span.Tracer().StartSpan(fmt.Sprintf("tikv.backoff.%s", typ), opentracing.ChildOf(span.Context())) + defer span1.Finish() + opentracing.ContextWithSpan(b.ctx, span1) + } + return b.BackoffWithMaxSleep(typ, -1, err) +} + +// BackoffTiKVRPC calls Backoff with boTiKVRPC. +func (b *Backoffer) BackoffTiKVRPC(err error) error { + return b.Backoff(boTiKVRPC, err) +} + +// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message +// and never sleep more than maxSleepMs for each sleep. +func (b *Backoffer) BackoffWithMaxSleep(typ BackoffType, maxSleepMs int, err error) error { + if strings.Contains(err.Error(), tikverr.MismatchClusterID) { + logutil.BgLogger().Fatal("critical error", zap.Error(err)) + } + select { + case <-b.ctx.Done(): + return errors.Trace(err) + default: + } + + b.errors = append(b.errors, errors.Errorf("%s at %s", err.Error(), time.Now().Format(time.RFC3339Nano))) + b.types = append(b.types, typ) + if b.noop || (b.maxSleep > 0 && b.totalSleep >= b.maxSleep) { + errMsg := fmt.Sprintf("%s backoffer.maxSleep %dms is exceeded, errors:", typ.String(), b.maxSleep) + for i, err := range b.errors { + // Print only last 3 errors for non-DEBUG log levels. + if log.GetLevel() == zapcore.DebugLevel || i >= len(b.errors)-3 { + errMsg += "\n" + err.Error() + } + } + logutil.BgLogger().Warn(errMsg) + // Use the first backoff type to generate a MySQL error. + return b.types[0].(BackoffType).TError() + } + + // Lazy initialize. + if b.fn == nil { + b.fn = make(map[BackoffType]func(context.Context, int) int) + } + f, ok := b.fn[typ] + if !ok { + f = typ.createFn(b.vars) + b.fn[typ] = f + } + + realSleep := f(b.ctx, maxSleepMs) + typ.metric().Observe(float64(realSleep) / 1000) + b.totalSleep += realSleep + if b.backoffSleepMS == nil { + b.backoffSleepMS = make(map[BackoffType]int) + } + b.backoffSleepMS[typ] += realSleep + if b.backoffTimes == nil { + b.backoffTimes = make(map[BackoffType]int) + } + b.backoffTimes[typ]++ + + stmtExec := b.ctx.Value(util.ExecDetailsKey) + if stmtExec != nil { + detail := stmtExec.(*util.ExecDetails) + atomic.AddInt64(&detail.BackoffDuration, int64(realSleep)*int64(time.Millisecond)) + atomic.AddInt64(&detail.BackoffCount, 1) + } + + if b.vars != nil && b.vars.Killed != nil { + if atomic.LoadUint32(b.vars.Killed) == 1 { + return tikverr.ErrQueryInterrupted + } + } + + var startTs interface{} + if ts := b.ctx.Value(TxnStartKey); ts != nil { + startTs = ts + } + logutil.Logger(b.ctx).Debug("retry later", + zap.Error(err), + zap.Int("totalSleep", b.totalSleep), + zap.Int("maxSleep", b.maxSleep), + zap.Stringer("type", typ), + zap.Reflect("txnStartTS", startTs)) + return nil +} + +func (b *Backoffer) String() string { + if b.totalSleep == 0 { + return "" + } + return fmt.Sprintf(" backoff(%dms %v)", b.totalSleep, b.types) +} + +// Clone creates a new Backoffer which keeps current Backoffer's sleep time and errors, and shares +// current Backoffer's context. +func (b *Backoffer) Clone() *Backoffer { + return &Backoffer{ + ctx: b.ctx, + maxSleep: b.maxSleep, + totalSleep: b.totalSleep, + errors: b.errors, + vars: b.vars, + } +} + +// Fork creates a new Backoffer which keeps current Backoffer's sleep time and errors, and holds +// a child context of current Backoffer's context. +func (b *Backoffer) Fork() (*Backoffer, context.CancelFunc) { + ctx, cancel := context.WithCancel(b.ctx) + return &Backoffer{ + ctx: ctx, + maxSleep: b.maxSleep, + totalSleep: b.totalSleep, + errors: b.errors, + vars: b.vars, + }, cancel +} + +// GetVars returns the binded vars. +func (b *Backoffer) GetVars() *kv.Variables { + return b.vars +} + +// GetTotalSleep returns total sleep time. +func (b *Backoffer) GetTotalSleep() int { + return b.totalSleep +} + +// GetTypes returns type list. +func (b *Backoffer) GetTypes() []fmt.Stringer { + return b.types +} + +// GetCtx returns the binded context. +func (b *Backoffer) GetCtx() context.Context { + return b.ctx +} + +// SetCtx sets the binded context to ctx. +func (b *Backoffer) SetCtx(ctx context.Context) { + b.ctx = ctx +} + +// GetBackoffTimes returns a map contains backoff time count by type. +func (b *Backoffer) GetBackoffTimes() map[BackoffType]int { + return b.backoffTimes +} + +// GetBackoffSleepMS returns a map contains backoff sleep time by type. +func (b *Backoffer) GetBackoffSleepMS() map[BackoffType]int { + return b.backoffSleepMS +} + +// ErrorsNum returns the number of errors. +func (b *Backoffer) ErrorsNum() int { + return len(b.errors) +} + +// SetVarsHook sets the vars.Hook is used for test to verify the variable take effect. +func (b *Backoffer) SetVarsHook(name string, vars *kv.Variables) { + if b.vars != nil && b.vars.Hook != nil { + b.vars.Hook(name, vars) + } +} diff --git a/store/tikv/backoff_test.go b/store/tikv/retry/backoff_test.go similarity index 98% rename from store/tikv/backoff_test.go rename to store/tikv/retry/backoff_test.go index 11254937abd72..f8dfb9ed120f3 100644 --- a/store/tikv/backoff_test.go +++ b/store/tikv/retry/backoff_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tikv +package retry import ( "context" diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 19a14b3f73819..6c43b7bdee7cd 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -22,6 +22,7 @@ import ( tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "go.uber.org/zap" ) @@ -85,9 +86,11 @@ func (s *Scanner) Value() []byte { return nil } +const scannerNextMaxBackoff = 20000 + // Next return next element. func (s *Scanner) Next() error { - bo := NewBackofferWithVars(context.WithValue(context.Background(), TxnStartKey, s.snapshot.version), scannerNextMaxBackoff, s.snapshot.vars) + bo := retry.NewBackofferWithVars(context.WithValue(context.Background(), retry.TxnStartKey, s.snapshot.version), scannerNextMaxBackoff, s.snapshot.vars) if !s.valid { return errors.New("scanner iterator is invalid") } @@ -223,7 +226,7 @@ func (s *Scanner) getData(bo *Backoffer) error { if regionErr != nil { logutil.BgLogger().Debug("scanner getData failed", zap.Stringer("regionErr", regionErr)) - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -251,7 +254,7 @@ func (s *Scanner) getData(bo *Backoffer) error { return errors.Trace(err) } if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(BoTxnLockFast, int(msBeforeExpired), errors.Errorf("key is locked during scanning")) + err = bo.BackoffWithMaxSleep(retry.BoTxnLockFast, int(msBeforeExpired), errors.Errorf("key is locked during scanning")) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 24149fdae44cb..ab3862fe4bf0b 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" @@ -126,6 +127,8 @@ func newTiKVSnapshot(store *KVStore, ts uint64, replicaReadSeed uint32) *KVSnaps } } +const batchGetMaxBackoff = 20000 + // SetSnapshotTS resets the timestamp for reads. func (s *KVSnapshot) SetSnapshotTS(ts uint64) { // Sanity check for snapshot version. @@ -170,8 +173,8 @@ func (s *KVSnapshot) BatchGet(ctx context.Context, keys [][]byte) (map[string][] // We want [][]byte instead of []kv.Key, use some magic to save memory. bytesKeys := *(*[][]byte)(unsafe.Pointer(&keys)) - ctx = context.WithValue(ctx, TxnStartKey, s.version) - bo := NewBackofferWithVars(ctx, batchGetMaxBackoff, s.vars) + ctx = context.WithValue(ctx, retry.TxnStartKey, s.version) + bo := retry.NewBackofferWithVars(ctx, batchGetMaxBackoff, s.vars) // Create a map to collect key-values from region servers. var mu sync.Mutex @@ -331,7 +334,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return errors.Trace(err) } @@ -378,7 +381,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec return errors.Trace(err) } if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(BoTxnLockFast, int(msBeforeExpired), errors.Errorf("batchGet lockedKeys: %d", len(lockedKeys))) + err = bo.BackoffWithMaxSleep(retry.BoTxnLockFast, int(msBeforeExpired), errors.Errorf("batchGet lockedKeys: %d", len(lockedKeys))) if err != nil { return errors.Trace(err) } @@ -394,6 +397,8 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec } } +const getMaxBackoff = 20000 + // Get gets the value for key k from snapshot. func (s *KVSnapshot) Get(ctx context.Context, k []byte) ([]byte, error) { @@ -401,8 +406,8 @@ func (s *KVSnapshot) Get(ctx context.Context, k []byte) ([]byte, error) { metrics.TxnCmdHistogramWithGet.Observe(time.Since(start).Seconds()) }(time.Now()) - ctx = context.WithValue(ctx, TxnStartKey, s.version) - bo := NewBackofferWithVars(ctx, getMaxBackoff, s.vars) + ctx = context.WithValue(ctx, retry.TxnStartKey, s.version) + bo := retry.NewBackofferWithVars(ctx, getMaxBackoff, s.vars) val, err := s.get(ctx, bo, k) s.recordBackoffInfo(bo) if err != nil { @@ -436,7 +441,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, opentracing.ContextWithSpan(ctx, span1) } failpoint.Inject("snapshot-get-cache-fail", func(_ failpoint.Value) { - if bo.ctx.Value("TestSnapshotCache") != nil { + if bo.GetCtx().Value("TestSnapshotCache") != nil { panic("cache miss") } }) @@ -488,7 +493,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, return nil, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { return nil, errors.Trace(err) } @@ -522,7 +527,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, return nil, errors.Trace(err) } if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(BoTxnLockFast, int(msBeforeExpired), errors.New(keyErr.String())) + err = bo.BackoffWithMaxSleep(retry.BoTxnLockFast, int(msBeforeExpired), errors.New(keyErr.String())) if err != nil { return nil, errors.Trace(err) } @@ -678,7 +683,7 @@ func extractKeyErr(keyErr *pb.KeyError) error { func (s *KVSnapshot) recordBackoffInfo(bo *Backoffer) { s.mu.RLock() - if s.mu.stats == nil || bo.totalSleep == 0 { + if s.mu.stats == nil || bo.GetTotalSleep() == 0 { s.mu.RUnlock() return } @@ -689,14 +694,14 @@ func (s *KVSnapshot) recordBackoffInfo(bo *Backoffer) { return } if s.mu.stats.backoffSleepMS == nil { - s.mu.stats.backoffSleepMS = bo.backoffSleepMS - s.mu.stats.backoffTimes = bo.backoffTimes + s.mu.stats.backoffSleepMS = bo.GetBackoffSleepMS() + s.mu.stats.backoffTimes = bo.GetBackoffTimes() return } - for k, v := range bo.backoffSleepMS { + for k, v := range bo.GetBackoffSleepMS() { s.mu.stats.backoffSleepMS[k] += v } - for k, v := range bo.backoffTimes { + for k, v := range bo.GetBackoffTimes() { s.mu.stats.backoffTimes[k] += v } } @@ -725,8 +730,8 @@ func (s *KVSnapshot) mergeRegionRequestStats(stats map[tikvrpc.CmdType]*RPCRunti // SnapshotRuntimeStats records the runtime stats of snapshot. type SnapshotRuntimeStats struct { rpcStats RegionRequestRuntimeStats - backoffSleepMS map[BackoffType]int - backoffTimes map[BackoffType]int + backoffSleepMS map[retry.BackoffType]int + backoffTimes map[retry.BackoffType]int scanDetail *util.ScanDetail timeDetail *util.TimeDetail } @@ -740,8 +745,8 @@ func (rs *SnapshotRuntimeStats) Clone() *SnapshotRuntimeStats { } } if len(rs.backoffSleepMS) > 0 { - newRs.backoffSleepMS = make(map[BackoffType]int) - newRs.backoffTimes = make(map[BackoffType]int) + newRs.backoffSleepMS = make(map[retry.BackoffType]int) + newRs.backoffTimes = make(map[retry.BackoffType]int) for k, v := range rs.backoffSleepMS { newRs.backoffSleepMS[k] += v } @@ -762,10 +767,10 @@ func (rs *SnapshotRuntimeStats) Merge(other *SnapshotRuntimeStats) { } if len(other.backoffSleepMS) > 0 { if rs.backoffSleepMS == nil { - rs.backoffSleepMS = make(map[BackoffType]int) + rs.backoffSleepMS = make(map[retry.BackoffType]int) } if rs.backoffTimes == nil { - rs.backoffTimes = make(map[BackoffType]int) + rs.backoffTimes = make(map[retry.BackoffType]int) } for k, v := range other.backoffSleepMS { rs.backoffSleepMS[k] += v diff --git a/store/tikv/split_region.go b/store/tikv/split_region.go index 5839aa4d73c96..38ce24917d1cf 100644 --- a/store/tikv/split_region.go +++ b/store/tikv/split_region.go @@ -27,6 +27,7 @@ import ( tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/util" pd "github.com/tikv/pd/client" @@ -56,7 +57,7 @@ func (s *KVStore) splitBatchRegionsReq(bo *Backoffer, keys [][]byte, scatter boo return nil, nil } // The first time it enters this function. - if bo.totalSleep == 0 { + if bo.GetTotalSleep() == 0 { logutil.BgLogger().Info("split batch regions request", zap.Int("split key count", len(keys)), zap.Int("batch count", len(batches)), @@ -76,8 +77,8 @@ func (s *KVStore) splitBatchRegionsReq(bo *Backoffer, keys [][]byte, scatter boo util.WithRecovery(func() { select { case ch <- s.batchSendSingleRegion(backoffer, b, scatter, tableID): - case <-bo.ctx.Done(): - ch <- singleBatchResp{err: bo.ctx.Err()} + case <-bo.GetCtx().Done(): + ch <- singleBatchResp{err: bo.GetCtx().Err()} } }, func(r interface{}) { if r != nil { @@ -110,8 +111,8 @@ func (s *KVStore) splitBatchRegionsReq(bo *Backoffer, keys [][]byte, scatter boo func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool, tableID *int64) singleBatchResp { if val, err := util.MockSplitRegionTimeout.Eval(); err == nil { if val.(bool) { - if _, ok := bo.ctx.Deadline(); ok { - <-bo.ctx.Done() + if _, ok := bo.GetCtx().Deadline(); ok { + <-bo.GetCtx().Done() } } } @@ -136,7 +137,7 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool return batchResp } if regionErr != nil { - err := bo.Backoff(BoRegionMiss, errors.New(regionErr.String())) + err := bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) if err != nil { batchResp.err = errors.Trace(err) return batchResp @@ -192,9 +193,14 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool return batchResp } +const ( + splitRegionBackoff = 20000 + maxSplitRegionsBackoff = 120000 +) + // SplitRegions splits regions by splitKeys. func (s *KVStore) SplitRegions(ctx context.Context, splitKeys [][]byte, scatter bool, tableID *int64) (regionIDs []uint64, err error) { - bo := NewBackofferWithVars(ctx, int(math.Min(float64(len(splitKeys))*splitRegionBackoff, maxSplitRegionsBackoff)), nil) + bo := retry.NewBackofferWithVars(ctx, int(math.Min(float64(len(splitKeys))*splitRegionBackoff, maxSplitRegionsBackoff)), nil) resp, err := s.splitBatchRegionsReq(bo, splitKeys, scatter, tableID) regionIDs = make([]uint64, 0, len(splitKeys)) if resp != nil && resp.Resp != nil { @@ -215,7 +221,7 @@ func (s *KVStore) scatterRegion(bo *Backoffer, regionID uint64, tableID *int64) if tableID != nil { opts = append(opts, pd.WithGroup(fmt.Sprintf("%v", *tableID))) } - _, err := s.pdClient.ScatterRegions(bo.ctx, []uint64{regionID}, opts...) + _, err := s.pdClient.ScatterRegions(bo.GetCtx(), []uint64{regionID}, opts...) if val, err2 := util.MockScatterRegionTimeout.Eval(); err2 == nil { if val.(bool) { @@ -226,7 +232,7 @@ func (s *KVStore) scatterRegion(bo *Backoffer, regionID uint64, tableID *int64) if err == nil { break } - err = bo.Backoff(BoPDRPC, errors.New(err.Error())) + err = bo.Backoff(retry.BoPDRPC, errors.New(err.Error())) if err != nil { return errors.Trace(err) } @@ -273,6 +279,8 @@ func (s *KVStore) preSplitRegion(ctx context.Context, group groupedMutations) bo return true } +const waitScatterRegionFinishBackoff = 120000 + // WaitScatterRegionFinish implements SplittableStore interface. // backOff is the back off time of the wait scatter region.(Milliseconds) // if backOff <= 0, the default wait scatter back off time will be used. @@ -283,7 +291,7 @@ func (s *KVStore) WaitScatterRegionFinish(ctx context.Context, regionID uint64, logutil.BgLogger().Info("wait scatter region", zap.Uint64("regionID", regionID), zap.Int("backoff(ms)", backOff)) - bo := NewBackofferWithVars(ctx, backOff, nil) + bo := retry.NewBackofferWithVars(ctx, backOff, nil) logFreq := 0 for { resp, err := s.pdClient.GetOperator(ctx, regionID) @@ -310,9 +318,9 @@ func (s *KVStore) WaitScatterRegionFinish(ctx context.Context, regionID uint64, logFreq++ } if err != nil { - err = bo.Backoff(BoRegionMiss, errors.New(err.Error())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(err.Error())) } else { - err = bo.Backoff(BoRegionMiss, errors.New("wait scatter region timeout")) + err = bo.Backoff(retry.BoRegionMiss, errors.New("wait scatter region timeout")) } if err != nil { return errors.Trace(err) @@ -322,7 +330,7 @@ func (s *KVStore) WaitScatterRegionFinish(ctx context.Context, regionID uint64, // CheckRegionInScattering uses to check whether scatter region finished. func (s *KVStore) CheckRegionInScattering(regionID uint64) (bool, error) { - bo := NewBackofferWithVars(context.Background(), locateRegionMaxBackoff, nil) + bo := retry.NewBackofferWithVars(context.Background(), locateRegionMaxBackoff, nil) for { resp, err := s.pdClient.GetOperator(context.Background(), regionID) if err == nil && resp != nil { @@ -331,7 +339,7 @@ func (s *KVStore) CheckRegionInScattering(regionID uint64) (bool, error) { } } if err != nil { - err = bo.Backoff(BoRegionMiss, errors.New(err.Error())) + err = bo.Backoff(retry.BoRegionMiss, errors.New(err.Error())) } else { return true, nil } diff --git a/store/tikv/test_probe.go b/store/tikv/test_probe.go index a6a4f8d826655..1a8dc5062218d 100644 --- a/store/tikv/test_probe.go +++ b/store/tikv/test_probe.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" pd "github.com/tikv/pd/client" @@ -66,7 +67,7 @@ func (s StoreProbe) ClearTxnLatches() { // SendTxnHeartbeat renews a txn's ttl. func (s StoreProbe) SendTxnHeartbeat(ctx context.Context, key []byte, startTS uint64, ttl uint64) (uint64, error) { - bo := NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil) return sendTxnHeartBeat(bo, s.KVStore, key, startTS, ttl) } @@ -266,12 +267,12 @@ func (c CommitterProbe) PrewriteAllMutations(ctx context.Context) error { // PrewriteMutations performs the first phase of commit for given keys. func (c CommitterProbe) PrewriteMutations(ctx context.Context, mutations CommitterMutations) error { - return c.prewriteMutations(NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), mutations) + return c.prewriteMutations(retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil), mutations) } // CommitMutations performs the second phase of commit. func (c CommitterProbe) CommitMutations(ctx context.Context) error { - return c.commitMutations(NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), nil), c.mutationsOfKeys([][]byte{c.primaryKey})) + return c.commitMutations(retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), nil), c.mutationsOfKeys([][]byte{c.primaryKey})) } // MutationsOfKeys returns mutations match the keys. @@ -281,7 +282,7 @@ func (c CommitterProbe) MutationsOfKeys(keys [][]byte) CommitterMutations { // PessimisticRollbackMutations rolls mutations back. func (c CommitterProbe) PessimisticRollbackMutations(ctx context.Context, muts CommitterMutations) error { - return c.pessimisticRollbackMutations(NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, nil), muts) + return c.pessimisticRollbackMutations(retry.NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, nil), muts) } // Cleanup cleans dirty data of a committer. @@ -366,7 +367,7 @@ func (c CommitterProbe) SetPrimaryKeyBlocker(ac, bk chan struct{}) { // CleanupMutations performs the clean up phase. func (c CommitterProbe) CleanupMutations(ctx context.Context) error { - bo := NewBackofferWithVars(ctx, cleanupMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, cleanupMaxBackoff, nil) return c.cleanupMutations(bo, c.mutations) } @@ -434,13 +435,13 @@ func (l LockResolverProbe) ResolveLockAsync(bo *Backoffer, lock *Lock, status Tx // ResolveLock resolves single lock. func (l LockResolverProbe) ResolveLock(ctx context.Context, lock *Lock) error { - bo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil) return l.resolveLock(bo, lock, TxnStatus{}, false, make(map[RegionVerID]struct{})) } // ResolvePessimisticLock resolves single pessimistic lock. func (l LockResolverProbe) ResolvePessimisticLock(ctx context.Context, lock *Lock) error { - bo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil) + bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, nil) return l.resolvePessimisticLock(bo, lock, make(map[RegionVerID]struct{})) } diff --git a/store/tikv/tests/lock_test.go b/store/tikv/tests/lock_test.go index 3c6c652d96041..f32991877fefd 100644 --- a/store/tikv/tests/lock_test.go +++ b/store/tikv/tests/lock_test.go @@ -493,7 +493,7 @@ func (s *testLockSuite) TestBatchResolveLocks(c *C) { c.Assert(msBeforeLockExpired, Greater, int64(0)) lr := s.store.NewLockResolver() - bo := tikv.NewBackofferWithVars(context.Background(), tikv.GcResolveLockMaxBackoff, nil) + bo := tikv.NewGcResolveLockMaxBackoffer(context.Background()) loc, err := s.store.GetRegionCache().LocateKey(bo, locks[0].Primary) c.Assert(err, IsNil) // Check BatchResolveLocks resolve the lock even the ttl is not expired. diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 76891cfa19bdb..cba091cbdc8da 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" @@ -443,7 +444,7 @@ func (txn *KVTxn) rollbackPessimisticLocks() error { if txn.lockedCnt == 0 { return nil } - bo := NewBackofferWithVars(context.Background(), cleanupMaxBackoff, txn.vars) + bo := retry.NewBackofferWithVars(context.Background(), cleanupMaxBackoff, txn.vars) keys := txn.collectLockedKeys() return txn.committer.pessimisticRollbackMutations(bo, &PlainMutations{keys: keys}) } @@ -584,16 +585,16 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput lockCtx.Stats = &util.LockKeysDetails{ LockKeys: int32(len(keys)), } - bo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, txn.vars) + bo := retry.NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, txn.vars) txn.committer.forUpdateTS = lockCtx.ForUpdateTS // If the number of keys greater than 1, it can be on different region, // concurrently execute on multiple regions may lead to deadlock. txn.committer.isFirstLock = txn.lockedCnt == 0 && len(keys) == 1 err = txn.committer.pessimisticLockMutations(bo, lockCtx, &PlainMutations{keys: keys}) - if bo.totalSleep > 0 { - atomic.AddInt64(&lockCtx.Stats.BackoffTime, int64(bo.totalSleep)*int64(time.Millisecond)) + if bo.GetTotalSleep() > 0 { + atomic.AddInt64(&lockCtx.Stats.BackoffTime, int64(bo.GetTotalSleep())*int64(time.Millisecond)) lockCtx.Stats.Mu.Lock() - lockCtx.Stats.Mu.BackoffTypes = append(lockCtx.Stats.Mu.BackoffTypes, bo.types...) + lockCtx.Stats.Mu.BackoffTypes = append(lockCtx.Stats.Mu.BackoffTypes, bo.GetTypes()...) lockCtx.Stats.Mu.Unlock() } if lockCtx.Killed != nil { @@ -663,6 +664,8 @@ func deduplicateKeys(keys [][]byte) [][]byte { return deduped } +const pessimisticRollbackMaxBackoff = 20000 + func (txn *KVTxn) asyncPessimisticRollback(ctx context.Context, keys [][]byte) *sync.WaitGroup { // Clone a new committer for execute in background. committer := &twoPhaseCommitter{ @@ -691,7 +694,7 @@ func (txn *KVTxn) asyncPessimisticRollback(ctx context.Context, keys [][]byte) * } }) - err := committer.pessimisticRollbackMutations(NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, txn.vars), &PlainMutations{keys: keys}) + err := committer.pessimisticRollbackMutations(retry.NewBackofferWithVars(ctx, pessimisticRollbackMaxBackoff, txn.vars), &PlainMutations{keys: keys}) if err != nil { logutil.Logger(ctx).Warn("[kv] pessimisticRollback failed.", zap.Error(err)) } From 9c3f1b70c250871db9212d8eb510ae984c02c5ff Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Thu, 13 May 2021 17:11:40 +0800 Subject: [PATCH 058/343] variables: init cte max recursive deeps in a new session (#24609) --- sessionctx/variable/session.go | 1 + sessionctx/variable/varsutil_test.go | 1 + 2 files changed, 2 insertions(+) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 815f42cc03ef3..97597997b36f5 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1063,6 +1063,7 @@ func NewSessionVars() *SessionVars { AnalyzeVersion: DefTiDBAnalyzeVersion, EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, AllowFallbackToTiKV: make(map[kv.StoreType]struct{}), + CTEMaxRecursionDepth: DefCTEMaxRecursionDepth, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index a96897140b9b4..90507f36539d6 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -105,6 +105,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.ShardAllocateStep, Equals, int64(DefTiDBShardAllocateStep)) c.Assert(vars.EnableChangeColumnType, Equals, DefTiDBChangeColumnType) c.Assert(vars.AnalyzeVersion, Equals, DefTiDBAnalyzeVersion) + c.Assert(vars.CTEMaxRecursionDepth, Equals, DefCTEMaxRecursionDepth) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.MemQuota)) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.BatchSize)) From d2bdfd51b71646dd7c78e437ee40458bef6649cd Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 17:49:39 +0800 Subject: [PATCH 059/343] store/tikv: move transaction options out to /kv (#24619) --- ddl/backfilling.go | 2 +- ddl/column.go | 3 +- ddl/index.go | 5 +-- executor/adapter.go | 3 +- executor/analyze.go | 18 ++++---- executor/batch_point_get.go | 12 ++--- executor/insert.go | 5 +-- executor/insert_common.go | 5 +-- executor/point_get.go | 14 +++--- executor/replace.go | 5 +-- executor/simple.go | 5 +-- executor/update.go | 5 +-- kv/mock_test.go | 3 +- kv/option.go | 62 ++++++++++++++++++++++++++ meta/meta.go | 5 +-- session/session.go | 30 ++++++------- sessionctx/binloginfo/binloginfo.go | 3 +- store/driver/txn/snapshot.go | 22 ++++----- store/driver/txn/txn_driver.go | 48 ++++++++++---------- store/tikv/kv/option.go | 48 -------------------- store/tikv/tests/snapshot_fail_test.go | 2 - 21 files changed, 153 insertions(+), 152 deletions(-) create mode 100644 kv/option.go diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 56512eec6ab65..ed279c68675cf 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -677,7 +677,7 @@ func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version ver := kv.Version{Ver: version} snap := store.GetSnapshot(ver) - snap.SetOption(tikvstore.Priority, priority) + snap.SetOption(kv.Priority, priority) it, err := snap.Iter(firstKey, upperBound) if err != nil { diff --git a/ddl/column.go b/ddl/column.go index 18c23b4d9c45a..e18c0c2d37e7b 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -1346,7 +1345,7 @@ func (w *updateColumnWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t errInTxn = kv.RunInNewTxn(context.Background(), w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error { taskCtx.addedCount = 0 taskCtx.scanCount = 0 - txn.SetOption(tikvstore.Priority, w.priority) + txn.SetOption(kv.Priority, w.priority) rowRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange) if err != nil { diff --git a/ddl/index.go b/ddl/index.go index f11a595aa8fb3..b1b4303d7a0f1 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -1117,7 +1116,7 @@ func (w *addIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (taskC errInTxn = kv.RunInNewTxn(context.Background(), w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error { taskCtx.addedCount = 0 taskCtx.scanCount = 0 - txn.SetOption(tikvstore.Priority, w.priority) + txn.SetOption(kv.Priority, w.priority) idxRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange) if err != nil { @@ -1329,7 +1328,7 @@ func (w *cleanUpIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t errInTxn = kv.RunInNewTxn(context.Background(), w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error { taskCtx.addedCount = 0 taskCtx.scanCount = 0 - txn.SetOption(tikvstore.Priority, w.priority) + txn.SetOption(kv.Priority, w.priority) idxRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange) if err != nil { diff --git a/executor/adapter.go b/executor/adapter.go index 44d00cd1efa1e..784696996cb94 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -43,7 +43,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" tikverr "github.com/pingcap/tidb/store/tikv/error" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -629,7 +628,7 @@ func UpdateForUpdateTS(seCtx sessionctx.Context, newForUpdateTS uint64) error { newForUpdateTS = version.Ver } seCtx.GetSessionVars().TxnCtx.SetForUpdateTS(newForUpdateTS) - txn.SetOption(tikvstore.SnapshotTS, seCtx.GetSessionVars().TxnCtx.GetForUpdateTS()) + txn.SetOption(kv.SnapshotTS, seCtx.GetSessionVars().TxnCtx.GetForUpdateTS()) return nil } diff --git a/executor/analyze.go b/executor/analyze.go index 9cf9c75b1261c..fec55d870bf95 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -1121,9 +1121,9 @@ func (e *AnalyzeFastExec) activateTxnForRowCount() (rollbackFn func() error, err return nil, errors.Trace(err) } } - txn.SetOption(tikvstore.Priority, kv.PriorityLow) - txn.SetOption(tikvstore.IsolationLevel, kv.RC) - txn.SetOption(tikvstore.NotFillCache, true) + txn.SetOption(kv.Priority, kv.PriorityLow) + txn.SetOption(kv.IsolationLevel, kv.RC) + txn.SetOption(kv.NotFillCache, true) return rollbackFn, nil } @@ -1322,7 +1322,7 @@ func (e *AnalyzeFastExec) handleScanIter(iter kv.Iterator) (scanKeysSize int, er func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) (keysSize int, err error) { snapshot := e.ctx.GetStore().GetSnapshot(kv.MaxVersion) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } for _, t := range e.scanTasks { iter, err := snapshot.Iter(kv.Key(t.StartKey), kv.Key(t.EndKey)) @@ -1341,11 +1341,11 @@ func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) (keysSize int, err func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { defer e.wg.Done() snapshot := e.ctx.GetStore().GetSnapshot(kv.MaxVersion) - snapshot.SetOption(tikvstore.NotFillCache, true) - snapshot.SetOption(tikvstore.IsolationLevel, kv.RC) - snapshot.SetOption(tikvstore.Priority, kv.PriorityLow) + snapshot.SetOption(kv.NotFillCache, true) + snapshot.SetOption(kv.IsolationLevel, kv.RC) + snapshot.SetOption(kv.Priority, kv.PriorityLow) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } rander := rand.New(rand.NewSource(e.randSeed)) @@ -1356,7 +1356,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { lower, upper := step-uint32(2*math.Sqrt(float64(step))), step step = uint32(rander.Intn(int(upper-lower))) + lower } - snapshot.SetOption(tikvstore.SampleStep, step) + snapshot.SetOption(kv.SampleStep, step) kvMap := make(map[string][]byte) var iter kv.Iterator iter, *err = snapshot.Iter(kv.Key(task.StartKey), kv.Key(task.EndKey)) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index f3ec18106fb21..23debe37404ee 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -113,17 +113,17 @@ func (e *BatchPointGetExec) Open(context.Context) error { e.stats = &runtimeStatsWithSnapshot{ SnapshotRuntimeStats: snapshotStats, } - snapshot.SetOption(tikvstore.CollectRuntimeStats, snapshotStats) + snapshot.SetOption(kv.CollectRuntimeStats, snapshotStats) e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } - snapshot.SetOption(tikvstore.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) + snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness - snapshot.SetOption(tikvstore.IsStalenessReadOnly, isStaleness) + snapshot.SetOption(kv.IsStalenessReadOnly, isStaleness) if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != oracle.GlobalTxnScope { - snapshot.SetOption(tikvstore.MatchStoreLabels, []*metapb.StoreLabel{ + snapshot.SetOption(kv.MatchStoreLabels, []*metapb.StoreLabel{ { Key: placement.DCLabelKey, Value: e.ctx.GetSessionVars().TxnCtx.TxnScope, @@ -149,7 +149,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { // Close implements the Executor interface. func (e *BatchPointGetExec) Close() error { if e.runtimeStats != nil && e.snapshot != nil { - e.snapshot.DelOption(tikvstore.CollectRuntimeStats) + e.snapshot.DelOption(kv.CollectRuntimeStats) } e.inited = 0 e.index = 0 diff --git a/executor/insert.go b/executor/insert.go index e8fdb9da3444e..178aefed5fb8b 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -215,8 +214,8 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D if e.collectRuntimeStatsEnabled() { if snapshot := txn.GetSnapshot(); snapshot != nil { - snapshot.SetOption(tikvstore.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) - defer snapshot.DelOption(tikvstore.CollectRuntimeStats) + snapshot.SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) + defer snapshot.DelOption(kv.CollectRuntimeStats) } } prefetchStart := time.Now() diff --git a/executor/insert_common.go b/executor/insert_common.go index 10fc6cb9edc59..258e873db89db 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" @@ -1049,8 +1048,8 @@ func (e *InsertValues) batchCheckAndInsert(ctx context.Context, rows [][]types.D } if e.collectRuntimeStatsEnabled() { if snapshot := txn.GetSnapshot(); snapshot != nil { - snapshot.SetOption(tikvstore.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) - defer snapshot.DelOption(tikvstore.CollectRuntimeStats) + snapshot.SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) + defer snapshot.DelOption(kv.CollectRuntimeStats) } } prefetchStart := time.Now() diff --git a/executor/point_get.go b/executor/point_get.go index c34987b7f0c1d..8857a4d253fd0 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -144,17 +144,17 @@ func (e *PointGetExecutor) Open(context.Context) error { e.stats = &runtimeStatsWithSnapshot{ SnapshotRuntimeStats: snapshotStats, } - e.snapshot.SetOption(tikvstore.CollectRuntimeStats, snapshotStats) + e.snapshot.SetOption(kv.CollectRuntimeStats, snapshotStats) e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - e.snapshot.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + e.snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } - e.snapshot.SetOption(tikvstore.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) + e.snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness - e.snapshot.SetOption(tikvstore.IsStalenessReadOnly, isStaleness) + e.snapshot.SetOption(kv.IsStalenessReadOnly, isStaleness) if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != oracle.GlobalTxnScope { - e.snapshot.SetOption(tikvstore.MatchStoreLabels, []*metapb.StoreLabel{ + e.snapshot.SetOption(kv.MatchStoreLabels, []*metapb.StoreLabel{ { Key: placement.DCLabelKey, Value: e.ctx.GetSessionVars().TxnCtx.TxnScope, @@ -167,7 +167,7 @@ func (e *PointGetExecutor) Open(context.Context) error { // Close implements the Executor interface. func (e *PointGetExecutor) Close() error { if e.runtimeStats != nil && e.snapshot != nil { - e.snapshot.DelOption(tikvstore.CollectRuntimeStats) + e.snapshot.DelOption(kv.CollectRuntimeStats) } if e.idxInfo != nil && e.tblInfo != nil { actRows := int64(0) @@ -391,7 +391,7 @@ func (e *PointGetExecutor) get(ctx context.Context, key kv.Key) ([]byte, error) } func (e *PointGetExecutor) verifyTxnScope() error { - txnScope := e.txn.GetOption(tikvstore.TxnScope).(string) + txnScope := e.txn.GetOption(kv.TxnScope).(string) if txnScope == "" || txnScope == oracle.GlobalTxnScope { return nil } diff --git a/executor/replace.go b/executor/replace.go index 20af75fe4a0ae..8f35be4d05dbd 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -221,8 +220,8 @@ func (e *ReplaceExec) exec(ctx context.Context, newRows [][]types.Datum) error { if e.collectRuntimeStatsEnabled() { if snapshot := txn.GetSnapshot(); snapshot != nil { - snapshot.SetOption(tikvstore.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) - defer snapshot.DelOption(tikvstore.CollectRuntimeStats) + snapshot.SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) + defer snapshot.DelOption(kv.CollectRuntimeStats) } } prefetchStart := time.Now() diff --git a/executor/simple.go b/executor/simple.go index 24cb857aec3d5..74063b2429c06 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -40,7 +40,6 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" @@ -606,10 +605,10 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { return err } if e.ctx.GetSessionVars().TxnCtx.IsPessimistic { - txn.SetOption(tikvstore.Pessimistic, true) + txn.SetOption(kv.Pessimistic, true) } if s.CausalConsistencyOnly { - txn.SetOption(tikvstore.GuaranteeLinearizability, false) + txn.SetOption(kv.GuaranteeLinearizability, false) } return nil } diff --git a/executor/update.go b/executor/update.go index b8c7e2a985142..7c4b07ab8e6f6 100644 --- a/executor/update.go +++ b/executor/update.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -261,7 +260,7 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { if e.collectRuntimeStatsEnabled() { txn, err := e.ctx.Txn(false) if err == nil && txn.GetSnapshot() != nil { - txn.GetSnapshot().SetOption(tikvstore.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) + txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) } } for rowIdx := 0; rowIdx < chk.NumRows(); rowIdx++ { @@ -408,7 +407,7 @@ func (e *UpdateExec) Close() error { if e.runtimeStats != nil && e.stats != nil { txn, err := e.ctx.Txn(false) if err == nil && txn.GetSnapshot() != nil { - txn.GetSnapshot().DelOption(tikvstore.CollectRuntimeStats) + txn.GetSnapshot().DelOption(kv.CollectRuntimeStats) } } return e.children[0].Close() diff --git a/kv/mock_test.go b/kv/mock_test.go index 45e45d5941251..eba059e763f82 100644 --- a/kv/mock_test.go +++ b/kv/mock_test.go @@ -17,7 +17,6 @@ import ( "context" . "github.com/pingcap/check" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" ) @@ -35,7 +34,7 @@ func (s testMockSuite) TestInterface(c *C) { snapshot := storage.GetSnapshot(version) _, err = snapshot.BatchGet(context.Background(), []Key{Key("abc"), Key("def")}) c.Check(err, IsNil) - snapshot.SetOption(tikvstore.Priority, PriorityNormal) + snapshot.SetOption(Priority, PriorityNormal) transaction, err := storage.Begin() c.Check(err, IsNil) diff --git a/kv/option.go b/kv/option.go new file mode 100644 index 0000000000000..5b04dfba06c95 --- /dev/null +++ b/kv/option.go @@ -0,0 +1,62 @@ +// Copyright 2021 PingCAP, Inc. + +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package kv + +// Transaction options +const ( + // BinlogInfo contains the binlog data and client. + BinlogInfo int = iota + 1 + // SchemaChecker is used for checking schema-validity. + SchemaChecker + // IsolationLevel sets isolation level for current transaction. The default level is SI. + IsolationLevel + // Priority marks the priority of this transaction. + Priority + // NotFillCache makes this request do not touch the LRU cache of the underlying storage. + NotFillCache + // SyncLog decides whether the WAL(write-ahead log) of this request should be synchronized. + SyncLog + // KeyOnly retrieve only keys, it can be used in scan now. + KeyOnly + // Pessimistic is defined for pessimistic lock + Pessimistic + // SnapshotTS is defined to set snapshot ts. + SnapshotTS + // Set replica read + ReplicaRead + // Set task ID + TaskID + // InfoSchema is schema version used by txn startTS. + InfoSchema + // CollectRuntimeStats is used to enable collect runtime stats. + CollectRuntimeStats + // SchemaAmender is used to amend mutations for pessimistic transactions + SchemaAmender + // SampleStep skips 'SampleStep - 1' number of keys after each returned key. + SampleStep + // CommitHook is a callback function called right after the transaction gets committed + CommitHook + // EnableAsyncCommit indicates whether async commit is enabled + EnableAsyncCommit + // Enable1PC indicates whether one-phase commit is enabled + Enable1PC + // GuaranteeLinearizability indicates whether to guarantee linearizability at the cost of an extra tso request before prewrite + GuaranteeLinearizability + // TxnScope indicates which @@txn_scope this transaction will work with. + TxnScope + // StalenessReadOnly indicates whether the transaction is staleness read only transaction + IsStalenessReadOnly + // MatchStoreLabels indicates the labels the store should be matched + MatchStoreLabels +) diff --git a/meta/meta.go b/meta/meta.go index 2682ed5b47d1e..3f76d2948e9b1 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/structure" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/logutil" @@ -94,8 +93,8 @@ type Meta struct { // NewMeta creates a Meta in transaction txn. // If the current Meta needs to handle a job, jobListKey is the type of the job's list. func NewMeta(txn kv.Transaction, jobListKeys ...JobListKeyType) *Meta { - txn.SetOption(tikvstore.Priority, kv.PriorityHigh) - txn.SetOption(tikvstore.SyncLog, struct{}{}) + txn.SetOption(kv.Priority, kv.PriorityHigh) + txn.SetOption(kv.SyncLog, struct{}{}) t := structure.NewStructure(txn, txn, mMetaPrefix) listKey := DefaultJobListKey if len(jobListKeys) != 0 { diff --git a/session/session.go b/session/session.go index 19312e5fc391e..8fc7c4c37eac8 100644 --- a/session/session.go +++ b/session/session.go @@ -497,7 +497,7 @@ func (s *session) doCommit(ctx context.Context) error { }, Client: s.sessionVars.BinlogClient, } - s.txn.SetOption(tikvstore.BinlogInfo, info) + s.txn.SetOption(kv.BinlogInfo, info) } } @@ -508,22 +508,22 @@ func (s *session) doCommit(ctx context.Context) error { physicalTableIDs = append(physicalTableIDs, id) } // Set this option for 2 phase commit to validate schema lease. - s.txn.SetOption(tikvstore.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.sessionVars.GetInfoSchema().SchemaMetaVersion(), physicalTableIDs)) - s.txn.SetOption(tikvstore.InfoSchema, s.sessionVars.TxnCtx.InfoSchema) - s.txn.SetOption(tikvstore.CommitHook, func(info string, _ error) { s.sessionVars.LastTxnInfo = info }) + s.txn.SetOption(kv.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.sessionVars.GetInfoSchema().SchemaMetaVersion(), physicalTableIDs)) + s.txn.SetOption(kv.InfoSchema, s.sessionVars.TxnCtx.InfoSchema) + s.txn.SetOption(kv.CommitHook, func(info string, _ error) { s.sessionVars.LastTxnInfo = info }) if s.GetSessionVars().EnableAmendPessimisticTxn { - s.txn.SetOption(tikvstore.SchemaAmender, NewSchemaAmenderForTikvTxn(s)) + s.txn.SetOption(kv.SchemaAmender, NewSchemaAmenderForTikvTxn(s)) } - s.txn.SetOption(tikvstore.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) - s.txn.SetOption(tikvstore.Enable1PC, s.GetSessionVars().Enable1PC) + s.txn.SetOption(kv.EnableAsyncCommit, s.GetSessionVars().EnableAsyncCommit) + s.txn.SetOption(kv.Enable1PC, s.GetSessionVars().Enable1PC) // priority of the sysvar is lower than `start transaction with causal consistency only` - if val := s.txn.GetOption(tikvstore.GuaranteeLinearizability); val == nil || val.(bool) { + if val := s.txn.GetOption(kv.GuaranteeLinearizability); val == nil || val.(bool) { // We needn't ask the TiKV client to guarantee linearizability for auto-commit transactions // because the property is naturally holds: // We guarantee the commitTS of any transaction must not exceed the next timestamp from the TSO. // An auto-commit transaction fetches its startTS from the TSO so its commitTS > its startTS > the commitTS // of any previously committed transactions. - s.txn.SetOption(tikvstore.GuaranteeLinearizability, + s.txn.SetOption(kv.GuaranteeLinearizability, s.GetSessionVars().TxnCtx.IsExplicit && s.GetSessionVars().GuaranteeLinearizability) } @@ -1883,7 +1883,7 @@ func (s *session) Txn(active bool) (kv.Transaction, error) { } s.sessionVars.TxnCtx.StartTS = s.txn.StartTS() if s.sessionVars.TxnCtx.IsPessimistic { - s.txn.SetOption(tikvstore.Pessimistic, true) + s.txn.SetOption(kv.Pessimistic, true) } if !s.sessionVars.IsAutocommit() { s.sessionVars.SetInTxn(true) @@ -1891,7 +1891,7 @@ func (s *session) Txn(active bool) (kv.Transaction, error) { s.sessionVars.TxnCtx.CouldRetry = s.isTxnRetryable() s.txn.SetVars(s.sessionVars.KVVars) if s.sessionVars.GetReplicaRead().IsFollowerRead() { - s.txn.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + s.txn.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } } return &s.txn, nil @@ -1955,7 +1955,7 @@ func (s *session) NewTxn(ctx context.Context) error { } txn.SetVars(s.sessionVars.KVVars) if s.GetSessionVars().GetReplicaRead().IsFollowerRead() { - txn.SetOption(tikvstore.ReplicaRead, tikvstore.ReplicaReadFollower) + txn.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) } s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() @@ -2763,7 +2763,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionctx.StalenessTxnOption) error { if s.txn.Valid() { txnID := s.txn.StartTS() - txnScope := s.txn.GetOption(tikvstore.TxnScope).(string) + txnScope := s.txn.GetOption(kv.TxnScope).(string) err := s.CommitTxn(ctx) if err != nil { return err @@ -2803,8 +2803,8 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc return s.NewTxn(ctx) } txn.SetVars(s.sessionVars.KVVars) - txn.SetOption(tikvstore.IsStalenessReadOnly, true) - txn.SetOption(tikvstore.TxnScope, txnScope) + txn.SetOption(kv.IsStalenessReadOnly, true) + txn.SetOption(kv.TxnScope, txnScope) s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() s.sessionVars.TxnCtx = &variable.TransactionContext{ diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 58313505e1c8e..163c22e4a6fb7 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tipb/go-binlog" @@ -295,7 +294,7 @@ func SetDDLBinlog(client *pumpcli.PumpsClient, txn kv.Transaction, jobID int64, }, Client: client, } - txn.SetOption(tikvstore.BinlogInfo, info) + txn.SetOption(kv.BinlogInfo, info) } const specialPrefix = `/*T! ` diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index a6a7d752a72fd..405067f5e082b 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -66,33 +66,33 @@ func (s *tikvSnapshot) IterReverse(k kv.Key) (kv.Iterator, error) { func (s *tikvSnapshot) SetOption(opt int, val interface{}) { switch opt { - case tikvstore.IsolationLevel: + case kv.IsolationLevel: level := getTiKVIsolationLevel(val.(kv.IsoLevel)) s.KVSnapshot.SetIsolationLevel(level) - case tikvstore.Priority: + case kv.Priority: s.KVSnapshot.SetPriority(getTiKVPriority(val.(int))) - case tikvstore.NotFillCache: + case kv.NotFillCache: s.KVSnapshot.SetNotFillCache(val.(bool)) - case tikvstore.SnapshotTS: + case kv.SnapshotTS: s.KVSnapshot.SetSnapshotTS(val.(uint64)) - case tikvstore.ReplicaRead: + case kv.ReplicaRead: s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) - case tikvstore.SampleStep: + case kv.SampleStep: s.KVSnapshot.SetSampleStep(val.(uint32)) - case tikvstore.TaskID: + case kv.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) - case tikvstore.CollectRuntimeStats: + case kv.CollectRuntimeStats: s.KVSnapshot.SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) - case tikvstore.IsStalenessReadOnly: + case kv.IsStalenessReadOnly: s.KVSnapshot.SetIsStatenessReadOnly(val.(bool)) - case tikvstore.MatchStoreLabels: + case kv.MatchStoreLabels: s.KVSnapshot.SetMatchStoreLabels(val.([]*metapb.StoreLabel)) } } func (s *tikvSnapshot) DelOption(opt int) { switch opt { - case tikvstore.CollectRuntimeStats: + case kv.CollectRuntimeStats: s.KVSnapshot.SetRuntimeStats(nil) } } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 0cd51a4480ee1..4d5ce77034312 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -126,51 +126,51 @@ func (txn *tikvTxn) GetUnionStore() kv.UnionStore { func (txn *tikvTxn) SetOption(opt int, val interface{}) { switch opt { - case tikvstore.BinlogInfo: + case kv.BinlogInfo: txn.SetBinlogExecutor(&binlogExecutor{ txn: txn.KVTxn, binInfo: val.(*binloginfo.BinlogInfo), // val cannot be other type. }) - case tikvstore.SchemaChecker: + case kv.SchemaChecker: txn.SetSchemaLeaseChecker(val.(tikv.SchemaLeaseChecker)) - case tikvstore.IsolationLevel: + case kv.IsolationLevel: level := getTiKVIsolationLevel(val.(kv.IsoLevel)) txn.KVTxn.GetSnapshot().SetIsolationLevel(level) - case tikvstore.Priority: + case kv.Priority: txn.KVTxn.SetPriority(getTiKVPriority(val.(int))) - case tikvstore.NotFillCache: + case kv.NotFillCache: txn.KVTxn.GetSnapshot().SetNotFillCache(val.(bool)) - case tikvstore.SyncLog: + case kv.SyncLog: txn.EnableForceSyncLog() - case tikvstore.Pessimistic: + case kv.Pessimistic: txn.SetPessimistic(val.(bool)) - case tikvstore.SnapshotTS: + case kv.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) - case tikvstore.ReplicaRead: + case kv.ReplicaRead: txn.KVTxn.GetSnapshot().SetReplicaRead(val.(tikvstore.ReplicaReadType)) - case tikvstore.TaskID: + case kv.TaskID: txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) - case tikvstore.InfoSchema: + case kv.InfoSchema: txn.SetSchemaVer(val.(tikv.SchemaVer)) - case tikvstore.CollectRuntimeStats: + case kv.CollectRuntimeStats: txn.KVTxn.GetSnapshot().SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) - case tikvstore.SchemaAmender: + case kv.SchemaAmender: txn.SetSchemaAmender(val.(tikv.SchemaAmender)) - case tikvstore.SampleStep: + case kv.SampleStep: txn.KVTxn.GetSnapshot().SetSampleStep(val.(uint32)) - case tikvstore.CommitHook: + case kv.CommitHook: txn.SetCommitCallback(val.(func(string, error))) - case tikvstore.EnableAsyncCommit: + case kv.EnableAsyncCommit: txn.SetEnableAsyncCommit(val.(bool)) - case tikvstore.Enable1PC: + case kv.Enable1PC: txn.SetEnable1PC(val.(bool)) - case tikvstore.GuaranteeLinearizability: + case kv.GuaranteeLinearizability: txn.SetCausalConsistency(!val.(bool)) - case tikvstore.TxnScope: + case kv.TxnScope: txn.SetScope(val.(string)) - case tikvstore.IsStalenessReadOnly: + case kv.IsStalenessReadOnly: txn.KVTxn.GetSnapshot().SetIsStatenessReadOnly(val.(bool)) - case tikvstore.MatchStoreLabels: + case kv.MatchStoreLabels: txn.KVTxn.GetSnapshot().SetMatchStoreLabels(val.([]*metapb.StoreLabel)) default: txn.KVTxn.SetOption(opt, val) @@ -179,9 +179,9 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { func (txn *tikvTxn) GetOption(opt int) interface{} { switch opt { - case tikvstore.GuaranteeLinearizability: + case kv.GuaranteeLinearizability: return !txn.KVTxn.IsCasualConsistency() - case tikvstore.TxnScope: + case kv.TxnScope: return txn.KVTxn.GetScope() default: return txn.KVTxn.GetOption(opt) @@ -190,7 +190,7 @@ func (txn *tikvTxn) GetOption(opt int) interface{} { func (txn *tikvTxn) DelOption(opt int) { switch opt { - case tikvstore.CollectRuntimeStats: + case kv.CollectRuntimeStats: txn.KVTxn.GetSnapshot().SetRuntimeStats(nil) default: txn.KVTxn.DelOption(opt) diff --git a/store/tikv/kv/option.go b/store/tikv/kv/option.go index bac9316d41773..7bd36733a568d 100644 --- a/store/tikv/kv/option.go +++ b/store/tikv/kv/option.go @@ -13,54 +13,6 @@ package kv -// Transaction options -const ( - // BinlogInfo contains the binlog data and client. - BinlogInfo int = iota + 1 - // SchemaChecker is used for checking schema-validity. - SchemaChecker - // IsolationLevel sets isolation level for current transaction. The default level is SI. - IsolationLevel - // Priority marks the priority of this transaction. - Priority - // NotFillCache makes this request do not touch the LRU cache of the underlying storage. - NotFillCache - // SyncLog decides whether the WAL(write-ahead log) of this request should be synchronized. - SyncLog - // KeyOnly retrieve only keys, it can be used in scan now. - KeyOnly - // Pessimistic is defined for pessimistic lock - Pessimistic - // SnapshotTS is defined to set snapshot ts. - SnapshotTS - // Set replica read - ReplicaRead - // Set task ID - TaskID - // InfoSchema is schema version used by txn startTS. - InfoSchema - // CollectRuntimeStats is used to enable collect runtime stats. - CollectRuntimeStats - // SchemaAmender is used to amend mutations for pessimistic transactions - SchemaAmender - // SampleStep skips 'SampleStep - 1' number of keys after each returned key. - SampleStep - // CommitHook is a callback function called right after the transaction gets committed - CommitHook - // EnableAsyncCommit indicates whether async commit is enabled - EnableAsyncCommit - // Enable1PC indicates whether one-phase commit is enabled - Enable1PC - // GuaranteeLinearizability indicates whether to guarantee linearizability at the cost of an extra tso request before prewrite - GuaranteeLinearizability - // TxnScope indicates which @@txn_scope this transaction will work with. - TxnScope - // StalenessReadOnly indicates whether the transaction is staleness read only transaction - IsStalenessReadOnly - // MatchStoreLabels indicates the labels the store should be matched - MatchStoreLabels -) - // Priority value for transaction priority. // TODO: remove after BR update. const ( diff --git a/store/tikv/tests/snapshot_fail_test.go b/store/tikv/tests/snapshot_fail_test.go index 9892061c44b8d..ed812b4f46e00 100644 --- a/store/tikv/tests/snapshot_fail_test.go +++ b/store/tikv/tests/snapshot_fail_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" - "github.com/pingcap/tidb/store/tikv/kv" ) type testSnapshotFailSuite struct { @@ -151,7 +150,6 @@ func (s *testSnapshotFailSuite) TestRetryMaxTsPointGetSkipLock(c *C) { c.Assert(err, IsNil) err = txn.Set([]byte("k2"), []byte("v2")) c.Assert(err, IsNil) - txn.SetOption(kv.EnableAsyncCommit, true) txn.SetEnableAsyncCommit(true) c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/asyncCommitDoNothing", "return"), IsNil) From 769062b3188d5b51240c2d0cda88d3718ea82634 Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 13 May 2021 18:35:39 +0800 Subject: [PATCH 060/343] =?UTF-8?q?store/driver:=20move=20backoff=20driver?= =?UTF-8?q?=20into=20single=20package=20so=20we=20can=20use=20i=E2=80=A6?= =?UTF-8?q?=20(#24624)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- store/copr/batch_coprocessor.go | 19 ++++--- store/copr/batch_request_sender.go | 4 +- store/copr/coprocessor.go | 23 ++++---- store/copr/coprocessor_test.go | 5 +- store/copr/mpp.go | 13 +++-- store/copr/store.go | 62 +------------------- store/driver/backoff/backoff.go | 90 ++++++++++++++++++++++++++++++ store/tikv/retry/backoff.go | 3 +- 8 files changed, 129 insertions(+), 90 deletions(-) create mode 100644 store/driver/backoff/backoff.go diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index c070f25a454da..b0c0ad5c9ea7b 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -97,7 +98,7 @@ type copTaskAndRPCContext struct { ctx *tikv.RPCContext } -func buildBatchCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { +func buildBatchCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { start := time.Now() const cmdType = tikvrpc.CmdBatchCop rangesLen := ranges.Len() @@ -178,7 +179,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V return copErrorResponse{errors.New("batch coprocessor cannot prove keep order or desc property")} } ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs) - bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) + bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildBatchCopTasks(bo, c.store.GetRegionCache(), ranges, req.StoreType) if err != nil { @@ -223,7 +224,7 @@ func (b *batchCopIterator) run(ctx context.Context) { // We run workers for every batch cop. for _, task := range b.tasks { b.wg.Add(1) - bo := newBackofferWithVars(ctx, copNextMaxBackoff, b.vars) + bo := backoff.NewBackofferWithVars(ctx, copNextMaxBackoff, b.vars) go b.handleTask(ctx, bo, task) } b.wg.Wait() @@ -293,7 +294,7 @@ func (b *batchCopIterator) Close() error { return nil } -func (b *batchCopIterator) handleTask(ctx context.Context, bo *backoffer, task *batchCopTask) { +func (b *batchCopIterator) handleTask(ctx context.Context, bo *Backoffer, task *batchCopTask) { tasks := []*batchCopTask{task} for idx := 0; idx < len(tasks); idx++ { ret, err := b.handleTaskOnce(ctx, bo, tasks[idx]) @@ -308,7 +309,7 @@ func (b *batchCopIterator) handleTask(ctx context.Context, bo *backoffer, task * } // Merge all ranges and request again. -func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { +func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *Backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { var ranges []tikvstore.KeyRange for _, taskCtx := range batchTask.copTasks { taskCtx.task.ranges.Do(func(ran *tikvstore.KeyRange) { @@ -318,7 +319,7 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *backoffer, return buildBatchCopTasks(bo, b.store.GetRegionCache(), tikv.NewKeyRanges(ranges), b.req.StoreType) } -func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *backoffer, task *batchCopTask) ([]*batchCopTask, error) { +func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *Backoffer, task *batchCopTask) ([]*batchCopTask, error) { sender := NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) var regionInfos = make([]*coprocessor.RegionInfo, 0, len(task.copTasks)) for _, task := range task.copTasks { @@ -363,7 +364,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *backoffer, ta return nil, b.handleStreamedBatchCopResponse(ctx, bo, resp.Resp.(*tikvrpc.BatchCopStreamResponse), task) } -func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (err error) { +func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, bo *Backoffer, response *tikvrpc.BatchCopStreamResponse, task *batchCopTask) (err error) { defer response.Close() resp := response.BatchResponse if resp == nil { @@ -381,7 +382,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b return nil } - if err1 := bo.b.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil { + if err1 := bo.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil { return errors.Trace(err) } @@ -396,7 +397,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b } } -func (b *batchCopIterator) handleBatchCopResponse(bo *backoffer, response *coprocessor.BatchResponse, task *batchCopTask) (err error) { +func (b *batchCopIterator) handleBatchCopResponse(bo *Backoffer, response *coprocessor.BatchResponse, task *batchCopTask) (err error) { if otherErr := response.GetOtherError(); otherErr != "" { err = errors.Errorf("other error: %s", otherErr) logutil.BgLogger().Warn("other error", diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index 139ee087ec290..422306382337d 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -38,7 +38,7 @@ func NewRegionBatchRequestSender(cache *tikv.RegionCache, client tikv.Client) *R } } -func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { +func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *Backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { // use the first ctx to send request, because every ctx has same address. cancel = func() {} rpcCtx := ctxs[0].ctx @@ -67,7 +67,7 @@ func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *backoffer, ctxs []co return } -func (ss *RegionBatchRequestSender) onSendFail(bo *backoffer, ctxs []copTaskAndRPCContext, err error) error { +func (ss *RegionBatchRequestSender) onSendFail(bo *Backoffer, ctxs []copTaskAndRPCContext, err error) error { // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { return errors.Trace(err) diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index dd8474fd75c3a..e9d9e6b8f1ebb 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -35,6 +35,7 @@ import ( "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/kv" tidbmetrics "github.com/pingcap/tidb/metrics" + "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -73,7 +74,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa return c.sendBatch(ctx, req, vars) } ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs) - bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) + bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) ranges := toTiKVKeyRanges(req.KeyRanges) tasks, err := buildCopTasks(bo, c.store.GetRegionCache(), ranges, req) if err != nil { @@ -144,7 +145,7 @@ func (r *copTask) String() string { // rangesPerTask limits the length of the ranges slice sent in one copTask. const rangesPerTask = 25000 -func buildCopTasks(bo *backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, req *kv.Request) ([]*copTask, error) { +func buildCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, req *kv.Request) ([]*copTask, error) { start := time.Now() cmdType := tikvrpc.CmdCop if req.Streaming { @@ -605,12 +606,12 @@ func (it *copIterator) Next(ctx context.Context) (kv.ResultSubset, error) { // Associate each region with an independent backoffer. In this way, when multiple regions are // unavailable, TiDB can execute very quickly without blocking -func chooseBackoffer(ctx context.Context, backoffermap map[uint64]*backoffer, task *copTask, worker *copIteratorWorker) *backoffer { +func chooseBackoffer(ctx context.Context, backoffermap map[uint64]*Backoffer, task *copTask, worker *copIteratorWorker) *Backoffer { bo, ok := backoffermap[task.region.GetID()] if ok { return bo } - newbo := newBackofferWithVars(ctx, copNextMaxBackoff, worker.vars) + newbo := backoff.NewBackofferWithVars(ctx, copNextMaxBackoff, worker.vars) backoffermap[task.region.GetID()] = newbo return newbo } @@ -629,7 +630,7 @@ func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, } }() remainTasks := []*copTask{task} - backoffermap := make(map[uint64]*backoffer) + backoffermap := make(map[uint64]*Backoffer) for len(remainTasks) > 0 { curTask := remainTasks[0] bo := chooseBackoffer(ctx, backoffermap, curTask, worker) @@ -657,7 +658,7 @@ func (worker *copIteratorWorker) handleTask(ctx context.Context, task *copTask, // handleTaskOnce handles single copTask, successful results are send to channel. // If error happened, returns error. If region split or meet lock, returns the remain tasks. -func (worker *copIteratorWorker) handleTaskOnce(bo *backoffer, task *copTask, ch chan<- *copResponse) ([]*copTask, error) { +func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch chan<- *copResponse) ([]*copTask, error) { failpoint.Inject("handleTaskOnceError", func(val failpoint.Value) { if val.(bool) { failpoint.Return(nil, errors.New("mock handleTaskOnce error")) @@ -747,7 +748,7 @@ const ( minLogKVProcessTime = 100 ) -func (worker *copIteratorWorker) logTimeCopTask(costTime time.Duration, task *copTask, bo *backoffer, resp *tikvrpc.Response) { +func (worker *copIteratorWorker) logTimeCopTask(costTime time.Duration, task *copTask, bo *Backoffer, resp *tikvrpc.Response) { logStr := fmt.Sprintf("[TIME_COP_PROCESS] resp_time:%s txnStartTS:%d region_id:%d store_addr:%s", costTime, worker.req.StartTs, task.region.GetID(), task.storeAddr) if bo.GetTotalSleep() > minLogBackoffTime { backoffTypes := strings.Replace(fmt.Sprintf("%v", bo.TiKVBackoffer().GetTypes()), " ", ",", -1) @@ -809,7 +810,7 @@ func appendScanDetail(logStr string, columnFamily string, scanInfo *kvrpcpb.Scan return logStr } -func (worker *copIteratorWorker) handleCopStreamResult(bo *backoffer, rpcCtx *tikv.RPCContext, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- *copResponse, costTime time.Duration) ([]*copTask, error) { +func (worker *copIteratorWorker) handleCopStreamResult(bo *Backoffer, rpcCtx *tikv.RPCContext, stream *tikvrpc.CopStreamResponse, task *copTask, ch chan<- *copResponse, costTime time.Duration) ([]*copTask, error) { defer stream.Close() var resp *coprocessor.Response var lastRange *coprocessor.KeyRange @@ -833,7 +834,7 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *backoffer, rpcCtx *ti if task.storeType == kv.TiFlash { err1 = bo.Backoff(tikv.BoTiFlashRPC, err1) } else { - err1 = bo.b.BackoffTiKVRPC(err1) + err1 = bo.BackoffTiKVRPC(err1) } if err1 != nil { @@ -858,7 +859,7 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *backoffer, rpcCtx *ti // returns more tasks when that happens, or handles the response if no error. // if we're handling streaming coprocessor response, lastRange is the range of last // successful response, otherwise it's nil. -func (worker *copIteratorWorker) handleCopResponse(bo *backoffer, rpcCtx *tikv.RPCContext, resp *copResponse, cacheKey []byte, cacheValue *coprCacheValue, task *copTask, ch chan<- *copResponse, lastRange *coprocessor.KeyRange, costTime time.Duration) ([]*copTask, error) { +func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.RPCContext, resp *copResponse, cacheKey []byte, cacheValue *coprCacheValue, task *copTask, ch chan<- *copResponse, lastRange *coprocessor.KeyRange, costTime time.Duration) ([]*copTask, error) { if regionErr := resp.pbResp.GetRegionError(); regionErr != nil { if rpcCtx != nil && task.storeType == kv.TiDB { resp.err = errors.Errorf("error: %v", regionErr) @@ -1015,7 +1016,7 @@ func (worker *copIteratorWorker) handleTiDBSendReqErr(err error, task *copTask, return nil } -func (worker *copIteratorWorker) buildCopTasksFromRemain(bo *backoffer, lastRange *coprocessor.KeyRange, task *copTask) ([]*copTask, error) { +func (worker *copIteratorWorker) buildCopTasksFromRemain(bo *Backoffer, lastRange *coprocessor.KeyRange, task *copTask) ([]*copTask, error) { remainedRanges := task.ranges if worker.req.Streaming && lastRange != nil { remainedRanges = worker.calculateRemain(task.ranges, lastRange, worker.req.Desc) diff --git a/store/copr/coprocessor_test.go b/store/copr/coprocessor_test.go index d7a6d52c5b4bb..3bd34f05d95f9 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -19,6 +19,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/driver/backoff" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" @@ -43,7 +44,7 @@ func (s *testCoprocessorSuite) TestBuildTasks(c *C) { cache := tikv.NewRegionCache(pdCli) defer cache.Close() - bo := newBackofferWithVars(context.Background(), 3000, nil) + bo := backoff.NewBackofferWithVars(context.Background(), 3000, nil) req := &kv.Request{} flashReq := &kv.Request{} @@ -212,7 +213,7 @@ func (s *testCoprocessorSuite) TestRebuild(c *C) { pdCli := &tikv.CodecPDClient{Client: mocktikv.NewPDClient(cluster)} cache := tikv.NewRegionCache(pdCli) defer cache.Close() - bo := newBackofferWithVars(context.Background(), 3000, nil) + bo := backoff.NewBackofferWithVars(context.Background(), 3000, nil) req := &kv.Request{} tasks, err := buildCopTasks(bo, cache, buildCopRanges("a", "z"), req) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 0d156de69fb20..9869fa501d430 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/mpp" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -56,7 +57,7 @@ func (c *MPPClient) selectAllTiFlashStore() []kv.MPPTaskMeta { // ConstructMPPTasks receives ScheduleRequest, which are actually collects of kv ranges. We allocates MPPTaskMeta for them and returns. func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasksRequest) ([]kv.MPPTaskMeta, error) { ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTS) - bo := newBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) + bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, nil) if req.KeyRanges == nil { return c.selectAllTiFlashStore(), nil } @@ -152,7 +153,7 @@ func (m *mppIterator) run(ctx context.Context) { break } m.wg.Add(1) - bo := newBackoffer(ctx, copNextMaxBackoff) + bo := backoff.NewBackoffer(ctx, copNextMaxBackoff) go m.handleDispatchReq(ctx, bo, task) } m.wg.Wait() @@ -176,7 +177,7 @@ func (m *mppIterator) sendToRespCh(resp *mppResponse) (exit bool) { // TODO:: Consider that which way is better: // - dispatch all tasks at once, and connect tasks at second. // - dispatch tasks and establish connection at the same time. -func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *backoffer, req *kv.MPPDispatchRequest) { +func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req *kv.MPPDispatchRequest) { defer func() { m.wg.Done() }() @@ -299,7 +300,7 @@ func (m *mppIterator) cancelMppTasks() { } } -func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchRequest, taskMeta *mpp.TaskMeta) { +func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchRequest, taskMeta *mpp.TaskMeta) { connReq := &mpp.EstablishMPPConnectionRequest{ SenderMeta: taskMeta, ReceiverMeta: &mpp.TaskMeta{ @@ -343,7 +344,7 @@ func (m *mppIterator) establishMPPConns(bo *backoffer, req *kv.MPPDispatchReques return } - if err1 := bo.b.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v", err)); err1 != nil { + if err1 := bo.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v", err)); err1 != nil { if errors.Cause(err) == context.Canceled { logutil.BgLogger().Info("stream recv timeout", zap.Error(err)) } else { @@ -366,7 +367,7 @@ func (m *mppIterator) Close() error { return nil } -func (m *mppIterator) handleMPPStreamResponse(bo *backoffer, response *mpp.MPPDataPacket, req *kv.MPPDispatchRequest) (err error) { +func (m *mppIterator) handleMPPStreamResponse(bo *Backoffer, response *mpp.MPPDataPacket, req *kv.MPPDispatchRequest) (err error) { if response.Error != nil { err = errors.Errorf("other error for mpp stream: %s", response.Error.Msg) logutil.BgLogger().Warn("other error", diff --git a/store/copr/store.go b/store/copr/store.go index d3f132f85238f..7fa4aeafb5135 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" @@ -122,62 +123,5 @@ func getEndPointType(t kv.StoreType) tikvrpc.EndpointType { } } -// backoffer wraps tikv.Backoffer and converts the error which returns by the functions of tikv.Backoffer to tidb error. -type backoffer struct { - b *tikv.Backoffer -} - -// newBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. -func newBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *backoffer { - b := tikv.NewBackofferWithVars(ctx, maxSleep, vars) - return &backoffer{b: b} -} - -func newBackoffer(ctx context.Context, maxSleep int) *backoffer { - b := tikv.NewBackoffer(ctx, maxSleep) - return &backoffer{b: b} -} - -// TiKVBackoffer returns tikv.Backoffer. -func (b *backoffer) TiKVBackoffer() *tikv.Backoffer { - return b.b -} - -// Backoff sleeps a while base on the backoffType and records the error message. -// It returns a retryable error if total sleep time exceeds maxSleep. -func (b *backoffer) Backoff(typ tikv.BackoffType, err error) error { - e := b.b.Backoff(typ, err) - return derr.ToTiDBErr(e) -} - -// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message -// and never sleep more than maxSleepMs for each sleep. -func (b *backoffer) BackoffWithMaxSleep(typ tikv.BackoffType, maxSleepMs int, err error) error { - e := b.b.BackoffWithMaxSleep(typ, maxSleepMs, err) - return derr.ToTiDBErr(e) -} - -// GetBackoffTimes returns a map contains backoff time count by type. -func (b *backoffer) GetBackoffTimes() map[tikv.BackoffType]int { - return b.b.GetBackoffTimes() -} - -// GetCtx returns the binded context. -func (b *backoffer) GetCtx() context.Context { - return b.b.GetCtx() -} - -// GetVars returns the binded vars. -func (b *backoffer) GetVars() *tikv.Variables { - return b.b.GetVars() -} - -// GetBackoffSleepMS returns a map contains backoff sleep time by type. -func (b *backoffer) GetBackoffSleepMS() map[tikv.BackoffType]int { - return b.b.GetBackoffSleepMS() -} - -// GetTotalSleep returns total sleep time. -func (b *backoffer) GetTotalSleep() int { - return b.b.GetTotalSleep() -} +// Backoffer wraps tikv.Backoffer and converts the error which returns by the functions of tikv.Backoffer to tidb error. +type Backoffer = backoff.Backoffer diff --git a/store/driver/backoff/backoff.go b/store/driver/backoff/backoff.go new file mode 100644 index 0000000000000..f634366381d06 --- /dev/null +++ b/store/driver/backoff/backoff.go @@ -0,0 +1,90 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package backoff + +import ( + "context" + + "github.com/pingcap/tidb/kv" + derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/tikv" +) + +// Backoffer wraps tikv.Backoffer and converts the error which returns by the functions of tikv.Backoffer to tidb error. +type Backoffer struct { + b *tikv.Backoffer +} + +// NewBackofferWithVars creates a Backoffer with maximum sleep time(in ms) and kv.Variables. +func NewBackofferWithVars(ctx context.Context, maxSleep int, vars *kv.Variables) *Backoffer { + b := tikv.NewBackofferWithVars(ctx, maxSleep, vars) + return &Backoffer{b: b} +} + +// NewBackoffer creates a Backoffer with maximum sleep time(in ms). +func NewBackoffer(ctx context.Context, maxSleep int) *Backoffer { + b := tikv.NewBackoffer(ctx, maxSleep) + return &Backoffer{b: b} +} + +// TiKVBackoffer returns tikv.Backoffer. +func (b *Backoffer) TiKVBackoffer() *tikv.Backoffer { + return b.b +} + +// Backoff sleeps a while base on the backoffType and records the error message. +// It returns a retryable error if total sleep time exceeds maxSleep. +func (b *Backoffer) Backoff(typ tikv.BackoffType, err error) error { + e := b.b.Backoff(typ, err) + return derr.ToTiDBErr(e) +} + +// BackoffTiKVRPC sleeps a while base on the TiKVRPC and records the error message. +// It returns a retryable error if total sleep time exceeds maxSleep. +func (b *Backoffer) BackoffTiKVRPC(err error) error { + e := b.b.BackoffTiKVRPC(err) + return derr.ToTiDBErr(e) +} + +// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message +// and never sleep more than maxSleepMs for each sleep. +func (b *Backoffer) BackoffWithMaxSleep(typ tikv.BackoffType, maxSleepMs int, err error) error { + e := b.b.BackoffWithMaxSleep(typ, maxSleepMs, err) + return derr.ToTiDBErr(e) +} + +// GetBackoffTimes returns a map contains backoff time count by type. +func (b *Backoffer) GetBackoffTimes() map[tikv.BackoffType]int { + return b.b.GetBackoffTimes() +} + +// GetCtx returns the binded context. +func (b *Backoffer) GetCtx() context.Context { + return b.b.GetCtx() +} + +// GetVars returns the binded vars. +func (b *Backoffer) GetVars() *tikv.Variables { + return b.b.GetVars() +} + +// GetBackoffSleepMS returns a map contains backoff sleep time by type. +func (b *Backoffer) GetBackoffSleepMS() map[tikv.BackoffType]int { + return b.b.GetBackoffSleepMS() +} + +// GetTotalSleep returns total sleep time. +func (b *Backoffer) GetTotalSleep() int { + return b.b.GetTotalSleep() +} diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go index 24dc9174f3fec..a563ec7359d22 100644 --- a/store/tikv/retry/backoff.go +++ b/store/tikv/retry/backoff.go @@ -279,7 +279,8 @@ func (b *Backoffer) Backoff(typ BackoffType, err error) error { return b.BackoffWithMaxSleep(typ, -1, err) } -// BackoffTiKVRPC calls Backoff with boTiKVRPC. +// BackoffTiKVRPC sleeps a while base on the TiKVRPC and records the error message. +// It returns a retryable error if total sleep time exceeds maxSleep. func (b *Backoffer) BackoffTiKVRPC(err error) error { return b.Backoff(boTiKVRPC, err) } From 1c4fbfce501d9f7061a5c6fee648eae9f2281892 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Thu, 13 May 2021 18:47:39 +0800 Subject: [PATCH 061/343] server: close the temporary session in HTTP API to avoid memory leak (#24339) --- server/http_handler.go | 53 ++++++++++++++---------------------- server/sql_info_fetcher.go | 1 + server/statistics_handler.go | 2 ++ 3 files changed, 23 insertions(+), 33 deletions(-) diff --git a/server/http_handler.go b/server/http_handler.go index 67babd1f05e8d..093a57e45c1ab 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -45,7 +45,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -355,11 +354,11 @@ func (t *tikvHandlerTool) getPartition(tableVal table.Table, partitionName strin } func (t *tikvHandlerTool) schema() (infoschema.InfoSchema, error) { - session, err := session.CreateSession(t.Store) + dom, err := session.GetDomain(t.Store) if err != nil { - return nil, errors.Trace(err) + return nil, err } - return domain.GetDomain(session.(sessionctx.Context)).InfoSchema(), nil + return dom.InfoSchema(), nil } func (t *tikvHandlerTool) handleMvccGetByHex(params map[string]string) (*mvccKV, error) { @@ -712,14 +711,13 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } } if asyncCommit := req.Form.Get("tidb_enable_async_commit"); asyncCommit != "" { - s, err := session.CreateSession(h.Store.(kv.Storage)) + s, err := session.CreateSession(h.Store) if err != nil { writeError(w, err) return } - if s != nil { - defer s.Close() - } + defer s.Close() + switch asyncCommit { case "0": err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnableAsyncCommit, variable.Off) @@ -735,14 +733,13 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } } if onePC := req.Form.Get("tidb_enable_1pc"); onePC != "" { - s, err := session.CreateSession(h.Store.(kv.Storage)) + s, err := session.CreateSession(h.Store) if err != nil { writeError(w, err) return } - if s != nil { - defer s.Close() - } + defer s.Close() + switch onePC { case "0": err = s.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(variable.TiDBEnable1PC, variable.Off) @@ -878,14 +875,11 @@ func (h flashReplicaHandler) getTiFlashReplicaInfo(tblInfo *model.TableInfo, rep } func (h flashReplicaHandler) getDropOrTruncateTableTiflash(currentSchema infoschema.InfoSchema) ([]*tableFlashReplicaInfo, error) { - s, err := session.CreateSession(h.Store.(kv.Storage)) + s, err := session.CreateSession(h.Store) if err != nil { return nil, errors.Trace(err) } - - if s != nil { - defer s.Close() - } + defer s.Close() store := domain.GetDomain(s).Store() txn, err := store.Begin() @@ -948,16 +942,18 @@ func (h flashReplicaHandler) handleStatusReport(w http.ResponseWriter, req *http writeError(w, err) return } - do, err := session.GetDomain(h.Store.(kv.Storage)) + do, err := session.GetDomain(h.Store) if err != nil { writeError(w, err) return } - s, err := session.CreateSession(h.Store.(kv.Storage)) + s, err := session.CreateSession(h.Store) if err != nil { writeError(w, err) return } + defer s.Close() + available := status.checkTableFlashReplicaAvailable() err = do.DDL().UpdateTableReplicaInfo(s, status.ID, available) if err != nil { @@ -1123,18 +1119,7 @@ func (h ddlHistoryJobHandler) ServeHTTP(w http.ResponseWriter, req *http.Request } func (h ddlHistoryJobHandler) getAllHistoryDDL() ([]*model.Job, error) { - s, err := session.CreateSession(h.Store.(kv.Storage)) - if err != nil { - return nil, errors.Trace(err) - } - - if s != nil { - defer s.Close() - } - - store := domain.GetDomain(s.(sessionctx.Context)).Store() - txn, err := store.Begin() - + txn, err := h.Store.Begin() if err != nil { return nil, errors.Trace(err) } @@ -1741,7 +1726,7 @@ type serverInfo struct { // ServeHTTP handles request of ddl server info. func (h serverInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { - do, err := session.GetDomain(h.Store.(kv.Storage)) + do, err := session.GetDomain(h.Store) if err != nil { writeError(w, errors.New("create session error")) log.Error(err) @@ -1771,7 +1756,7 @@ type clusterServerInfo struct { // ServeHTTP handles request of all ddl servers info. func (h allServerInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { - do, err := session.GetDomain(h.Store.(kv.Storage)) + do, err := session.GetDomain(h.Store) if err != nil { writeError(w, errors.New("create session error")) log.Error(err) @@ -1872,6 +1857,8 @@ func (h profileHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { writeError(w, err) return } + defer sctx.Close() + var start, end time.Time if req.FormValue("end") != "" { end, err = time.ParseInLocation(time.RFC3339, req.FormValue("end"), sctx.GetSessionVars().Location()) diff --git a/server/sql_info_fetcher.go b/server/sql_info_fetcher.go index a7be33ea00154..6fc80daf506d6 100644 --- a/server/sql_info_fetcher.go +++ b/server/sql_info_fetcher.go @@ -81,6 +81,7 @@ func (sh *sqlInfoFetcher) zipInfoForSQL(w http.ResponseWriter, r *http.Request) return } defer sh.s.Close() + sh.do = domain.GetDomain(sh.s) reqCtx := r.Context() sql := r.FormValue("sql") diff --git a/server/statistics_handler.go b/server/statistics_handler.go index 733a0559f4943..55e9e4f16df18 100644 --- a/server/statistics_handler.go +++ b/server/statistics_handler.go @@ -92,6 +92,8 @@ func (sh StatsHistoryHandler) ServeHTTP(w http.ResponseWriter, req *http.Request writeError(w, err) return } + defer se.Close() + se.GetSessionVars().StmtCtx.TimeZone = time.Local t, err := types.ParseTime(se.GetSessionVars().StmtCtx, params[pSnapshot], mysql.TypeTimestamp, 6) if err != nil { From aa3e64d08bf19c7230cc3b53c3a57f68a50ae3c3 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Thu, 13 May 2021 19:31:39 +0800 Subject: [PATCH 062/343] store/tikv: use latest PD TS plus one as min commit ts (#24579) --- store/tikv/2pc.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 9e4c28ad6fd34..19f3e4faf40e3 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -996,17 +996,18 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { // If we want to use async commit or 1PC and also want linearizability across // all nodes, we have to make sure the commit TS of this transaction is greater // than the snapshot TS of all existent readers. So we get a new timestamp - // from PD as our MinCommitTS. + // from PD and plus one as our MinCommitTS. if commitTSMayBeCalculated && c.needLinearizability() { failpoint.Inject("getMinCommitTSFromTSO", nil) - minCommitTS, err := c.store.oracle.GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + latestTS, err := c.store.oracle.GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) // If we fail to get a timestamp from PD, we just propagate the failure // instead of falling back to the normal 2PC because a normal 2PC will // also be likely to fail due to the same timestamp issue. if err != nil { return errors.Trace(err) } - c.minCommitTS = minCommitTS + // Plus 1 to avoid producing the same commit TS with previously committed transactions + c.minCommitTS = latestTS + 1 } // Calculate maxCommitTS if necessary if commitTSMayBeCalculated { From 1df03a6808b3d69add58a1c55d61146bca13b8f2 Mon Sep 17 00:00:00 2001 From: sylzd Date: Thu, 13 May 2021 19:51:38 +0800 Subject: [PATCH 063/343] planner: fix incorrect TableDual plan built from nulleq (#24596) --- util/ranger/points.go | 3 ++- util/ranger/ranger_test.go | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/util/ranger/points.go b/util/ranger/points.go index d98b548dcbb7a..9c33ccef7feb3 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -246,7 +246,8 @@ func (r *builder) buildFormBinOp(expr *expression.ScalarFunction) []*point { if col.RetType.EvalType() == types.ETString && (value.Kind() == types.KindString || value.Kind() == types.KindBinaryLiteral) { value.SetString(value.GetString(), col.RetType.Collate) } - if col.GetType().Tp == mysql.TypeYear { + // If nulleq with null value, values.ToInt64 will return err + if col.GetType().Tp == mysql.TypeYear && !value.IsNull() { // If the original value is adjusted, we need to change the condition. // For example, col < 2156. Since the max year is 2155, 2156 is changed to 2155. // col < 2155 is wrong. It should be col <= 2155. diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 194ba1e779dc3..575d7f196fbf9 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -1504,12 +1504,13 @@ func (s *testRangerSuite) TestIndexRangeForYear(c *C) { // test index range testKit.MustExec("DROP TABLE IF EXISTS t") testKit.MustExec("CREATE TABLE t (a year(4), key(a))") - testKit.MustExec("INSERT INTO t VALUES (1), (70), (99), (0), ('0')") + testKit.MustExec("INSERT INTO t VALUES (1), (70), (99), (0), ('0'), (NULL)") testKit.MustQuery("SELECT * FROM t WHERE a < 15698").Check(testkit.Rows("0", "1970", "1999", "2000", "2001")) testKit.MustQuery("SELECT * FROM t WHERE a <= 0").Check(testkit.Rows("0")) testKit.MustQuery("SELECT * FROM t WHERE a <= 1").Check(testkit.Rows("0", "1970", "1999", "2000", "2001")) testKit.MustQuery("SELECT * FROM t WHERE a < 2000").Check(testkit.Rows("0", "1970", "1999")) testKit.MustQuery("SELECT * FROM t WHERE a > -1").Check(testkit.Rows("0", "1970", "1999", "2000", "2001")) + testKit.MustQuery("SELECT * FROM t WHERE a <=> NULL").Check(testkit.Rows("")) tests := []struct { indexPos int From aecff1c42e3a05b911311f695db66fce228db369 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 13 May 2021 20:03:38 +0800 Subject: [PATCH 064/343] ranger: fix the case which could have duplicate ranges (#24590) --- util/ranger/ranger.go | 6 +++--- util/ranger/ranger_test.go | 10 +++++++++- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 4f1efef7a7d90..d69c3dbc02392 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -474,7 +474,7 @@ func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bo for _, ran := range ranges { lowTail := len(ran.LowVal) - 1 for i := 0; i < lowTail; i++ { - CutDatumByPrefixLen(&ran.LowVal[i], lengths[i], tp[i]) + hasCut = CutDatumByPrefixLen(&ran.LowVal[i], lengths[i], tp[i]) || hasCut } lowCut := CutDatumByPrefixLen(&ran.LowVal[lowTail], lengths[lowTail], tp[lowTail]) // If the length of the last column of LowVal is equal to the prefix length, LowExclude should be set false. @@ -485,13 +485,13 @@ func fixPrefixColRange(ranges []*Range, lengths []int, tp []*types.FieldType) bo } highTail := len(ran.HighVal) - 1 for i := 0; i < highTail; i++ { - CutDatumByPrefixLen(&ran.HighVal[i], lengths[i], tp[i]) + hasCut = CutDatumByPrefixLen(&ran.HighVal[i], lengths[i], tp[i]) || hasCut } highCut := CutDatumByPrefixLen(&ran.HighVal[highTail], lengths[highTail], tp[highTail]) if highCut { ran.HighExclude = false } - hasCut = lowCut || highCut + hasCut = hasCut || lowCut || highCut } return hasCut } diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 575d7f196fbf9..ed4722566033a 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -351,7 +351,8 @@ create table t( index idx_cb(c, a), index idx_d(d(2)), index idx_e(e(2)), - index idx_f(f) + index idx_f(f), + index idx_de(d(2), e) )`) tests := []struct { @@ -620,6 +621,13 @@ create table t( filterConds: "[like(test.t.f, @%, 92)]", resultStr: "[[NULL,+inf]]", }, + { + indexPos: 5, + exprStr: "d in ('aab', 'aac') and e = 'a'", + accessConds: "[in(test.t.d, aab, aac) eq(test.t.e, a)]", + filterConds: "[in(test.t.d, aab, aac)]", + resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", + }, } collate.SetNewCollationEnabledForTest(true) From 5d40ea459a4bf3fc862d193c246dda96da976fc4 Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Thu, 13 May 2021 20:43:38 +0800 Subject: [PATCH 065/343] executor, store: Pass the SQL digest down to pessimistic lock request (#24380) --- executor/executor.go | 3 + store/mockstore/unistore/tikv/deadlock.go | 32 ++++- store/mockstore/unistore/tikv/detector.go | 48 ++++++- .../mockstore/unistore/tikv/detector_test.go | 37 +++-- store/mockstore/unistore/tikv/errors.go | 2 + store/mockstore/unistore/tikv/mvcc.go | 12 +- store/mockstore/unistore/tikv/server.go | 12 +- store/tikv/kv/kv.go | 1 + store/tikv/pessimistic.go | 2 +- store/tikv/region_request_test.go | 8 +- store/tikv/tests/lock_test.go | 132 ++++++++++++++++++ store/tikv/txn.go | 21 +-- util/resourcegrouptag/resource_group_tag.go | 85 +++++++++++ .../resource_group_tag_test.go | 111 +++++++++++++++ 14 files changed, 465 insertions(+), 41 deletions(-) create mode 100644 util/resourcegrouptag/resource_group_tag.go create mode 100644 util/resourcegrouptag/resource_group_tag_test.go diff --git a/executor/executor.go b/executor/executor.go index e5d5d44efefe3..1666f6955bba9 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -62,6 +62,7 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/resourcegrouptag" "go.uber.org/zap" ) @@ -971,6 +972,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { } func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.LockCtx { + _, sqlDigest := seVars.StmtCtx.SQLDigest() return &tikvstore.LockCtx{ Killed: &seVars.Killed, ForUpdateTS: seVars.TxnCtx.GetForUpdateTS(), @@ -980,6 +982,7 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.Loc LockKeysDuration: &seVars.StmtCtx.LockKeysDuration, LockKeysCount: &seVars.StmtCtx.LockKeysCount, LockExpired: &seVars.TxnCtx.LockExpire, + ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(sqlDigest), } } diff --git a/store/mockstore/unistore/tikv/deadlock.go b/store/mockstore/unistore/tikv/deadlock.go index 6641a500e2cc1..de2eaf8fa61d9 100644 --- a/store/mockstore/unistore/tikv/deadlock.go +++ b/store/mockstore/unistore/tikv/deadlock.go @@ -44,7 +44,10 @@ type DetectorServer struct { func (ds *DetectorServer) Detect(req *deadlockPb.DeadlockRequest) *deadlockPb.DeadlockResponse { switch req.Tp { case deadlockPb.DeadlockRequestType_Detect: - err := ds.Detector.Detect(req.Entry.Txn, req.Entry.WaitForTxn, req.Entry.KeyHash) + err := ds.Detector.Detect(req.Entry.Txn, req.Entry.WaitForTxn, req.Entry.KeyHash, diagnosticContext{ + key: req.Entry.Key, + resourceGroupTag: req.Entry.ResourceGroupTag, + }) if err != nil { resp := convertErrToResp(err, req.Entry.Txn, req.Entry.WaitForTxn, req.Entry.KeyHash) return resp @@ -178,30 +181,35 @@ func (dt *DetectorClient) recvLoop(streamCli deadlockPb.Deadlock_DetectClient) { } func (dt *DetectorClient) handleRemoteTask(requestType deadlockPb.DeadlockRequestType, - txnTs uint64, waitForTxnTs uint64, keyHash uint64) { + txnTs uint64, waitForTxnTs uint64, keyHash uint64, diagCtx diagnosticContext) { detectReq := &deadlockPb.DeadlockRequest{} detectReq.Tp = requestType detectReq.Entry.Txn = txnTs detectReq.Entry.WaitForTxn = waitForTxnTs detectReq.Entry.KeyHash = keyHash + detectReq.Entry.Key = diagCtx.key + detectReq.Entry.ResourceGroupTag = diagCtx.resourceGroupTag dt.sendCh <- detectReq } // CleanUp processes cleaup task on local detector // user interfaces func (dt *DetectorClient) CleanUp(startTs uint64) { - dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUp, startTs, 0, 0) + dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUp, startTs, 0, 0, diagnosticContext{}) } // CleanUpWaitFor cleans up the specific wait edge in detector's wait map func (dt *DetectorClient) CleanUpWaitFor(txnTs, waitForTxn, keyHash uint64) { - dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUpWaitFor, txnTs, waitForTxn, keyHash) + dt.handleRemoteTask(deadlockPb.DeadlockRequestType_CleanUpWaitFor, txnTs, waitForTxn, keyHash, diagnosticContext{}) } // Detect post the detection request to local deadlock detector or remote first region leader, // the caller should use `waiter.ch` to receive possible deadlock response -func (dt *DetectorClient) Detect(txnTs uint64, waitForTxnTs uint64, keyHash uint64) { - dt.handleRemoteTask(deadlockPb.DeadlockRequestType_Detect, txnTs, waitForTxnTs, keyHash) +func (dt *DetectorClient) Detect(txnTs uint64, waitForTxnTs uint64, keyHash uint64, key []byte, resourceGroupTag []byte) { + dt.handleRemoteTask(deadlockPb.DeadlockRequestType_Detect, txnTs, waitForTxnTs, keyHash, diagnosticContext{ + key: key, + resourceGroupTag: resourceGroupTag, + }) } // convertErrToResp converts `ErrDeadlock` to `DeadlockResponse` proto type @@ -213,6 +221,18 @@ func convertErrToResp(errDeadlock *ErrDeadlock, txnTs, waitForTxnTs, keyHash uin resp := &deadlockPb.DeadlockResponse{} resp.Entry = entry resp.DeadlockKeyHash = errDeadlock.DeadlockKeyHash + + resp.WaitChain = make([]*deadlockPb.WaitForEntry, 0, len(errDeadlock.WaitChain)) + for _, item := range errDeadlock.WaitChain { + resp.WaitChain = append(resp.WaitChain, &deadlockPb.WaitForEntry{ + Txn: item.Txn, + WaitForTxn: item.WaitForTxn, + KeyHash: item.KeyHash, + Key: item.Key, + ResourceGroupTag: item.ResourceGroupTag, + }) + } + return resp } diff --git a/store/mockstore/unistore/tikv/detector.go b/store/mockstore/unistore/tikv/detector.go index 0273bed5fe6a8..a27adb3f35b6c 100644 --- a/store/mockstore/unistore/tikv/detector.go +++ b/store/mockstore/unistore/tikv/detector.go @@ -30,6 +30,7 @@ import ( "sync" "time" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/log" "go.uber.org/zap" ) @@ -54,6 +55,12 @@ type txnKeyHashPair struct { txn uint64 keyHash uint64 registerTime time.Time + diagCtx diagnosticContext +} + +type diagnosticContext struct { + key []byte + resourceGroupTag []byte } func (p *txnKeyHashPair) isExpired(ttl time.Duration, nowTime time.Time) bool { @@ -75,13 +82,27 @@ func NewDetector(ttl time.Duration, urgentSize uint64, expireInterval time.Durat } // Detect detects deadlock for the sourceTxn on a locked key. -func (d *Detector) Detect(sourceTxn, waitForTxn, keyHash uint64) *ErrDeadlock { +func (d *Detector) Detect(sourceTxn, waitForTxn, keyHash uint64, diagCtx diagnosticContext) *ErrDeadlock { d.lock.Lock() nowTime := time.Now() d.activeExpire(nowTime) err := d.doDetect(nowTime, sourceTxn, waitForTxn) if err == nil { - d.register(sourceTxn, waitForTxn, keyHash) + d.register(sourceTxn, waitForTxn, keyHash, diagCtx) + } else { + // Reverse the wait chain so that the order will be each one waiting for the next one, and append the current + // entry that finally caused the deadlock. + for i := 0; i < len(err.WaitChain)/2; i++ { + j := len(err.WaitChain) - i - 1 + err.WaitChain[i], err.WaitChain[j] = err.WaitChain[j], err.WaitChain[i] + } + err.WaitChain = append(err.WaitChain, &deadlockPB.WaitForEntry{ + Txn: sourceTxn, + Key: diagCtx.key, + KeyHash: keyHash, + ResourceGroupTag: diagCtx.resourceGroupTag, + WaitForTxn: waitForTxn, + }) } d.lock.Unlock() return err @@ -103,9 +124,26 @@ func (d *Detector) doDetect(nowTime time.Time, sourceTxn, waitForTxn uint64) *Er continue } if keyHashPair.txn == sourceTxn { - return &ErrDeadlock{DeadlockKeyHash: keyHashPair.keyHash} + return &ErrDeadlock{DeadlockKeyHash: keyHashPair.keyHash, + WaitChain: []*deadlockPB.WaitForEntry{ + { + Txn: waitForTxn, + Key: keyHashPair.diagCtx.key, + KeyHash: keyHashPair.keyHash, + ResourceGroupTag: keyHashPair.diagCtx.resourceGroupTag, + WaitForTxn: keyHashPair.txn, + }, + }, + } } if err := d.doDetect(nowTime, sourceTxn, keyHashPair.txn); err != nil { + err.WaitChain = append(err.WaitChain, &deadlockPB.WaitForEntry{ + Txn: waitForTxn, + Key: keyHashPair.diagCtx.key, + KeyHash: keyHashPair.keyHash, + ResourceGroupTag: keyHashPair.diagCtx.resourceGroupTag, + WaitForTxn: keyHashPair.txn, + }) return err } } @@ -115,9 +153,9 @@ func (d *Detector) doDetect(nowTime time.Time, sourceTxn, waitForTxn uint64) *Er return nil } -func (d *Detector) register(sourceTxn, waitForTxn, keyHash uint64) { +func (d *Detector) register(sourceTxn, waitForTxn, keyHash uint64, diagCtx diagnosticContext) { val := d.waitForMap[sourceTxn] - pair := txnKeyHashPair{txn: waitForTxn, keyHash: keyHash, registerTime: time.Now()} + pair := txnKeyHashPair{txn: waitForTxn, keyHash: keyHash, registerTime: time.Now(), diagCtx: diagCtx} if val == nil { newList := &txnList{txns: list.New()} newList.txns.PushBack(&pair) diff --git a/store/mockstore/unistore/tikv/detector_test.go b/store/mockstore/unistore/tikv/detector_test.go index 1768cc377ec7c..b0d3a074ff840 100644 --- a/store/mockstore/unistore/tikv/detector_test.go +++ b/store/mockstore/unistore/tikv/detector_test.go @@ -31,6 +31,7 @@ import ( "time" . "github.com/pingcap/check" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" ) func TestT(t *testing.T) { @@ -42,19 +43,38 @@ var _ = Suite(&testDeadlockSuite{}) type testDeadlockSuite struct{} func (s *testDeadlockSuite) TestDeadlock(c *C) { + makeDiagCtx := func(key string, resourceGroupTag string) diagnosticContext { + return diagnosticContext{ + key: []byte(key), + resourceGroupTag: []byte(resourceGroupTag), + } + } + checkWaitChainEntry := func(entry *deadlockPB.WaitForEntry, txn, waitForTxn uint64, key, resourceGroupTag string) { + c.Assert(entry.Txn, Equals, txn) + c.Assert(entry.WaitForTxn, Equals, waitForTxn) + c.Assert(string(entry.Key), Equals, key) + c.Assert(string(entry.ResourceGroupTag), Equals, resourceGroupTag) + } + ttl := 50 * time.Millisecond expireInterval := 100 * time.Millisecond urgentSize := uint64(1) detector := NewDetector(ttl, urgentSize, expireInterval) - err := detector.Detect(1, 2, 100) + err := detector.Detect(1, 2, 100, makeDiagCtx("k1", "tag1")) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(1)) - err = detector.Detect(2, 3, 200) + err = detector.Detect(2, 3, 200, makeDiagCtx("k2", "tag2")) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(2)) - err = detector.Detect(3, 1, 300) + err = detector.Detect(3, 1, 300, makeDiagCtx("k3", "tag3")) c.Assert(err, NotNil) c.Assert(err.Error(), Equals, fmt.Sprintf("deadlock")) + c.Assert(len(err.WaitChain), Equals, 3) + // The order of entries in the wait chain is specific: each item is waiting for the next one. + checkWaitChainEntry(err.WaitChain[0], 1, 2, "k1", "tag1") + checkWaitChainEntry(err.WaitChain[1], 2, 3, "k2", "tag2") + checkWaitChainEntry(err.WaitChain[2], 3, 1, "k3", "tag3") + c.Assert(detector.totalSize, Equals, uint64(2)) detector.CleanUp(2) list2 := detector.waitForMap[2] @@ -62,20 +82,21 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { c.Assert(detector.totalSize, Equals, uint64(1)) // After cycle is broken, no deadlock now. - err = detector.Detect(3, 1, 300) + diagCtx := diagnosticContext{} + err = detector.Detect(3, 1, 300, diagCtx) c.Assert(err, IsNil) list3 := detector.waitForMap[3] c.Assert(list3.txns.Len(), Equals, 1) c.Assert(detector.totalSize, Equals, uint64(2)) // Different keyHash grows the list. - err = detector.Detect(3, 1, 400) + err = detector.Detect(3, 1, 400, diagCtx) c.Assert(err, IsNil) c.Assert(list3.txns.Len(), Equals, 2) c.Assert(detector.totalSize, Equals, uint64(3)) // Same waitFor and key hash doesn't grow the list. - err = detector.Detect(3, 1, 400) + err = detector.Detect(3, 1, 400, diagCtx) c.Assert(err, IsNil) c.Assert(list3.txns.Len(), Equals, 2) c.Assert(detector.totalSize, Equals, uint64(3)) @@ -90,7 +111,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { // after 100ms, all entries expired, detect non exist edges time.Sleep(100 * time.Millisecond) - err = detector.Detect(100, 200, 100) + err = detector.Detect(100, 200, 100, diagCtx) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(1)) c.Assert(len(detector.waitForMap), Equals, 1) @@ -98,7 +119,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { // expired entry should not report deadlock, detect will remove this entry // not dependent on expire check interval time.Sleep(60 * time.Millisecond) - err = detector.Detect(200, 100, 200) + err = detector.Detect(200, 100, 200, diagCtx) c.Assert(err, IsNil) c.Assert(detector.totalSize, Equals, uint64(1)) c.Assert(len(detector.waitForMap), Equals, 1) diff --git a/store/mockstore/unistore/tikv/errors.go b/store/mockstore/unistore/tikv/errors.go index 01d28fb73c896..98a70951871d5 100644 --- a/store/mockstore/unistore/tikv/errors.go +++ b/store/mockstore/unistore/tikv/errors.go @@ -16,6 +16,7 @@ package tikv import ( "fmt" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/mockstore/unistore/tikv/mvcc" ) @@ -90,6 +91,7 @@ type ErrDeadlock struct { LockKey []byte LockTS uint64 DeadlockKeyHash uint64 + WaitChain []*deadlockPB.WaitForEntry } func (e ErrDeadlock) Error() string { diff --git a/store/mockstore/unistore/tikv/mvcc.go b/store/mockstore/unistore/tikv/mvcc.go index 4e3eb4f7d7df8..fe5a75b549945 100644 --- a/store/mockstore/unistore/tikv/mvcc.go +++ b/store/mockstore/unistore/tikv/mvcc.go @@ -239,7 +239,11 @@ func (store *MVCCStore) PessimisticLock(reqCtx *requestCtx, req *kvrpcpb.Pessimi for _, m := range mutations { lock, err := store.checkConflictInLockStore(reqCtx, m, startTS) if err != nil { - return store.handleCheckPessimisticErr(startTS, err, req.IsFirstLock, req.WaitTimeout) + var resourceGroupTag []byte = nil + if req.Context != nil { + resourceGroupTag = req.Context.ResourceGroupTag + } + return store.handleCheckPessimisticErr(startTS, err, req.IsFirstLock, req.WaitTimeout, m.Key, resourceGroupTag) } if lock != nil { if lock.Op != uint8(kvrpcpb.Op_PessimisticLock) { @@ -533,11 +537,13 @@ func (store *MVCCStore) CheckSecondaryLocks(reqCtx *requestCtx, keys [][]byte, s func (store *MVCCStore) normalizeWaitTime(lockWaitTime int64) time.Duration { if lockWaitTime > store.conf.PessimisticTxn.WaitForLockTimeout { lockWaitTime = store.conf.PessimisticTxn.WaitForLockTimeout + } else if lockWaitTime == 0 { + lockWaitTime = store.conf.PessimisticTxn.WaitForLockTimeout } return time.Duration(lockWaitTime) * time.Millisecond } -func (store *MVCCStore) handleCheckPessimisticErr(startTS uint64, err error, isFirstLock bool, lockWaitTime int64) (*lockwaiter.Waiter, error) { +func (store *MVCCStore) handleCheckPessimisticErr(startTS uint64, err error, isFirstLock bool, lockWaitTime int64, key []byte, resourceGroupTag []byte) (*lockwaiter.Waiter, error) { if locked, ok := err.(*ErrLocked); ok { if lockWaitTime != lockwaiter.LockNoWait { keyHash := farm.Fingerprint64(locked.Key) @@ -546,7 +552,7 @@ func (store *MVCCStore) handleCheckPessimisticErr(startTS uint64, err error, isF log.S().Debugf("%d blocked by %d on key %d", startTS, lock.StartTS, keyHash) waiter := store.lockWaiterManager.NewWaiter(startTS, lock.StartTS, keyHash, waitTimeDuration) if !isFirstLock { - store.DeadlockDetectCli.Detect(startTS, lock.StartTS, keyHash) + store.DeadlockDetectCli.Detect(startTS, lock.StartTS, keyHash, key, resourceGroupTag) } return waiter, err } diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index adf3049330897..036d824a39ff9 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -217,6 +217,7 @@ func (svr *Server) KvPessimisticLock(ctx context.Context, req *kvrpcpb.Pessimist LockKey: errLocked.Key, LockTS: errLocked.Lock.StartTS, DeadlockKeyHash: result.DeadlockResp.DeadlockKeyHash, + WaitChain: result.DeadlockResp.WaitChain, } resp.Errors, resp.RegionError = convertToPBErrors(deadlockErr) return resp, nil @@ -845,11 +846,6 @@ func (svr *Server) ReadIndex(context.Context, *kvrpcpb.ReadIndexRequest) (*kvrpc return &kvrpcpb.ReadIndexResponse{}, nil } -// GetLockWaitInfo implements implements the tikvpb.TikvServer interface. -func (svr *Server) GetLockWaitInfo(ctx context.Context, _ *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { - return &kvrpcpb.GetLockWaitInfoResponse{}, nil -} - // transaction debugger commands. // MvccGetByKey implements implements the tikvpb.TikvServer interface. @@ -976,6 +972,11 @@ func (svr *Server) GetStoreSafeTS(context.Context, *kvrpcpb.StoreSafeTSRequest) return &kvrpcpb.StoreSafeTSResponse{}, nil } +// GetLockWaitInfo implements the tikvpb.TikvServer interface. +func (svr *Server) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + panic("unimplemented") +} + func convertToKeyError(err error) *kvrpcpb.KeyError { if err == nil { return nil @@ -1011,6 +1012,7 @@ func convertToKeyError(err error) *kvrpcpb.KeyError { LockKey: x.LockKey, LockTs: x.LockTS, DeadlockKeyHash: x.DeadlockKeyHash, + WaitChain: x.WaitChain, }, } case *ErrCommitExpire: diff --git a/store/tikv/kv/kv.go b/store/tikv/kv/kv.go index 2b7e87ecd2e47..8ba36a749db4f 100644 --- a/store/tikv/kv/kv.go +++ b/store/tikv/kv/kv.go @@ -27,4 +27,5 @@ type LockCtx struct { ValuesLock sync.Mutex LockExpired *uint32 Stats *util.LockKeysDetails + ResourceGroupTag []byte } diff --git a/store/tikv/pessimistic.go b/store/tikv/pessimistic.go index 445ced93ff904..2da8e93dad946 100644 --- a/store/tikv/pessimistic.go +++ b/store/tikv/pessimistic.go @@ -101,7 +101,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * WaitTimeout: action.LockWaitTime, ReturnValues: action.ReturnValues, MinCommitTs: c.forUpdateTS + 1, - }, pb.Context{Priority: c.priority, SyncLog: c.syncLog}) + }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: action.LockCtx.ResourceGroupTag}) lockWaitStartTime := action.WaitStartTime for { // if lockWaitTime set, refine the request `WaitTimeout` field based on timeout limit diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index f7fd2a149060d..bbe7ff8d19479 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -463,10 +463,6 @@ func (s *mockTikvGrpcServer) SplitRegion(context.Context, *kvrpcpb.SplitRegionRe return nil, errors.New("unreachable") } -func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { - return nil, errors.New("unreachable") -} - func (s *mockTikvGrpcServer) CoprocessorStream(*coprocessor.Request, tikvpb.Tikv_CoprocessorStreamServer) error { return errors.New("unreachable") } @@ -495,6 +491,10 @@ func (s *mockTikvGrpcServer) CoprocessorV2(context.Context, *coprocessor_v2.RawC return nil, errors.New("unreachable") } +func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { + return nil, errors.New("unreachable") +} + func (s *testRegionRequestToSingleStoreSuite) TestNoReloadRegionForGrpcWhenCtxCanceled(c *C) { // prepare a mock tikv grpc server addr := "localhost:56341" diff --git a/store/tikv/tests/lock_test.go b/store/tikv/tests/lock_test.go index f32991877fefd..d64c1d102e6d1 100644 --- a/store/tikv/tests/lock_test.go +++ b/store/tikv/tests/lock_test.go @@ -19,13 +19,17 @@ import ( "fmt" "math" "runtime" + "sync" "time" . "github.com/pingcap/check" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" + deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) @@ -640,3 +644,131 @@ func (s *testLockSuite) TestBatchResolveTxnFallenBackFromAsyncCommit(c *C) { _, err = t3.Get(context.Background(), []byte("fb2")) c.Assert(tikverr.IsErrNotFound(err), IsTrue) } + +func (s *testLockSuite) TestDeadlockReportWaitChain(c *C) { + // Utilities to make the test logic clear and simple. + type txnWrapper struct { + tikv.TxnProbe + wg sync.WaitGroup + } + + makeLockCtx := func(txn *txnWrapper, resourceGroupTag string) *kv.LockCtx { + return &kv.LockCtx{ + ForUpdateTS: txn.StartTS(), + WaitStartTime: time.Now(), + LockWaitTime: 1000, + ResourceGroupTag: []byte(resourceGroupTag), + } + } + + // Prepares several transactions and each locks a key. + prepareTxns := func(num int) []*txnWrapper { + res := make([]*txnWrapper, 0, num) + for i := 0; i < num; i++ { + txnProbe, err := s.store.Begin() + c.Assert(err, IsNil) + txn := &txnWrapper{TxnProbe: txnProbe} + txn.SetPessimistic(true) + tag := fmt.Sprintf("tag-init%v", i) + key := []byte{'k', byte(i)} + err = txn.LockKeys(context.Background(), makeLockCtx(txn, tag), key) + c.Assert(err, IsNil) + + res = append(res, txn) + } + return res + } + + // Let the i-th trnasaction lock the key that has been locked by j-th transaction + tryLock := func(txns []*txnWrapper, i int, j int) error { + c.Logf("txn %v try locking %v", i, j) + txn := txns[i] + tag := fmt.Sprintf("tag-%v-%v", i, j) + key := []byte{'k', byte(j)} + return txn.LockKeys(context.Background(), makeLockCtx(txn, tag), key) + } + + // Asserts the i-th transaction waits for the j-th transaction. + makeWaitFor := func(txns []*txnWrapper, i int, j int) { + txns[i].wg.Add(1) + go func() { + defer txns[i].wg.Done() + err := tryLock(txns, i, j) + // After the lock being waited for is released, the transaction returns a WriteConflict error + // unconditionally, which is by design. + c.Assert(err, NotNil) + c.Logf("txn %v wait for %v finished, err: %s", i, j, err.Error()) + _, ok := errors.Cause(err).(*tikverr.ErrWriteConflict) + c.Assert(ok, IsTrue) + }() + } + + waitAndRollback := func(txns []*txnWrapper, i int) { + // It's expected that each transaction should be rolled back after its blocker, so that `Rollback` will not + // run when there's concurrent `LockKeys` running. + // If it's blocked on the `Wait` forever, it means the transaction's blocker is not rolled back. + c.Logf("rollback txn %v", i) + txns[i].wg.Wait() + err := txns[i].Rollback() + c.Assert(err, IsNil) + } + + // Check the given WaitForEntry is caused by txn[i] waiting for txn[j]. + checkWaitChainEntry := func(txns []*txnWrapper, entry *deadlockPB.WaitForEntry, i, j int) { + c.Assert(entry.Txn, Equals, txns[i].StartTS()) + c.Assert(entry.WaitForTxn, Equals, txns[j].StartTS()) + c.Assert(entry.Key, BytesEquals, []byte{'k', byte(j)}) + c.Assert(string(entry.ResourceGroupTag), Equals, fmt.Sprintf("tag-%v-%v", i, j)) + } + + c.Log("test case 1: 1->0->1") + + txns := prepareTxns(2) + + makeWaitFor(txns, 0, 1) + // Sleep for a while to make sure it has been blocked. + time.Sleep(time.Millisecond * 100) + + // txn2 tries locking k1 and encounters deadlock error. + err := tryLock(txns, 1, 0) + c.Assert(err, NotNil) + dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock) + c.Assert(ok, IsTrue) + + waitChain := dl.GetWaitChain() + c.Assert(len(waitChain), Equals, 2) + checkWaitChainEntry(txns, waitChain[0], 0, 1) + checkWaitChainEntry(txns, waitChain[1], 1, 0) + + // Each transaction should be rolled back after its blocker being rolled back + waitAndRollback(txns, 1) + waitAndRollback(txns, 0) + + c.Log("test case 2: 3->2->0->1->3") + txns = prepareTxns(4) + + makeWaitFor(txns, 0, 1) + makeWaitFor(txns, 2, 0) + makeWaitFor(txns, 1, 3) + // Sleep for a while to make sure it has been blocked. + time.Sleep(time.Millisecond * 100) + + err = tryLock(txns, 3, 2) + c.Assert(err, NotNil) + dl, ok = errors.Cause(err).(*tikverr.ErrDeadlock) + c.Assert(ok, IsTrue) + + waitChain = dl.GetWaitChain() + c.Assert(len(waitChain), Equals, 4) + c.Logf("wait chain: \n** %v\n**%v\n**%v\n**%v\n", waitChain[0], waitChain[1], waitChain[2], waitChain[3]) + checkWaitChainEntry(txns, waitChain[0], 2, 0) + checkWaitChainEntry(txns, waitChain[1], 0, 1) + checkWaitChainEntry(txns, waitChain[2], 1, 3) + checkWaitChainEntry(txns, waitChain[3], 3, 2) + + // Each transaction should be rolled back after its blocker being rolled back + waitAndRollback(txns, 3) + waitAndRollback(txns, 1) + waitAndRollback(txns, 0) + waitAndRollback(txns, 2) +} diff --git a/store/tikv/txn.go b/store/tikv/txn.go index cba091cbdc8da..20bf0491ed294 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -613,15 +613,18 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput // If there is only 1 key and lock fails, no need to do pessimistic rollback. if len(keys) > 1 || keyMayBeLocked { wg := txn.asyncPessimisticRollback(ctx, keys) - if dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok && hashInKeys(dl.DeadlockKeyHash, keys) { - dl.IsRetryable = true - // Wait for the pessimistic rollback to finish before we retry the statement. - wg.Wait() - // Sleep a little, wait for the other transaction that blocked by this transaction to acquire the lock. - time.Sleep(time.Millisecond * 5) - failpoint.Inject("SingleStmtDeadLockRetrySleep", func() { - time.Sleep(300 * time.Millisecond) - }) + if dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok { + logutil.Logger(ctx).Debug("deadlock error received", zap.Uint64("startTS", txn.startTS), zap.Stringer("deadlockInfo", dl)) + if hashInKeys(dl.DeadlockKeyHash, keys) { + dl.IsRetryable = true + // Wait for the pessimistic rollback to finish before we retry the statement. + wg.Wait() + // Sleep a little, wait for the other transaction that blocked by this transaction to acquire the lock. + time.Sleep(time.Millisecond * 5) + failpoint.Inject("SingleStmtDeadLockRetrySleep", func() { + time.Sleep(300 * time.Millisecond) + }) + } } } if assignedPrimaryKey { diff --git a/util/resourcegrouptag/resource_group_tag.go b/util/resourcegrouptag/resource_group_tag.go new file mode 100644 index 0000000000000..cacbf574b91fb --- /dev/null +++ b/util/resourcegrouptag/resource_group_tag.go @@ -0,0 +1,85 @@ +package resourcegrouptag + +import ( + "encoding/hex" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + +const ( + resourceGroupTagPrefixSQLDigest = byte(1) +) + +// EncodeResourceGroupTag encodes sqlDigest into resource group tag. +// A resource group tag can be carried in the Context field of TiKV requests, which is a byte array, and sent to TiKV as +// diagnostic information. Currently it contains only the SQL Digest, and the codec method is naive but extendable. +// This function doesn't return error. When there's some error, which can only be caused by unexpected format of the +// arguments, it simply returns an empty result. +// The format: +// +-----------+-----------------------+----------------------------+---------------+----------------+---- +// | version=1 | field1 prefix (1byte) | field1 content (var bytes) | field2 prefix | field2 content | ... +// +-----------+-----------------------+----------------------------+---------------+----------------+---- +// The `version` section marks the codec version, which makes it easier for changing the format in the future. +// Each field starts with a byte to mark what field it is, and the length of the content depends on the field's +// definition. +// Currently there's only one field (SQL Digest), and its content starts with a byte `B` describing it's length, and +// then follows by exactly `B` bytes. +func EncodeResourceGroupTag(sqlDigest string) []byte { + if len(sqlDigest) == 0 { + return nil + } + if len(sqlDigest) >= 512 { + logutil.BgLogger().Warn("failed to encode sql digest to resource group tag: length too long", zap.String("sqlDigest", sqlDigest)) + return nil + } + + res := make([]byte, 3+len(sqlDigest)/2) + + const encodingVersion = 1 + res[0] = encodingVersion + + res[1] = resourceGroupTagPrefixSQLDigest + // The SQL Digest is expected to be a hex string. Convert it back to bytes to save half of the memory. + res[2] = byte(len(sqlDigest) / 2) + _, err := hex.Decode(res[3:], []byte(sqlDigest)) + if err != nil { + logutil.BgLogger().Warn("failed to encode sql digest to resource group tag: invalid hex string", zap.String("sqlDigest", sqlDigest)) + return nil + } + + return res +} + +// DecodeResourceGroupTag decodes a resource group tag into various information contained in it. Currently it contains +// only the SQL Digest. +func DecodeResourceGroupTag(data []byte) (sqlDigest string, err error) { + if len(data) == 0 { + return "", nil + } + + encodingVersion := data[0] + if encodingVersion != 1 { + return "", errors.Errorf("unsupported resource group tag version %v", data[0]) + } + rem := data[1:] + + for len(rem) > 0 { + switch rem[0] { + case resourceGroupTagPrefixSQLDigest: + // There must be one more byte at rem[1] to represent the content's length, and the remaining bytes should + // not be shorter than the length specified by rem[1]. + if len(rem) < 2 || len(rem)-2 < int(rem[1]) { + return "", errors.Errorf("cannot parse resource group tag: field length mismatch, tag: %v", hex.EncodeToString(data)) + } + fieldLen := int(rem[1]) + sqlDigest = hex.EncodeToString(rem[2 : 2+fieldLen]) + rem = rem[2+fieldLen:] + default: + return "", errors.Errorf("resource group tag field not recognized, prefix: %v, tag: %v", rem[0], hex.EncodeToString(data)) + } + } + + return +} diff --git a/util/resourcegrouptag/resource_group_tag_test.go b/util/resourcegrouptag/resource_group_tag_test.go new file mode 100644 index 0000000000000..a979b92fce315 --- /dev/null +++ b/util/resourcegrouptag/resource_group_tag_test.go @@ -0,0 +1,111 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package resourcegrouptag + +import ( + "math/rand" + "testing" + + . "github.com/pingcap/check" +) + +type testUtilsSuite struct{} + +var _ = Suite(&testUtilsSuite{}) + +func TestT(t *testing.T) { + TestingT(t) +} + +func (s *testUtilsSuite) TestResourceGroupTagEncoding(c *C) { + sqlDigest := "" + tag := EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + decodedSQLDigest, err := DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(len(decodedSQLDigest), Equals, 0) + + sqlDigest = "aa" + tag = EncodeResourceGroupTag(sqlDigest) + // version(1) + prefix(1) + length(1) + content(2hex -> 1byte) + c.Assert(len(tag), Equals, 4) + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, sqlDigest) + + sqlDigest = genRandHex(64) + tag = EncodeResourceGroupTag(sqlDigest) + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, sqlDigest) + + sqlDigest = genRandHex(510) + tag = EncodeResourceGroupTag(sqlDigest) + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, sqlDigest) + + // The max supported length is 255 bytes (510 hex digits). + sqlDigest = genRandHex(512) + tag = EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + + // A hex string can't have odd length. + sqlDigest = genRandHex(15) + tag = EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + + // Non-hexadecimal character is invalid + sqlDigest = "aabbccddgg" + tag = EncodeResourceGroupTag(sqlDigest) + c.Assert(len(tag), Equals, 0) + + // A tag should start with a supported version + tag = []byte("\x00") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) + + // The fields should have format like `[prefix, length, content...]`, otherwise decoding it should returns error. + tag = []byte("\x01\x01") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) + + tag = []byte("\x01\x01\x02") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) + + tag = []byte("\x01\x01\x02AB") + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(decodedSQLDigest, Equals, "4142") + + tag = []byte("\x01\x01\x00") + decodedSQLDigest, err = DecodeResourceGroupTag(tag) + c.Assert(err, IsNil) + c.Assert(len(decodedSQLDigest), Equals, 0) + + // Unsupported field + tag = []byte("\x01\x99") + _, err = DecodeResourceGroupTag(tag) + c.Assert(err, NotNil) +} + +func genRandHex(length int) string { + const chars = "0123456789abcdef" + res := make([]byte, length) + for i := 0; i < length; i++ { + res[i] = chars[rand.Intn(len(chars))] + } + return string(res) +} From b7c22aa31d062b77a96493519a6b50ca9b627971 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 13 May 2021 21:29:39 +0800 Subject: [PATCH 066/343] kv: remove UnionStore interface (#24625) --- ddl/index.go | 2 +- ddl/index_change_test.go | 2 +- executor/admin.go | 2 +- executor/admin_test.go | 44 +++++++++++++-------------- executor/distsql_test.go | 2 +- executor/executor_test.go | 4 +-- kv/interface_mock_test.go | 4 --- kv/kv.go | 2 -- kv/union_store.go | 17 +---------- store/driver/txn/txn_driver.go | 4 --- store/driver/txn/unionstore_driver.go | 36 ---------------------- table/index.go | 6 ++-- table/tables/index.go | 27 ++++++++-------- table/tables/index_test.go | 12 ++++---- table/tables/tables.go | 4 +-- util/admin/admin.go | 2 +- util/mock/context.go | 8 ----- 17 files changed, 54 insertions(+), 124 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index b1b4303d7a0f1..fdba6c65008f6 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1343,7 +1343,7 @@ func (w *cleanUpIndexWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t // we fetch records row by row, so records will belong to // index[0], index[1] ... index[n-1], index[0], index[1] ... // respectively. So indexes[i%n] is the index of idxRecords[i]. - err := w.indexes[i%n].Delete(w.sessCtx.GetSessionVars().StmtCtx, txn.GetUnionStore(), idxRecord.vals, idxRecord.handle) + err := w.indexes[i%n].Delete(w.sessCtx.GetSessionVars().StmtCtx, txn, idxRecord.vals, idxRecord.handle) if err != nil { return errors.Trace(err) } diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index 0a54b6b25e694..dfdfc7111c372 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -198,7 +198,7 @@ func checkIndexExists(ctx sessionctx.Context, tbl table.Table, indexValue interf if err != nil { return errors.Trace(err) } - doesExist, _, err := idx.Exist(ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), types.MakeDatums(indexValue), kv.IntHandle(handle)) + doesExist, _, err := idx.Exist(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(indexValue), kv.IntHandle(handle)) if err != nil { return errors.Trace(err) } diff --git a/executor/admin.go b/executor/admin.go index 7e15a24e667ca..be46e39a1d2a3 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -575,7 +575,7 @@ func (e *CleanupIndexExec) deleteDanglingIdx(txn kv.Transaction, values map[stri return errors.Trace(errors.Errorf("batch keys are inconsistent with handles")) } for _, handleIdxVals := range handleIdxValsGroup.([][]types.Datum) { - if err := e.index.Delete(e.ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), handleIdxVals, handle); err != nil { + if err := e.index.Delete(e.ctx.GetSessionVars().StmtCtx, txn, handleIdxVals, handle); err != nil { return err } e.removeCnt++ diff --git a/executor/admin_test.go b/executor/admin_test.go index 20095eb59a0ba..35e3d08345d63 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -135,7 +135,7 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { sc := s.ctx.GetSessionVars().StmtCtx txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(1), kv.IntHandle(1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(1), kv.IntHandle(1)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -158,7 +158,7 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(10), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(10), kv.IntHandle(10)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -172,15 +172,15 @@ func (s *testSuite5) TestAdminRecoverIndex(c *C) { txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(1), kv.IntHandle(1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(1), kv.IntHandle(1)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(2), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(2), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(3), kv.IntHandle(3)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(3), kv.IntHandle(3)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(10), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(10), kv.IntHandle(10)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(20), kv.IntHandle(20)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(20)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -236,7 +236,7 @@ func (s *testSuite5) TestClusteredIndexAdminRecoverIndex(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) cHandle := testutil.MustNewCommonHandle(c, "1", "3") - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(2), cHandle) + err = indexOpr.Delete(sc, txn, types.MakeDatums(2), cHandle) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -269,7 +269,7 @@ func (s *testSuite5) TestAdminRecoverPartitionTableIndex(c *C) { sc := s.ctx.GetSessionVars().StmtCtx txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(idxValue), kv.IntHandle(idxValue)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(idxValue), kv.IntHandle(idxValue)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -345,13 +345,13 @@ func (s *testSuite5) TestAdminRecoverIndex1(c *C) { txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("1"), kv.IntHandle(1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("1"), kv.IntHandle(1)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("2"), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("2"), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("3"), kv.IntHandle(3)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("3"), kv.IntHandle(3)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums("10"), kv.IntHandle(4)) + err = indexOpr.Delete(sc, txn, types.MakeDatums("10"), kv.IntHandle(4)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -745,7 +745,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { indexOpr := tables.NewIndex(tblInfo.GetPartitionInfo().Definitions[partitionIdx].ID, tblInfo, idxInfo) txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(i), kv.IntHandle(i)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i), kv.IntHandle(i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -784,7 +784,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(i+8), kv.IntHandle(i+8)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), kv.IntHandle(i+8)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -807,7 +807,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(i+8), kv.IntHandle(i)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(i+8), kv.IntHandle(i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -842,7 +842,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Index c2 is missing 11. txn, err := s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(-10), kv.IntHandle(-1)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(-10), kv.IntHandle(-1)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -873,7 +873,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Index c2 has two more values than table data: 10, 13, and these handles have correlative record. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(0), kv.IntHandle(0)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(0), kv.IntHandle(0)) c.Assert(err, IsNil) // Make sure the index value "19" is smaller "21". Then we scan to "19" before "21". _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(19), kv.IntHandle(10), nil) @@ -890,9 +890,9 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Two indices have the same handle. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(13), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(13), kv.IntHandle(2)) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(12), kv.IntHandle(2)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(12), kv.IntHandle(2)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -906,7 +906,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(12), kv.IntHandle(2), nil) c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(20), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(20), kv.IntHandle(10)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) @@ -917,7 +917,7 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { // Recover records. txn, err = s.store.Begin() c.Assert(err, IsNil) - err = indexOpr.Delete(sc, txn.GetUnionStore(), types.MakeDatums(19), kv.IntHandle(10)) + err = indexOpr.Delete(sc, txn, types.MakeDatums(19), kv.IntHandle(10)) c.Assert(err, IsNil) _, err = indexOpr.Create(s.ctx, txn, types.MakeDatums(20), kv.IntHandle(10), nil) c.Assert(err, IsNil) diff --git a/executor/distsql_test.go b/executor/distsql_test.go index eca6e1d016b40..6f06fd550f0b4 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -241,7 +241,7 @@ func (s *testSuite3) TestInconsistentIndex(c *C) { for i := 0; i < 10; i++ { txn, err := s.store.Begin() c.Assert(err, IsNil) - err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), types.MakeDatums(i+10), kv.IntHandle(100+i)) + err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(i+10), kv.IntHandle(100+i)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/executor/executor_test.go b/executor/executor_test.go index 3b168636606ed..7b433e71e60ac 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3852,9 +3852,9 @@ func (s *testSuite) TestCheckIndex(c *C) { // table data (handle, data): (1, 10), (2, 20), (4, 40) txn, err = s.store.Begin() c.Assert(err, IsNil) - err = idx.Delete(sc, txn.GetUnionStore(), types.MakeDatums(int64(30)), kv.IntHandle(3)) + err = idx.Delete(sc, txn, types.MakeDatums(int64(30)), kv.IntHandle(3)) c.Assert(err, IsNil) - err = idx.Delete(sc, txn.GetUnionStore(), types.MakeDatums(int64(20)), kv.IntHandle(2)) + err = idx.Delete(sc, txn, types.MakeDatums(int64(20)), kv.IntHandle(2)) c.Assert(err, IsNil) err = txn.Commit(context.Background()) c.Assert(err, IsNil) diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index 2388c4f48b9f3..e1d41f1693088 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -106,10 +106,6 @@ func (t *mockTxn) GetSnapshot() Snapshot { return nil } -func (t *mockTxn) GetUnionStore() UnionStore { - return nil -} - func (t *mockTxn) NewStagingBuffer() MemBuffer { return nil } diff --git a/kv/kv.go b/kv/kv.go index 1fad79d641009..e5ab4eed6f812 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -174,8 +174,6 @@ type Transaction interface { GetMemBuffer() MemBuffer // GetSnapshot returns the Snapshot binding to this transaction. GetSnapshot() Snapshot - // GetUnionStore returns the UnionStore binding to this transaction. - GetUnionStore() UnionStore // SetVars sets variables to the transaction. SetVars(vars interface{}) // GetVars gets variables from the transaction. diff --git a/kv/union_store.go b/kv/union_store.go index 0e9a6768c5ebc..1d12e0a7fd37b 100644 --- a/kv/union_store.go +++ b/kv/union_store.go @@ -15,21 +15,6 @@ package kv // UnionStore is a store that wraps a snapshot for read and a MemBuffer for buffered write. // Also, it provides some transaction related utilities. +// TODO: Remove after upgrading BR. type UnionStore interface { - Retriever - - // HasPresumeKeyNotExists returns whether the key presumed key not exists error for the lazy check. - HasPresumeKeyNotExists(k Key) bool - // UnmarkPresumeKeyNotExists deletes the key presume key not exists error flag for the lazy check. - UnmarkPresumeKeyNotExists(k Key) - - // SetOption sets an option with a value, when val is nil, uses the default - // value of this option. - SetOption(opt int, val interface{}) - // DelOption deletes an option. - DelOption(opt int) - // GetOption gets an option. - GetOption(opt int) interface{} - // GetMemBuffer return the MemBuffer binding to this unionStore. - GetMemBuffer() MemBuffer } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 4d5ce77034312..72c1aac8c8e71 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -120,10 +120,6 @@ func (txn *tikvTxn) GetMemBuffer() kv.MemBuffer { return newMemBuffer(txn.KVTxn.GetMemBuffer()) } -func (txn *tikvTxn) GetUnionStore() kv.UnionStore { - return &tikvUnionStore{txn.KVTxn.GetUnionStore()} -} - func (txn *tikvTxn) SetOption(opt int, val interface{}) { switch opt { case kv.BinlogInfo: diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index 5a2f56bfe4233..b54bffa139700 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -111,42 +111,6 @@ func (m *memBuffer) SnapshotGetter() kv.Getter { return newKVGetter(m.MemDB.SnapshotGetter()) } -//tikvUnionStore implements kv.UnionStore -type tikvUnionStore struct { - *unionstore.KVUnionStore -} - -func (u *tikvUnionStore) GetMemBuffer() kv.MemBuffer { - return newMemBuffer(u.KVUnionStore.GetMemBuffer()) -} - -func (u *tikvUnionStore) Get(ctx context.Context, k kv.Key) ([]byte, error) { - data, err := u.KVUnionStore.Get(ctx, k) - return data, derr.ToTiDBErr(err) -} - -func (u *tikvUnionStore) HasPresumeKeyNotExists(k kv.Key) bool { - return u.KVUnionStore.HasPresumeKeyNotExists(k) -} - -func (u *tikvUnionStore) UnmarkPresumeKeyNotExists(k kv.Key) { - u.KVUnionStore.UnmarkPresumeKeyNotExists(k) -} - -func (u *tikvUnionStore) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { - it, err := u.KVUnionStore.Iter(k, upperBound) - return newKVIterator(it), derr.ToTiDBErr(err) -} - -// IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. -// The returned iterator will iterate from greater key to smaller key. -// If k is nil, the returned iterator will be positioned at the last key. -// TODO: Add lower bound limit -func (u *tikvUnionStore) IterReverse(k kv.Key) (kv.Iterator, error) { - it, err := u.KVUnionStore.IterReverse(k) - return newKVIterator(it), derr.ToTiDBErr(err) -} - type tikvGetter struct { unionstore.Getter } diff --git a/table/index.go b/table/index.go index 5a9f32fbbfd3f..336efb7f574c2 100644 --- a/table/index.go +++ b/table/index.go @@ -66,11 +66,11 @@ type Index interface { // Create supports insert into statement. Create(ctx sessionctx.Context, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle, handleRestoreData []types.Datum, opts ...CreateIdxOptFunc) (kv.Handle, error) // Delete supports delete from statement. - Delete(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) error + Delete(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) error // Drop supports drop table, drop index statements. - Drop(us kv.UnionStore) error + Drop(txn kv.Transaction) error // Exist supports check index exists or not. - Exist(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) + Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) // GenIndexKey generates an index key. GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.Datum, h kv.Handle, buf []byte) (key []byte, distinct bool, err error) // Seek supports where clause. diff --git a/table/tables/index.go b/table/tables/index.go index 8b4630d47f70d..ab6296390fbfd 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -184,9 +184,8 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue return nil, err } - us := txn.GetUnionStore() if !distinct || skipCheck || opt.Untouched { - err = us.GetMemBuffer().Set(key, idxVal) + err = txn.GetMemBuffer().Set(key, idxVal) return nil, err } @@ -202,18 +201,18 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue var value []byte if sctx.GetSessionVars().LazyCheckKeyNotExists() { - value, err = us.GetMemBuffer().Get(ctx, key) + value, err = txn.GetMemBuffer().Get(ctx, key) } else { - value, err = us.Get(ctx, key) + value, err = txn.Get(ctx, key) } if err != nil && !kv.IsErrNotFound(err) { return nil, err } if err != nil || len(value) == 0 { if sctx.GetSessionVars().LazyCheckKeyNotExists() && err != nil { - err = us.GetMemBuffer().SetWithFlags(key, idxVal, tikvstore.SetPresumeKeyNotExists) + err = txn.GetMemBuffer().SetWithFlags(key, idxVal, tikvstore.SetPresumeKeyNotExists) } else { - err = us.GetMemBuffer().Set(key, idxVal) + err = txn.GetMemBuffer().Set(key, idxVal) } return nil, err } @@ -226,22 +225,22 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue } // Delete removes the entry for handle h and indexedValues from KV index. -func (c *index) Delete(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) error { +func (c *index) Delete(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) error { key, distinct, err := c.GenIndexKey(sc, indexedValues, h, nil) if err != nil { return err } if distinct { - err = us.GetMemBuffer().DeleteWithFlags(key, tikvstore.SetNeedLocked) + err = txn.GetMemBuffer().DeleteWithFlags(key, tikvstore.SetNeedLocked) } else { - err = us.GetMemBuffer().Delete(key) + err = txn.GetMemBuffer().Delete(key) } return err } // Drop removes the KV index from store. -func (c *index) Drop(us kv.UnionStore) error { - it, err := us.Iter(c.prefix, c.prefix.PrefixNext()) +func (c *index) Drop(txn kv.Transaction) error { + it, err := txn.Iter(c.prefix, c.prefix.PrefixNext()) if err != nil { return err } @@ -252,7 +251,7 @@ func (c *index) Drop(us kv.UnionStore) error { if !it.Key().HasPrefix(c.prefix) { break } - err := us.GetMemBuffer().Delete(it.Key()) + err := txn.GetMemBuffer().Delete(it.Key()) if err != nil { return err } @@ -298,13 +297,13 @@ func (c *index) SeekFirst(r kv.Retriever) (iter table.IndexIterator, err error) return &indexIter{it: it, idx: c, prefix: c.prefix, colInfos: colInfos, tps: tps}, nil } -func (c *index) Exist(sc *stmtctx.StatementContext, us kv.UnionStore, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) { +func (c *index) Exist(sc *stmtctx.StatementContext, txn kv.Transaction, indexedValues []types.Datum, h kv.Handle) (bool, kv.Handle, error) { key, distinct, err := c.GenIndexKey(sc, indexedValues, h, nil) if err != nil { return false, nil, err } - value, err := us.Get(context.TODO(), key) + value, err := txn.Get(context.TODO(), key) if kv.IsErrNotFound(err) { return false, nil, nil } diff --git a/table/tables/index_test.go b/table/tables/index_test.go index 9345e86bab185..2c0a417746d42 100644 --- a/table/tables/index_test.go +++ b/table/tables/index_test.go @@ -104,15 +104,15 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(h.IntValue(), Equals, int64(1)) it.Close() sc := &stmtctx.StatementContext{TimeZone: time.Local} - exist, _, err := index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(100)) + exist, _, err := index.Exist(sc, txn, values, kv.IntHandle(100)) c.Assert(err, IsNil) c.Assert(exist, IsFalse) - exist, _, err = index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(1)) + exist, _, err = index.Exist(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) c.Assert(exist, IsTrue) - err = index.Delete(sc, txn.GetUnionStore(), values, kv.IntHandle(1)) + err = index.Delete(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) it, err = index.SeekFirst(txn) @@ -132,7 +132,7 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(err, IsNil) c.Assert(hit, IsFalse) - err = index.Drop(txn.GetUnionStore()) + err = index.Drop(txn) c.Assert(err, IsNil) it, hit, err = index.Seek(sc, txn, values) @@ -194,12 +194,12 @@ func (s *testIndexSuite) TestIndex(c *C) { c.Assert(h.IntValue(), Equals, int64(1)) it.Close() - exist, h, err = index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(1)) + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(1)) c.Assert(err, IsNil) c.Assert(h.IntValue(), Equals, int64(1)) c.Assert(exist, IsTrue) - exist, h, err = index.Exist(sc, txn.GetUnionStore(), values, kv.IntHandle(2)) + exist, h, err = index.Exist(sc, txn, values, kv.IntHandle(2)) c.Assert(err, NotNil) c.Assert(h.IntValue(), Equals, int64(1)) c.Assert(exist, IsTrue) diff --git a/table/tables/tables.go b/table/tables/tables.go index a6a4180401d6a..37e6ab1103a89 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1152,7 +1152,7 @@ func (t *TableCommon) removeRowIndices(ctx sessionctx.Context, h kv.Handle, rec logutil.BgLogger().Info("remove row index failed", zap.Any("index", v.Meta()), zap.Uint64("txnStartTS", txn.StartTS()), zap.String("handle", h.String()), zap.Any("record", rec), zap.Error(err)) return err } - if err = v.Delete(ctx.GetSessionVars().StmtCtx, txn.GetUnionStore(), vals, h); err != nil { + if err = v.Delete(ctx.GetSessionVars().StmtCtx, txn, vals, h); err != nil { if v.Meta().State != model.StatePublic && kv.ErrNotExist.Equal(err) { // If the index is not in public state, we may have not created the index, // or already deleted the index, so skip ErrNotExist error. @@ -1167,7 +1167,7 @@ func (t *TableCommon) removeRowIndices(ctx sessionctx.Context, h kv.Handle, rec // removeRowIndex implements table.Table RemoveRowIndex interface. func (t *TableCommon) removeRowIndex(sc *stmtctx.StatementContext, h kv.Handle, vals []types.Datum, idx table.Index, txn kv.Transaction) error { - return idx.Delete(sc, txn.GetUnionStore(), vals, h) + return idx.Delete(sc, txn, vals, h) } // buildIndexForRow implements table.Table BuildIndexForRow interface. diff --git a/util/admin/admin.go b/util/admin/admin.go index 20217a53c1b6d..608040a89d6b0 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -389,7 +389,7 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table vals1[i] = colDefVal } } - isExist, h2, err := idx.Exist(sc, txn.GetUnionStore(), vals1, h1) + isExist, h2, err := idx.Exist(sc, txn, vals1, h1) if kv.ErrKeyExists.Equal(err) { record1 := &RecordData{Handle: h1, Values: vals1} record2 := &RecordData{Handle: h2, Values: vals1} diff --git a/util/mock/context.go b/util/mock/context.go index 4b329e0ff1f55..d6a5f1d913902 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -59,14 +59,6 @@ func (txn *wrapTxn) Valid() bool { return txn.Transaction != nil && txn.Transaction.Valid() } -// GetUnionStore implements GetUnionStore -func (txn *wrapTxn) GetUnionStore() kv.UnionStore { - if txn.Transaction == nil { - return nil - } - return txn.Transaction.GetUnionStore() -} - func (txn *wrapTxn) CacheTableInfo(id int64, info *model.TableInfo) { if txn.Transaction == nil { return From 18cbfaac15f8478902726e1e64971aa96b862462 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 14 May 2021 09:15:38 +0800 Subject: [PATCH 067/343] *: enable gosimple linter (#24617) --- Makefile | 1 + ddl/db_test.go | 8 +++---- ddl/ddl_api.go | 2 +- ddl/serial_test.go | 10 ++++---- executor/aggfuncs/func_percentile.go | 4 +--- executor/concurrent_map.go | 1 - executor/delete.go | 5 +--- executor/executor_test.go | 4 ++-- executor/explainfor_test.go | 24 +++++++++---------- executor/grant_test.go | 4 ++-- executor/infoschema_reader.go | 1 - executor/insert_test.go | 2 +- executor/join.go | 2 +- executor/memtable_reader.go | 4 ++-- executor/merge_join_test.go | 4 ++-- executor/parallel_apply_test.go | 1 - executor/partition_table_test.go | 2 +- executor/show.go | 8 +++---- expression/integration_test.go | 2 +- go.mod | 2 +- go.sum | 5 ++-- infoschema/metrics_schema.go | 14 +++-------- planner/cascades/implementation_rules.go | 10 ++------ planner/cascades/transformation_rules.go | 5 +--- planner/core/exhaust_physical_plans.go | 1 + planner/core/partition_pruner_test.go | 8 +++---- planner/core/pb_to_plan.go | 22 ++++++++--------- planner/core/physical_plans.go | 4 +--- planner/core/rule_partition_processor.go | 8 +++---- planner/core/stringer.go | 12 +++++----- planner/core/util.go | 2 +- plugin/conn_ip_example/conn_ip_example.go | 5 ---- server/http_handler_test.go | 6 ++--- server/server_test.go | 6 ++--- sessionctx/binloginfo/binloginfo.go | 20 +++++++--------- statistics/cmsketch.go | 9 +++---- statistics/feedback.go | 2 -- statistics/handle/handle.go | 1 - statistics/handle/handle_test.go | 2 +- statistics/handle/update_test.go | 4 ++-- store/copr/mpp.go | 8 ++----- .../unistore/cophandler/closure_exec.go | 4 +--- .../unistore/tikv/dbreader/db_reader.go | 1 - store/mockstore/unistore/tikv/detector.go | 5 +--- .../mockstore/unistore/tikv/detector_test.go | 3 +-- store/tikv/region_request.go | 5 +--- store/tikv/txn.go | 2 +- util/chunk/row_container.go | 2 -- util/execdetails/execdetails.go | 4 +--- util/expensivequery/memory_usage_alarm.go | 1 - util/profile/trackerRecorder.go | 10 ++++---- util/rowcodec/decoder.go | 5 +--- 52 files changed, 111 insertions(+), 176 deletions(-) diff --git a/Makefile b/Makefile index dd0e5f521acab..d6681083dc284 100644 --- a/Makefile +++ b/Makefile @@ -57,6 +57,7 @@ check-static: tools/bin/golangci-lint --enable=unused \ --enable=structcheck \ --enable=deadcode \ + --enable=gosimple \ $$($(PACKAGE_DIRECTORIES)) check-slow:tools/bin/gometalinter tools/bin/gosec diff --git a/ddl/db_test.go b/ddl/db_test.go index eddad6d0d635d..ecef0d0144215 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2280,10 +2280,8 @@ func (s *testDBSuite6) TestDropColumn(c *C) { testddlutil.ExecMultiSQLInGoroutine(c, s.store, "drop_col_db", []string{"insert into t2 set c1 = 1, c2 = 1, c3 = 1, c4 = 1"}, dmlDone) } for i := 0; i < num; i++ { - select { - case err := <-ddlDone: - c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) - } + err := <-ddlDone + c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) } // Test for drop partition table column. @@ -6575,7 +6573,7 @@ func (s *testSerialDBSuite) TestModifyColumnTypeWhenInterception(c *C) { count := defaultBatchSize * 4 // Add some rows. - dml := fmt.Sprintf("insert into t values") + dml := "insert into t values" for i := 1; i <= count; i++ { dml += fmt.Sprintf("(%d, %f)", i, 11.22) if i != count { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e6d77c9e674e9..528a6087638fc 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4040,7 +4040,7 @@ func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNe autoid.MaxAutoRandomBits, newRandBits, specNewColumn.Name.Name.O) return 0, ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) } - break // Increasing auto_random shard bits is allowed. + // increasing auto_random shard bits is allowed. case oldRandBits > newRandBits: if newRandBits == 0 { return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg) diff --git a/ddl/serial_test.go b/ddl/serial_test.go index e532bfc2352af..a8d245007f39c 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -503,15 +503,15 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) { // for failure table cases tk.MustExec("use ctwl_db") - failSQL := fmt.Sprintf("create table t1 like test_not_exist.t") + failSQL := "create table t1 like test_not_exist.t" tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable) - failSQL = fmt.Sprintf("create table t1 like test.t_not_exist") + failSQL = "create table t1 like test.t_not_exist" tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable) - failSQL = fmt.Sprintf("create table t1 (like test_not_exist.t)") + failSQL = "create table t1 (like test_not_exist.t)" tk.MustGetErrCode(failSQL, mysql.ErrNoSuchTable) - failSQL = fmt.Sprintf("create table test_not_exis.t1 like ctwl_db.t") + failSQL = "create table test_not_exis.t1 like ctwl_db.t" tk.MustGetErrCode(failSQL, mysql.ErrBadDB) - failSQL = fmt.Sprintf("create table t1 like ctwl_db.t") + failSQL = "create table t1 like ctwl_db.t" tk.MustGetErrCode(failSQL, mysql.ErrTableExists) // test failure for wrong object cases diff --git a/executor/aggfuncs/func_percentile.go b/executor/aggfuncs/func_percentile.go index 31855f791fb0f..ff13392276c61 100644 --- a/executor/aggfuncs/func_percentile.go +++ b/executor/aggfuncs/func_percentile.go @@ -53,9 +53,7 @@ func (e *basePercentile) AllocPartialResult() (pr PartialResult, memDelta int64) return } -func (e *basePercentile) ResetPartialResult(pr PartialResult) { - return -} +func (e *basePercentile) ResetPartialResult(pr PartialResult) {} func (e *basePercentile) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup []chunk.Row, pr PartialResult) (memDelta int64, err error) { return diff --git a/executor/concurrent_map.go b/executor/concurrent_map.go index 27f13a4f21dcb..3d6ef1082f605 100644 --- a/executor/concurrent_map.go +++ b/executor/concurrent_map.go @@ -56,7 +56,6 @@ func (m concurrentMap) Insert(key uint64, value *entry) { shard.items[key] = value } shard.Unlock() - return } // UpsertCb : Callback to return new element to be inserted into the map diff --git a/executor/delete.go b/executor/delete.go index 1fe9c26b8ac82..16f0e9c421b19 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -188,10 +188,7 @@ func (e *DeleteExec) removeRowsInTblRowMap(tblRowMap tableRowMapType) error { var err error rowMap.Range(func(h kv.Handle, val interface{}) bool { err = e.removeRow(e.ctx, e.tblID2Table[id], h, val.([]types.Datum)) - if err != nil { - return false - } - return true + return err == nil }) if err != nil { return err diff --git a/executor/executor_test.go b/executor/executor_test.go index 7b433e71e60ac..65af164174e6f 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5809,7 +5809,7 @@ func (s *testRecoverTable) TestRecoverTable(c *C) { // Test for recover one table multiple time. tk.MustExec("drop table t_recover") tk.MustExec("flashback table t_recover to t_recover_tmp") - _, err = tk.Exec(fmt.Sprintf("recover table t_recover")) + _, err = tk.Exec("recover table t_recover") c.Assert(infoschema.ErrTableExists.Equal(err), IsTrue) gcEnable, err := gcutil.CheckGCEnable(tk.Se) @@ -5876,7 +5876,7 @@ func (s *testRecoverTable) TestFlashbackTable(c *C) { tk.MustQuery("select a,_tidb_rowid from t_flashback2;").Check(testkit.Rows("1 1", "2 2", "3 3", "4 5001", "5 5002", "6 5003", "7 10001", "8 10002", "9 10003")) // Test for flashback one table multiple time. - _, err = tk.Exec(fmt.Sprintf("flashback table t_flashback to t_flashback4")) + _, err = tk.Exec("flashback table t_flashback to t_flashback4") c.Assert(infoschema.ErrTableExists.Equal(err), IsTrue) // Test for flashback truncated table to new table. diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index e29a7a3e24cee..46df545b1ff47 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -181,11 +181,11 @@ func (s *testSuite) TestExplainMemTablePredicate(c *C) { func (s *testSuite) TestExplainClusterTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where type in ('tikv', 'tidb')")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where type in ('tikv', 'tidb')").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG node_types:["tidb","tikv"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where instance='192.168.1.7:2379'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where instance='192.168.1.7:2379'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG instances:["192.168.1.7:2379"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.cluster_config where type='tidb' and instance='192.168.1.7:2379'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.cluster_config where type='tidb' and instance='192.168.1.7:2379'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:CLUSTER_CONFIG node_types:["tidb"], instances:["192.168.1.7:2379"]`)) } @@ -203,11 +203,11 @@ func (s *testSuite) TestInspectionResultTable(c *C) { func (s *testSuite) TestInspectionRuleTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES node_types:["inspection"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection' or type='summary'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection' or type='summary'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES node_types:["inspection","summary"]`)) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.inspection_rules where type='inspection' and type='summary'")).Check(testkit.Rows( + tk.MustQuery("desc select * from information_schema.inspection_rules where type='inspection' and type='summary'").Check(testkit.Rows( `MemTableScan_5 10000.00 root table:INSPECTION_RULES skip_request: true`)) } @@ -355,12 +355,12 @@ func (s *testPrepareSerialSuite) TestExplainDotForQuery(c *C) { func (s *testSuite) TestExplainTableStorage(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"]"))) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_NAME = 'schemata'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]"))) - tk.MustQuery(fmt.Sprintf("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'")).Check(testkit.Rows( - fmt.Sprintf("MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]"))) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"]")) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_NAME = 'schemata'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS table:[\"schemata\"]")) + tk.MustQuery("desc select * from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'information_schema' and TABLE_NAME = 'schemata'").Check(testkit.Rows( + "MemTableScan_5 10000.00 root table:TABLE_STORAGE_STATS schema:[\"information_schema\"], table:[\"schemata\"]")) } func (s *testSuite) TestInspectionSummaryTable(c *C) { diff --git a/executor/grant_test.go b/executor/grant_test.go index 13686494feddc..0e41c75e44353 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -68,7 +68,7 @@ func (s *testSuite3) TestGrantDBScope(c *C) { createUserSQL := `CREATE USER 'testDB'@'localhost' IDENTIFIED BY '123';` tk.MustExec(createUserSQL) // Make sure all the db privs for new user is empty. - sql := fmt.Sprintf("SELECT * FROM mysql.db WHERE User=\"testDB\" and host=\"localhost\"") + sql := `SELECT * FROM mysql.db WHERE User="testDB" and host="localhost"` tk.MustQuery(sql).Check(testkit.Rows()) // Grant each priv to the user. @@ -101,7 +101,7 @@ func (s *testSuite3) TestWithGrantOption(c *C) { createUserSQL := `CREATE USER 'testWithGrant'@'localhost' IDENTIFIED BY '123';` tk.MustExec(createUserSQL) // Make sure all the db privs for new user is empty. - sql := fmt.Sprintf("SELECT * FROM mysql.db WHERE User=\"testWithGrant\" and host=\"localhost\"") + sql := `SELECT * FROM mysql.db WHERE User="testWithGrant" and host="localhost"` tk.MustQuery(sql).Check(testkit.Rows()) // Grant select priv to the user, with grant option. diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index ae338bdd644d2..4d56cc55accac 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1856,7 +1856,6 @@ func (e *memtableRetriever) dataForTableTiFlashReplica(ctx sessionctx.Context, s } } e.rows = rows - return } func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, tableName string) error { diff --git a/executor/insert_test.go b/executor/insert_test.go index ffcfdc214bdb9..351b337bd84ae 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -426,7 +426,7 @@ func (s *testSuite3) TestInsertDateTimeWithTimeZone(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (ts timestamp)") tk.MustExec("insert into t values ('2020-10-22T12:00:00Z'), ('2020-10-22T13:00:00Z'), ('2020-10-22T14:00:00Z')") - tk.MustQuery(fmt.Sprintf("select count(*) from t where ts > '2020-10-22T12:00:00Z'")).Check(testkit.Rows("2")) + tk.MustQuery("select count(*) from t where ts > '2020-10-22T12:00:00Z'").Check(testkit.Rows("2")) // test for datetime with fsp fspCases := []struct { diff --git a/executor/join.go b/executor/join.go index c1a8045aba9a3..1a3f62de47ac1 100644 --- a/executor/join.go +++ b/executor/join.go @@ -1078,7 +1078,7 @@ func (e *joinRuntimeStats) String() string { if e.cache.useCache { buf.WriteString(fmt.Sprintf(", cache:ON, cacheHitRatio:%.3f%%", e.cache.hitRatio*100)) } else { - buf.WriteString(fmt.Sprintf(", cache:OFF")) + buf.WriteString(", cache:OFF") } } if e.hasHashStat { diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index 24e2001131580..76ea478d91e71 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -230,9 +230,9 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String continue } var str string - switch val.(type) { + switch val := val.(type) { case string: // remove quotes - str = val.(string) + str = val default: tmp, err := json.Marshal(val) if err != nil { diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 94b9c33a5a9d7..cf5328fc93263 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -277,7 +277,6 @@ func (s *testSerialSuite1) TestShuffleMergeJoinInDisk(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), Greater, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.BytesConsumed(), Equals, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), Greater, int64(0)) - return } func (s *testSerialSuite1) TestMergeJoinInDisk(c *C) { defer config.RestoreFunc()() @@ -313,7 +312,6 @@ func (s *testSerialSuite1) TestMergeJoinInDisk(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), Greater, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.BytesConsumed(), Equals, int64(0)) c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), Greater, int64(0)) - return } func (s *testSuite2) TestMergeJoin(c *C) { @@ -726,6 +724,7 @@ func (s *testSuite2) TestMergeJoinDifferentTypes(c *C) { } // TestVectorizedMergeJoin is used to test vectorized merge join with some corner cases. +//nolint:gosimple // generates false positive fmt.Sprintf warnings which keep aligned func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -841,6 +840,7 @@ func (s *testSuiteJoin3) TestVectorizedMergeJoin(c *C) { } // TestVectorizedShuffleMergeJoin is used to test vectorized shuffle merge join with some corner cases. +//nolint:gosimple // generates false positive fmt.Sprintf warnings which keep aligned func (s *testSuiteJoin3) TestVectorizedShuffleMergeJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@session.tidb_merge_join_concurrency = 4;") diff --git a/executor/parallel_apply_test.go b/executor/parallel_apply_test.go index b849d3d961043..c0ecb19783273 100644 --- a/executor/parallel_apply_test.go +++ b/executor/parallel_apply_test.go @@ -43,7 +43,6 @@ func checkApplyPlan(c *C, tk *testkit.TestKit, sql string, parallel int) { } } c.Assert(containApply, IsTrue) - return } func (s *testSuite) TestParallelApply(c *C) { diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 5be39c3a04d54..ab5a19ca823e9 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -263,7 +263,7 @@ func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { } // test empty PointGet - queryHash := fmt.Sprintf("select a from thash where a=200") + queryHash := "select a from thash where a=200" c.Assert(tk.HasPlan(queryHash, "Point_Get"), IsTrue) // check if PointGet is used tk.MustQuery(queryHash).Check(testkit.Rows()) diff --git a/executor/show.go b/executor/show.go index c5df3bb290268..2bd9b786fcffb 100644 --- a/executor/show.go +++ b/executor/show.go @@ -862,10 +862,10 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } if pkCol != nil { - // If PKIsHanle, pk info is not in tb.Indices(). We should handle it here. + // If PKIsHandle, pk info is not in tb.Indices(). We should handle it here. buf.WriteString(",\n") fmt.Fprintf(buf, " PRIMARY KEY (%s)", stringutil.Escape(pkCol.Name.O, sqlMode)) - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] CLUSTERED */")) + buf.WriteString(" /*T![clustered_index] CLUSTERED */") } publicIndices := make([]*model.IndexInfo, 0, len(tableInfo.Indices)) @@ -906,9 +906,9 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } if idxInfo.Primary { if tableInfo.PKIsHandle || tableInfo.IsCommonHandle { - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] CLUSTERED */")) + buf.WriteString(" /*T![clustered_index] CLUSTERED */") } else { - buf.WriteString(fmt.Sprintf(" /*T![clustered_index] NONCLUSTERED */")) + buf.WriteString(" /*T![clustered_index] NONCLUSTERED */") } } if i != len(publicIndices)-1 { diff --git a/expression/integration_test.go b/expression/integration_test.go index a3d983069cce9..8e0f70823f327 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -5833,7 +5833,7 @@ func (s *testIntegrationSuite) TestDecodetoChunkReuse(c *C) { tk.MustExec("create table chk (a int,b varchar(20))") for i := 0; i < 200; i++ { if i%5 == 0 { - tk.MustExec(fmt.Sprintf("insert chk values (NULL,NULL)")) + tk.MustExec("insert chk values (NULL,NULL)") continue } tk.MustExec(fmt.Sprintf("insert chk values (%d,'%s')", i, strconv.Itoa(i))) diff --git a/go.mod b/go.mod index c4b668a81dcf1..fe8e08ae42e47 100644 --- a/go.mod +++ b/go.mod @@ -82,7 +82,7 @@ require ( gopkg.in/natefinch/lumberjack.v2 v2.0.0 gopkg.in/yaml.v2 v2.4.0 // indirect gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 // indirect - honnef.co/go/tools v0.1.3 // indirect + honnef.co/go/tools v0.1.4 // indirect modernc.org/mathutil v1.2.2 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 diff --git a/go.sum b/go.sum index 212917e414404..3ee71da011a54 100644 --- a/go.sum +++ b/go.sum @@ -500,7 +500,6 @@ github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQD github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= -github.com/sergi/go-diff v1.1.0 h1:we8PVUC3FE2uYfodKH/nBHMSetSfHDR6scGdBi+erh0= github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v3.21.2+incompatible h1:U+YvJfjCh6MslYlIAXvPtzhW3YZEtc9uncueUNpD/0A= @@ -933,8 +932,8 @@ honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.1.3 h1:qTakTkI6ni6LFD5sBwwsdSO+AQqbSIxOauHTTQKZ/7o= -honnef.co/go/tools v0.1.3/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= +honnef.co/go/tools v0.1.4 h1:SadWOkti5uVN1FAMgxn165+Mw00fuQKyk4Gyn/inxNQ= +honnef.co/go/tools v0.1.4/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= modernc.org/mathutil v1.2.2 h1:+yFk8hBprV+4c0U9GjFtL+dV3N8hOJ8JCituQcMShFY= modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= diff --git a/infoschema/metrics_schema.go b/infoschema/metrics_schema.go index 49a57e4ac9eeb..3b4654f90f7f2 100644 --- a/infoschema/metrics_schema.go +++ b/infoschema/metrics_schema.go @@ -100,17 +100,9 @@ func (def *MetricTableDef) genColumnInfos() []columnInfo { // GenPromQL generates the promQL. func (def *MetricTableDef) GenPromQL(sctx sessionctx.Context, labels map[string]set.StringSet, quantile float64) string { promQL := def.PromQL - if strings.Contains(promQL, promQLQuantileKey) { - promQL = strings.Replace(promQL, promQLQuantileKey, strconv.FormatFloat(quantile, 'f', -1, 64), -1) - } - - if strings.Contains(promQL, promQLLabelConditionKey) { - promQL = strings.Replace(promQL, promQLLabelConditionKey, def.genLabelCondition(labels), -1) - } - - if strings.Contains(promQL, promQRangeDurationKey) { - promQL = strings.Replace(promQL, promQRangeDurationKey, strconv.FormatInt(sctx.GetSessionVars().MetricSchemaRangeDuration, 10)+"s", -1) - } + promQL = strings.Replace(promQL, promQLQuantileKey, strconv.FormatFloat(quantile, 'f', -1, 64), -1) + promQL = strings.Replace(promQL, promQLLabelConditionKey, def.genLabelCondition(labels), -1) + promQL = strings.Replace(promQL, promQRangeDurationKey, strconv.FormatInt(sctx.GetSessionVars().MetricSchemaRangeDuration, 10)+"s", -1) return promQL } diff --git a/planner/cascades/implementation_rules.go b/planner/cascades/implementation_rules.go index d7a08b4fabaab..56c2141cfb213 100644 --- a/planner/cascades/implementation_rules.go +++ b/planner/cascades/implementation_rules.go @@ -95,10 +95,7 @@ type ImplTableDual struct { // Match implements ImplementationRule Match interface. func (r *ImplTableDual) Match(expr *memo.GroupExpr, prop *property.PhysicalProperty) (matched bool) { - if !prop.IsEmpty() { - return false - } - return true + return prop.IsEmpty() } // OnImplement implements ImplementationRule OnImplement interface. @@ -116,10 +113,7 @@ type ImplMemTableScan struct { // Match implements ImplementationRule Match interface. func (r *ImplMemTableScan) Match(expr *memo.GroupExpr, prop *property.PhysicalProperty) (matched bool) { - if !prop.IsEmpty() { - return false - } - return true + return prop.IsEmpty() } // OnImplement implements ImplementationRule OnImplement interface. diff --git a/planner/cascades/transformation_rules.go b/planner/cascades/transformation_rules.go index 6d23e063f5877..9961509299a52 100644 --- a/planner/cascades/transformation_rules.go +++ b/planner/cascades/transformation_rules.go @@ -1503,10 +1503,7 @@ func NewRuleMergeAggregationProjection() Transformation { // Match implements Transformation interface. func (r *MergeAggregationProjection) Match(old *memo.ExprIter) bool { proj := old.Children[0].GetExpr().ExprNode.(*plannercore.LogicalProjection) - if plannercore.ExprsHasSideEffects(proj.Exprs) { - return false - } - return true + return !plannercore.ExprsHasSideEffects(proj.Exprs) } // OnTransform implements Transformation interface. diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index cd227657a75d9..7c7493eedd5c5 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1416,6 +1416,7 @@ func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges []*ranger.Range, p func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool) (ranges []*ranger.Range, emptyRange bool, err error) { pointLength := matchedKeyCnt + len(eqAndInFuncs) + //nolint:gosimple // false positive unnecessary nil check if nextColRange != nil { for _, colRan := range nextColRange { // The range's exclude status is the same with last col's. diff --git a/planner/core/partition_pruner_test.go b/planner/core/partition_pruner_test.go index dd614239de55d..e151efdecce03 100644 --- a/planner/core/partition_pruner_test.go +++ b/planner/core/partition_pruner_test.go @@ -327,7 +327,7 @@ func (s *testPartitionPruneSuit) TestListColumnsPartitionPrunerRandom(c *C) { tk1.MustExec(insert) // Test query without condition - query := fmt.Sprintf("select * from t1 order by id,a,b") + query := "select * from t1 order by id,a,b" tk.MustQuery(query).Check(tk1.MustQuery(query).Rows()) } @@ -467,9 +467,9 @@ func (s *testPartitionPruneSuit) TestRangePartitionPredicatePruner(c *C) { tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeIntOnly tk.MustExec(`create table t (a int(11) default null) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin partition by range(a) ( - partition p0 values less than (1), - partition p1 values less than (2), - partition p2 values less than (3), + partition p0 values less than (1), + partition p1 values less than (2), + partition p2 values less than (3), partition p_max values less than (maxvalue));`) var input []string diff --git a/planner/core/pb_to_plan.go b/planner/core/pb_to_plan.go index 3cfaf6708affe..a453596a0f288 100644 --- a/planner/core/pb_to_plan.go +++ b/planner/core/pb_to_plan.go @@ -255,15 +255,15 @@ func (b *PBPlanBuilder) pbToKill(e *tipb.Executor) (PhysicalPlan, error) { return &PhysicalSimpleWrapper{Inner: simple}, nil } -func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { - if p == nil { - return predicates, p +func (b *PBPlanBuilder) predicatePushDown(physicalPlan PhysicalPlan, predicates []expression.Expression) ([]expression.Expression, PhysicalPlan) { + if physicalPlan == nil { + return predicates, physicalPlan } - switch p.(type) { + switch plan := physicalPlan.(type) { case *PhysicalMemTable: - memTable := p.(*PhysicalMemTable) + memTable := plan if memTable.Extractor == nil { - return predicates, p + return predicates, plan } names := make([]*types.FieldName, 0, len(memTable.Columns)) for _, col := range memTable.Columns { @@ -284,8 +284,8 @@ func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expressio predicates = memTable.Extractor.Extract(b.sctx, memTable.schema, names, predicates) return predicates, memTable case *PhysicalSelection: - selection := p.(*PhysicalSelection) - conditions, child := b.predicatePushDown(p.Children()[0], selection.Conditions) + selection := plan + conditions, child := b.predicatePushDown(plan.Children()[0], selection.Conditions) if len(conditions) > 0 { selection.Conditions = conditions selection.SetChildren(child) @@ -293,10 +293,10 @@ func (b *PBPlanBuilder) predicatePushDown(p PhysicalPlan, predicates []expressio } return predicates, child default: - if children := p.Children(); len(children) > 0 { + if children := plan.Children(); len(children) > 0 { _, child := b.predicatePushDown(children[0], nil) - p.SetChildren(child) + plan.SetChildren(child) } - return predicates, p + return predicates, plan } } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 43304971b4680..f4e6769ee8d70 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -911,9 +911,7 @@ func (p *PhysicalMergeJoin) Clone() (PhysicalPlan, error) { return nil, err } cloned.basePhysicalJoin = *base - for _, cf := range p.CompareFuncs { - cloned.CompareFuncs = append(cloned.CompareFuncs, cf) - } + cloned.CompareFuncs = append(cloned.CompareFuncs, p.CompareFuncs...) cloned.Desc = p.Desc return cloned, nil } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 57858679b7795..17f8c5b2a88b1 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -297,7 +297,7 @@ func (s *partitionProcessor) reconstructTableColNames(ds *DataSource) ([]*types. }) continue } - return nil, errors.New(fmt.Sprintf("information of column %v is not found", colExpr.String())) + return nil, fmt.Errorf("information of column %v is not found", colExpr.String()) } return names, nil } @@ -1345,9 +1345,9 @@ func appendWarnForUnknownPartitions(ctx sessionctx.Context, hintName string, unk if len(unknownPartitions) == 0 { return } - ctx.GetSessionVars().StmtCtx.AppendWarning( - errors.New(fmt.Sprintf("Unknown partitions (%s) in optimizer hint %s", - strings.Join(unknownPartitions, ","), hintName))) + + warning := fmt.Errorf("Unknown partitions (%s) in optimizer hint %s", strings.Join(unknownPartitions, ","), hintName) + ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } func (s *partitionProcessor) checkHintsApplicable(ds *DataSource, partitionSet set.StringSet) { diff --git a/planner/core/stringer.go b/planner/core/stringer.go index 4c63f6ff244b9..346b5b50e5742 100644 --- a/planner/core/stringer.go +++ b/planner/core/stringer.go @@ -270,31 +270,31 @@ func toString(in Plan, strs []string, idxs []int) ([]string, []int) { case *PhysicalShuffleReceiverStub: str = fmt.Sprintf("PartitionReceiverStub(%s)", x.ExplainInfo()) case *PointGetPlan: - str = fmt.Sprintf("PointGet(") + str = "PointGet(" if x.IndexInfo != nil { str += fmt.Sprintf("Index(%s.%s)%v)", x.TblInfo.Name.L, x.IndexInfo.Name.L, x.IndexValues) } else { str += fmt.Sprintf("Handle(%s.%s)%v)", x.TblInfo.Name.L, x.TblInfo.GetPkName().L, x.Handle) } case *BatchPointGetPlan: - str = fmt.Sprintf("BatchPointGet(") + str = "BatchPointGet(" if x.IndexInfo != nil { str += fmt.Sprintf("Index(%s.%s)%v)", x.TblInfo.Name.L, x.IndexInfo.Name.L, x.IndexValues) } else { str += fmt.Sprintf("Handle(%s.%s)%v)", x.TblInfo.Name.L, x.TblInfo.GetPkName().L, x.Handles) } case *PhysicalExchangeReceiver: - str = fmt.Sprintf("Recv(") + str = "Recv(" for _, task := range x.Tasks { str += fmt.Sprintf("%d, ", task.ID) } - str += fmt.Sprintf(")") + str += ")" case *PhysicalExchangeSender: - str = fmt.Sprintf("Send(") + str = "Send(" for _, task := range x.TargetTasks { str += fmt.Sprintf("%d, ", task.ID) } - str += fmt.Sprintf(")") + str += ")" default: str = fmt.Sprintf("%T", in) } diff --git a/planner/core/util.go b/planner/core/util.go index 753445f07ee42..19ce0a47673a2 100644 --- a/planner/core/util.go +++ b/planner/core/util.go @@ -291,7 +291,7 @@ func extractStringFromStringSet(set set.StringSet) string { l = append(l, fmt.Sprintf(`"%s"`, k)) } sort.Strings(l) - return fmt.Sprintf("%s", strings.Join(l, ",")) + return strings.Join(l, ",") } func tableHasDirtyContent(ctx sessionctx.Context, tableInfo *model.TableInfo) bool { diff --git a/plugin/conn_ip_example/conn_ip_example.go b/plugin/conn_ip_example/conn_ip_example.go index bae1b3ff37497..24d0bf04b0309 100644 --- a/plugin/conn_ip_example/conn_ip_example.go +++ b/plugin/conn_ip_example/conn_ip_example.go @@ -62,19 +62,14 @@ func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugi switch event { case plugin.Log: fmt.Println("---- event: Log") - break case plugin.Error: fmt.Println("---- event: Error") - break case plugin.Result: fmt.Println("---- event: Result") - break case plugin.Status: fmt.Println("---- event: Status") - break default: fmt.Println("---- event: unrecognized") - break } fmt.Printf("---- cmd: %s\n", cmd) } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index a7495b987f24a..7cdd7137138bd 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -578,7 +578,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { ts.prepareData(c) defer ts.stopServer(c) - resp, err := ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1")) + resp, err := ts.fetchStatus("/mvcc/key/tidb/test/1") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) var data mvccKV @@ -621,7 +621,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { c.Assert(err, IsNil) c.Assert(data2, DeepEquals, data) - resp, err = ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1?decode=true")) + resp, err = ts.fetchStatus("/mvcc/key/tidb/test/1?decode=true") c.Assert(err, IsNil) decoder = json.NewDecoder(resp.Body) var data3 map[string]interface{} @@ -667,7 +667,7 @@ func (ts *HTTPHandlerTestSuite) TestGetMVCCNotFound(c *C) { ts.startServer(c) ts.prepareData(c) defer ts.stopServer(c) - resp, err := ts.fetchStatus(fmt.Sprintf("/mvcc/key/tidb/test/1234")) + resp, err := ts.fetchStatus("/mvcc/key/tidb/test/1234") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) var data mvccKV diff --git a/server/server_test.go b/server/server_test.go index 20c0c2b508213..a342dab77d79a 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -499,14 +499,14 @@ func (cli *testServerClient) runTestLoadDataForSlowLog(c *C, server *Server) { } // Test for record slow log for load data statement. - rows := dbt.mustQuery(fmt.Sprintf("select plan from information_schema.slow_query where query like 'load data local infile %% into table t_slow;' order by time desc limit 1")) + rows := dbt.mustQuery("select plan from information_schema.slow_query where query like 'load data local infile % into table t_slow;' order by time desc limit 1") expectedPlan := ".*LoadData.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.* commit_txn.*" checkPlan(rows, expectedPlan) // Test for record statements_summary for load data statement. - rows = dbt.mustQuery(fmt.Sprintf("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %%' limit 1")) + rows = dbt.mustQuery("select plan from information_schema.STATEMENTS_SUMMARY where QUERY_SAMPLE_TEXT like 'load data local infile %' limit 1") checkPlan(rows, expectedPlan) // Test log normal statement after executing load date. - rows = dbt.mustQuery(fmt.Sprintf("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1")) + rows = dbt.mustQuery("select plan from information_schema.slow_query where query = 'insert ignore into t_slow values (1,1);' order by time desc limit 1") expectedPlan = ".*Insert.* time.* loops.* prepare.* check_insert.* mem_insert_time:.* prefetch.* rpc.*" checkPlan(rows, expectedPlan) }) diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 163c22e4a6fb7..5645b7496ebcc 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -154,17 +154,15 @@ func WaitBinlogRecover(timeout time.Duration) error { defer ticker.Stop() start := time.Now() for { - select { - case <-ticker.C: - if atomic.LoadInt32(&skippedCommitterCounter) == 0 { - logutil.BgLogger().Warn("[binloginfo] binlog recovered") - return nil - } - if time.Since(start) > timeout { - logutil.BgLogger().Warn("[binloginfo] waiting for binlog recovering timed out", - zap.Duration("duration", timeout)) - return errors.New("timeout") - } + <-ticker.C + if atomic.LoadInt32(&skippedCommitterCounter) == 0 { + logutil.BgLogger().Warn("[binloginfo] binlog recovered") + return nil + } + if time.Since(start) > timeout { + logutil.BgLogger().Warn("[binloginfo] waiting for binlog recovering timed out", + zap.Duration("duration", timeout)) + return errors.New("timeout") } } } diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index f682a1507a4bf..68791a7a0787f 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -818,14 +818,11 @@ func MergeTopN(topNs []*TopN, n uint32) (*TopN, []TopNMeta) { } func checkEmptyTopNs(topNs []*TopN) bool { - totCnt := uint64(0) + count := uint64(0) for _, topN := range topNs { - totCnt += topN.TotalCount() + count += topN.TotalCount() } - if totCnt == 0 { - return true - } - return false + return count == 0 } func getMergedTopNFromSortedSlice(sorted []TopNMeta, n uint32) (*TopN, []TopNMeta) { diff --git a/statistics/feedback.go b/statistics/feedback.go index 89aeab32152b3..5b69163e21b2b 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -107,7 +107,6 @@ func (m *QueryFeedbackMap) Append(q *QueryFeedback) { Tp: q.Tp, } m.append(k, []*QueryFeedback{q}) - return } // MaxQueryFeedbackCount is the max number of feedbacks that are cached in memory. @@ -136,7 +135,6 @@ func (m *QueryFeedbackMap) Merge(r *QueryFeedbackMap) { break } } - return } var ( diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index a8f51f8924e81..fe3644739c1b8 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -554,7 +554,6 @@ func (sc statsCache) initMemoryUsage() { sum += tb.MemoryUsage() } sc.memUsage = sum - return } // update updates the statistics table cache using copy on write. diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 2bf1d93ff4246..6d39e42b076e0 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -2377,7 +2377,7 @@ func (s *testStatsSuite) TestIndexFMSketch(c *C) { // test NDV checkNDV := func(rows, ndv int) { tk.MustExec("analyze table t") - rs := tk.MustQuery(fmt.Sprintf("select value from mysql.stats_fm_sketch")).Rows() + rs := tk.MustQuery("select value from mysql.stats_fm_sketch").Rows() c.Assert(len(rs), Equals, rows) for i := range rs { fm, err := statistics.DecodeFMSketch([]byte(rs[i][0].(string))) diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index b105738098f4b..a1de28e78eeef 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -577,8 +577,8 @@ func (s *testSerialStatsSuite) TestAutoAnalyzeOnEmptyTable(c *C) { // test if it will be limited by the time range c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsFalse) - tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_start_time='00:00 +0000'")) - tk.MustExec(fmt.Sprintf("set global tidb_auto_analyze_end_time='23:59 +0000'")) + tk.MustExec("set global tidb_auto_analyze_start_time='00:00 +0000'") + tk.MustExec("set global tidb_auto_analyze_end_time='23:59 +0000'") c.Assert(s.do.StatsHandle().HandleAutoAnalyze(s.do.InfoSchema()), IsTrue) } diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 9869fa501d430..2aaf4223ed8e5 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -144,14 +144,10 @@ func (m *mppIterator) run(ctx context.Context) { break } m.mu.Lock() - switch task.State { - case kv.MppTaskReady: + if task.State == kv.MppTaskReady { task.State = kv.MppTaskRunning - m.mu.Unlock() - default: - m.mu.Unlock() - break } + m.mu.Unlock() m.wg.Add(1) bo := backoff.NewBackoffer(ctx, copNextMaxBackoff) go m.handleDispatchReq(ctx, bo, task) diff --git a/store/mockstore/unistore/cophandler/closure_exec.go b/store/mockstore/unistore/cophandler/closure_exec.go index 152362c13d5e3..696a1497b068e 100644 --- a/store/mockstore/unistore/cophandler/closure_exec.go +++ b/store/mockstore/unistore/cophandler/closure_exec.go @@ -124,9 +124,7 @@ func buildClosureExecutorFromExecutorList(dagCtx *dagContext, executors []*tipb. outputFieldTypes = append(outputFieldTypes, originalOutputFieldTypes[idx]) } } else { - for _, tp := range originalOutputFieldTypes { - outputFieldTypes = append(outputFieldTypes, tp) - } + outputFieldTypes = append(outputFieldTypes, originalOutputFieldTypes...) } if len(executors) == 1 { ce.resultFieldType = outputFieldTypes diff --git a/store/mockstore/unistore/tikv/dbreader/db_reader.go b/store/mockstore/unistore/tikv/dbreader/db_reader.go index 158f295cc1bdf..6e3909ada740f 100644 --- a/store/mockstore/unistore/tikv/dbreader/db_reader.go +++ b/store/mockstore/unistore/tikv/dbreader/db_reader.go @@ -165,7 +165,6 @@ func (r *DBReader) BatchGet(keys [][]byte, startTS uint64, f BatchGetFunc) { } f(key, val, err) } - return } // ErrScanBreak is returned by ScanFunc to break the scan loop. diff --git a/store/mockstore/unistore/tikv/detector.go b/store/mockstore/unistore/tikv/detector.go index a27adb3f35b6c..6a56a0d9fd73c 100644 --- a/store/mockstore/unistore/tikv/detector.go +++ b/store/mockstore/unistore/tikv/detector.go @@ -64,10 +64,7 @@ type diagnosticContext struct { } func (p *txnKeyHashPair) isExpired(ttl time.Duration, nowTime time.Time) bool { - if p.registerTime.Add(ttl).Before(nowTime) { - return true - } - return false + return p.registerTime.Add(ttl).Before(nowTime) } // NewDetector creates a new Detector. diff --git a/store/mockstore/unistore/tikv/detector_test.go b/store/mockstore/unistore/tikv/detector_test.go index b0d3a074ff840..189a8b00b8217 100644 --- a/store/mockstore/unistore/tikv/detector_test.go +++ b/store/mockstore/unistore/tikv/detector_test.go @@ -26,7 +26,6 @@ package tikv import ( - "fmt" "testing" "time" @@ -68,7 +67,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { c.Assert(detector.totalSize, Equals, uint64(2)) err = detector.Detect(3, 1, 300, makeDiagCtx("k3", "tag3")) c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, fmt.Sprintf("deadlock")) + c.Assert(err.Error(), Equals, "deadlock") c.Assert(len(err.WaitChain), Equals, 3) // The order of entries in the wait chain is specific: each item is waiting for the next one. checkWaitChainEntry(err.WaitChain[0], 1, 2, "k1", "tag1") diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index ec5b92f81c5ad..b8b61aac05fc8 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -477,10 +477,7 @@ func (s *RegionRequestSender) sendReqToRegion(bo *Backoffer, rpcCtx *RPCContext, if val.(bool) { ctx1, cancel := context.WithCancel(context.Background()) cancel() - select { - case <-ctx1.Done(): - } - + <-ctx1.Done() ctx = ctx1 err = ctx.Err() resp = nil diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 20bf0491ed294..e444f5adda7f6 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -642,7 +642,7 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput // PointGet and BatchPointGet will return value in pessimistic lock response, the value may not exist. // For other lock modes, the locked key values always exist. if lockCtx.ReturnValues { - val, _ := lockCtx.Values[string(key)] + val := lockCtx.Values[string(key)] if len(val.Value) == 0 { valExists = tikv.SetKeyLockedValueNotExists } diff --git a/util/chunk/row_container.go b/util/chunk/row_container.go index f67cbb36b76da..0ef0b573e1bb2 100644 --- a/util/chunk/row_container.go +++ b/util/chunk/row_container.go @@ -90,7 +90,6 @@ func (c *RowContainer) SpillToDisk() { } } c.m.records.Clear() - return } // Reset resets RowContainer. @@ -447,7 +446,6 @@ func (c *SortedRowContainer) Sort() { func (c *SortedRowContainer) sortAndSpillToDisk() { c.Sort() c.RowContainer.SpillToDisk() - return } // Add appends a chunk into the SortedRowContainer. diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index ca045352dbd33..676f0a241d489 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -703,9 +703,7 @@ func (e *RuntimeStatsWithConcurrencyInfo) SetConcurrencyInfo(infos ...*Concurren e.Lock() defer e.Unlock() e.concurrency = e.concurrency[:0] - for _, info := range infos { - e.concurrency = append(e.concurrency, info) - } + e.concurrency = append(e.concurrency, infos...) } // Clone implements the RuntimeStats interface. diff --git a/util/expensivequery/memory_usage_alarm.go b/util/expensivequery/memory_usage_alarm.go index fbe9b6c5ff438..03d53c12342c9 100644 --- a/util/expensivequery/memory_usage_alarm.go +++ b/util/expensivequery/memory_usage_alarm.go @@ -83,7 +83,6 @@ func (record *memoryUsageAlarm) initMemoryUsageAlarmRecord() { } } record.initialized = true - return } // If Performance.ServerMemoryQuota is set, use `ServerMemoryQuota * MemoryUsageAlarmRatio` to check oom risk. diff --git a/util/profile/trackerRecorder.go b/util/profile/trackerRecorder.go index c5e5390a51bff..7c449ffa9113c 100644 --- a/util/profile/trackerRecorder.go +++ b/util/profile/trackerRecorder.go @@ -29,12 +29,10 @@ func HeapProfileForGlobalMemTracker(d time.Duration) { t := time.NewTicker(d) defer t.Stop() for { - select { - case <-t.C: - err := heapProfileForGlobalMemTracker() - if err != nil { - log.Warn("profile memory into tracker failed", zap.Error(err)) - } + <-t.C + err := heapProfileForGlobalMemTracker() + if err != nil { + log.Warn("profile memory into tracker failed", zap.Error(err)) } } } diff --git a/util/rowcodec/decoder.go b/util/rowcodec/decoder.go index 69a78d1de7d43..0efd50ecaf27c 100644 --- a/util/rowcodec/decoder.go +++ b/util/rowcodec/decoder.go @@ -260,10 +260,7 @@ func (decoder *ChunkDecoder) tryAppendHandleColumn(colIdx int, col *ColInfo, han } coder := codec.NewDecoder(chk, decoder.loc) _, err := coder.DecodeOne(handle.EncodedCol(i), colIdx, col.Ft) - if err != nil { - return false - } - return true + return err == nil } } return false From e628bad698b58f8b1268c7fdf5305f25d16b1c84 Mon Sep 17 00:00:00 2001 From: cfzjywxk Date: Fri, 14 May 2021 10:31:38 +0800 Subject: [PATCH 068/343] txn: avoid the gc resolving pessimistic locks of ongoing transactions (#24601) --- store/gcworker/gc_worker.go | 13 ++++++-- store/gcworker/gc_worker_test.go | 51 +++++++++++++++++++++++++++++++- 2 files changed, 61 insertions(+), 3 deletions(-) diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index e0aa993558b6c..b408f279be98a 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -409,12 +409,21 @@ func (w *GCWorker) calcSafePointByMinStartTS(ctx context.Context, safePoint uint return safePoint } - if globalMinStartTS < safePoint { + // If the lock.ts <= max_ts(safePoint), it will be collected and resolved by the gc worker, + // the locks of ongoing pessimistic transactions could be resolved by the gc worker and then + // the transaction is aborted, decrement the value by 1 to avoid this. + globalMinStartAllowedTS := globalMinStartTS + if globalMinStartTS > 0 { + globalMinStartAllowedTS = globalMinStartTS - 1 + } + + if globalMinStartAllowedTS < safePoint { logutil.Logger(ctx).Info("[gc worker] gc safepoint blocked by a running session", zap.String("uuid", w.uuid), zap.Uint64("globalMinStartTS", globalMinStartTS), + zap.Uint64("globalMinStartAllowedTS", globalMinStartAllowedTS), zap.Uint64("safePoint", safePoint)) - safePoint = globalMinStartTS + safePoint = globalMinStartAllowedTS } return safePoint } diff --git a/store/gcworker/gc_worker_test.go b/store/gcworker/gc_worker_test.go index bc09651e0d379..39abe369f82fb 100644 --- a/store/gcworker/gc_worker_test.go +++ b/store/gcworker/gc_worker_test.go @@ -262,7 +262,7 @@ func (s *testGCWorkerSuite) TestMinStartTS(c *C) { strconv.FormatUint(now-oracle.EncodeTSO(20000), 10)) c.Assert(err, IsNil) sp = s.gcWorker.calcSafePointByMinStartTS(ctx, now-oracle.EncodeTSO(10000)) - c.Assert(sp, Equals, now-oracle.EncodeTSO(20000)) + c.Assert(sp, Equals, now-oracle.EncodeTSO(20000)-1) } func (s *testGCWorkerSuite) TestPrepareGC(c *C) { @@ -1589,3 +1589,52 @@ func (s *testGCWorkerSuite) TestGCPlacementRules(c *C) { c.Assert(pid, Equals, int64(1)) c.Assert(err, IsNil) } + +func (s *testGCWorkerSuite) TestGCWithPendingTxn(c *C) { + ctx := context.Background() + gcSafePointCacheInterval = 0 + err := s.gcWorker.saveValueToSysTable(gcEnableKey, booleanFalse) + c.Assert(err, IsNil) + + k1 := []byte("tk1") + v1 := []byte("v1") + txn, err := s.store.Begin() + c.Assert(err, IsNil) + txn.SetOption(kv.Pessimistic, true) + lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()} + + // Lock the key. + err = txn.Set(k1, v1) + c.Assert(err, IsNil) + err = txn.LockKeys(ctx, lockCtx, k1) + c.Assert(err, IsNil) + + // Prepare to run gc with txn's startTS as the safepoint ts. + spkv := s.tikvStore.GetSafePointKV() + err = spkv.Put(fmt.Sprintf("%s/%s", infosync.ServerMinStartTSPath, "a"), strconv.FormatUint(txn.StartTS(), 10)) + c.Assert(err, IsNil) + s.mustSetTiDBServiceSafePoint(c, txn.StartTS(), txn.StartTS()) + veryLong := gcDefaultLifeTime * 100 + err = s.gcWorker.saveTime(gcLastRunTimeKey, oracle.GetTimeFromTS(s.mustAllocTs(c)).Add(-veryLong)) + c.Assert(err, IsNil) + s.gcWorker.lastFinish = time.Now().Add(-veryLong) + s.oracle.AddOffset(time.Minute * 10) + err = s.gcWorker.saveValueToSysTable(gcEnableKey, booleanTrue) + c.Assert(err, IsNil) + + // Trigger the tick let the gc job start. + err = s.gcWorker.leaderTick(ctx) + c.Assert(err, IsNil) + // Wait for GC finish + select { + case err = <-s.gcWorker.done: + s.gcWorker.gcIsRunning = false + break + case <-time.After(time.Second * 10): + err = errors.New("receive from s.gcWorker.done timeout") + } + c.Assert(err, IsNil) + + err = txn.Commit(ctx) + c.Assert(err, IsNil) +} From d169a57ba5a4e677196d12b7fef797561315b29b Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Fri, 14 May 2021 11:55:38 +0800 Subject: [PATCH 069/343] util: fix wrong enum building for index range (#24632) --- expression/integration_test.go | 10 ++++++++++ util/ranger/points.go | 2 +- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 8e0f70823f327..095e21119ab30 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9344,4 +9344,14 @@ func (s *testIntegrationSuite) TestEnumIndex(c *C) { testkit.Rows( "OFJHCEKCQGT:MXI7P3[YO4N0DF=2XJWJ4Z9Z;HQ8TMUTZV8YLQAHWJ4BDZHR3A -30 ", "ZOHBSCRMZPOI`IVTSEZAIDAF7DS@1TT20AP9 -30 ")) + + // issue 24576 + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(col1 enum('a','b','c'), col2 enum('a','b','c'), col3 int, index idx(col1,col2));") + tk.MustExec("insert into t values(1,1,1),(2,2,2),(3,3,3);") + tk.MustQuery("select /*+ use_index(t,idx) */ col3 from t where col2 between 'b' and 'b' and col1 is not null;").Check( + testkit.Rows("2")) + tk.MustQuery("select /*+ use_index(t,idx) */ col3 from t where col2 = 'b' and col1 is not null;").Check( + testkit.Rows("2")) } diff --git a/util/ranger/points.go b/util/ranger/points.go index 9c33ccef7feb3..46a4283dd3222 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -459,7 +459,7 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val tmpEnum := types.Enum{} for i := range ft.Elems { tmpEnum.Name = ft.Elems[i] - tmpEnum.Value = uint64(i) + tmpEnum.Value = uint64(i) + 1 d := types.NewMysqlEnumDatum(tmpEnum) if v, err := d.CompareDatum(sc, &val); err == nil { switch op { From 80a557eb4956da36f21ba1c81ad12595dcf716b1 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Thu, 13 May 2021 22:43:38 -0600 Subject: [PATCH 070/343] sessionctx: change innodb large prefix default (#24555) --- expression/integration_test.go | 6 ++++++ sessionctx/variable/noop.go | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 095e21119ab30..b1bded2a17719 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9086,9 +9086,15 @@ func (s *testIntegrationSuite) TestEnumPushDown(c *C) { func (s *testIntegrationSuite) TestJiraSetInnoDBDefaultRowFormat(c *C) { // For issue #23541 // JIRA needs to be able to set this to be happy. + // See: https://nova.moe/run-jira-on-tidb/ tk := testkit.NewTestKit(c, s.store) tk.MustExec("set global innodb_default_row_format = dynamic") tk.MustExec("set global innodb_default_row_format = 'dynamic'") + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_default_row_format'").Check(testkit.Rows("innodb_default_row_format dynamic")) + tk.MustQuery("SHOW VARIABLES LIKE 'character_set_server'").Check(testkit.Rows("character_set_server utf8mb4")) + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_file_format'").Check(testkit.Rows("innodb_file_format Barracuda")) + tk.MustQuery("SHOW VARIABLES LIKE 'innodb_large_prefix'").Check(testkit.Rows("innodb_large_prefix ON")) + } func (s *testIntegrationSerialSuite) TestCollationForBinaryLiteral(c *C) { diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index c510d9c73ce3a..1ad37b512d807 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -312,7 +312,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "datetime_format", Value: "%Y-%m-%d %H:%i:%s"}, {Scope: ScopeGlobal, Name: "log_syslog", Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: "transaction_alloc_block_size", Value: "8192"}, - {Scope: ScopeGlobal, Name: "innodb_large_prefix", Type: TypeBool, Value: Off}, + {Scope: ScopeGlobal, Name: "innodb_large_prefix", Type: TypeBool, Value: On}, {Scope: ScopeNone, Name: "performance_schema_max_cond_classes", Value: "80"}, {Scope: ScopeGlobal, Name: "innodb_io_capacity", Value: "200"}, {Scope: ScopeGlobal, Name: "max_binlog_cache_size", Value: "18446744073709547520"}, From ea7f0ca1ba80674a1bec449ca2c3d0f79d48b735 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Fri, 14 May 2021 15:49:38 +0800 Subject: [PATCH 071/343] store: fix data race about KVStore.tikvClient (#24655) --- store/tikv/commit.go | 2 +- store/tikv/kv.go | 25 ++++++++++++++++--------- store/tikv/prewrite.go | 2 +- store/tikv/scan.go | 2 +- store/tikv/split_region.go | 2 +- 5 files changed, 20 insertions(+), 13 deletions(-) diff --git a/store/tikv/commit.go b/store/tikv/commit.go index 449081860c029..10c60d9f6d4bd 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -48,7 +48,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch CommitVersion: c.commitTS, }, pb.Context{Priority: c.priority, SyncLog: c.syncLog}) - sender := NewRegionRequestSender(c.store.regionCache, c.store.client) + sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) // If we fail to receive response for the request that commits primary key, it will be undetermined whether this diff --git a/store/tikv/kv.go b/store/tikv/kv.go index f61db4168ef7d..bbf8517a42a8c 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -66,10 +66,13 @@ var oracleUpdateInterval = 2000 // KVStore contains methods to interact with a TiKV cluster. type KVStore struct { - clusterID uint64 - uuid string - oracle oracle.Oracle - client Client + clusterID uint64 + uuid string + oracle oracle.Oracle + clientMu struct { + sync.RWMutex + client Client + } pdClient pd.Client regionCache *RegionCache lockResolver *LockResolver @@ -133,7 +136,6 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client clusterID: pdClient.GetClusterID(context.TODO()), uuid: uuid, oracle: o, - client: reqCollapse{client}, pdClient: pdClient, regionCache: NewRegionCache(pdClient), kv: spkv, @@ -142,6 +144,7 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client closed: make(chan struct{}), replicaReadSeed: rand.Uint32(), } + store.clientMu.client = reqCollapse{client} store.lockResolver = newLockResolver(store) go store.runSafePointChecker() @@ -205,7 +208,7 @@ func (s *KVStore) Close() error { s.pdClient.Close() close(s.closed) - if err := s.client.Close(); err != nil { + if err := s.GetTiKVClient().Close(); err != nil { return errors.Trace(err) } @@ -312,7 +315,7 @@ func (s *KVStore) SupportDeleteRange() (supported bool) { // SendReq sends a request to region. func (s *KVStore) SendReq(bo *Backoffer, req *tikvrpc.Request, regionID RegionVerID, timeout time.Duration) (*tikvrpc.Response, error) { - sender := NewRegionRequestSender(s.regionCache, s.client) + sender := NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) return sender.SendReq(bo, req, regionID, timeout) } @@ -343,12 +346,16 @@ func (s *KVStore) SetOracle(oracle oracle.Oracle) { // SetTiKVClient resets the client instance. func (s *KVStore) SetTiKVClient(client Client) { - s.client = client + s.clientMu.Lock() + defer s.clientMu.Unlock() + s.clientMu.client = client } // GetTiKVClient gets the client instance. func (s *KVStore) GetTiKVClient() (client Client) { - return s.client + s.clientMu.RLock() + defer s.clientMu.RUnlock() + return s.clientMu.client } func (s *KVStore) getSafeTS(storeID uint64) uint64 { diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index 7097ba5dbcd3e..ffb47e1fb46fa 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -157,7 +157,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff req := c.buildPrewriteRequest(batch, txnSize) for { - sender := NewRegionRequestSender(c.store.regionCache, c.store.client) + sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) // If we fail to receive response for async commit prewrite, it will be undetermined whether this diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 6c43b7bdee7cd..035291a783aec 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -164,7 +164,7 @@ func (s *Scanner) getData(bo *Backoffer) error { zap.String("nextEndKey", kv.StrKey(s.nextEndKey)), zap.Bool("reverse", s.reverse), zap.Uint64("txnStartTS", s.startTS())) - sender := NewRegionRequestSender(s.snapshot.store.regionCache, s.snapshot.store.client) + sender := NewRegionRequestSender(s.snapshot.store.regionCache, s.snapshot.store.GetTiKVClient()) var reqEndKey, reqStartKey []byte var loc *KeyLocation var err error diff --git a/store/tikv/split_region.go b/store/tikv/split_region.go index 38ce24917d1cf..c33a89efc19be 100644 --- a/store/tikv/split_region.go +++ b/store/tikv/split_region.go @@ -123,7 +123,7 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool Priority: kvrpcpb.CommandPri_Normal, }) - sender := NewRegionRequestSender(s.regionCache, s.client) + sender := NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.regionID, ReadTimeoutShort) batchResp := singleBatchResp{resp: resp} From d9f28c7f765662a575e8a2bf6f139229cce1d360 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Fri, 14 May 2021 02:07:39 -0600 Subject: [PATCH 072/343] executor, privileges: Add dynamic privileges to SHOW PRIVILEGES (#24646) --- executor/executor_test.go | 9 ++++++++- executor/show.go | 4 ++++ privilege/privileges/privileges.go | 11 +++++++++++ privilege/privileges/privileges_test.go | 12 ++++++++++++ 4 files changed, 35 insertions(+), 1 deletion(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 65af164174e6f..d67bb6b48b8f4 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -350,7 +350,14 @@ func (s *testSuiteP1) TestShow(c *C) { "Trigger Tables To use triggers", "Create tablespace Server Admin To create/alter/drop tablespaces", "Update Tables To update existing rows", - "Usage Server Admin No privileges - allow connect only")) + "Usage Server Admin No privileges - allow connect only", + "BACKUP_ADMIN Server Admin ", + "SYSTEM_VARIABLES_ADMIN Server Admin ", + "ROLE_ADMIN Server Admin ", + "CONNECTION_ADMIN Server Admin ", + "RESTRICTED_TABLES_ADMIN Server Admin ", + "RESTRICTED_STATUS_ADMIN Server Admin ", + )) c.Assert(len(tk.MustQuery("show table status").Rows()), Equals, 1) } diff --git a/executor/show.go b/executor/show.go index 2bd9b786fcffb..ab08e5ba4cbf5 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1412,6 +1412,10 @@ func (e *ShowExec) fetchShowPrivileges() error { e.appendRow([]interface{}{"Create tablespace", "Server Admin", "To create/alter/drop tablespaces"}) e.appendRow([]interface{}{"Update", "Tables", "To update existing rows"}) e.appendRow([]interface{}{"Usage", "Server Admin", "No privileges - allow connect only"}) + + for _, priv := range privileges.GetDynamicPrivileges() { + e.appendRow([]interface{}{priv, "Server Admin", ""}) + } return nil } diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index c5ec2f8394385..6ac58e04e44e4 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -535,3 +535,14 @@ func RegisterDynamicPrivilege(privNameInUpper string) error { dynamicPrivs = append(dynamicPrivs, privNameInUpper) return nil } + +// GetDynamicPrivileges returns the list of registered DYNAMIC privileges +// for use in meta data commands (i.e. SHOW PRIVILEGES) +func GetDynamicPrivileges() []string { + dynamicPrivLock.Lock() + defer dynamicPrivLock.Unlock() + + privCopy := make([]string, len(dynamicPrivs)) + copy(privCopy, dynamicPrivs) + return privCopy +} diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 3038aad397076..c7a825a4d894d 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1427,3 +1427,15 @@ func (s *testPrivilegeSuite) TestViewDefiner(c *C) { tk.MustExec("select * from test_view") tk.MustExec("select * from test_view2") } + +func (s *testPrivilegeSuite) TestDynamicPrivsRegistration(c *C) { + se := newSession(c, s.store, s.dbName) + pm := privilege.GetPrivilegeManager(se) + + count := len(privileges.GetDynamicPrivileges()) + + c.Assert(pm.IsDynamicPrivilege("ACDC_ADMIN"), IsFalse) + privileges.RegisterDynamicPrivilege("ACDC_ADMIN") + c.Assert(pm.IsDynamicPrivilege("ACDC_ADMIN"), IsTrue) + c.Assert(len(privileges.GetDynamicPrivileges()), Equals, count+1) +} From 031a9fa18b909c1501c99d6a8a98448beb94a962 Mon Sep 17 00:00:00 2001 From: xhe Date: Fri, 14 May 2021 17:05:38 +0800 Subject: [PATCH 073/343] ddl: refactor rule [4/6] (#24007) --- ddl/ddl_api.go | 14 +-- ddl/placement/errors.go | 6 + ddl/placement/rule.go | 132 +++++++++++++++++++++ ddl/placement/rule_test.go | 206 +++++++++++++++++++++++++++++++++ ddl/placement/types.go | 36 ------ ddl/placement/types_test.go | 12 -- ddl/placement/utils.go | 4 +- ddl/placement/utils_test.go | 14 +-- ddl/placement_rule_test.go | 36 +++--- ddl/placement_sql_test.go | 6 +- executor/infoschema_reader.go | 2 +- expression/integration_test.go | 2 +- infoschema/tables_test.go | 2 +- session/session_test.go | 2 +- 14 files changed, 385 insertions(+), 89 deletions(-) create mode 100644 ddl/placement/rule.go create mode 100644 ddl/placement/rule_test.go diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 528a6087638fc..a3f8bb7f9c622 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5895,8 +5895,8 @@ func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]* } rules = append(rules, &placement.Rule{ - Count: int(replicas), - LabelConstraints: labelConstraints, + Count: int(replicas), + Constraints: labelConstraints, }) return rules, nil @@ -5925,8 +5925,8 @@ func buildPlacementSpecReplicasAndConstraint(replicas uint64, cnstr string) ([]* } rules = append(rules, &placement.Rule{ - Count: cnt, - LabelConstraints: labelConstraints, + Count: cnt, + Constraints: labelConstraints, }) } @@ -6051,14 +6051,14 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, newRules := bundle.Rules[:0] for i, rule := range bundle.Rules { // merge all empty constraints - if len(rule.LabelConstraints) == 0 { + if len(rule.Constraints) == 0 { extraCnt[rule.Role] += rule.Count continue } // refer to tidb#22065. // add -engine=tiflash to every rule to avoid schedules to tiflash instances. // placement rules in SQL is not compatible with `set tiflash replica` yet - if err := rule.LabelConstraints.Add(placement.Constraint{ + if err := rule.Constraints.Add(placement.Constraint{ Op: placement.NotIn, Key: placement.EngineLabelKey, Values: []string{placement.EngineLabelTiFlash}, @@ -6083,7 +6083,7 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, Count: cnt, StartKeyHex: startKey, EndKeyHex: endKey, - LabelConstraints: []placement.Constraint{{ + Constraints: []placement.Constraint{{ Op: placement.NotIn, Key: placement.EngineLabelKey, Values: []string{placement.EngineLabelTiFlash}, diff --git a/ddl/placement/errors.go b/ddl/placement/errors.go index 19797022a609c..95fce4591c961 100644 --- a/ddl/placement/errors.go +++ b/ddl/placement/errors.go @@ -24,4 +24,10 @@ var ( ErrUnsupportedConstraint = errors.New("unsupported label constraint") // ErrConflictingConstraints is from constraints.go. ErrConflictingConstraints = errors.New("conflicting label constraints") + // ErrInvalidConstraintsMapcnt is from rule.go. + ErrInvalidConstraintsMapcnt = errors.New("label constraints in map syntax have invalid replicas") + // ErrInvalidConstraintsFormat is from rule.go. + ErrInvalidConstraintsFormat = errors.New("invalid label constraints format") + // ErrInvalidConstraintsRelicas is from rule.go. + ErrInvalidConstraintsRelicas = errors.New("label constraints with invalid REPLICAS") ) diff --git a/ddl/placement/rule.go b/ddl/placement/rule.go new file mode 100644 index 0000000000000..134bdd5a610f9 --- /dev/null +++ b/ddl/placement/rule.go @@ -0,0 +1,132 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package placement + +import ( + "fmt" + "strings" + + "github.com/go-yaml/yaml" +) + +// PeerRoleType is the expected peer type of the placement rule. +type PeerRoleType string + +const ( + // Voter can either match a leader peer or follower peer. + Voter PeerRoleType = "voter" + // Leader matches a leader. + Leader PeerRoleType = "leader" + // Follower matches a follower. + Follower PeerRoleType = "follower" + // Learner matches a learner. + Learner PeerRoleType = "learner" +) + +// Rule is the core placement rule struct. Check https://github.com/tikv/pd/blob/master/server/schedule/placement/rule.go. +type Rule struct { + GroupID string `json:"group_id"` + ID string `json:"id"` + Index int `json:"index,omitempty"` + Override bool `json:"override,omitempty"` + StartKeyHex string `json:"start_key"` + EndKeyHex string `json:"end_key"` + Role PeerRoleType `json:"role"` + Count int `json:"count"` + Constraints Constraints `json:"label_constraints,omitempty"` + LocationLabels []string `json:"location_labels,omitempty"` + IsolationLevel string `json:"isolation_level,omitempty"` +} + +// NewRules constructs []*Rule from a yaml-compatible representation of +// array or map of constraints. It converts 'CONSTRAINTS' field in RFC +// https://github.com/pingcap/tidb/blob/master/docs/design/2020-06-24-placement-rules-in-sql.md to structs. +func NewRules(replicas uint64, cnstr string) ([]*Rule, error) { + rules := []*Rule{} + + cnstbytes := []byte(cnstr) + + constraints1 := []string{} + err1 := yaml.UnmarshalStrict(cnstbytes, &constraints1) + if err1 == nil { + // can not emit REPLICAS with an array or empty label + if replicas == 0 { + return rules, fmt.Errorf("%w: should be positive", ErrInvalidConstraintsRelicas) + } + + labelConstraints, err := NewConstraints(constraints1) + if err != nil { + return rules, err + } + + rules = append(rules, &Rule{ + Count: int(replicas), + Constraints: labelConstraints, + }) + + return rules, nil + } + + constraints2 := map[string]int{} + err2 := yaml.UnmarshalStrict(cnstbytes, &constraints2) + if err2 == nil { + ruleCnt := 0 + for labels, cnt := range constraints2 { + if cnt <= 0 { + return rules, fmt.Errorf("%w: count of labels '%s' should be positive, but got %d", ErrInvalidConstraintsMapcnt, labels, cnt) + } + ruleCnt += cnt + } + + if replicas == 0 { + replicas = uint64(ruleCnt) + } + + if int(replicas) < ruleCnt { + return rules, fmt.Errorf("%w: should be larger or equal to the number of total replicas, but REPLICAS=%d < total=%d", ErrInvalidConstraintsRelicas, replicas, ruleCnt) + } + + for labels, cnt := range constraints2 { + labelConstraints, err := NewConstraints(strings.Split(labels, ",")) + if err != nil { + return rules, err + } + + rules = append(rules, &Rule{ + Count: cnt, + Constraints: labelConstraints, + }) + } + + remain := int(replicas) - ruleCnt + if remain > 0 { + rules = append(rules, &Rule{ + Count: remain, + }) + } + + return rules, nil + } + + return nil, fmt.Errorf("%w: should be [constraint1, ...] (error %s), {constraint1: cnt1, ...} (error %s), or any yaml compatible representation", ErrInvalidConstraintsFormat, err1, err2) +} + +// Clone is used to duplicate a RuleOp for safe modification. +// Note that it is a shallow copy: LocationLabels and Constraints +// is not cloned. +func (r *Rule) Clone() *Rule { + n := &Rule{} + *n = *r + return n +} diff --git a/ddl/placement/rule_test.go b/ddl/placement/rule_test.go new file mode 100644 index 0000000000000..85dd492f348e7 --- /dev/null +++ b/ddl/placement/rule_test.go @@ -0,0 +1,206 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package placement + +import ( + "encoding/json" + "errors" + + . "github.com/pingcap/check" +) + +var _ = Suite(&testRuleSuite{}) + +type testRuleSuite struct{} + +func (t *testRuleSuite) TestClone(c *C) { + rule := &Rule{ID: "434"} + newRule := rule.Clone() + newRule.ID = "121" + + c.Assert(rule, DeepEquals, &Rule{ID: "434"}) + c.Assert(newRule, DeepEquals, &Rule{ID: "121"}) +} + +func matchRule(r1 *Rule, t2 []*Rule) bool { + for _, r2 := range t2 { + if ok, _ := DeepEquals.Check([]interface{}{r1, r2}, nil); ok { + return true + } + } + return false +} + +func matchRules(t1, t2 []*Rule, prefix string, c *C) { + expected, err := json.Marshal(t1) + c.Assert(err, IsNil) + got, err := json.Marshal(t2) + c.Assert(err, IsNil) + comment := Commentf("%s, expected %s\nbut got %s", prefix, expected, got) + c.Assert(len(t1), Equals, len(t2), comment) + for _, r1 := range t1 { + c.Assert(matchRule(r1, t2), IsTrue, comment) + } +} + +func (t *testRuleSuite) TestNewRules(c *C) { + type TestCase struct { + name string + input string + replicas uint64 + output []*Rule + err error + } + tests := []TestCase{} + + tests = append(tests, TestCase{ + name: "empty constraints", + input: "", + replicas: 3, + output: []*Rule{ + { + Count: 3, + Constraints: Constraints{}, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "zero replicas", + input: "", + replicas: 0, + err: ErrInvalidConstraintsRelicas, + }) + + labels, err := NewConstraints([]string{"+zone=sh", "+zone=sh"}) + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal array constraints", + input: `["+zone=sh", "+zone=sh"]`, + replicas: 3, + output: []*Rule{ + { + Count: 3, + Constraints: labels, + }, + }, + }) + + labels1, err := NewConstraints([]string{"+zone=sh", "-zone=bj"}) + c.Assert(err, IsNil) + labels2, err := NewConstraints([]string{"+zone=sh"}) + c.Assert(err, IsNil) + tests = append(tests, TestCase{ + name: "normal object constraints", + input: `{"+zone=sh,-zone=bj":2, "+zone=sh": 1}`, + replicas: 3, + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "normal object constraints, with extra count", + input: "{'+zone=sh,-zone=bj':2, '+zone=sh': 1}", + replicas: 4, + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + { + Count: 1, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "normal object constraints, without count", + input: "{'+zone=sh,-zone=bj':2, '+zone=sh': 1}", + output: []*Rule{ + { + Count: 2, + Constraints: labels1, + }, + { + Count: 1, + Constraints: labels2, + }, + }, + }) + + tests = append(tests, TestCase{ + name: "zero count in object constraints", + input: `{"+zone=sh,-zone=bj":0, "+zone=sh": 1}`, + replicas: 3, + err: ErrInvalidConstraintsMapcnt, + }) + + tests = append(tests, TestCase{ + name: "overlarge total count in object constraints", + input: `{"+ne=sh,-zone=bj":1, "+zone=sh": 4}`, + replicas: 3, + err: ErrInvalidConstraintsRelicas, + }) + + tests = append(tests, TestCase{ + name: "invalid array", + input: `["+ne=sh", "+zone=sh"`, + replicas: 3, + err: ErrInvalidConstraintsFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid array constraints", + input: `["ne=sh", "+zone=sh"]`, + replicas: 3, + err: ErrInvalidConstraintFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid map", + input: `{+ne=sh,-zone=bj:1, "+zone=sh": 4`, + replicas: 5, + err: ErrInvalidConstraintsFormat, + }) + + tests = append(tests, TestCase{ + name: "invalid map constraints", + input: `{"nesh,-zone=bj":1, "+zone=sh": 4}`, + replicas: 6, + err: ErrInvalidConstraintFormat, + }) + + for _, t := range tests { + comment := Commentf("%s", t.name) + output, err := NewRules(t.replicas, t.input) + if t.err == nil { + c.Assert(err, IsNil, comment) + matchRules(t.output, output, comment.CheckCommentString(), c) + } else { + c.Assert(errors.Is(err, t.err), IsTrue, comment) + } + } +} diff --git a/ddl/placement/types.go b/ddl/placement/types.go index 3bb9da96e3890..72093a2c19c78 100644 --- a/ddl/placement/types.go +++ b/ddl/placement/types.go @@ -22,42 +22,6 @@ import ( // After all, placement rules are communicated using an HTTP API. Loose // coupling is a good feature. -// PeerRoleType is the expected peer type of the placement rule. -type PeerRoleType string - -const ( - // Voter can either match a leader peer or follower peer. - Voter PeerRoleType = "voter" - // Leader matches a leader. - Leader PeerRoleType = "leader" - // Follower matches a follower. - Follower PeerRoleType = "follower" - // Learner matches a learner. - Learner PeerRoleType = "learner" -) - -// Rule is the placement rule. Check https://github.com/tikv/pd/blob/master/server/schedule/placement/rule.go. -type Rule struct { - GroupID string `json:"group_id"` - ID string `json:"id"` - Index int `json:"index,omitempty"` - Override bool `json:"override,omitempty"` - StartKeyHex string `json:"start_key"` - EndKeyHex string `json:"end_key"` - Role PeerRoleType `json:"role"` - Count int `json:"count"` - LabelConstraints Constraints `json:"label_constraints,omitempty"` - LocationLabels []string `json:"location_labels,omitempty"` - IsolationLevel string `json:"isolation_level,omitempty"` -} - -// Clone is used to duplicate a RuleOp for safe modification. -func (r *Rule) Clone() *Rule { - n := &Rule{} - *n = *r - return n -} - // Bundle is a group of all rules and configurations. It is used to support rule cache. type Bundle struct { ID string `json:"group_id"` diff --git a/ddl/placement/types_test.go b/ddl/placement/types_test.go index 77153cb29b692..93ed1a5a80f43 100644 --- a/ddl/placement/types_test.go +++ b/ddl/placement/types_test.go @@ -18,7 +18,6 @@ import ( ) var _ = Suite(&testBundleSuite{}) -var _ = Suite(&testRuleSuite{}) type testBundleSuite struct{} @@ -49,14 +48,3 @@ func (t *testBundleSuite) TestClone(c *C) { c.Assert(bundle, DeepEquals, &Bundle{ID: GroupID(1), Rules: []*Rule{{ID: "434"}}}) c.Assert(newBundle, DeepEquals, &Bundle{ID: GroupID(2), Rules: []*Rule{{ID: "121"}}}) } - -type testRuleSuite struct{} - -func (t *testRuleSuite) TestClone(c *C) { - rule := &Rule{ID: "434"} - newRule := rule.Clone() - newRule.ID = "121" - - c.Assert(rule, DeepEquals, &Rule{ID: "434"}) - c.Assert(newRule, DeepEquals, &Rule{ID: "121"}) -} diff --git a/ddl/placement/utils.go b/ddl/placement/utils.go index 16c0a424dde53..5b12f10e2d243 100644 --- a/ddl/placement/utils.go +++ b/ddl/placement/utils.go @@ -61,7 +61,7 @@ func BuildPlacementCopyBundle(oldBundle *Bundle, newID int64) *Bundle { func GetLeaderDCByBundle(bundle *Bundle, dcLabelKey string) (string, bool) { for _, rule := range bundle.Rules { if isValidLeaderRule(rule, dcLabelKey) { - return rule.LabelConstraints[0].Values[0], true + return rule.Constraints[0].Values[0], true } } return "", false @@ -69,7 +69,7 @@ func GetLeaderDCByBundle(bundle *Bundle, dcLabelKey string) (string, bool) { func isValidLeaderRule(rule *Rule, dcLabelKey string) bool { if rule.Role == Leader && rule.Count == 1 { - for _, con := range rule.LabelConstraints { + for _, con := range rule.Constraints { if con.Op == In && con.Key == dcLabelKey && len(con.Values) == 1 { return true } diff --git a/ddl/placement/utils_test.go b/ddl/placement/utils_test.go index 964382846485e..10941e0663455 100644 --- a/ddl/placement/utils_test.go +++ b/ddl/placement/utils_test.go @@ -58,7 +58,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -84,7 +84,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Voter, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -110,7 +110,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -127,7 +127,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "12", Role: Voter, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, @@ -153,7 +153,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "fake", Op: In, @@ -179,7 +179,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: NotIn, @@ -205,7 +205,7 @@ func (t *testUtilsSuite) TestGetLeaderDCByBundle(c *C) { { ID: "11", Role: Leader, - LabelConstraints: []Constraint{ + Constraints: []Constraint{ { Key: "zone", Op: In, diff --git a/ddl/placement_rule_test.go b/ddl/placement_rule_test.go index b051092a776e9..a9a916cb5a199 100644 --- a/ddl/placement_rule_test.go +++ b/ddl/placement_rule_test.go @@ -52,7 +52,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, @@ -67,9 +67,9 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { Constraints: "", }}, output: []*placement.Rule{{ - Role: placement.Voter, - Count: 3, - LabelConstraints: []placement.Constraint{}, + Role: placement.Voter, + Count: 3, + Constraints: []placement.Constraint{}, }}, }, @@ -83,14 +83,14 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, { Role: placement.Voter, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, }, }, @@ -108,7 +108,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -127,7 +127,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -154,7 +154,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, @@ -162,7 +162,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Follower, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "in", Values: []string{"bj"}}, }, @@ -189,7 +189,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 2, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "notIn", Values: []string{"sh"}}, {Key: "zone", Op: "in", Values: []string{"bj"}}, }, @@ -214,14 +214,14 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { }, output: []*placement.Rule{ { - Role: placement.Voter, - Count: 1, - LabelConstraints: []placement.Constraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, + Role: placement.Voter, + Count: 1, + Constraints: []placement.Constraint{{Key: "zone", Op: "notIn", Values: []string{"sh"}}}, }, { - Role: placement.Voter, - Count: 1, - LabelConstraints: []placement.Constraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, + Role: placement.Voter, + Count: 1, + Constraints: []placement.Constraint{{Key: "zone", Op: "in", Values: []string{"bj"}}}, }, { Role: placement.Voter, @@ -306,7 +306,7 @@ func (s *testPlacementSuite) TestPlacementBuild(c *C) { { Role: placement.Voter, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ {Key: "zone", Op: "in", Values: []string{"sh"}}, {Key: "zone", Op: "notIn", Values: []string{"bj"}}, }, diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index e77b0ba99d5cf..fb7158681714f 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -404,7 +404,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -423,7 +423,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Follower, Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, @@ -619,7 +619,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 4d56cc55accac..4f788a3d7bd1d 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1915,7 +1915,7 @@ func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) er continue } for _, rule := range bundle.Rules { - constraint, err := rule.LabelConstraints.Restore() + constraint, err := rule.Constraints.Restore() if err != nil { return errors.Wrapf(err, "Restore rule %s in bundle %s failed", rule.ID, bundle.ID) } diff --git a/expression/integration_test.go b/expression/integration_test.go index b1bded2a17719..80e39b76ce746 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -8755,7 +8755,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 6cc24300c1be4..ebe4a0620256f 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1449,7 +1449,7 @@ func (s *testTableSuite) TestPlacementPolicy(c *C) { ID: "0", Role: "voter", Count: 3, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: "zone", Op: "in", diff --git a/session/session_test.go b/session/session_test.go index 3baee4f0ef6f1..a8861dfddd79c 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -3319,7 +3319,7 @@ PARTITION BY RANGE (c) ( GroupID: groupID, Role: placement.Leader, Count: 1, - LabelConstraints: []placement.Constraint{ + Constraints: []placement.Constraint{ { Key: placement.DCLabelKey, Op: placement.In, From 392df99f04d5bc51560993408f39de81ffd3c672 Mon Sep 17 00:00:00 2001 From: HuaiyuXu <391585975@qq.com> Date: Fri, 14 May 2021 17:21:38 +0800 Subject: [PATCH 074/343] cmd: ddl_test modify retryCnt from 5 to 20 (#24662) --- cmd/ddltest/ddl_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/ddltest/ddl_test.go b/cmd/ddltest/ddl_test.go index 52009b10de142..36922638950a8 100644 --- a/cmd/ddltest/ddl_test.go +++ b/cmd/ddltest/ddl_test.go @@ -143,7 +143,7 @@ func (s *TestDDLSuite) SetUpSuite(c *C) { s.procs = make([]*server, *serverNum) // Set server restart retry count. - s.retryCount = 5 + s.retryCount = 20 createLogFiles(c, *serverNum) err = s.startServers() From 4d7fcbb3dffb7372150e47257842eb30efd33bf7 Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Fri, 14 May 2021 17:33:38 +0800 Subject: [PATCH 075/343] executor: add correctness tests about direct reading with ORDER BY and LIMIT (#24455) --- executor/partition_table_test.go | 79 ++++++++++++++++++++++++++++++++ 1 file changed, 79 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index ab5a19ca823e9..df0c52cfb55cc 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -227,6 +227,85 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { tk.MustQuery("select * from t_info_null where (date = '2020-10-02' or date = '2020-10-06') and app = 'xxx' and media = '19003006'").Check(testkit.Rows()) } +func (s *partitionTableSuite) TestOrderByandLimit(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_orderby_limit") + tk.MustExec("use test_orderby_limit") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // range partition table + tk.MustExec(`create table trange(a int, b int, index idx_a(a)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table + tk.MustExec("create table thash(a int, b int, index idx_a(a), index idx_b(b)) partition by hash(a) partitions 4;") + + // regular table + tk.MustExec("create table tregular(a int, b int, index idx_a(a))") + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000))) + } + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular values " + strings.Join(vals, ",")) + + // test indexLookUp + for i := 0; i < 100; i++ { + // explain select * from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select * from t where a > {y} use index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select * from trange use index(idx_a) where a > %v order by a, b limit %v;", x, y) + queryRegular := fmt.Sprintf("select * from tregular use index(idx_a) where a > %v order by a, b limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "IndexLookUp"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test tableReader + for i := 0; i < 100; i++ { + // explain select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select * from t where a > {y} ignore index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select * from trange ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + queryRegular := fmt.Sprintf("select * from tregular ignore index(idx_a) where a > %v order by a, b limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "TableReader"), IsTrue) // check if tableReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test indexReader + for i := 0; i < 100; i++ { + // explain select a from t where a > {y} use index(idx_a) order by a limit {x}; // check if IndexLookUp is used + // select a from t where a > {y} use index(idx_a) order by a limit {x}; // it can return the correct result + x := rand.Intn(1099) + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select a from trange use index(idx_a) where a > %v order by a limit %v;", x, y) + queryRegular := fmt.Sprintf("select a from tregular use index(idx_a) where a > %v order by a limit %v;", x, y) + c.Assert(tk.HasPlan(queryPartition, "IndexReader"), IsTrue) // check if indexReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } + + // test indexMerge + for i := 0; i < 100; i++ { + // explain select /*+ use_index_merge(t) */ * from t where a > 2 or b < 5 order by a limit {x}; // check if IndexMerge is used + // select /*+ use_index_merge(t) */ * from t where a > 2 or b < 5 order by a limit {x}; // can return the correct value + y := rand.Intn(2000) + 1 + queryPartition := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > 2 or b < 5 order by a, b limit %v;", y) + queryRegular := fmt.Sprintf("select * from tregular where a > 2 or b < 5 order by a, b limit %v;", y) + c.Assert(tk.HasPlan(queryPartition, "IndexMerge"), IsTrue) // check if indexMerge is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + } +} + func (s *partitionTableSuite) TestBatchGetandPointGetwithHashPartition(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 42b12f7cc324c18e31e64e7c3829588cb4c96237 Mon Sep 17 00:00:00 2001 From: disksing Date: Fri, 14 May 2021 18:01:38 +0800 Subject: [PATCH 076/343] store/tikv: remove options from unionstore (#24629) --- store/driver/txn/txn_driver.go | 6 +----- store/tikv/txn.go | 16 ---------------- store/tikv/unionstore/union_store.go | 24 ------------------------ 3 files changed, 1 insertion(+), 45 deletions(-) diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 72c1aac8c8e71..7c940561f82fe 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -168,8 +168,6 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.GetSnapshot().SetIsStatenessReadOnly(val.(bool)) case kv.MatchStoreLabels: txn.KVTxn.GetSnapshot().SetMatchStoreLabels(val.([]*metapb.StoreLabel)) - default: - txn.KVTxn.SetOption(opt, val) } } @@ -180,7 +178,7 @@ func (txn *tikvTxn) GetOption(opt int) interface{} { case kv.TxnScope: return txn.KVTxn.GetScope() default: - return txn.KVTxn.GetOption(opt) + return nil } } @@ -188,8 +186,6 @@ func (txn *tikvTxn) DelOption(opt int) { switch opt { case kv.CollectRuntimeStats: txn.KVTxn.GetSnapshot().SetRuntimeStats(nil) - default: - txn.KVTxn.DelOption(opt) } } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index e444f5adda7f6..aafaa2b323d24 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -221,22 +221,6 @@ func (txn *KVTxn) Delete(k []byte) error { return txn.us.GetMemBuffer().Delete(k) } -// SetOption sets an option with a value, when val is nil, uses the default -// value of this option. -func (txn *KVTxn) SetOption(opt int, val interface{}) { - txn.us.SetOption(opt, val) -} - -// GetOption returns the option -func (txn *KVTxn) GetOption(opt int) interface{} { - return txn.us.GetOption(opt) -} - -// DelOption deletes an option. -func (txn *KVTxn) DelOption(opt int) { - txn.us.DelOption(opt) -} - // SetSchemaLeaseChecker sets a hook to check schema version. func (txn *KVTxn) SetSchemaLeaseChecker(checker SchemaLeaseChecker) { txn.schemaLeaseChecker = checker diff --git a/store/tikv/unionstore/union_store.go b/store/tikv/unionstore/union_store.go index f9a077d1c1352..08354975e38c5 100644 --- a/store/tikv/unionstore/union_store.go +++ b/store/tikv/unionstore/union_store.go @@ -59,7 +59,6 @@ type uSnapshot interface { type KVUnionStore struct { memBuffer *MemDB snapshot uSnapshot - opts options } // NewUnionStore builds a new unionStore. @@ -67,7 +66,6 @@ func NewUnionStore(snapshot uSnapshot) *KVUnionStore { return &KVUnionStore{ snapshot: snapshot, memBuffer: newMemDB(), - opts: make(map[int]interface{}), } } @@ -131,30 +129,8 @@ func (us *KVUnionStore) UnmarkPresumeKeyNotExists(k []byte) { us.memBuffer.UpdateFlags(k, kv.DelPresumeKeyNotExists) } -// SetOption implements the unionStore SetOption interface. -func (us *KVUnionStore) SetOption(opt int, val interface{}) { - us.opts[opt] = val -} - -// DelOption implements the unionStore DelOption interface. -func (us *KVUnionStore) DelOption(opt int) { - delete(us.opts, opt) -} - -// GetOption implements the unionStore GetOption interface. -func (us *KVUnionStore) GetOption(opt int) interface{} { - return us.opts[opt] -} - // SetEntrySizeLimit sets the size limit for each entry and total buffer. func (us *KVUnionStore) SetEntrySizeLimit(entryLimit, bufferLimit uint64) { us.memBuffer.entrySizeLimit = entryLimit us.memBuffer.bufferSizeLimit = bufferLimit } - -type options map[int]interface{} - -func (opts options) Get(opt int) (interface{}, bool) { - v, ok := opts[opt] - return v, ok -} From e92df204b53310029b7c32c5143ab6cebbf7ff18 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Fri, 14 May 2021 19:09:38 +0800 Subject: [PATCH 077/343] planner: fix wrongly check for update statement (#24614) --- executor/write_test.go | 5 ++-- planner/core/integration_test.go | 14 ++++++++++ planner/core/logical_plan_builder.go | 39 +++++++++++++++------------- planner/core/logical_plan_test.go | 1 - 4 files changed, 37 insertions(+), 22 deletions(-) diff --git a/executor/write_test.go b/executor/write_test.go index 27ea70ae748a5..b832e52a9935c 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1554,7 +1554,7 @@ func (s *testSuite8) TestUpdate(c *C) { _, err = tk.Exec("UPDATE t SET c2=16777215 WHERE c1>= -8388608 AND c1 < -9 ORDER BY c1 LIMIT 2") c.Assert(err, IsNil) - tk.MustExec("update (select * from t) t set c1 = 1111111") + tk.MustGetErrCode("update (select * from t) t set c1 = 1111111", mysql.ErrNonUpdatableTable) // test update ignore for bad null error tk.MustExec("drop table if exists t;") @@ -1604,8 +1604,7 @@ func (s *testSuite8) TestUpdate(c *C) { tk.MustExec("drop view v") tk.MustExec("create sequence seq") - _, err = tk.Exec("update seq set minvalue=1") - c.Assert(err.Error(), Equals, "update sequence seq is not supported now.") + tk.MustGetErrCode("update seq set minvalue=1", mysql.ErrBadField) tk.MustExec("drop sequence seq") tk.MustExec("drop table if exists t1, t2") diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 9e8eaa9204af9..62e5b032c86df 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -196,6 +196,20 @@ func (s *testIntegrationSuite) TestIssue22298(c *C) { tk.MustGetErrMsg(`select * from t where 0 and c = 10;`, "[planner:1054]Unknown column 'c' in 'where clause'") } +func (s *testIntegrationSuite) TestIssue24571(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec(`create view v as select 1 as b;`) + tk.MustExec(`create table t (a int);`) + tk.MustExec(`update v, t set a=2;`) + tk.MustGetErrCode(`update v, t set b=2;`, mysql.ErrNonUpdatableTable) + tk.MustExec("create database db1") + tk.MustExec("use db1") + tk.MustExec("update test.t, (select 1 as a) as t set test.t.a=1;") + // bug in MySQL: ERROR 1288 (HY000): The target table t of the UPDATE is not updatable + tk.MustExec("update (select 1 as a) as t, test.t set test.t.a=1;") +} + func (s *testIntegrationSuite) TestIssue22828(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 7ff0e2ac6c6aa..64bc0c41407e1 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -4239,17 +4239,6 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( b.popTableHints() }() - // update subquery table should be forbidden - var notUpdatableTbl []string - notUpdatableTbl = extractTableSourceAsNames(update.TableRefs.TableRefs, notUpdatableTbl, true) - for _, asName := range notUpdatableTbl { - for _, assign := range update.List { - if assign.Column.Table.L == asName { - return nil, ErrNonUpdatableTable.GenWithStackByArgs(asName, "UPDATE") - } - } - } - b.inUpdateStmt = true b.isForUpdateRead = true @@ -4265,12 +4254,6 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( if dbName == "" { dbName = b.ctx.GetSessionVars().CurrentDB } - if t.TableInfo.IsView() { - return nil, errors.Errorf("update view %s is not supported now.", t.Name.O) - } - if t.TableInfo.IsSequence() { - return nil, errors.Errorf("update sequence %s is not supported now.", t.Name.O) - } b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, dbName, t.Name.L, "", nil) } @@ -4314,6 +4297,10 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( proj.SetChildren(p) p = proj + // update subquery table should be forbidden + var notUpdatableTbl []string + notUpdatableTbl = extractTableSourceAsNames(update.TableRefs.TableRefs, notUpdatableTbl, true) + var updateTableList []*ast.TableName updateTableList = extractTableList(update.TableRefs.TableRefs, updateTableList, true) orderedList, np, allAssignmentsAreConstant, err := b.buildUpdateLists(ctx, updateTableList, update.List, p, notUpdatableTbl) @@ -4417,6 +4404,21 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab columnsIdx[assign.Column] = idx } name := p.OutputNames()[idx] + for _, tl := range tableList { + if (tl.Schema.L == "" || tl.Schema.L == name.DBName.L) && (tl.Name.L == name.TblName.L) { + if tl.TableInfo.IsView() || tl.TableInfo.IsSequence() { + return nil, nil, false, ErrNonUpdatableTable.GenWithStackByArgs(name.TblName.O, "UPDATE") + } + // may be a subquery + if tl.Schema.L == "" { + for _, nTbl := range notUpdatableTbl { + if nTbl == name.TblName.L { + return nil, nil, false, ErrNonUpdatableTable.GenWithStackByArgs(name.TblName.O, "UPDATE") + } + } + } + } + } columnFullName := fmt.Sprintf("%s.%s.%s", name.DBName.L, name.TblName.L, name.ColName.L) // We save a flag for the column in map `modifyColumns` // This flag indicated if assign keyword `DEFAULT` to the column @@ -4439,9 +4441,10 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab break } } - if !updatable { + if !updatable || tn.TableInfo.IsView() || tn.TableInfo.IsSequence() { continue } + tableInfo := tn.TableInfo tableVal, found := b.is.TableByID(tableInfo.ID) if !found { diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 11a116bb4fac8..921f1c99b34ec 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1459,7 +1459,6 @@ func (s *testPlanSuite) TestNameResolver(c *C) { {"delete a from (select * from t ) as a, t", "[planner:1288]The target table a of the DELETE is not updatable"}, {"delete b from (select * from t ) as a, t", "[planner:1109]Unknown table 'b' in MULTI DELETE"}, {"select '' as fakeCol from t group by values(fakeCol)", "[planner:1054]Unknown column '' in 'VALUES() function'"}, - {"update t, (select * from t) as b set b.a = t.a", "[planner:1288]The target table b of the UPDATE is not updatable"}, {"select row_number() over () from t group by 1", "[planner:1056]Can't group on 'row_number() over ()'"}, {"select row_number() over () as x from t group by 1", "[planner:1056]Can't group on 'x'"}, {"select sum(a) as x from t group by 1", "[planner:1056]Can't group on 'x'"}, From 557c94b9ba6e7e4d1094be7e5e2a72bf1535e0f5 Mon Sep 17 00:00:00 2001 From: disksing Date: Mon, 17 May 2021 11:13:39 +0800 Subject: [PATCH 078/343] store/tikv: remove CompareTS (#24657) --- executor/stale_txn_test.go | 2 +- store/tikv/oracle/oracle.go | 19 ------------------- store/tikv/txn.go | 2 +- 3 files changed, 2 insertions(+), 21 deletions(-) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index ce5202ae58a75..493bda06c5de2 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -312,7 +312,7 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { if testcase.useSafeTS { c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.injectSafeTS) } else { - c.Assert(oracle.CompareTS(tk.Se.GetSessionVars().TxnCtx.StartTS, testcase.injectSafeTS), Equals, 1) + c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Greater, testcase.injectSafeTS) } tk.MustExec("commit") failpoint.Disable("github.com/pingcap/tidb/store/tikv/injectSafeTS") diff --git a/store/tikv/oracle/oracle.go b/store/tikv/oracle/oracle.go index 0a6865cf59039..daf00c66814ca 100644 --- a/store/tikv/oracle/oracle.go +++ b/store/tikv/oracle/oracle.go @@ -148,25 +148,6 @@ func GoTimeToTS(t time.Time) uint64 { return uint64(ts) } -// CompareTS is used to compare two timestamps. -// If tsoOne > tsoTwo, returns 1. -// If tsoOne = tsoTwo, returns 0. -// If tsoOne < tsoTwo, returns -1. -func CompareTS(tsoOne, tsoTwo uint64) int { - tsOnePhy := ExtractPhysical(tsoOne) - tsOneLog := ExtractLogical(tsoOne) - tsTwoPhy := ExtractPhysical(tsoTwo) - tsTwoLog := ExtractLogical(tsoTwo) - - if tsOnePhy > tsTwoPhy || (tsOnePhy == tsTwoPhy && tsOneLog > tsTwoLog) { - return 1 - } - if tsOnePhy == tsTwoPhy && tsOneLog == tsTwoLog { - return 0 - } - return -1 -} - // GoTimeToLowerLimitStartTS returns the min start_ts of the uncommitted transaction. // maxTxnTimeUse means the max time a Txn May use (in ms) from its begin to commit. func GoTimeToLowerLimitStartTS(now time.Time, maxTxnTimeUse int64) uint64 { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index aafaa2b323d24..beeeafe66a063 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -119,7 +119,7 @@ func extractStartTs(store *KVStore, options kv.TransactionOption) (uint64, error startTs = *options.MinStartTS // If the safeTS is larger than the minStartTS, we will use safeTS as StartTS, otherwise we will use // minStartTS directly. - if oracle.CompareTS(startTs, safeTS) < 0 { + if startTs < safeTS { startTs = safeTS } } else if options.MaxPrevSec != nil { From f2cb145af2b893295797b36ec8e8603f784e69e6 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Sun, 16 May 2021 21:25:39 -0600 Subject: [PATCH 079/343] planner, privilege: Add security enhanced mode part 4 (#24416) --- executor/executor_test.go | 1 + planner/core/planbuilder.go | 40 ++++++++++++++++++++- privilege/privilege.go | 3 ++ privilege/privileges/privileges.go | 16 +++++++++ privilege/privileges/privileges_test.go | 47 +++++++++++++++++++++++++ 5 files changed, 106 insertions(+), 1 deletion(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index d67bb6b48b8f4..e359837d8fb92 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -357,6 +357,7 @@ func (s *testSuiteP1) TestShow(c *C) { "CONNECTION_ADMIN Server Admin ", "RESTRICTED_TABLES_ADMIN Server Admin ", "RESTRICTED_STATUS_ADMIN Server Admin ", + "RESTRICTED_USER_ADMIN Server Admin ", )) c.Assert(len(tk.MustQuery("show table status").Rows()), Equals, 1) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 6fc98bc522508..43997217da54b 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -36,6 +37,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" + "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -2276,9 +2278,16 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { p.setSchemaAndNames(buildBRIESchema()) err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or BACKUP_ADMIN") b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "BACKUP_ADMIN", false, err) - case *ast.GrantRoleStmt, *ast.RevokeRoleStmt: + case *ast.GrantRoleStmt: err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or ROLE_ADMIN") b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "ROLE_ADMIN", false, err) + case *ast.RevokeRoleStmt: + err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or ROLE_ADMIN") + b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "ROLE_ADMIN", false, err) + // Check if any of the users are RESTRICTED + for _, user := range raw.Users { + b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, user, "RESTRICTED_USER_ADMIN") + } case *ast.RevokeStmt: b.visitInfo = collectVisitInfoFromRevokeStmt(b.ctx, b.visitInfo, raw) case *ast.KillStmt: @@ -2292,12 +2301,23 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or CONNECTION_ADMIN") b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "CONNECTION_ADMIN", false, err) } + b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, &auth.UserIdentity{Username: pi.User, Hostname: pi.Host}, "RESTRICTED_CONNECTION_ADMIN") } } case *ast.UseStmt: if raw.DBName == "" { return nil, ErrNoDB } + case *ast.DropUserStmt: + // The main privilege checks for DROP USER are currently performed in executor/simple.go + // because they use complex OR conditions (not supported by visitInfo). + for _, user := range raw.UserList { + b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, user, "RESTRICTED_USER_ADMIN") + } + case *ast.SetPwdStmt: + if raw.User != nil { + b.visitInfo = appendVisitInfoIsRestrictedUser(b.visitInfo, b.ctx, raw.User, "RESTRICTED_USER_ADMIN") + } case *ast.ShutdownStmt: b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShutdownPriv, "", "", "", nil) } @@ -2339,6 +2359,10 @@ func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stm for _, priv := range allPrivs { vi = appendVisitInfo(vi, priv, dbName, tableName, "", nil) } + for _, u := range stmt.Users { + // For SEM, make sure the users are not restricted + vi = appendVisitInfoIsRestrictedUser(vi, sctx, u.User, "RESTRICTED_USER_ADMIN") + } if nonDynamicPrivilege { // Dynamic privileges use their own GRANT OPTION. If there were any non-dynamic privilege requests, // we need to attach the "GLOBAL" version of the GRANT OPTION. @@ -2347,6 +2371,20 @@ func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stm return vi } +// appendVisitInfoIsRestrictedUser appends additional visitInfo if the user has a +// special privilege called "RESTRICTED_USER_ADMIN". It only applies when SEM is enabled. +func appendVisitInfoIsRestrictedUser(visitInfo []visitInfo, sctx sessionctx.Context, user *auth.UserIdentity, priv string) []visitInfo { + if !sem.IsEnabled() { + return visitInfo + } + checker := privilege.GetPrivilegeManager(sctx) + if checker != nil && checker.RequestDynamicVerificationWithUser("RESTRICTED_USER_ADMIN", false, user) { + err := ErrSpecificAccessDenied.GenWithStackByArgs(priv) + visitInfo = appendDynamicVisitInfo(visitInfo, priv, false, err) + } + return visitInfo +} + func collectVisitInfoFromGrantStmt(sctx sessionctx.Context, vi []visitInfo, stmt *ast.GrantStmt) []visitInfo { // To use GRANT, you must have the GRANT OPTION privilege, // and you must have the privileges that you are granting. diff --git a/privilege/privilege.go b/privilege/privilege.go index cf59ce9b0314e..f732d9da1199b 100644 --- a/privilege/privilege.go +++ b/privilege/privilege.go @@ -50,6 +50,9 @@ type Manager interface { // Dynamic privileges are only assignable globally, and have their own grantable attribute. RequestDynamicVerification(activeRoles []*auth.RoleIdentity, privName string, grantable bool) bool + // RequestDynamicVerification verifies a DYNAMIC privilege for a specific user. + RequestDynamicVerificationWithUser(privName string, grantable bool, user *auth.UserIdentity) bool + // ConnectionVerification verifies user privilege for connection. ConnectionVerification(user, host string, auth, salt []byte, tlsState *tls.ConnectionState) (string, string, bool) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 6ac58e04e44e4..e0c63cfb14bfc 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -45,6 +45,7 @@ var dynamicPrivs = []string{ "CONNECTION_ADMIN", "RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled "RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled. + "RESTRICTED_USER_ADMIN", // User can not have their access revoked by SUPER users. } var dynamicPrivLock sync.Mutex @@ -56,6 +57,21 @@ type UserPrivileges struct { *Handle } +// RequestDynamicVerificationWithUser implements the Manager interface. +func (p *UserPrivileges) RequestDynamicVerificationWithUser(privName string, grantable bool, user *auth.UserIdentity) bool { + if SkipWithGrant { + return true + } + + if user == nil { + return false + } + + mysqlPriv := p.Handle.Get() + roles := mysqlPriv.getDefaultRoles(user.Username, user.Hostname) + return mysqlPriv.RequestDynamicVerification(roles, user.Username, user.Hostname, privName, grantable) +} + // RequestDynamicVerification implements the Manager interface. func (p *UserPrivileges) RequestDynamicVerification(activeRoles []*auth.RoleIdentity, privName string, grantable bool) bool { if SkipWithGrant { diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index c7a825a4d894d..2af31f3699f7d 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1428,6 +1428,53 @@ func (s *testPrivilegeSuite) TestViewDefiner(c *C) { tk.MustExec("select * from test_view2") } +func (s *testPrivilegeSuite) TestSecurityEnhancedModeRestrictedUsers(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE USER ruroot1, ruroot2, ruroot3") + tk.MustExec("CREATE ROLE notimportant") + tk.MustExec("GRANT SUPER, CREATE USER ON *.* to ruroot1 WITH GRANT OPTION") + tk.MustExec("SET tidb_enable_dynamic_privileges=1") + tk.MustExec("GRANT SUPER, RESTRICTED_USER_ADMIN, CREATE USER ON *.* to ruroot2 WITH GRANT OPTION") + tk.MustExec("GRANT RESTRICTED_USER_ADMIN ON *.* to ruroot3") + tk.MustExec("GRANT notimportant TO ruroot2, ruroot3") + + sem.Enable() + defer sem.Disable() + + stmts := []string{ + "SET PASSWORD for ruroot3 = 'newpassword'", + "REVOKE notimportant FROM ruroot3", + "REVOKE SUPER ON *.* FROM ruroot3", + "DROP USER ruroot3", + } + + // ruroot1 has SUPER but in SEM will be restricted + tk.Se.Auth(&auth.UserIdentity{ + Username: "ruroot1", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + + for _, stmt := range stmts { + err := tk.ExecToErr(stmt) + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_USER_ADMIN privilege(s) for this operation") + } + + // Switch to ruroot2, it should be permitted + tk.Se.Auth(&auth.UserIdentity{ + Username: "ruroot2", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + + for _, stmt := range stmts { + err := tk.ExecToErr(stmt) + c.Assert(err, IsNil) + } +} + func (s *testPrivilegeSuite) TestDynamicPrivsRegistration(c *C) { se := newSession(c, s.store, s.dbName) pm := privilege.GetPrivilegeManager(se) From 0eefeecd1bce8aad16af6f9831dad5d85b982243 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 May 2021 11:37:39 +0800 Subject: [PATCH 080/343] executor: add some test cases about partition table dynamic-mode with split-region (#24665) --- executor/partition_table_test.go | 37 ++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index df0c52cfb55cc..5dc23e4b8e624 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -612,6 +612,43 @@ func (s *partitionTableSuite) TestAddDropPartitions(c *C) { tk.MustPartition(`select * from t where a < 20`, "p1,p2,p3").Sort().Check(testkit.Rows("12", "15", "7")) } +func (s *partitionTableSuite) TestSplitRegion(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_split_region") + tk.MustExec("use test_split_region") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table tnormal (a int, b int)`) + tk.MustExec(`create table thash (a int, b int, index(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, index(a)) partition by range(a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + vals := make([]string, 0, 1000) + for i := 0; i < 1000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into tnormal values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into trange values ` + strings.Join(vals, ", ")) + + tk.MustExec(`SPLIT TABLE thash INDEX a BETWEEN (1) AND (25000) REGIONS 10`) + tk.MustExec(`SPLIT TABLE trange INDEX a BETWEEN (1) AND (25000) REGIONS 10`) + + result := tk.MustQuery(`select * from tnormal where a>=1 and a<=15000`).Sort().Rows() + tk.MustPartition(`select * from trange where a>=1 and a<=15000`, "p0,p1").Sort().Check(result) + tk.MustPartition(`select * from thash where a>=1 and a<=15000`, "all").Sort().Check(result) + + result = tk.MustQuery(`select * from tnormal where a in (1, 10001, 20001)`).Sort().Rows() + tk.MustPartition(`select * from trange where a in (1, 10001, 20001)`, "p0,p1,p2").Sort().Check(result) + tk.MustPartition(`select * from thash where a in (1, 10001, 20001)`, "p1").Sort().Check(result) +} + func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 09e95b90284edcfffe34382d854c2b9f0c587025 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 17 May 2021 11:51:39 +0800 Subject: [PATCH 081/343] planner: fix wrong column offsets when processing dynamic pruning for IndexJoin (#24659) --- executor/builder.go | 33 +++++++++++++++--- executor/index_lookup_join.go | 10 +++--- executor/partition_table_test.go | 59 ++++++++++++++++++++++++++++++++ 3 files changed, 94 insertions(+), 8 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 3324e52f894ff..54e2dfb93012a 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -49,6 +49,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" @@ -2476,11 +2477,14 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) outerKeyCols[i] = v.OuterJoinKeys[i].Index } innerKeyCols := make([]int, len(v.InnerJoinKeys)) + innerKeyColIDs := make([]int64, len(v.InnerJoinKeys)) for i := 0; i < len(v.InnerJoinKeys); i++ { innerKeyCols[i] = v.InnerJoinKeys[i].Index + innerKeyColIDs[i] = v.InnerJoinKeys[i].ID } e.outerCtx.keyCols = outerKeyCols e.innerCtx.keyCols = innerKeyCols + e.innerCtx.keyColIDs = innerKeyColIDs outerHashCols, innerHashCols := make([]int, len(v.OuterHashKeys)), make([]int, len(v.InnerHashKeys)) for i := 0; i < len(v.OuterHashKeys); i++ { @@ -2785,7 +2789,6 @@ func keyColumnsIncludeAllPartitionColumns(keyColumns []int, pe *tables.Partition func prunePartitionForInnerExecutor(ctx sessionctx.Context, tbl table.Table, schema *expression.Schema, partitionInfo *plannercore.PartitionInfo, lookUpContent []*indexJoinLookUpContent) (usedPartition []table.PhysicalTable, canPrune bool, contentPos []int64, err error) { partitionTbl := tbl.(table.PartitionedTable) - locateKey := make([]types.Datum, schema.Len()) // TODO: condition based pruning can be do in advance. condPruneResult, err := partitionPruning(ctx, partitionTbl, partitionInfo.PruningConds, partitionInfo.PartitionNames, partitionInfo.Columns, partitionInfo.ColumnNames) if err != nil { @@ -2800,22 +2803,44 @@ func prunePartitionForInnerExecutor(ctx sessionctx.Context, tbl table.Table, sch if err != nil { return nil, false, nil, err } + + // recalculate key column offsets + if lookUpContent[0].keyColIDs == nil { + return nil, false, nil, + dbterror.ClassOptimizer.NewStd(mysql.ErrInternal).GenWithStack("cannot get column IDs when dynamic pruning") + } + keyColOffsets := make([]int, len(lookUpContent[0].keyColIDs)) + for i, colID := range lookUpContent[0].keyColIDs { + offset := -1 + for j, col := range partitionTbl.Cols() { + if colID == col.ID { + offset = j + break + } + } + if offset == -1 { + return nil, false, nil, + dbterror.ClassOptimizer.NewStd(mysql.ErrInternal).GenWithStack("invalid column offset when dynamic pruning") + } + keyColOffsets[i] = offset + } + offsetMap := make(map[int]bool) - for _, offset := range lookUpContent[0].keyCols { + for _, offset := range keyColOffsets { offsetMap[offset] = true } for _, offset := range pe.ColumnOffset { if _, ok := offsetMap[offset]; !ok { - logutil.BgLogger().Warn("can not runtime prune in index join") return condPruneResult, false, nil, nil } } + locateKey := make([]types.Datum, len(partitionTbl.Cols())) partitions := make(map[int64]table.PhysicalTable) contentPos = make([]int64, len(lookUpContent)) for idx, content := range lookUpContent { for i, date := range content.keys { - locateKey[content.keyCols[i]] = date + locateKey[keyColOffsets[i]] = date } p, err := partitionTbl.GetPartitionByRow(ctx, locateKey) if err != nil { diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 9bec8e118515a..0e31280b6632a 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -95,6 +95,7 @@ type innerCtx struct { readerBuilder *dataReaderBuilder rowTypes []*types.FieldType keyCols []int + keyColIDs []int64 // the original ID in its table, used by dynamic partition pruning hashCols []int colLens []int hasPrefixCol bool @@ -472,9 +473,10 @@ func (iw *innerWorker) run(ctx context.Context, wg *sync.WaitGroup) { } type indexJoinLookUpContent struct { - keys []types.Datum - row chunk.Row - keyCols []int + keys []types.Datum + row chunk.Row + keyCols []int + keyColIDs []int64 // the original ID in its table, used by dynamic partition pruning } func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) error { @@ -545,7 +547,7 @@ func (iw *innerWorker) constructLookupContent(task *lookUpJoinTask) ([]*indexJoi // dLookUpKey is sorted and deduplicated at sortAndDedupLookUpContents. // So we don't need to do it here. } - lookUpContents = append(lookUpContents, &indexJoinLookUpContent{keys: dLookUpKey, row: chk.GetRow(rowIdx), keyCols: iw.keyCols}) + lookUpContents = append(lookUpContents, &indexJoinLookUpContent{keys: dLookUpKey, row: chk.GetRow(rowIdx), keyCols: iw.keyCols, keyColIDs: iw.keyColIDs}) } } diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 5dc23e4b8e624..cf53b7cd82688 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -435,6 +435,65 @@ func (s *partitionTableSuite) TestView(c *C) { } } +func (s *partitionTableSuite) TestDynamicPruningUnderIndexJoin(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database pruing_under_index_join") + tk.MustExec("use pruing_under_index_join") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table tnormal (a int, b int, c int, primary key(a), index idx_b(b))`) + tk.MustExec(`create table thash (a int, b int, c int, primary key(a), index idx_b(b)) partition by hash(a) partitions 4`) + tk.MustExec(`create table touter (a int, b int, c int)`) + + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v, %v)", i, rand.Intn(10000), rand.Intn(10000))) + } + tk.MustExec(`insert into tnormal values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into touter values ` + strings.Join(vals, ", ")) + + // case 1: IndexReader in the inner side + tk.MustQuery(`explain format='brief' select /*+ INL_JOIN(touter, thash) */ thash.b from touter join thash use index(idx_b) on touter.b = thash.b`).Check(testkit.Rows( + `IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:pruing_under_index_join.touter.b, inner key:pruing_under_index_join.thash.b, equal cond:eq(pruing_under_index_join.touter.b, pruing_under_index_join.thash.b)`, + `├─TableReader(Build) 9990.00 root data:Selection`, + `│ └─Selection 9990.00 cop[tikv] not(isnull(pruing_under_index_join.touter.b))`, + `│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + `└─IndexReader(Probe) 1.25 root partition:all index:Selection`, + ` └─Selection 1.25 cop[tikv] not(isnull(pruing_under_index_join.thash.b))`, + ` └─IndexRangeScan 1.25 cop[tikv] table:thash, index:idx_b(b) range: decided by [eq(pruing_under_index_join.thash.b, pruing_under_index_join.touter.b)], keep order:false, stats:pseudo`)) + tk.MustQuery(`select /*+ INL_JOIN(touter, thash) */ thash.b from touter join thash use index(idx_b) on touter.b = thash.b`).Sort().Check( + tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b`).Sort().Rows()) + + // case 2: TableReader in the inner side + tk.MustQuery(`explain format='brief' select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(primary) on touter.b = thash.a`).Check(testkit.Rows( + `IndexJoin 12487.50 root inner join, inner:TableReader, outer key:pruing_under_index_join.touter.b, inner key:pruing_under_index_join.thash.a, equal cond:eq(pruing_under_index_join.touter.b, pruing_under_index_join.thash.a)`, + `├─TableReader(Build) 9990.00 root data:Selection`, + `│ └─Selection 9990.00 cop[tikv] not(isnull(pruing_under_index_join.touter.b))`, + `│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + `└─TableReader(Probe) 1.00 root partition:all data:TableRangeScan`, + ` └─TableRangeScan 1.00 cop[tikv] table:thash range: decided by [pruing_under_index_join.touter.b], keep order:false, stats:pseudo`)) + tk.MustQuery(`select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(primary) on touter.b = thash.a`).Sort().Check( + tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.* from touter join tnormal use index(primary) on touter.b = tnormal.a`).Sort().Rows()) + + // case 3: IndexLookUp in the inner side + read all inner columns + tk.MustQuery(`explain format='brief' select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(idx_b) on touter.b = thash.b`).Check(testkit.Rows( + `IndexJoin 12487.50 root inner join, inner:IndexLookUp, outer key:pruing_under_index_join.touter.b, inner key:pruing_under_index_join.thash.b, equal cond:eq(pruing_under_index_join.touter.b, pruing_under_index_join.thash.b)`, + `├─TableReader(Build) 9990.00 root data:Selection`, + `│ └─Selection 9990.00 cop[tikv] not(isnull(pruing_under_index_join.touter.b))`, + `│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + `└─IndexLookUp(Probe) 1.25 root partition:all `, + ` ├─Selection(Build) 1.25 cop[tikv] not(isnull(pruing_under_index_join.thash.b))`, + ` │ └─IndexRangeScan 1.25 cop[tikv] table:thash, index:idx_b(b) range: decided by [eq(pruing_under_index_join.thash.b, pruing_under_index_join.touter.b)], keep order:false, stats:pseudo`, + ` └─TableRowIDScan(Probe) 1.25 cop[tikv] table:thash keep order:false, stats:pseudo`)) + tk.MustQuery(`select /*+ INL_JOIN(touter, thash) */ thash.* from touter join thash use index(idx_b) on touter.b = thash.b`).Sort().Check( + tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.* from touter join tnormal use index(idx_b) on touter.b = tnormal.b`).Sort().Rows()) +} + func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 8ad868f801fce6252e41bb2662087318980af448 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Sun, 16 May 2021 22:45:38 -0600 Subject: [PATCH 082/343] *: Add security enhanced mode part 3 (#24412) --- docs/design/2021-03-09-dynamic-privileges.md | 4 +- .../2021-03-09-security-enhanced-mode.md | 4 +- executor/executor_test.go | 1 + executor/show.go | 15 +++++- planner/core/expression_rewriter.go | 5 ++ planner/core/planbuilder.go | 4 ++ privilege/privileges/privileges.go | 7 +-- privilege/privileges/privileges_test.go | 49 +++++++++++++++++++ sessionctx/variable/sysvar.go | 2 +- sessionctx/variable/tidb_vars.go | 2 - tidb-server/main.go | 3 ++ util/sem/sem.go | 32 ++++++++++++ util/sem/sem_test.go | 25 ++++++++++ 13 files changed, 141 insertions(+), 12 deletions(-) diff --git a/docs/design/2021-03-09-dynamic-privileges.md b/docs/design/2021-03-09-dynamic-privileges.md index 7ad0d59d2c54e..c85c0dc0c5305 100644 --- a/docs/design/2021-03-09-dynamic-privileges.md +++ b/docs/design/2021-03-09-dynamic-privileges.md @@ -1,7 +1,7 @@ # Proposal: - Author(s): [morgo](https://github.com/morgo) -- Last updated: April 25, 2021 +- Last updated: May 04, 2021 - Discussion at: N/A ## Table of Contents @@ -238,7 +238,7 @@ No change | Privilege Name | Description | Notes | | --------------- | --------------- | --------------- | -| `RESTRICTED_SYSTEM_VARIABLES_ADMIN` | Allows changing a restricted `GLOBAL` system variable. | Currently in SEM all high risk variables are unloaded. TBD, it might be required in future that they are only visible/settable to those with this privilege and not SUPER. | +| `RESTRICTED_VARIABLES_ADMIN` | Allows changing a restricted `GLOBAL` system variable. | Currently in SEM all high risk variables are unloaded. TBD, it might be required in future that they are only visible/settable to those with this privilege and not SUPER. | | `RESTRICTED_STATUS_ADMIN` | Allows observing restricted status variables. | i.e. `SHOW GLOBAL STATUS` by default hides some statistics when `SEM` is enabled. | | `RESTRICTED_CONNECTION_ADMIN` | A special privilege to say that their connections, etc. can’t be killed by SUPER users AND they can kill connections by all other users. Affects `KILL`, `KILL TIDB` commands. | It is intended for the CloudAdmin user in DBaaS. | | `RESTRICTED_USER_ADMIN` | A special privilege to say that their access can’t be changed by `SUPER` users. Statements `DROP USER`, `SET PASSWORD`, `ALTER USER`, `REVOKE` are all limited. | It is intended for the CloudAdmin user in DbaaS. | diff --git a/docs/design/2021-03-09-security-enhanced-mode.md b/docs/design/2021-03-09-security-enhanced-mode.md index e939fec67c154..efc5b79f499e4 100644 --- a/docs/design/2021-03-09-security-enhanced-mode.md +++ b/docs/design/2021-03-09-security-enhanced-mode.md @@ -1,7 +1,7 @@ # Proposal: - Author(s): [morgo](https://github.com/morgo) -- Last updated: April 25, 2021 +- Last updated: May 04, 2021 - Discussion at: N/A ## Table of Contents @@ -49,7 +49,7 @@ A boolean option called `EnableEnhancedSecurity` (default `FALSE`) will be added ### System Variables -The following system variables will be hidden unless the user has the `RESTRICTED_SYSTEM_VARIABLES_ADMIN` privilege: +The following system variables will be hidden unless the user has the `RESTRICTED_VARIABLES_ADMIN` privilege: * variable.TiDBDDLSlowOprThreshold, * variable.TiDBAllowRemoveAutoInc, diff --git a/executor/executor_test.go b/executor/executor_test.go index e359837d8fb92..5e6b4490f5eb6 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -357,6 +357,7 @@ func (s *testSuiteP1) TestShow(c *C) { "CONNECTION_ADMIN Server Admin ", "RESTRICTED_TABLES_ADMIN Server Admin ", "RESTRICTED_STATUS_ADMIN Server Admin ", + "RESTRICTED_VARIABLES_ADMIN Server Admin ", "RESTRICTED_USER_ADMIN Server Admin ", )) c.Assert(len(tk.MustQuery("show table status").Rows()), Equals, 1) diff --git a/executor/show.go b/executor/show.go index ab08e5ba4cbf5..09e3d0c71e7b4 100644 --- a/executor/show.go +++ b/executor/show.go @@ -661,6 +661,17 @@ func (e *ShowExec) fetchShowMasterStatus() error { return nil } +func (e *ShowExec) sysVarHiddenForSem(sysVarNameInLower string) bool { + if !sem.IsEnabled() || !sem.IsInvisibleSysVar(sysVarNameInLower) { + return false + } + checker := privilege.GetPrivilegeManager(e.ctx) + if checker == nil || checker.RequestDynamicVerification(e.ctx.GetSessionVars().ActiveRoles, "RESTRICTED_VARIABLES_ADMIN", false) { + return false + } + return true +} + func (e *ShowExec) fetchShowVariables() (err error) { var ( value string @@ -673,7 +684,7 @@ func (e *ShowExec) fetchShowVariables() (err error) { // otherwise, fetch the value from table `mysql.Global_Variables`. for _, v := range variable.GetSysVars() { if v.Scope != variable.ScopeSession { - if v.Hidden { + if v.Hidden || e.sysVarHiddenForSem(v.Name) { continue } value, err = variable.GetGlobalSystemVar(sessionVars, v.Name) @@ -690,7 +701,7 @@ func (e *ShowExec) fetchShowVariables() (err error) { // If it is a session only variable, use the default value defined in code, // otherwise, fetch the value from table `mysql.Global_Variables`. for _, v := range variable.GetSysVars() { - if v.Hidden { + if v.Hidden || e.sysVarHiddenForSem(v.Name) { continue } value, err = variable.GetSessionOrGlobalSystemVar(sessionVars, v.Name) diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 8b52318a260a1..eb154d0201ecb 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -42,6 +42,7 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/hint" + "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/stringutil" ) @@ -1220,6 +1221,10 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { er.err = variable.ErrUnknownSystemVar.GenWithStackByArgs(name) return } + if sem.IsEnabled() && sem.IsInvisibleSysVar(sysVar.Name) { + err := ErrSpecificAccessDenied.GenWithStackByArgs("RESTRICTED_VARIABLES_ADMIN") + er.b.visitInfo = appendDynamicVisitInfo(er.b.visitInfo, "RESTRICTED_VARIABLES_ADMIN", false, err) + } if v.ExplicitScope && !sysVar.HasNoneScope() { if v.IsGlobal && !sysVar.HasGlobalScope() { er.err = variable.ErrIncorrectScope.GenWithStackByArgs(name, "GLOBAL") diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 43997217da54b..7dc2459dace33 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -722,6 +722,10 @@ func (b *PlanBuilder) buildSet(ctx context.Context, v *ast.SetStmt) (Plan, error err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or SYSTEM_VARIABLES_ADMIN") b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "SYSTEM_VARIABLES_ADMIN", false, err) } + if sem.IsEnabled() && sem.IsInvisibleSysVar(strings.ToLower(vars.Name)) { + err := ErrSpecificAccessDenied.GenWithStackByArgs("RESTRICTED_VARIABLES_ADMIN") + b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "RESTRICTED_VARIABLES_ADMIN", false, err) + } assign := &expression.VarAssignment{ Name: vars.Name, IsGlobal: vars.IsGlobal, diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index e0c63cfb14bfc..0e8d88a90c5a1 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -43,9 +43,10 @@ var dynamicPrivs = []string{ "SYSTEM_VARIABLES_ADMIN", "ROLE_ADMIN", "CONNECTION_ADMIN", - "RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled - "RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled. - "RESTRICTED_USER_ADMIN", // User can not have their access revoked by SUPER users. + "RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled + "RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled. + "RESTRICTED_VARIABLES_ADMIN", // Can see all variables when SEM is enabled + "RESTRICTED_USER_ADMIN", // User can not have their access revoked by SUPER users. } var dynamicPrivLock sync.Mutex diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 2af31f3699f7d..2f6cbef8af2cf 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1400,6 +1400,55 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeStatusVars(c *C) { }, nil, nil) } +func (s *testPrivilegeSuite) TestSecurityEnhancedModeSysVars(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE USER svroot1, svroot2") + tk.MustExec("GRANT SUPER ON *.* to svroot1 WITH GRANT OPTION") + tk.MustExec("SET tidb_enable_dynamic_privileges=1") + tk.MustExec("GRANT SUPER, RESTRICTED_VARIABLES_ADMIN ON *.* to svroot2") + + sem.Enable() + defer sem.Disable() + + // svroot1 has SUPER but in SEM will be restricted + tk.Se.Auth(&auth.UserIdentity{ + Username: "svroot1", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + + tk.MustQuery(`SHOW VARIABLES LIKE 'tidb_force_priority'`).Check(testkit.Rows()) + tk.MustQuery(`SHOW GLOBAL VARIABLES LIKE 'tidb_enable_telemetry'`).Check(testkit.Rows()) + + _, err := tk.Exec("SET tidb_force_priority = 'NO_PRIORITY'") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_VARIABLES_ADMIN privilege(s) for this operation") + _, err = tk.Exec("SET GLOBAL tidb_enable_telemetry = OFF") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_VARIABLES_ADMIN privilege(s) for this operation") + + _, err = tk.Exec("SELECT @@session.tidb_force_priority") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_VARIABLES_ADMIN privilege(s) for this operation") + _, err = tk.Exec("SELECT @@global.tidb_enable_telemetry") + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the RESTRICTED_VARIABLES_ADMIN privilege(s) for this operation") + + tk.Se.Auth(&auth.UserIdentity{ + Username: "svroot2", + Hostname: "localhost", + AuthUsername: "uroot", + AuthHostname: "%", + }, nil, nil) + + tk.MustQuery(`SHOW VARIABLES LIKE 'tidb_force_priority'`).Check(testkit.Rows("tidb_force_priority NO_PRIORITY")) + tk.MustQuery(`SHOW GLOBAL VARIABLES LIKE 'tidb_enable_telemetry'`).Check(testkit.Rows("tidb_enable_telemetry ON")) + + // should not actually make any change. + tk.MustExec("SET tidb_force_priority = 'NO_PRIORITY'") + tk.MustExec("SET GLOBAL tidb_enable_telemetry = ON") + + tk.MustQuery(`SELECT @@session.tidb_force_priority`).Check(testkit.Rows("NO_PRIORITY")) + tk.MustQuery(`SELECT @@global.tidb_enable_telemetry`).Check(testkit.Rows("1")) +} + // TestViewDefiner tests that default roles are correctly applied in the algorithm definer // See: https://github.com/pingcap/tidb/issues/24414 func (s *testPrivilegeSuite) TestViewDefiner(c *C) { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 73a8ca0066450..574e649656205 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -575,7 +575,7 @@ var defaultSysVars = []*SysVar{ } return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(ForeignKeyChecks, originalValue) }}, - {Scope: ScopeNone, Name: Hostname, Value: ServerHostname}, + {Scope: ScopeNone, Name: Hostname, Value: DefHostname}, {Scope: ScopeSession, Name: Timestamp, Value: ""}, {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetFilesystem, Value: "binary", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCharacterSet(normalizedValue, CharacterSetFilesystem) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index e416f9a695fc3..7fdccecb5a97e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -15,7 +15,6 @@ package variable import ( "math" - "os" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" @@ -694,7 +693,6 @@ var ( // DDLSlowOprThreshold is the threshold for ddl slow operations, uint is millisecond. DDLSlowOprThreshold uint32 = DefTiDBDDLSlowOprThreshold ForcePriority = int32(DefTiDBForcePriority) - ServerHostname, _ = os.Hostname() MaxOfMaxAllowedPacket uint64 = 1073741824 ExpensiveQueryTimeThreshold uint64 = DefTiDBExpensiveQueryTimeThreshold MinExpensiveQueryTimeThreshold uint64 = 10 // 10s diff --git a/tidb-server/main.go b/tidb-server/main.go index 6429ba960a0cb..05f4ecc59c25a 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -537,6 +537,9 @@ func setGlobalVars() { variable.SetSysVar(variable.TiDBSlowQueryFile, cfg.Log.SlowQueryFile) variable.SetSysVar(variable.TiDBIsolationReadEngines, strings.Join(cfg.IsolationRead.Engines, ", ")) variable.MemoryUsageAlarmRatio.Store(cfg.Performance.MemoryUsageAlarmRatio) + if hostname, err := os.Hostname(); err != nil { + variable.SetSysVar(variable.Hostname, hostname) + } if cfg.Security.EnableSEM { sem.Enable() diff --git a/util/sem/sem.go b/util/sem/sem.go index 8c3d2b456d991..d29d29b601559 100644 --- a/util/sem/sem.go +++ b/util/sem/sem.go @@ -14,6 +14,7 @@ package sem import ( + "os" "strings" "sync/atomic" @@ -70,6 +71,7 @@ var ( func Enable() { atomic.StoreInt32(&semEnabled, 1) variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.On) + variable.SetSysVar(variable.Hostname, variable.DefHostname) // write to log so users understand why some operations are weird. logutil.BgLogger().Info("tidb-server is operating with security enhanced mode (SEM) enabled") } @@ -79,6 +81,9 @@ func Enable() { func Disable() { atomic.StoreInt32(&semEnabled, 0) variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.Off) + if hostname, err := os.Hostname(); err != nil { + variable.SetSysVar(variable.Hostname, hostname) + } } // IsEnabled checks if Security Enhanced Mode (SEM) is enabled @@ -125,6 +130,33 @@ func IsInvisibleStatusVar(varName string) bool { return varName == tidbGCLeaderDesc } +// IsInvisibleSysVar returns true if the sysvar needs to be hidden +func IsInvisibleSysVar(varNameInLower string) bool { + switch varNameInLower { + case variable.TiDBDDLSlowOprThreshold, // ddl_slow_threshold + variable.TiDBAllowRemoveAutoInc, + variable.TiDBCheckMb4ValueInUTF8, + variable.TiDBConfig, + variable.TiDBEnableSlowLog, + variable.TiDBExpensiveQueryTimeThreshold, + variable.TiDBForcePriority, + variable.TiDBGeneralLog, + variable.TiDBMetricSchemaRangeDuration, + variable.TiDBMetricSchemaStep, + variable.TiDBOptWriteRowID, + variable.TiDBPProfSQLCPU, + variable.TiDBRecordPlanInSlowLog, + variable.TiDBSlowQueryFile, + variable.TiDBSlowLogThreshold, + variable.TiDBEnableCollectExecutionInfo, + variable.TiDBMemoryUsageAlarmRatio, + variable.TiDBEnableTelemetry, + variable.TiDBRowFormatVersion: + return true + } + return false +} + // IsRestrictedPrivilege returns true if the privilege shuld not be satisfied by SUPER // As most dynamic privileges are. func IsRestrictedPrivilege(privNameInUpper string) bool { diff --git a/util/sem/sem_test.go b/util/sem/sem_test.go index c303d2195c7f4..073a195139c37 100644 --- a/util/sem/sem_test.go +++ b/util/sem/sem_test.go @@ -17,6 +17,7 @@ import ( "testing" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/sessionctx/variable" . "github.com/pingcap/check" ) @@ -74,3 +75,27 @@ func (s *testSecurity) TestIsInvisibleStatusVar(c *C) { c.Assert(IsInvisibleStatusVar("ddl_schema_version"), IsFalse) c.Assert(IsInvisibleStatusVar("Ssl_version"), IsFalse) } + +func (s *testSecurity) TestIsInvisibleSysVar(c *C) { + c.Assert(IsInvisibleSysVar(variable.Hostname), IsFalse) // changes the value to default, but is not invisible + c.Assert(IsInvisibleSysVar(variable.TiDBEnableEnhancedSecurity), IsFalse) // should be able to see the mode is on. + + c.Assert(IsInvisibleSysVar(variable.TiDBAllowRemoveAutoInc), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBCheckMb4ValueInUTF8), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBConfig), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBEnableSlowLog), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBExpensiveQueryTimeThreshold), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBForcePriority), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBGeneralLog), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBMetricSchemaRangeDuration), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBMetricSchemaStep), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBOptWriteRowID), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBPProfSQLCPU), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBRecordPlanInSlowLog), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBSlowQueryFile), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBSlowLogThreshold), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBEnableCollectExecutionInfo), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBMemoryUsageAlarmRatio), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBEnableTelemetry), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBRowFormatVersion), IsTrue) +} From 2df8c3bfd1b76c2cb2ec7f2bdd31b45125568ab4 Mon Sep 17 00:00:00 2001 From: disksing Date: Mon, 17 May 2021 14:09:39 +0800 Subject: [PATCH 083/343] store/tikv: resolve ReplicaReadType dependencies (#24653) --- distsql/request_builder_test.go | 18 +++++++--------- executor/analyze.go | 5 ++--- executor/analyze_test.go | 4 ++-- executor/batch_point_get.go | 2 +- executor/point_get.go | 2 +- kv/kv.go | 2 +- kv/option.go | 17 +++++++++++++++ planner/optimize.go | 3 +-- session/session.go | 5 ++--- session/session_test.go | 13 ++++++----- sessionctx/variable/session.go | 10 ++++----- sessionctx/variable/sysvar.go | 6 +++--- sessionctx/variable/varsutil_test.go | 8 +++---- store/copr/coprocessor.go | 3 ++- store/driver/options/options.go | 32 ++++++++++++++++++++++++++++ store/driver/txn/snapshot.go | 5 +++-- store/driver/txn/txn_driver.go | 4 +++- store/tikv/kv/store_vars.go | 7 +++--- 18 files changed, 96 insertions(+), 50 deletions(-) create mode 100644 store/driver/options/options.go diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 18c1ee8fc24e4..ed921c391d6a3 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -324,7 +323,7 @@ func (s *testSuite) TestRequestBuilder1(c *C) { NotFillCache: false, SyncLog: false, Streaming: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -400,7 +399,7 @@ func (s *testSuite) TestRequestBuilder2(c *C) { NotFillCache: false, SyncLog: false, Streaming: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -447,7 +446,7 @@ func (s *testSuite) TestRequestBuilder3(c *C) { NotFillCache: false, SyncLog: false, Streaming: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -494,7 +493,7 @@ func (s *testSuite) TestRequestBuilder4(c *C) { Streaming: true, NotFillCache: false, SyncLog: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -577,10 +576,10 @@ func (s *testSuite) TestRequestBuilder6(c *C) { } func (s *testSuite) TestRequestBuilder7(c *C) { - for _, replicaRead := range []tikvstore.ReplicaReadType{ - tikvstore.ReplicaReadLeader, - tikvstore.ReplicaReadFollower, - tikvstore.ReplicaReadMixed, + for _, replicaRead := range []kv.ReplicaReadType{ + kv.ReplicaReadLeader, + kv.ReplicaReadFollower, + kv.ReplicaReadMixed, } { vars := variable.NewSessionVars() vars.SetReplicaRead(replicaRead) @@ -626,7 +625,6 @@ func (s *testSuite) TestRequestBuilder8(c *C) { IsolationLevel: 0, Priority: 0, MemTracker: (*memory.Tracker)(nil), - ReplicaRead: 0x1, SchemaVar: 10000, } c.Assert(actual, DeepEquals, expect) diff --git a/executor/analyze.go b/executor/analyze.go index fec55d870bf95..633f6e4dcc69e 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -45,7 +45,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -1322,7 +1321,7 @@ func (e *AnalyzeFastExec) handleScanIter(iter kv.Iterator) (scanKeysSize int, er func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) (keysSize int, err error) { snapshot := e.ctx.GetStore().GetSnapshot(kv.MaxVersion) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } for _, t := range e.scanTasks { iter, err := snapshot.Iter(kv.Key(t.StartKey), kv.Key(t.EndKey)) @@ -1345,7 +1344,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { snapshot.SetOption(kv.IsolationLevel, kv.RC) snapshot.SetOption(kv.Priority, kv.PriorityLow) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } rander := rand.New(rand.NewSource(e.randSeed)) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 39ec524dbacc4..b527875c1498b 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" @@ -38,7 +39,6 @@ import ( "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/table" @@ -121,7 +121,7 @@ func (s *testSuite1) TestAnalyzeReplicaReadFollower(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") ctx := tk.Se.(sessionctx.Context) - ctx.GetSessionVars().SetReplicaRead(tikvstore.ReplicaReadFollower) + ctx.GetSessionVars().SetReplicaRead(kv.ReplicaReadFollower) tk.MustExec("analyze table t") } diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 23debe37404ee..2137884c69745 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -117,7 +117,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness diff --git a/executor/point_get.go b/executor/point_get.go index 8857a4d253fd0..fc8326555bf01 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -148,7 +148,7 @@ func (e *PointGetExecutor) Open(context.Context) error { e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - e.snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + e.snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } e.snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness diff --git a/kv/kv.go b/kv/kv.go index e5ab4eed6f812..572fe104024bc 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -273,7 +273,7 @@ type Request struct { // call would not corresponds to a whole region result. Streaming bool // ReplicaRead is used for reading data from replicas, only follower is supported at this time. - ReplicaRead tikvstore.ReplicaReadType + ReplicaRead ReplicaReadType // StoreType represents this request is sent to the which type of store. StoreType StoreType // Cacheable is true if the request can be cached. Currently only deterministic DAG requests can be cached. diff --git a/kv/option.go b/kv/option.go index 5b04dfba06c95..dc0d700666d5a 100644 --- a/kv/option.go +++ b/kv/option.go @@ -60,3 +60,20 @@ const ( // MatchStoreLabels indicates the labels the store should be matched MatchStoreLabels ) + +// ReplicaReadType is the type of replica to read data from +type ReplicaReadType byte + +const ( + // ReplicaReadLeader stands for 'read from leader'. + ReplicaReadLeader ReplicaReadType = iota + // ReplicaReadFollower stands for 'read from follower'. + ReplicaReadFollower + // ReplicaReadMixed stands for 'read from leader and follower and learner'. + ReplicaReadMixed +) + +// IsFollowerRead checks if follower is going to be used to read data. +func (r ReplicaReadType) IsFollowerRead() bool { + return r != ReplicaReadLeader +} diff --git a/planner/optimize.go b/planner/optimize.go index 4e0f7334ae5ae..ec9bfef67d0a7 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/logutil" @@ -533,7 +532,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin warns = append(warns, warn) } stmtHints.HasReplicaReadHint = true - stmtHints.ReplicaRead = byte(tikvstore.ReplicaReadFollower) + stmtHints.ReplicaRead = byte(kv.ReplicaReadFollower) } // Handle MAX_EXECUTION_TIME if maxExecutionTimeCnt != 0 { diff --git a/session/session.go b/session/session.go index 8fc7c4c37eac8..af3f41c863dc0 100644 --- a/session/session.go +++ b/session/session.go @@ -69,7 +69,6 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" @@ -1891,7 +1890,7 @@ func (s *session) Txn(active bool) (kv.Transaction, error) { s.sessionVars.TxnCtx.CouldRetry = s.isTxnRetryable() s.txn.SetVars(s.sessionVars.KVVars) if s.sessionVars.GetReplicaRead().IsFollowerRead() { - s.txn.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + s.txn.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } } return &s.txn, nil @@ -1955,7 +1954,7 @@ func (s *session) NewTxn(ctx context.Context) error { } txn.SetVars(s.sessionVars.KVVars) if s.GetSessionVars().GetReplicaRead().IsFollowerRead() { - txn.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + txn.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() diff --git a/session/session_test.go b/session/session_test.go index a8861dfddd79c..4870215f33c9e 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -51,7 +51,6 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" @@ -3064,11 +3063,11 @@ func (s *testSessionSuite2) TestReplicaRead(c *C) { tk := testkit.NewTestKit(c, s.store) tk.Se, err = session.CreateSession4Test(s.store) c.Assert(err, IsNil) - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadLeader) tk.MustExec("set @@tidb_replica_read = 'follower';") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) tk.MustExec("set @@tidb_replica_read = 'leader';") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadLeader) } func (s *testSessionSuite3) TestIsolationRead(c *C) { @@ -3153,12 +3152,12 @@ func (s *testSessionSuite2) TestStmtHints(c *C) { c.Assert(tk.Se.GetSessionVars().GetEnableCascadesPlanner(), IsTrue) // Test READ_CONSISTENT_REPLICA hint - tk.Se.GetSessionVars().SetReplicaRead(tikvstore.ReplicaReadLeader) + tk.Se.GetSessionVars().SetReplicaRead(kv.ReplicaReadLeader) tk.MustExec("select /*+ READ_CONSISTENT_REPLICA() */ 1;") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) tk.MustExec("select /*+ READ_CONSISTENT_REPLICA(), READ_CONSISTENT_REPLICA() */ 1;") c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) } func (s *testSessionSuite3) TestPessimisticLockOnPartition(c *C) { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 97597997b36f5..0c6c74d90a26d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -723,7 +723,7 @@ type SessionVars struct { enableIndexMerge bool // replicaRead is used for reading data from replicas, only follower is supported at this time. - replicaRead tikvstore.ReplicaReadType + replicaRead kv.ReplicaReadType // IsolationReadEngines is used to isolation read, tidb only read from the stores whose engine type is in the engines. IsolationReadEngines map[kv.StoreType]struct{} @@ -1029,7 +1029,7 @@ func NewSessionVars() *SessionVars { WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, enableIndexMerge: false, EnableNoopFuncs: DefTiDBEnableNoopFuncs, - replicaRead: tikvstore.ReplicaReadLeader, + replicaRead: kv.ReplicaReadLeader, AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, UsePlanBaselines: DefTiDBUsePlanBaselines, EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, @@ -1179,15 +1179,15 @@ func (s *SessionVars) SetEnableIndexMerge(val bool) { } // GetReplicaRead get ReplicaRead from sql hints and SessionVars.replicaRead. -func (s *SessionVars) GetReplicaRead() tikvstore.ReplicaReadType { +func (s *SessionVars) GetReplicaRead() kv.ReplicaReadType { if s.StmtCtx.HasReplicaReadHint { - return tikvstore.ReplicaReadType(s.StmtCtx.ReplicaRead) + return kv.ReplicaReadType(s.StmtCtx.ReplicaRead) } return s.replicaRead } // SetReplicaRead set SessionVars.replicaRead. -func (s *SessionVars) SetReplicaRead(val tikvstore.ReplicaReadType) { +func (s *SessionVars) SetReplicaRead(val kv.ReplicaReadType) { s.replicaRead = val } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 574e649656205..98518fe4af0f0 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1246,11 +1246,11 @@ var defaultSysVars = []*SysVar{ }}, {Scope: ScopeSession, Name: TiDBReplicaRead, Value: "leader", Type: TypeEnum, PossibleValues: []string{"leader", "follower", "leader-and-follower"}, SetSession: func(s *SessionVars, val string) error { if strings.EqualFold(val, "follower") { - s.SetReplicaRead(tikvstore.ReplicaReadFollower) + s.SetReplicaRead(kv.ReplicaReadFollower) } else if strings.EqualFold(val, "leader-and-follower") { - s.SetReplicaRead(tikvstore.ReplicaReadMixed) + s.SetReplicaRead(kv.ReplicaReadMixed) } else if strings.EqualFold(val, "leader") || len(val) == 0 { - s.SetReplicaRead(tikvstore.ReplicaReadLeader) + s.SetReplicaRead(kv.ReplicaReadLeader) } return nil }}, diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 90507f36539d6..1d8e629b7df4a 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/testleak" ) @@ -431,19 +431,19 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "follower") - c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(v.GetReplicaRead(), Equals, kv.ReplicaReadFollower) err = SetSessionSystemVar(v, TiDBReplicaRead, "leader") c.Assert(err, IsNil) val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "leader") - c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) + c.Assert(v.GetReplicaRead(), Equals, kv.ReplicaReadLeader) err = SetSessionSystemVar(v, TiDBReplicaRead, "leader-and-follower") c.Assert(err, IsNil) val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "leader-and-follower") - c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadMixed) + c.Assert(v.GetReplicaRead(), Equals, kv.ReplicaReadMixed) err = SetSessionSystemVar(v, TiDBEnableStmtSummary, "ON") c.Assert(err, IsNil) diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index e9d9e6b8f1ebb..989a6d835ce0f 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -37,6 +37,7 @@ import ( tidbmetrics "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/driver/options" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -697,7 +698,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch } } - req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, worker.req.ReplicaRead, &worker.replicaReadSeed, kvrpcpb.Context{ + req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, options.GetTiKVReplicaReadType(worker.req.ReplicaRead), &worker.replicaReadSeed, kvrpcpb.Context{ IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel), Priority: priorityToPB(worker.req.Priority), NotFillCache: worker.req.NotFillCache, diff --git a/store/driver/options/options.go b/store/driver/options/options.go new file mode 100644 index 0000000000000..dc16f7793ed91 --- /dev/null +++ b/store/driver/options/options.go @@ -0,0 +1,32 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package options + +import ( + "github.com/pingcap/tidb/kv" + storekv "github.com/pingcap/tidb/store/tikv/kv" +) + +// GetTiKVReplicaReadType maps kv.ReplicaReadType to tikv/kv.ReplicaReadType. +func GetTiKVReplicaReadType(t kv.ReplicaReadType) storekv.ReplicaReadType { + switch t { + case kv.ReplicaReadLeader: + return storekv.ReplicaReadLeader + case kv.ReplicaReadFollower: + return storekv.ReplicaReadFollower + case kv.ReplicaReadMixed: + return storekv.ReplicaReadMixed + } + return 0 +} diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 405067f5e082b..6692f45a749a3 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/driver/options" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" ) type tikvSnapshot struct { @@ -76,7 +76,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { case kv.SnapshotTS: s.KVSnapshot.SetSnapshotTS(val.(uint64)) case kv.ReplicaRead: - s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) + t := options.GetTiKVReplicaReadType(val.(kv.ReplicaReadType)) + s.KVSnapshot.SetReplicaRead(t) case kv.SampleStep: s.KVSnapshot.SetSampleStep(val.(uint32)) case kv.TaskID: diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 7c940561f82fe..7f05f80139c12 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/binloginfo" derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/driver/options" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -143,7 +144,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { case kv.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) case kv.ReplicaRead: - txn.KVTxn.GetSnapshot().SetReplicaRead(val.(tikvstore.ReplicaReadType)) + t := options.GetTiKVReplicaReadType(val.(kv.ReplicaReadType)) + txn.KVTxn.GetSnapshot().SetReplicaRead(t) case kv.TaskID: txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case kv.InfoSchema: diff --git a/store/tikv/kv/store_vars.go b/store/tikv/kv/store_vars.go index 5f65f927bffb9..02d87018213a9 100644 --- a/store/tikv/kv/store_vars.go +++ b/store/tikv/kv/store_vars.go @@ -25,15 +25,14 @@ type ReplicaReadType byte const ( // ReplicaReadLeader stands for 'read from leader'. - ReplicaReadLeader ReplicaReadType = 1 << iota + ReplicaReadLeader ReplicaReadType = iota // ReplicaReadFollower stands for 'read from follower'. ReplicaReadFollower // ReplicaReadMixed stands for 'read from leader and follower and learner'. ReplicaReadMixed ) -// IsFollowerRead checks if leader is going to be used to read data. +// IsFollowerRead checks if follower is going to be used to read data. func (r ReplicaReadType) IsFollowerRead() bool { - // In some cases the default value is 0, which should be treated as `ReplicaReadLeader`. - return r != ReplicaReadLeader && r != 0 + return r != ReplicaReadLeader } From c20d496dd780f1f3454d84716cfa60f4ac3b7eb2 Mon Sep 17 00:00:00 2001 From: rebelice Date: Mon, 17 May 2021 14:23:39 +0800 Subject: [PATCH 084/343] executor: add test cases about partition table with `expression` (#24628) --- executor/partition_table_test.go | 255 ++++++++++++++++++++++++++++++- util/testkit/testkit.go | 7 +- 2 files changed, 253 insertions(+), 9 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index cf53b7cd82688..e87e1044278e3 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -573,12 +573,12 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { } func createTable4DynamicPruneModeTestWithExpression(tk *testkit.TestKit) { - tk.MustExec("create table trange(a int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11));") - tk.MustExec("create table thash(a int) partition by hash(a) partitions 4;") - tk.MustExec("create table t(a int)") - tk.MustExec("insert into trange values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") - tk.MustExec("insert into thash values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") - tk.MustExec("insert into t values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);") + tk.MustExec("create table trange(a int, b int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11));") + tk.MustExec("create table thash(a int, b int) partition by hash(a) partitions 4;") + tk.MustExec("create table t(a int, b int)") + tk.MustExec("insert into trange values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1);") + tk.MustExec("insert into thash values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1);") + tk.MustExec("insert into t values(1, NULL), (1, NULL), (1, 1), (2, 1), (3, 2), (4, 3), (5, 5), (6, 7), (7, 7), (7, 7), (10, NULL), (NULL, NULL), (NULL, 1);") tk.MustExec("set session tidb_partition_prune_mode='dynamic'") tk.MustExec("analyze table trange") tk.MustExec("analyze table thash") @@ -590,7 +590,7 @@ type testData4Expression struct { partitions []string } -func (s *partitionTableSuite) TestDynamicPruneModeWithEqualExpression(c *C) { +func (s *partitionTableSuite) TestDynamicPruneModeWithExpression(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop database if exists db_equal_expression") tk.MustExec("create database db_equal_expression") @@ -627,6 +627,247 @@ func (s *partitionTableSuite) TestDynamicPruneModeWithEqualExpression(c *C) { "p0", }, }, + { + sql: "select * from %s where b is NULL", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a > -1", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a >= 4 and a <= 5", + partitions: []string{ + "p1,p2", + "p0,p1", + }, + }, + { + sql: "select * from %s where a > 10", + partitions: []string{ + "dual", + "all", + }, + }, + { + sql: "select * from %s where a >=2 and a <= 3", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a between 2 and 3", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a < 2", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where a <= 3", + partitions: []string{ + "p0,p1", + "all", + }, + }, + { + sql: "select * from %s where a in (2, 3)", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a in (1, 5)", + partitions: []string{ + "p0,p2", + "p1", + }, + }, + { + sql: "select * from %s where a not in (1, 5)", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a = 2 and a = 2", + partitions: []string{ + "p0", + "p2", + }, + }, + { + sql: "select * from %s where a = 2 and a = 3", + partitions: []string{ + // This means that we have no partition-read plan + "", + "", + }, + }, + { + sql: "select * from %s where a < 2 and a > 0", + partitions: []string{ + "p0", + "p1", + }, + }, + { + sql: "select * from %s where a < 2 and a < 3", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where a > 1 and a > 2", + partitions: []string{ + "p1,p2", + "all", + }, + }, + { + sql: "select * from %s where a = 2 or a = 3", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a = 2 or a in (3)", + partitions: []string{ + "p0,p1", + "p2,p3", + }, + }, + { + sql: "select * from %s where a = 2 or a > 3", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a = 2 or a <= 1", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where a = 2 or a between 2 and 2", + partitions: []string{ + "p0", + "p2", + }, + }, + { + sql: "select * from %s where a != 2", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a != 2 and a > 4", + partitions: []string{ + "p2", + "all", + }, + }, + { + sql: "select * from %s where a != 2 and a != 3", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a != 2 and a = 3", + partitions: []string{ + "p1", + "p3", + }, + }, + { + sql: "select * from %s where not (a = 2)", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where not (a > 2)", + partitions: []string{ + "p0", + "all", + }, + }, + { + sql: "select * from %s where not (a < 2)", + partitions: []string{ + "all", + "all", + }, + }, + // cases that partition pruning can not work + { + sql: "select * from %s where a + 1 > 4", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a - 1 > 0", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a * 2 < 0", + partitions: []string{ + "all", + "all", + }, + }, + { + sql: "select * from %s where a << 1 < 0", + partitions: []string{ + "all", + "all", + }, + }, + // comparison between int column and string column + { + sql: "select * from %s where a > '10'", + partitions: []string{ + "dual", + "all", + }, + }, + { + sql: "select * from %s where a > '10ab'", + partitions: []string{ + "dual", + "all", + }, + }, } for _, t := range tests { diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index d6d6e41bb6e9f..7cacaf211375e 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -258,9 +258,12 @@ func (tk *TestKit) MustNoGlobalStats(table string) bool { // MustPartition checks if the result execution plan must read specific partitions. func (tk *TestKit) MustPartition(sql string, partitions string, args ...interface{}) *Result { rs := tk.MustQuery("explain "+sql, args...) - ok := false + ok := len(partitions) == 0 for i := range rs.rows { - if strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 { + if len(partitions) == 0 && strings.Contains(rs.rows[i][3], "partition:") { + ok = false + } + if len(partitions) != 0 && strings.Compare(rs.rows[i][3], "partition:"+partitions) == 0 { ok = true } } From ae36fbdb316ffd955cc72dcc8731ba1fa8b9919a Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 17 May 2021 14:35:39 +0800 Subject: [PATCH 085/343] tablecodec: fix write wrong prefix index value when collation is ascii_bin/latin1_bin (#24578) --- executor/write_test.go | 19 +++++++++++++++++++ tablecodec/tablecodec.go | 3 ++- 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/executor/write_test.go b/executor/write_test.go index b832e52a9935c..cf7a51985a450 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -3930,6 +3930,25 @@ func (s *testSerialSuite) TestIssue20840(c *C) { tk.MustExec("drop table t1") } +func (s *testSerialSuite) TestIssueInsertPrefixIndexForNonUTF8Collation(c *C) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2, t3") + tk.MustExec("create table t1 ( c_int int, c_str varchar(40) character set ascii collate ascii_bin, primary key(c_int, c_str(8)) clustered , unique key(c_str))") + tk.MustExec("create table t2 ( c_int int, c_str varchar(40) character set latin1 collate latin1_bin, primary key(c_int, c_str(8)) clustered , unique key(c_str))") + tk.MustExec("insert into t1 values (3, 'fervent brattain')") + tk.MustExec("insert into t2 values (3, 'fervent brattain')") + tk.MustExec("admin check table t1") + tk.MustExec("admin check table t2") + + tk.MustExec("create table t3 (x varchar(40) CHARACTER SET ascii COLLATE ascii_bin, UNIQUE KEY uk(x(4)))") + tk.MustExec("insert into t3 select 'abc '") + tk.MustGetErrCode("insert into t3 select 'abc d'", 1062) +} + func (s *testSerialSuite) TestIssue22496(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index 215f8d05c27fa..de766831bc245 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -1292,6 +1292,7 @@ func TruncateIndexValue(v *types.Datum, idxCol *model.IndexColumn, tblCol *model if notStringType { return } + originalKind := v.Kind() isUTF8Charset := tblCol.Charset == charset.CharsetUTF8 || tblCol.Charset == charset.CharsetUTF8MB4 if isUTF8Charset && utf8.RuneCount(v.GetBytes()) > idxCol.Length { rs := bytes.Runes(v.GetBytes()) @@ -1303,7 +1304,7 @@ func TruncateIndexValue(v *types.Datum, idxCol *model.IndexColumn, tblCol *model } } else if !isUTF8Charset && len(v.GetBytes()) > idxCol.Length { v.SetBytes(v.GetBytes()[:idxCol.Length]) - if v.Kind() == types.KindString { + if originalKind == types.KindString { v.SetString(v.GetString(), tblCol.Collate) } } From 2ca98e393cb6873727a177cb5c974fa8d3ca61c5 Mon Sep 17 00:00:00 2001 From: xhe Date: Mon, 17 May 2021 19:53:39 +0800 Subject: [PATCH 086/343] *: compatibility with staleread (#24285) --- ddl/column_change_test.go | 21 ++-- ddl/column_test.go | 3 +- ddl/ddl.go | 20 ++-- ddl/ddl_api.go | 30 +++--- ddl/ddl_test.go | 5 + ddl/ddl_worker_test.go | 10 +- ddl/index_change_test.go | 22 ++-- ddl/options.go | 16 +-- ddl/options_test.go | 6 +- ddl/partition.go | 81 +++++++-------- ddl/reorg_test.go | 2 +- ddl/restart_test.go | 4 +- ddl/schema.go | 8 +- ddl/schema_test.go | 4 +- ddl/stat_test.go | 2 +- ddl/table.go | 54 +++++----- ddl/table_test.go | 2 +- ddl/util/syncer_test.go | 7 ++ domain/domain.go | 182 +++++++++++++++------------------- domain/domain_test.go | 2 +- executor/simple.go | 21 ---- executor/stale_txn_test.go | 91 ++++------------- infoschema/builder.go | 47 +++++---- infoschema/cache.go | 95 ++++++++++++++++++ infoschema/cache_test.go | 119 ++++++++++++++++++++++ infoschema/infoschema.go | 36 ------- infoschema/infoschema_test.go | 54 ++-------- metrics/domain.go | 13 +++ metrics/metrics.go | 1 + owner/manager_test.go | 13 +++ session/session.go | 9 +- session/session_test.go | 10 +- 32 files changed, 535 insertions(+), 455 deletions(-) create mode 100644 infoschema/cache.go create mode 100644 infoschema/cache_test.go diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index 94e8787a2bdc4..6bd5a94f7235e 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -47,15 +47,18 @@ type testColumnChangeSuite struct { func (s *testColumnChangeSuite) SetUpSuite(c *C) { SetWaitTimeWhenErrorOccurred(1 * time.Microsecond) s.store = testCreateStore(c, "test_column_change") - s.dbInfo = &model.DBInfo{ - Name: model.NewCIStr("test_column_change"), - ID: 1, - } - err := kv.RunInNewTxn(context.Background(), s.store, true, func(ctx context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) - return errors.Trace(t.CreateDatabase(s.dbInfo)) - }) - c.Check(err, IsNil) + d := testNewDDLAndStart( + context.Background(), + c, + WithStore(s.store), + WithLease(testLease), + ) + defer func() { + err := d.Stop() + c.Assert(err, IsNil) + }() + s.dbInfo = testSchemaInfo(c, d, "test_index_change") + testCreateSchema(c, testNewContext(d), d, s.dbInfo) } func (s *testColumnChangeSuite) TearDownSuite(c *C) { diff --git a/ddl/column_test.go b/ddl/column_test.go index 862fb4aa04c59..f3eaa26d22385 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -54,8 +54,7 @@ func (s *testColumnSuite) SetUpSuite(c *C) { s.dbInfo = testSchemaInfo(c, d, "test_column") testCreateSchema(c, testNewContext(d), d, s.dbInfo) - err := d.Stop() - c.Assert(err, IsNil) + c.Assert(d.Stop(), IsNil) } func (s *testColumnSuite) TearDownSuite(c *C) { diff --git a/ddl/ddl.go b/ddl/ddl.go index 6f20fe25ccc07..9eb05b86741ed 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -202,7 +202,7 @@ type ddlCtx struct { ddlEventCh chan<- *util.Event lease time.Duration // lease is schema lease. binlogCli *pumpcli.PumpsClient // binlogCli is used for Binlog. - infoHandle *infoschema.Handle + infoCache *infoschema.InfoCache statsHandle *handle.Handle tableLockCkr util.DeadTableLockChecker etcdCli *clientv3.Client @@ -282,6 +282,15 @@ func newDDL(ctx context.Context, options ...Option) *ddl { deadLockCkr = util.NewDeadTableLockChecker(etcdCli) } + // TODO: make store and infoCache explicit arguments + // these two should be ensured to exist + if opt.Store == nil { + panic("store should not be nil") + } + if opt.InfoCache == nil { + panic("infoCache should not be nil") + } + ddlCtx := &ddlCtx{ uuid: id, store: opt.Store, @@ -290,7 +299,7 @@ func newDDL(ctx context.Context, options ...Option) *ddl { ownerManager: manager, schemaSyncer: syncer, binlogCli: binloginfo.GetPumpsClient(), - infoHandle: opt.InfoHandle, + infoCache: opt.InfoCache, tableLockCkr: deadLockCkr, etcdCli: opt.EtcdCli, } @@ -411,7 +420,7 @@ func (d *ddl) GetLease() time.Duration { // Please don't use this function, it is used by TestParallelDDLBeforeRunDDLJob to intercept the calling of d.infoHandle.Get(), use d.infoHandle.Get() instead. // Otherwise, the TestParallelDDLBeforeRunDDLJob will hang up forever. func (d *ddl) GetInfoSchemaWithInterceptor(ctx sessionctx.Context) infoschema.InfoSchema { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() d.mu.RLock() defer d.mu.RUnlock() @@ -649,10 +658,7 @@ func (d *ddl) startCleanDeadTableLock() { if !d.ownerManager.IsOwner() { continue } - if d.infoHandle == nil || !d.infoHandle.IsValid() { - continue - } - deadLockTables, err := d.tableLockCkr.GetDeadLockedTables(d.ctx, d.infoHandle.Get().AllSchemas()) + deadLockTables, err := d.tableLockCkr.GetDeadLockedTables(d.ctx, d.infoCache.GetLatest().AllSchemas()) if err != nil { logutil.BgLogger().Info("[ddl] get dead table lock failed.", zap.Error(err)) continue diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index a3f8bb7f9c622..d0289dc19e39f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2367,7 +2367,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A return errors.Trace(err) } - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() if is.TableIsView(ident.Schema, ident.Name) || is.TableIsSequence(ident.Schema, ident.Name) { return ErrWrongObject.GenWithStackByArgs(ident.Schema, ident.Name, "BASE TABLE") } @@ -2898,7 +2898,7 @@ func (d *ddl) AddColumns(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alte // AddTablePartitions will add a new partition to the table. func (d *ddl) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) @@ -2959,7 +2959,7 @@ func (d *ddl) AddTablePartitions(ctx sessionctx.Context, ident ast.Ident, spec * // CoalescePartitions coalesce partitions can be used with a table that is partitioned by hash or key to reduce the number of partitions by number. func (d *ddl) CoalescePartitions(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) @@ -2991,7 +2991,7 @@ func (d *ddl) CoalescePartitions(ctx sessionctx.Context, ident ast.Ident, spec * } func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) @@ -3039,7 +3039,7 @@ func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, sp } func (d *ddl) DropTablePartition(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists.GenWithStackByArgs(schema)) @@ -3752,7 +3752,7 @@ func processAndCheckDefaultValueAndColumn(ctx sessionctx.Context, col *table.Col func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, originalColName model.CIStr, spec *ast.AlterTableSpec) (*model.Job, error) { specNewColumn := spec.NewColumns[0] - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return nil, errors.Trace(infoschema.ErrDatabaseNotExists) @@ -4203,7 +4203,7 @@ func (d *ddl) ModifyColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Al func (d *ddl) AlterColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { specNewColumn := spec.NewColumns[0] - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name) @@ -4257,7 +4257,7 @@ func (d *ddl) AlterColumn(ctx sessionctx.Context, ident ast.Ident, spec *ast.Alt // AlterTableComment updates the table comment information. func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) @@ -4310,7 +4310,7 @@ func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Iden return ErrUnknownCharacterSet.GenWithStackByArgs(toCharset) } - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) @@ -4471,7 +4471,7 @@ func (d *ddl) AlterTableDropStatistics(ctx sessionctx.Context, ident ast.Ident, // UpdateTableReplicaInfo updates the table flash replica infos. func (d *ddl) UpdateTableReplicaInfo(ctx sessionctx.Context, physicalID int64, available bool) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() tb, ok := is.TableByID(physicalID) if !ok { tb, _, _ = is.FindTableByPartitionID(physicalID) @@ -4574,7 +4574,7 @@ func checkAlterTableCharset(tblInfo *model.TableInfo, dbInfo *model.DBInfo, toCh // In TiDB, indexes are case-insensitive (so index 'a' and 'A" are considered the same index), // but index names are case-sensitive (we can rename index 'a' to 'A') func (d *ddl) RenameIndex(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ident.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) @@ -5232,7 +5232,7 @@ func buildFKInfo(fkName model.CIStr, keys []*ast.IndexPartSpecification, refer * } func (d *ddl) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model.CIStr, keys []*ast.IndexPartSpecification, refer *ast.ReferenceDef) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ti.Schema) @@ -5264,7 +5264,7 @@ func (d *ddl) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName mode } func (d *ddl) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model.CIStr) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ti.Schema) @@ -5290,7 +5290,7 @@ func (d *ddl) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model. } func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CIStr, ifExists bool) error { - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() schema, ok := is.SchemaByName(ti.Schema) if !ok { return errors.Trace(infoschema.ErrDatabaseNotExists) @@ -6036,7 +6036,7 @@ func (d *ddl) AlterTableAlterPartition(ctx sessionctx.Context, ident ast.Ident, return errors.Trace(err) } - oldBundle := infoschema.GetBundle(d.infoHandle.Get(), []int64{partitionID, meta.ID, schema.ID}) + oldBundle := infoschema.GetBundle(d.infoCache.GetLatest(), []int64{partitionID, meta.ID, schema.ID}) oldBundle.ID = placement.GroupID(partitionID) diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index b77c3300c2700..79635bfc0933b 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" @@ -86,6 +87,10 @@ func TestT(t *testing.T) { } func testNewDDLAndStart(ctx context.Context, c *C, options ...Option) *ddl { + // init infoCache and a stub infoSchema + ic := infoschema.NewCache(2) + ic.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) + options = append(options, WithInfoCache(ic)) d := newDDL(ctx, options...) err := d.Start(nil) c.Assert(err, IsNil) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 6e745820b04b9..72fef7c96c19e 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -247,7 +247,7 @@ func (s *testDDLSuite) TestTableError(c *C) { // Schema ID is wrong, so dropping table is failed. doDDLJobErr(c, -1, 1, model.ActionDropTable, nil, ctx, d) // Table ID is wrong, so dropping table is failed. - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_ddl") testCreateSchema(c, testNewContext(d), d, dbInfo) job := doDDLJobErr(c, dbInfo.ID, -1, model.ActionDropTable, nil, ctx, d) @@ -295,7 +295,7 @@ func (s *testDDLSuite) TestViewError(c *C) { c.Assert(err, IsNil) }() ctx := testNewContext(d) - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_ddl") testCreateSchema(c, testNewContext(d), d, dbInfo) // Table ID or schema ID is wrong, so getting table is failed. @@ -363,7 +363,7 @@ func (s *testDDLSuite) TestForeignKeyError(c *C) { doDDLJobErr(c, -1, 1, model.ActionAddForeignKey, nil, ctx, d) doDDLJobErr(c, -1, 1, model.ActionDropForeignKey, nil, ctx, d) - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_ddl") tblInfo := testTableInfo(c, d, "t", 3) testCreateSchema(c, ctx, d, dbInfo) testCreateTable(c, ctx, d, dbInfo, tblInfo) @@ -393,7 +393,7 @@ func (s *testDDLSuite) TestIndexError(c *C) { doDDLJobErr(c, -1, 1, model.ActionAddIndex, nil, ctx, d) doDDLJobErr(c, -1, 1, model.ActionDropIndex, nil, ctx, d) - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_ddl") tblInfo := testTableInfo(c, d, "t", 3) testCreateSchema(c, ctx, d, dbInfo) testCreateTable(c, ctx, d, dbInfo, tblInfo) @@ -435,7 +435,7 @@ func (s *testDDLSuite) TestColumnError(c *C) { }() ctx := testNewContext(d) - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_ddl") tblInfo := testTableInfo(c, d, "t", 3) testCreateSchema(c, ctx, d, dbInfo) testCreateTable(c, ctx, d, dbInfo, tblInfo) diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index dfdfc7111c372..6a34599137c10 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -38,15 +37,18 @@ type testIndexChangeSuite struct { func (s *testIndexChangeSuite) SetUpSuite(c *C) { s.store = testCreateStore(c, "test_index_change") - s.dbInfo = &model.DBInfo{ - Name: model.NewCIStr("test_index_change"), - ID: 1, - } - err := kv.RunInNewTxn(context.Background(), s.store, true, func(ctx context.Context, txn kv.Transaction) error { - t := meta.NewMeta(txn) - return errors.Trace(t.CreateDatabase(s.dbInfo)) - }) - c.Check(err, IsNil, Commentf("err %v", errors.ErrorStack(err))) + d := testNewDDLAndStart( + context.Background(), + c, + WithStore(s.store), + WithLease(testLease), + ) + defer func() { + err := d.Stop() + c.Assert(err, IsNil) + }() + s.dbInfo = testSchemaInfo(c, d, "test_index_change") + testCreateSchema(c, testNewContext(d), d, s.dbInfo) } func (s *testIndexChangeSuite) TearDownSuite(c *C) { diff --git a/ddl/options.go b/ddl/options.go index 8613a8e9affa9..9238a7c8542ff 100644 --- a/ddl/options.go +++ b/ddl/options.go @@ -26,11 +26,11 @@ type Option func(*Options) // Options represents all the options of the DDL module needs type Options struct { - EtcdCli *clientv3.Client - Store kv.Storage - InfoHandle *infoschema.Handle - Hook Callback - Lease time.Duration + EtcdCli *clientv3.Client + Store kv.Storage + InfoCache *infoschema.InfoCache + Hook Callback + Lease time.Duration } // WithEtcdClient specifies the `clientv3.Client` of DDL used to request the etcd service @@ -47,10 +47,10 @@ func WithStore(store kv.Storage) Option { } } -// WithInfoHandle specifies the `infoschema.Handle` -func WithInfoHandle(ih *infoschema.Handle) Option { +// WithInfoCache specifies the `infoschema.InfoCache` +func WithInfoCache(ic *infoschema.InfoCache) Option { return func(options *Options) { - options.InfoHandle = ih + options.InfoCache = ic } } diff --git a/ddl/options_test.go b/ddl/options_test.go index 294d68731e4c3..22a451d622c71 100644 --- a/ddl/options_test.go +++ b/ddl/options_test.go @@ -33,14 +33,14 @@ func (s *ddlOptionsSuite) TestOptions(c *C) { callback := &ddl.BaseCallback{} lease := time.Second * 3 store := &mock.Store{} - infoHandle := infoschema.NewHandle(store) + infoHandle := infoschema.NewCache(16) options := []ddl.Option{ ddl.WithEtcdClient(client), ddl.WithHook(callback), ddl.WithLease(lease), ddl.WithStore(store), - ddl.WithInfoHandle(infoHandle), + ddl.WithInfoCache(infoHandle), } opt := &ddl.Options{} @@ -52,5 +52,5 @@ func (s *ddlOptionsSuite) TestOptions(c *C) { c.Assert(opt.Hook, Equals, callback) c.Assert(opt.Lease, Equals, lease) c.Assert(opt.Store, Equals, store) - c.Assert(opt.InfoHandle, Equals, infoHandle) + c.Assert(opt.InfoCache, Equals, infoHandle) } diff --git a/ddl/partition.go b/ddl/partition.go index 0cafa9d2ff525..4e55ec1779e21 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -911,18 +911,15 @@ func getTableInfoWithDroppingPartitions(t *model.TableInfo) *model.TableInfo { } func dropRuleBundles(d *ddlCtx, physicalTableIDs []int64) error { - if d.infoHandle != nil && d.infoHandle.IsValid() { - bundles := make([]*placement.Bundle, 0, len(physicalTableIDs)) - for _, ID := range physicalTableIDs { - oldBundle, ok := d.infoHandle.Get().BundleByName(placement.GroupID(ID)) - if ok && !oldBundle.IsEmpty() { - bundles = append(bundles, placement.BuildPlacementDropBundle(ID)) - } + bundles := make([]*placement.Bundle, 0, len(physicalTableIDs)) + for _, ID := range physicalTableIDs { + oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(ID)) + if ok && !oldBundle.IsEmpty() { + bundles = append(bundles, placement.BuildPlacementDropBundle(ID)) } - err := infosync.PutRuleBundles(context.TODO(), bundles) - return err } - return nil + err := infosync.PutRuleBundles(context.TODO(), bundles) + return err } // onDropTablePartition deletes old partition meta. @@ -1095,22 +1092,20 @@ func onTruncateTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) (int64, e } } - if d.infoHandle != nil && d.infoHandle.IsValid() { - bundles := make([]*placement.Bundle, 0, len(oldIDs)) + bundles := make([]*placement.Bundle, 0, len(oldIDs)) - for i, oldID := range oldIDs { - oldBundle, ok := d.infoHandle.Get().BundleByName(placement.GroupID(oldID)) - if ok && !oldBundle.IsEmpty() { - bundles = append(bundles, placement.BuildPlacementDropBundle(oldID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newPartitions[i].ID)) - } + for i, oldID := range oldIDs { + oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(oldID)) + if ok && !oldBundle.IsEmpty() { + bundles = append(bundles, placement.BuildPlacementDropBundle(oldID)) + bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newPartitions[i].ID)) } + } - err = infosync.PutRuleBundles(context.TODO(), bundles) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Wrapf(err, "failed to notify PD the placement rules") - } + err = infosync.PutRuleBundles(context.TODO(), bundles) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } newIDs := make([]int64, len(oldIDs)) @@ -1299,27 +1294,25 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo // the follow code is a swap function for rules of two partitions // though partitions has exchanged their ID, swap still take effect - if d.infoHandle != nil && d.infoHandle.IsValid() { - bundles := make([]*placement.Bundle, 0, 2) - ptBundle, ptOK := d.infoHandle.Get().BundleByName(placement.GroupID(partDef.ID)) - ptOK = ptOK && !ptBundle.IsEmpty() - ntBundle, ntOK := d.infoHandle.Get().BundleByName(placement.GroupID(nt.ID)) - ntOK = ntOK && !ntBundle.IsEmpty() - if ptOK && ntOK { - bundles = append(bundles, placement.BuildPlacementCopyBundle(ptBundle, nt.ID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(ntBundle, partDef.ID)) - } else if ptOK { - bundles = append(bundles, placement.BuildPlacementDropBundle(partDef.ID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(ptBundle, nt.ID)) - } else if ntOK { - bundles = append(bundles, placement.BuildPlacementDropBundle(nt.ID)) - bundles = append(bundles, placement.BuildPlacementCopyBundle(ntBundle, partDef.ID)) - } - err = infosync.PutRuleBundles(context.TODO(), bundles) - if err != nil { - job.State = model.JobStateCancelled - return ver, errors.Wrapf(err, "failed to notify PD the placement rules") - } + bundles := make([]*placement.Bundle, 0, 2) + ptBundle, ptOK := d.infoCache.GetLatest().BundleByName(placement.GroupID(partDef.ID)) + ptOK = ptOK && !ptBundle.IsEmpty() + ntBundle, ntOK := d.infoCache.GetLatest().BundleByName(placement.GroupID(nt.ID)) + ntOK = ntOK && !ntBundle.IsEmpty() + if ptOK && ntOK { + bundles = append(bundles, placement.BuildPlacementCopyBundle(ptBundle, nt.ID)) + bundles = append(bundles, placement.BuildPlacementCopyBundle(ntBundle, partDef.ID)) + } else if ptOK { + bundles = append(bundles, placement.BuildPlacementDropBundle(partDef.ID)) + bundles = append(bundles, placement.BuildPlacementCopyBundle(ptBundle, nt.ID)) + } else if ntOK { + bundles = append(bundles, placement.BuildPlacementDropBundle(nt.ID)) + bundles = append(bundles, placement.BuildPlacementCopyBundle(ntBundle, partDef.ID)) + } + err = infosync.PutRuleBundles(context.TODO(), bundles) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Wrapf(err, "failed to notify PD the placement rules") } ver, err = updateSchemaVersion(t, job) diff --git a/ddl/reorg_test.go b/ddl/reorg_test.go index 18dd9a975fceb..4c28540e7ad3b 100644 --- a/ddl/reorg_test.go +++ b/ddl/reorg_test.go @@ -217,7 +217,7 @@ func (s *testDDLSuite) TestReorgOwner(c *C) { c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d1, "test") + dbInfo := testSchemaInfo(c, d1, "test_reorg") testCreateSchema(c, ctx, d1, dbInfo) tblInfo := testTableInfo(c, d1, "t", 3) diff --git a/ddl/restart_test.go b/ddl/restart_test.go index b587d54b80cc8..b7791ef7679bd 100644 --- a/ddl/restart_test.go +++ b/ddl/restart_test.go @@ -120,7 +120,7 @@ func (s *testSchemaSuite) TestSchemaResume(c *C) { testCheckOwner(c, d1, true) - dbInfo := testSchemaInfo(c, d1, "test") + dbInfo := testSchemaInfo(c, d1, "test_restart") job := &model.Job{ SchemaID: dbInfo.ID, Type: model.ActionCreateSchema, @@ -157,7 +157,7 @@ func (s *testStatSuite) TestStat(c *C) { c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_restart") testCreateSchema(c, testNewContext(d), d, dbInfo) // TODO: Get this information from etcd. diff --git a/ddl/schema.go b/ddl/schema.go index 823e12a551900..a4b14a49bdbc3 100644 --- a/ddl/schema.go +++ b/ddl/schema.go @@ -68,16 +68,12 @@ func onCreateSchema(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error } func checkSchemaNotExists(d *ddlCtx, t *meta.Meta, schemaID int64, dbInfo *model.DBInfo) error { - // d.infoHandle maybe nil in some test. - if d.infoHandle == nil { - return checkSchemaNotExistsFromStore(t, schemaID, dbInfo) - } // Try to use memory schema info to check first. currVer, err := t.GetSchemaVersion() if err != nil { return err } - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() if is.SchemaMetaVersion() == currVer { return checkSchemaNotExistsFromInfoSchema(is, schemaID, dbInfo) } @@ -169,7 +165,7 @@ func onDropSchema(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) oldIDs := getIDs(tables) bundles := make([]*placement.Bundle, 0, len(oldIDs)+1) for _, ID := range append(oldIDs, dbInfo.ID) { - oldBundle, ok := d.infoHandle.Get().BundleByName(placement.GroupID(ID)) + oldBundle, ok := d.infoCache.GetLatest().BundleByName(placement.GroupID(ID)) if ok && !oldBundle.IsEmpty() { bundles = append(bundles, placement.BuildPlacementDropBundle(ID)) } diff --git a/ddl/schema_test.go b/ddl/schema_test.go index c70a0b793bb35..b4c8efee7b089 100644 --- a/ddl/schema_test.go +++ b/ddl/schema_test.go @@ -139,7 +139,7 @@ func (s *testSchemaSuite) TestSchema(c *C) { c.Assert(err, IsNil) }() ctx := testNewContext(d) - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_schema") // create a database. job := testCreateSchema(c, ctx, d, dbInfo) @@ -228,7 +228,7 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) { // d2 must not be owner. d2.ownerManager.RetireOwner() - dbInfo := testSchemaInfo(c, d2, "test") + dbInfo := testSchemaInfo(c, d2, "test_schema") testCreateSchema(c, ctx, d2, dbInfo) testCheckSchemaState(c, d2, dbInfo, model.StatePublic) diff --git a/ddl/stat_test.go b/ddl/stat_test.go index fe562a0ae0fb8..1ed3cbfe4c7fc 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -61,7 +61,7 @@ func (s *testSerialStatSuite) TestDDLStatsInfo(c *C) { c.Assert(err, IsNil) }() - dbInfo := testSchemaInfo(c, d, "test") + dbInfo := testSchemaInfo(c, d, "test_stat") testCreateSchema(c, testNewContext(d), d, dbInfo) tblInfo := testTableInfo(c, d, "t", 2) ctx := testNewContext(d) diff --git a/ddl/table.go b/ddl/table.go index 668de3ac41c05..424dd040a0de9 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -487,34 +487,32 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro } } - if d.infoHandle != nil && d.infoHandle.IsValid() { - is := d.infoHandle.Get() - - bundles := make([]*placement.Bundle, 0, len(oldPartitionIDs)+1) - if oldBundle, ok := is.BundleByName(placement.GroupID(tableID)); ok { - bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newTableID)) - } - - if pi := tblInfo.GetPartitionInfo(); pi != nil { - oldIDs := make([]int64, 0, len(oldPartitionIDs)) - newIDs := make([]int64, 0, len(oldPartitionIDs)) - newDefs := pi.Definitions - for i := range oldPartitionIDs { - newID := newDefs[i].ID - if oldBundle, ok := is.BundleByName(placement.GroupID(oldPartitionIDs[i])); ok && !oldBundle.IsEmpty() { - oldIDs = append(oldIDs, oldPartitionIDs[i]) - newIDs = append(newIDs, newID) - bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newID)) - } + is := d.infoCache.GetLatest() + + bundles := make([]*placement.Bundle, 0, len(oldPartitionIDs)+1) + if oldBundle, ok := is.BundleByName(placement.GroupID(tableID)); ok { + bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newTableID)) + } + + if pi := tblInfo.GetPartitionInfo(); pi != nil { + oldIDs := make([]int64, 0, len(oldPartitionIDs)) + newIDs := make([]int64, 0, len(oldPartitionIDs)) + newDefs := pi.Definitions + for i := range oldPartitionIDs { + newID := newDefs[i].ID + if oldBundle, ok := is.BundleByName(placement.GroupID(oldPartitionIDs[i])); ok && !oldBundle.IsEmpty() { + oldIDs = append(oldIDs, oldPartitionIDs[i]) + newIDs = append(newIDs, newID) + bundles = append(bundles, placement.BuildPlacementCopyBundle(oldBundle, newID)) } - job.CtxVars = []interface{}{oldIDs, newIDs} } + job.CtxVars = []interface{}{oldIDs, newIDs} + } - err = infosync.PutRuleBundles(context.TODO(), bundles) - if err != nil { - job.State = model.JobStateCancelled - return 0, errors.Wrapf(err, "failed to notify PD the placement rules") - } + err = infosync.PutRuleBundles(context.TODO(), bundles) + if err != nil { + job.State = model.JobStateCancelled + return 0, errors.Wrapf(err, "failed to notify PD the placement rules") } // Clear the tiflash replica available status. @@ -967,16 +965,12 @@ func onUpdateFlashReplicaStatus(t *meta.Meta, job *model.Job) (ver int64, _ erro } func checkTableNotExists(d *ddlCtx, t *meta.Meta, schemaID int64, tableName string) error { - // d.infoHandle maybe nil in some test. - if d.infoHandle == nil || !d.infoHandle.IsValid() { - return checkTableNotExistsFromStore(t, schemaID, tableName) - } // Try to use memory schema info to check first. currVer, err := t.GetSchemaVersion() if err != nil { return err } - is := d.infoHandle.Get() + is := d.infoCache.GetLatest() if is.SchemaMetaVersion() == currVer { return checkTableNotExistsFromInfoSchema(is, schemaID, tableName) } diff --git a/ddl/table_test.go b/ddl/table_test.go index 5760fc2b152b5..10927908f5289 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -355,7 +355,7 @@ func (s *testTableSuite) SetUpSuite(c *C) { WithLease(testLease), ) - s.dbInfo = testSchemaInfo(c, s.d, "test") + s.dbInfo = testSchemaInfo(c, s.d, "test_table") testCreateSchema(c, testNewContext(s.d), s.d, s.dbInfo) } diff --git a/ddl/util/syncer_test.go b/ddl/util/syncer_test.go index b552488ad49de..5a9d41d47e3b8 100644 --- a/ddl/util/syncer_test.go +++ b/ddl/util/syncer_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/parser/terror" . "github.com/pingcap/tidb/ddl" . "github.com/pingcap/tidb/ddl/util" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/store/mockstore" "go.etcd.io/etcd/clientv3" @@ -69,11 +70,14 @@ func TestSyncerSimple(t *testing.T) { defer clus.Terminate(t) cli := clus.RandClient() ctx := goctx.Background() + ic := infoschema.NewCache(2) + ic.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d := NewDDL( ctx, WithEtcdClient(cli), WithStore(store), WithLease(testLease), + WithInfoCache(ic), ) err = d.Start(nil) if err != nil { @@ -110,11 +114,14 @@ func TestSyncerSimple(t *testing.T) { t.Fatalf("client get global version result not match, err %v", err) } + ic2 := infoschema.NewCache(2) + ic2.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d1 := NewDDL( ctx, WithEtcdClient(cli), WithStore(store), WithLease(testLease), + WithInfoCache(ic2), ) err = d1.Start(nil) if err != nil { diff --git a/domain/domain.go b/domain/domain.go index f4b0ac8900f24..e6ea3d1e2d949 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -67,7 +67,7 @@ import ( // Multiple domains can be used in parallel without synchronization. type Domain struct { store kv.Storage - infoHandle *infoschema.Handle + infoCache *infoschema.InfoCache privHandle *privileges.Handle bindHandle *bindinfo.BindHandle statsHandle unsafe.Pointer @@ -92,78 +92,75 @@ type Domain struct { isLostConnectionToPD sync2.AtomicInt32 // !0: true, 0: false. } -// loadInfoSchema loads infoschema at startTS into handle, usedSchemaVersion is the currently used -// infoschema version, if it is the same as the schema version at startTS, we don't need to reload again. -// It returns the latest schema version, the changed table IDs, whether it's a full load and an error. -func (do *Domain) loadInfoSchema(handle *infoschema.Handle, usedSchemaVersion int64, - startTS uint64) (neededSchemaVersion int64, change *tikv.RelatedSchemaChange, fullLoad bool, err error) { +// loadInfoSchema loads infoschema at startTS. +// It returns: +// 1. the needed infoschema +// 2. cache hit indicator +// 3. currentSchemaVersion(before loading) +// 4. the changed table IDs if it is not full load +// 5. an error if any +func (do *Domain) loadInfoSchema(startTS uint64) (infoschema.InfoSchema, bool, int64, *tikv.RelatedSchemaChange, error) { snapshot := do.store.GetSnapshot(kv.NewVersion(startTS)) m := meta.NewSnapshotMeta(snapshot) - neededSchemaVersion, err = m.GetSchemaVersion() + neededSchemaVersion, err := m.GetSchemaVersion() if err != nil { - return 0, nil, fullLoad, err - } - if usedSchemaVersion != 0 && usedSchemaVersion == neededSchemaVersion { - return neededSchemaVersion, nil, fullLoad, nil + return nil, false, 0, nil, err } - // Update self schema version to etcd. - defer func() { - // There are two possibilities for not updating the self schema version to etcd. - // 1. Failed to loading schema information. - // 2. When users use history read feature, the neededSchemaVersion isn't the latest schema version. - if err != nil || neededSchemaVersion < do.InfoSchema().SchemaMetaVersion() { - logutil.BgLogger().Info("do not update self schema version to etcd", - zap.Int64("usedSchemaVersion", usedSchemaVersion), - zap.Int64("neededSchemaVersion", neededSchemaVersion), zap.Error(err)) - return - } + if is := do.infoCache.GetByVersion(neededSchemaVersion); is != nil { + return is, true, 0, nil, nil + } - err = do.ddl.SchemaSyncer().UpdateSelfVersion(context.Background(), neededSchemaVersion) - if err != nil { - logutil.BgLogger().Info("update self version failed", - zap.Int64("usedSchemaVersion", usedSchemaVersion), - zap.Int64("neededSchemaVersion", neededSchemaVersion), zap.Error(err)) - } - }() + currentSchemaVersion := int64(0) + if oldInfoSchema := do.infoCache.GetLatest(); oldInfoSchema != nil { + currentSchemaVersion = oldInfoSchema.SchemaMetaVersion() + } + // TODO: tryLoadSchemaDiffs has potential risks of failure. And it becomes worse in history reading cases. + // It is only kept because there is no alternative diff/partial loading solution. + // And it is only used to diff upgrading the current latest infoschema, if: + // 1. Not first time bootstrap loading, which needs a full load. + // 2. It is newer than the current one, so it will be "the current one" after this function call. + // 3. There are less 100 diffs. startTime := time.Now() - ok, relatedChanges, err := do.tryLoadSchemaDiffs(m, usedSchemaVersion, neededSchemaVersion) - if err != nil { + if currentSchemaVersion != 0 && neededSchemaVersion > currentSchemaVersion && neededSchemaVersion-currentSchemaVersion < 100 { + is, relatedChanges, err := do.tryLoadSchemaDiffs(m, currentSchemaVersion, neededSchemaVersion) + if err == nil { + do.infoCache.Insert(is) + logutil.BgLogger().Info("diff load InfoSchema success", + zap.Int64("currentSchemaVersion", currentSchemaVersion), + zap.Int64("neededSchemaVersion", neededSchemaVersion), + zap.Duration("start time", time.Since(startTime)), + zap.Int64s("phyTblIDs", relatedChanges.PhyTblIDS), + zap.Uint64s("actionTypes", relatedChanges.ActionTypes)) + return is, false, currentSchemaVersion, relatedChanges, nil + } // We can fall back to full load, don't need to return the error. logutil.BgLogger().Error("failed to load schema diff", zap.Error(err)) } - if ok { - logutil.BgLogger().Info("diff load InfoSchema success", - zap.Int64("usedSchemaVersion", usedSchemaVersion), - zap.Int64("neededSchemaVersion", neededSchemaVersion), - zap.Duration("start time", time.Since(startTime)), - zap.Int64s("phyTblIDs", relatedChanges.PhyTblIDS), - zap.Uint64s("actionTypes", relatedChanges.ActionTypes)) - return neededSchemaVersion, relatedChanges, fullLoad, nil - } - fullLoad = true schemas, err := do.fetchAllSchemasWithTables(m) if err != nil { - return 0, nil, fullLoad, err + return nil, false, currentSchemaVersion, nil, err } bundles, err := infosync.GetAllRuleBundles(context.TODO()) if err != nil { - return 0, nil, fullLoad, err + return nil, false, currentSchemaVersion, nil, err } - newISBuilder, err := infoschema.NewBuilder(handle).InitWithDBInfos(schemas, bundles, neededSchemaVersion) + newISBuilder, err := infoschema.NewBuilder(do.Store()).InitWithDBInfos(schemas, bundles, neededSchemaVersion) if err != nil { - return 0, nil, fullLoad, err + return nil, false, currentSchemaVersion, nil, err } logutil.BgLogger().Info("full load InfoSchema success", - zap.Int64("usedSchemaVersion", usedSchemaVersion), + zap.Int64("currentSchemaVersion", currentSchemaVersion), zap.Int64("neededSchemaVersion", neededSchemaVersion), zap.Duration("start time", time.Since(startTime))) - newISBuilder.Build() - return neededSchemaVersion, nil, fullLoad, nil + + is := newISBuilder.Build() + do.infoCache.Insert(is) + return is, false, currentSchemaVersion, nil, nil } func (do *Domain) fetchAllSchemasWithTables(m *meta.Meta) ([]*model.DBInfo, error) { @@ -238,48 +235,31 @@ func (do *Domain) fetchSchemasWithTables(schemas []*model.DBInfo, m *meta.Meta, done <- nil } -const ( - initialVersion = 0 - maxNumberOfDiffsToLoad = 100 -) - -func isTooOldSchema(usedVersion, newVersion int64) bool { - if usedVersion == initialVersion || newVersion-usedVersion > maxNumberOfDiffsToLoad { - return true - } - return false -} - // tryLoadSchemaDiffs tries to only load latest schema changes. // Return true if the schema is loaded successfully. // Return false if the schema can not be loaded by schema diff, then we need to do full load. // The second returned value is the delta updated table and partition IDs. -func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64) (bool, *tikv.RelatedSchemaChange, error) { - // If there isn't any used version, or used version is too old, we do full load. - // And when users use history read feature, we will set usedVersion to initialVersion, then full load is needed. - if isTooOldSchema(usedVersion, newVersion) { - return false, nil, nil - } +func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64) (infoschema.InfoSchema, *tikv.RelatedSchemaChange, error) { var diffs []*model.SchemaDiff for usedVersion < newVersion { usedVersion++ diff, err := m.GetSchemaDiff(usedVersion) if err != nil { - return false, nil, err + return nil, nil, err } if diff == nil { // If diff is missing for any version between used and new version, we fall back to full reload. - return false, nil, nil + return nil, nil, fmt.Errorf("failed to get schemadiff") } diffs = append(diffs, diff) } - builder := infoschema.NewBuilder(do.infoHandle).InitWithOldInfoSchema() + builder := infoschema.NewBuilder(do.Store()).InitWithOldInfoSchema(do.infoCache.GetLatest()) phyTblIDs := make([]int64, 0, len(diffs)) actions := make([]uint64, 0, len(diffs)) for _, diff := range diffs { IDs, err := builder.ApplyDiff(m, diff) if err != nil { - return false, nil, err + return nil, nil, err } if canSkipSchemaCheckerDDL(diff.Type) { continue @@ -289,11 +269,11 @@ func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64 actions = append(actions, uint64(1< staleVer { - return errors.New("schema version changed after the staleness startTS") - } // With START TRANSACTION, autocommit remains disabled until you end // the transaction with COMMIT or ROLLBACK. The autocommit mode then diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 493bda06c5de2..7cf235bd3c0f7 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -18,7 +18,6 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/store/tikv/oracle" @@ -26,12 +25,6 @@ import ( ) func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") - c.Assert(err, IsNil) - }() - testcases := []struct { name string preSQL string @@ -117,8 +110,6 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { } func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -155,7 +146,7 @@ func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", fmt.Sprintf(`return("%v")`, testcase.zone)) failpoint.Enable("github.com/pingcap/tidb/store/tikv/assertStoreLabels", fmt.Sprintf(`return("%v_%v")`, placement.DCLabelKey, testcase.txnScope)) failpoint.Enable("github.com/pingcap/tidb/store/tikv/assertStaleReadFlag", `return(true)`) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:20';`) + tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) tk.MustQuery(testcase.sql) tk.MustExec(`commit`) failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") @@ -165,12 +156,6 @@ func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { } func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") - c.Assert(err, IsNil) - }() - tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. @@ -193,62 +178,7 @@ func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { tk.MustExec("commit") } -func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { - testcases := []struct { - name string - sql string - expectErr error - }{ - { - name: "ddl change before stale txn", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:03'`, - expectErr: errors.New("schema version changed after the staleness startTS"), - }, - { - name: "ddl change before stale txn", - sql: fmt.Sprintf("START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '%v'", - time.Now().Truncate(3*time.Second).Format("2006-01-02 15:04:05")), - expectErr: errors.New(".*schema version changed after the staleness startTS.*"), - }, - { - name: "ddl change before stale txn", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:03'`, - expectErr: nil, - }, - } - tk := testkit.NewTestKitWithInit(c, s.store) - for _, testcase := range testcases { - check := func() { - if testcase.expectErr != nil { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(true)"), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") - c.Assert(err, IsNil) - }() - - } else { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer func() { - err := failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") - c.Assert(err, IsNil) - }() - - } - _, err := tk.Exec(testcase.sql) - if testcase.expectErr != nil { - c.Assert(err, NotNil) - c.Assert(err.Error(), Matches, testcase.expectErr.Error()) - } else { - c.Assert(err, IsNil) - } - } - check() - } -} - func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -318,3 +248,22 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { failpoint.Disable("github.com/pingcap/tidb/store/tikv/injectSafeTS") } } + +func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id int primary key);") + + schemaVer1 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() + time.Sleep(time.Second) + tk.MustExec("drop table if exists t") + schemaVer2 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() + // confirm schema changed + c.Assert(schemaVer1, Less, schemaVer2) + + tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:01'`) + schemaVer3 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() + // got an old infoSchema + c.Assert(schemaVer3, Equals, schemaVer1) +} diff --git a/infoschema/builder.go b/infoschema/builder.go index 28591d8679baf..88e8b71add319 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/table" @@ -35,8 +36,10 @@ import ( // Builder builds a new InfoSchema. type Builder struct { - is *infoSchema - handle *Handle + is *infoSchema + // TODO: store is only used by autoid allocators + // detach allocators from storage, use passed transaction in the feature + store kv.Storage } // ApplyDiff applies SchemaDiff to the new InfoSchema. @@ -352,14 +355,14 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i ConvertOldVersionUTF8ToUTF8MB4IfNeed(tblInfo) if len(allocs) == 0 { - allocs = autoid.NewAllocatorsFromTblInfo(b.handle.store, dbInfo.ID, tblInfo) + allocs = autoid.NewAllocatorsFromTblInfo(b.store, dbInfo.ID, tblInfo) } else { switch tp { case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType) + newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType) allocs = append(allocs, newAlloc) case model.ActionRebaseAutoRandomBase: - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) + newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) allocs = append(allocs, newAlloc) case model.ActionModifyColumn: // Change column attribute from auto_increment to auto_random. @@ -368,7 +371,7 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i allocs = allocs.Filter(func(a autoid.Allocator) bool { return a.GetType() != autoid.AutoIncrementType && a.GetType() != autoid.RowIDAllocType }) - newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) + newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) allocs = append(allocs, newAlloc) } } @@ -470,9 +473,14 @@ func (b *Builder) applyPlacementUpdate(id string) error { return nil } +// Build builds and returns the built infoschema. +func (b *Builder) Build() InfoSchema { + return b.is +} + // InitWithOldInfoSchema initializes an empty new InfoSchema by copies all the data from old InfoSchema. -func (b *Builder) InitWithOldInfoSchema() *Builder { - oldIS := b.handle.Get().(*infoSchema) +func (b *Builder) InitWithOldInfoSchema(oldSchema InfoSchema) *Builder { + oldIS := oldSchema.(*infoSchema) b.is.schemaMetaVersion = oldIS.schemaMetaVersion b.copySchemasMap(oldIS) b.copyBundlesMap(oldIS) @@ -549,7 +557,7 @@ func (b *Builder) createSchemaTablesForDB(di *model.DBInfo, tableFromMeta tableF b.is.schemaMap[di.Name.L] = schTbls for _, t := range di.Tables { - allocs := autoid.NewAllocatorsFromTblInfo(b.handle.store, di.ID, t) + allocs := autoid.NewAllocatorsFromTblInfo(b.store, di.ID, t) var tbl table.Table tbl, err := tableFromMeta(allocs, t) if err != nil { @@ -574,21 +582,16 @@ func RegisterVirtualTable(dbInfo *model.DBInfo, tableFromMeta tableFromMetaFunc) drivers = append(drivers, &virtualTableDriver{dbInfo, tableFromMeta}) } -// Build sets new InfoSchema to the handle in the Builder. -func (b *Builder) Build() { - b.handle.value.Store(b.is) -} - // NewBuilder creates a new Builder with a Handle. -func NewBuilder(handle *Handle) *Builder { - b := new(Builder) - b.handle = handle - b.is = &infoSchema{ - schemaMap: map[string]*schemaTables{}, - ruleBundleMap: map[string]*placement.Bundle{}, - sortedTablesBuckets: make([]sortedTables, bucketCount), +func NewBuilder(store kv.Storage) *Builder { + return &Builder{ + store: store, + is: &infoSchema{ + schemaMap: map[string]*schemaTables{}, + ruleBundleMap: map[string]*placement.Bundle{}, + sortedTablesBuckets: make([]sortedTables, bucketCount), + }, } - return b } func tableBucketIdx(tableID int64) int { diff --git a/infoschema/cache.go b/infoschema/cache.go new file mode 100644 index 0000000000000..4c3371b1bc354 --- /dev/null +++ b/infoschema/cache.go @@ -0,0 +1,95 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package infoschema + +import ( + "sort" + "sync" + + "github.com/pingcap/tidb/metrics" +) + +// InfoCache handles information schema, including getting and setting. +// The cache behavior, however, is transparent and under automatic management. +// It only promised to cache the infoschema, if it is newer than all the cached. +type InfoCache struct { + mu sync.RWMutex + // cache is sorted by SchemaVersion in descending order + cache []InfoSchema +} + +// NewCache creates a new InfoCache. +func NewCache(capcity int) *InfoCache { + return &InfoCache{cache: make([]InfoSchema, 0, capcity)} +} + +// GetLatest gets the newest information schema. +func (h *InfoCache) GetLatest() InfoSchema { + h.mu.RLock() + defer h.mu.RUnlock() + metrics.InfoCacheCounters.WithLabelValues("get").Inc() + if len(h.cache) > 0 { + metrics.InfoCacheCounters.WithLabelValues("hit").Inc() + return h.cache[0] + } + return nil +} + +// GetByVersion gets the information schema based on schemaVersion. Returns nil if it is not loaded. +func (h *InfoCache) GetByVersion(version int64) InfoSchema { + h.mu.RLock() + defer h.mu.RUnlock() + metrics.InfoCacheCounters.WithLabelValues("get").Inc() + i := sort.Search(len(h.cache), func(i int) bool { + return h.cache[i].SchemaMetaVersion() <= version + }) + if i < len(h.cache) && h.cache[i].SchemaMetaVersion() == version { + metrics.InfoCacheCounters.WithLabelValues("hit").Inc() + return h.cache[i] + } + return nil +} + +// Insert will **TRY** to insert the infoschema into the cache. +// It only promised to cache the newest infoschema. +// It returns 'true' if it is cached, 'false' otherwise. +func (h *InfoCache) Insert(is InfoSchema) bool { + h.mu.Lock() + defer h.mu.Unlock() + + version := is.SchemaMetaVersion() + i := sort.Search(len(h.cache), func(i int) bool { + return h.cache[i].SchemaMetaVersion() <= version + }) + + // cached entry + if i < len(h.cache) && h.cache[i].SchemaMetaVersion() == version { + return true + } + + if len(h.cache) < cap(h.cache) { + // has free space, grown the slice + h.cache = h.cache[:len(h.cache)+1] + copy(h.cache[i+1:], h.cache[i:]) + h.cache[i] = is + return true + } else if i < len(h.cache) { + // drop older schema + copy(h.cache[i+1:], h.cache[i:]) + h.cache[i] = is + return true + } + // older than all cached schemas, refuse to cache it + return false +} diff --git a/infoschema/cache_test.go b/infoschema/cache_test.go new file mode 100644 index 0000000000000..a8e9ddcc0df5a --- /dev/null +++ b/infoschema/cache_test.go @@ -0,0 +1,119 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package infoschema_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/infoschema" +) + +var _ = Suite(&testInfoCacheSuite{}) + +type testInfoCacheSuite struct { +} + +func (s *testInfoCacheSuite) TestNewCache(c *C) { + ic := infoschema.NewCache(16) + c.Assert(ic, NotNil) +} + +func (s *testInfoCacheSuite) TestInsert(c *C) { + ic := infoschema.NewCache(3) + c.Assert(ic, NotNil) + + is2 := infoschema.MockInfoSchemaWithSchemaVer(nil, 2) + ic.Insert(is2) + c.Assert(ic.GetByVersion(2), NotNil) + + // newer + is5 := infoschema.MockInfoSchemaWithSchemaVer(nil, 5) + ic.Insert(is5) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(2), NotNil) + + // older + is0 := infoschema.MockInfoSchemaWithSchemaVer(nil, 0) + ic.Insert(is0) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(2), NotNil) + c.Assert(ic.GetByVersion(0), NotNil) + + // replace 5, drop 0 + is6 := infoschema.MockInfoSchemaWithSchemaVer(nil, 6) + ic.Insert(is6) + c.Assert(ic.GetByVersion(6), NotNil) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(2), NotNil) + c.Assert(ic.GetByVersion(0), IsNil) + + // replace 2, drop 2 + is3 := infoschema.MockInfoSchemaWithSchemaVer(nil, 3) + ic.Insert(is3) + c.Assert(ic.GetByVersion(6), NotNil) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(3), NotNil) + c.Assert(ic.GetByVersion(2), IsNil) + c.Assert(ic.GetByVersion(0), IsNil) + + // insert 2, but failed silently + ic.Insert(is2) + c.Assert(ic.GetByVersion(6), NotNil) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(3), NotNil) + c.Assert(ic.GetByVersion(2), IsNil) + c.Assert(ic.GetByVersion(0), IsNil) + + // insert 5, but it is already in + ic.Insert(is5) + c.Assert(ic.GetByVersion(6), NotNil) + c.Assert(ic.GetByVersion(5), NotNil) + c.Assert(ic.GetByVersion(3), NotNil) + c.Assert(ic.GetByVersion(2), IsNil) + c.Assert(ic.GetByVersion(0), IsNil) +} + +func (s *testInfoCacheSuite) TestGetByVersion(c *C) { + ic := infoschema.NewCache(2) + c.Assert(ic, NotNil) + is1 := infoschema.MockInfoSchemaWithSchemaVer(nil, 1) + ic.Insert(is1) + is3 := infoschema.MockInfoSchemaWithSchemaVer(nil, 3) + ic.Insert(is3) + + c.Assert(ic.GetByVersion(1), Equals, is1) + c.Assert(ic.GetByVersion(3), Equals, is3) + c.Assert(ic.GetByVersion(0), IsNil, Commentf("index == 0, but not found")) + c.Assert(ic.GetByVersion(2), IsNil, Commentf("index in the middle, but not found")) + c.Assert(ic.GetByVersion(4), IsNil, Commentf("index == length, but not found")) +} + +func (s *testInfoCacheSuite) TestGetLatest(c *C) { + ic := infoschema.NewCache(16) + c.Assert(ic, NotNil) + c.Assert(ic.GetLatest(), IsNil) + + is1 := infoschema.MockInfoSchemaWithSchemaVer(nil, 1) + ic.Insert(is1) + c.Assert(ic.GetLatest(), Equals, is1) + + // newer change the newest + is2 := infoschema.MockInfoSchemaWithSchemaVer(nil, 2) + ic.Insert(is2) + c.Assert(ic.GetLatest(), Equals, is2) + + // older schema doesn't change the newest + is0 := infoschema.MockInfoSchemaWithSchemaVer(nil, 0) + ic.Insert(is0) + c.Assert(ic.GetLatest(), Equals, is2) +} diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index ac8afd14605f1..2494e89b4d57f 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -17,12 +17,10 @@ import ( "fmt" "sort" "sync" - "sync/atomic" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/ddl/placement" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util" @@ -312,40 +310,6 @@ func (is *infoSchema) SequenceByName(schema, sequence model.CIStr) (util.Sequenc return tbl.(util.SequenceTable), nil } -// Handle handles information schema, including getting and setting. -type Handle struct { - value atomic.Value - store kv.Storage -} - -// NewHandle creates a new Handle. -func NewHandle(store kv.Storage) *Handle { - h := &Handle{ - store: store, - } - return h -} - -// Get gets information schema from Handle. -func (h *Handle) Get() InfoSchema { - v := h.value.Load() - schema, _ := v.(InfoSchema) - return schema -} - -// IsValid uses to check whether handle value is valid. -func (h *Handle) IsValid() bool { - return h.value.Load() != nil -} - -// EmptyClone creates a new Handle with the same store and memSchema, but the value is not set. -func (h *Handle) EmptyClone() *Handle { - newHandle := &Handle{ - store: h.store, - } - return newHandle -} - func init() { // Initialize the information shema database and register the driver to `drivers` dbID := autoid.InformationSchemaDBID diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index 6aa0c5526f467..87276ef1452b9 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -15,7 +15,6 @@ package infoschema_test import ( "context" - "sync" "testing" . "github.com/pingcap/check" @@ -57,7 +56,6 @@ func (*testSuite) TestT(c *C) { c.Assert(err, IsNil) defer dom.Close() - handle := infoschema.NewHandle(store) dbName := model.NewCIStr("Test") tbName := model.NewCIStr("T") colName := model.NewCIStr("A") @@ -116,7 +114,7 @@ func (*testSuite) TestT(c *C) { }) c.Assert(err, IsNil) - builder, err := infoschema.NewBuilder(handle).InitWithDBInfos(dbInfos, nil, 1) + builder, err := infoschema.NewBuilder(dom.Store()).InitWithDBInfos(dbInfos, nil, 1) c.Assert(err, IsNil) txn, err := store.Begin() @@ -126,8 +124,7 @@ func (*testSuite) TestT(c *C) { err = txn.Rollback() c.Assert(err, IsNil) - builder.Build() - is := handle.Get() + is := builder.Build() schemaNames := is.AllSchemaNames() c.Assert(schemaNames, HasLen, 4) @@ -213,14 +210,10 @@ func (*testSuite) TestT(c *C) { c.Assert(err, IsNil) err = txn.Rollback() c.Assert(err, IsNil) - builder.Build() - is = handle.Get() + is = builder.Build() schema, ok = is.SchemaByID(dbID) c.Assert(ok, IsTrue) c.Assert(len(schema.Tables), Equals, 1) - - emptyHandle := handle.EmptyClone() - c.Assert(emptyHandle.Get(), IsNil) } func (testSuite) TestMockInfoSchema(c *C) { @@ -258,32 +251,6 @@ func checkApplyCreateNonExistsTableDoesNotPanic(c *C, txn kv.Transaction, builde c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) } -// TestConcurrent makes sure it is safe to concurrently create handle on multiple stores. -func (testSuite) TestConcurrent(c *C) { - defer testleak.AfterTest(c)() - storeCount := 5 - stores := make([]kv.Storage, storeCount) - for i := 0; i < storeCount; i++ { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - stores[i] = store - } - defer func() { - for _, store := range stores { - store.Close() - } - }() - var wg sync.WaitGroup - wg.Add(storeCount) - for _, store := range stores { - go func(s kv.Storage) { - defer wg.Done() - _ = infoschema.NewHandle(s) - }(store) - } - wg.Wait() -} - // TestInfoTables makes sure that all tables of information_schema could be found in infoschema handle. func (*testSuite) TestInfoTables(c *C) { defer testleak.AfterTest(c)() @@ -293,12 +260,10 @@ func (*testSuite) TestInfoTables(c *C) { err := store.Close() c.Assert(err, IsNil) }() - handle := infoschema.NewHandle(store) - builder, err := infoschema.NewBuilder(handle).InitWithDBInfos(nil, nil, 0) + + builder, err := infoschema.NewBuilder(store).InitWithDBInfos(nil, nil, 0) c.Assert(err, IsNil) - builder.Build() - is := handle.Get() - c.Assert(is, NotNil) + is := builder.Build() infoTables := []string{ "SCHEMATA", @@ -360,12 +325,9 @@ func (*testSuite) TestGetBundle(c *C) { c.Assert(err, IsNil) }() - handle := infoschema.NewHandle(store) - builder, err := infoschema.NewBuilder(handle).InitWithDBInfos(nil, nil, 0) + builder, err := infoschema.NewBuilder(store).InitWithDBInfos(nil, nil, 0) c.Assert(err, IsNil) - builder.Build() - - is := handle.Get() + is := builder.Build() bundle := &placement.Bundle{ ID: placement.PDBundleID, diff --git a/metrics/domain.go b/metrics/domain.go index dd3912555d59c..f30dbd59e5d32 100644 --- a/metrics/domain.go +++ b/metrics/domain.go @@ -38,6 +38,19 @@ var ( Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), // 1ms ~ 524s }) + // InfoCacheCounters are the counters of get/hit. + InfoCacheCounters = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "domain", + Name: "infocache_counters", + Help: "Counters of infoCache: get/hit.", + }, []string{LblType}) + // InfoCacheCounterGet is the total number of getting entry. + InfoCacheCounterGet = "get" + // InfoCacheCounterHit is the cache hit numbers for get. + InfoCacheCounterHit = "hit" + // LoadPrivilegeCounter records the counter of load privilege. LoadPrivilegeCounter = prometheus.NewCounterVec( prometheus.CounterOpts{ diff --git a/metrics/metrics.go b/metrics/metrics.go index ff2ac3b1aa08d..4a879b5d5423c 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -98,6 +98,7 @@ func RegisterMetrics() { prometheus.MustRegister(JobsGauge) prometheus.MustRegister(KeepAliveCounter) prometheus.MustRegister(LoadPrivilegeCounter) + prometheus.MustRegister(InfoCacheCounters) prometheus.MustRegister(LoadSchemaCounter) prometheus.MustRegister(LoadSchemaDuration) prometheus.MustRegister(MetaHistogram) diff --git a/owner/manager_test.go b/owner/manager_test.go index e25b204e6bbb4..e239419057291 100644 --- a/owner/manager_test.go +++ b/owner/manager_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/terror" . "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/logutil" @@ -72,11 +73,14 @@ func TestSingle(t *testing.T) { defer clus.Terminate(t) cli := clus.RandClient() ctx := goctx.Background() + ic := infoschema.NewCache(2) + ic.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d := NewDDL( ctx, WithEtcdClient(cli), WithStore(store), WithLease(testLease), + WithInfoCache(ic), ) err = d.Start(nil) if err != nil { @@ -142,11 +146,14 @@ func TestCluster(t *testing.T) { defer clus.Terminate(t) cli := clus.Client(0) + ic := infoschema.NewCache(2) + ic.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d := NewDDL( goctx.Background(), WithEtcdClient(cli), WithStore(store), WithLease(testLease), + WithInfoCache(ic), ) err = d.Start(nil) if err != nil { @@ -157,11 +164,14 @@ func TestCluster(t *testing.T) { t.Fatalf("expect true, got isOwner:%v", isOwner) } cli1 := clus.Client(1) + ic2 := infoschema.NewCache(2) + ic2.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d1 := NewDDL( goctx.Background(), WithEtcdClient(cli1), WithStore(store), WithLease(testLease), + WithInfoCache(ic2), ) err = d1.Start(nil) if err != nil { @@ -189,11 +199,14 @@ func TestCluster(t *testing.T) { // d3 (not owner) stop cli3 := clus.Client(3) + ic3 := infoschema.NewCache(2) + ic3.Insert(infoschema.MockInfoSchemaWithSchemaVer(nil, 0)) d3 := NewDDL( goctx.Background(), WithEtcdClient(cli3), WithStore(store), WithLease(testLease), + WithInfoCache(ic3), ) err = d3.Start(nil) if err != nil { diff --git a/session/session.go b/session/session.go index af3f41c863dc0..84ff7e4eec424 100644 --- a/session/session.go +++ b/session/session.go @@ -1935,7 +1935,7 @@ func (s *session) isTxnRetryable() bool { func (s *session) NewTxn(ctx context.Context) error { if s.txn.Valid() { - txnID := s.txn.StartTS() + txnStartTS := s.txn.StartTS() txnScope := s.GetSessionVars().TxnCtx.TxnScope err := s.CommitTxn(ctx) if err != nil { @@ -1944,7 +1944,7 @@ func (s *session) NewTxn(ctx context.Context) error { vars := s.GetSessionVars() logutil.Logger(ctx).Info("NewTxn() inside a transaction auto commit", zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), - zap.Uint64("txnStartTS", txnID), + zap.Uint64("txnStartTS", txnStartTS), zap.String("txnScope", txnScope)) } @@ -2805,7 +2805,10 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc txn.SetOption(kv.IsStalenessReadOnly, true) txn.SetOption(kv.TxnScope, txnScope) s.txn.changeInvalidToValid(txn) - is := domain.GetDomain(s).InfoSchema() + is, err := domain.GetDomain(s).GetSnapshotInfoSchema(txn.StartTS()) + if err != nil { + return errors.Trace(err) + } s.sessionVars.TxnCtx = &variable.TransactionContext{ InfoSchema: is, CreateTime: time.Now(), diff --git a/session/session_test.go b/session/session_test.go index 4870215f33c9e..a897cb7db07f3 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -3903,9 +3903,7 @@ func (s *testSessionSerialSuite) TestIssue21943(c *C) { c.Assert(err.Error(), Equals, "[variable:1238]Variable 'last_plan_from_cache' is a read only variable") } -func (s *testSessionSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") +func (s *testSessionSerialSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { testcases := []struct { name string sql string @@ -4036,7 +4034,7 @@ func (s *testSessionSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { tk.MustExec(`set @@tidb_enable_noop_functions=1;`) for _, testcase := range testcases { c.Log(testcase.name) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`) + tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) if testcase.isValidate { _, err := tk.Exec(testcase.sql) c.Assert(err, IsNil) @@ -4050,8 +4048,6 @@ func (s *testSessionSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { } func (s *testSessionSerialSuite) TestSpecialSQLInStalenessTxn(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer", "return(false)"), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/executor/mockStalenessTxnSchemaVer") tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") testcases := []struct { @@ -4098,7 +4094,7 @@ func (s *testSessionSerialSuite) TestSpecialSQLInStalenessTxn(c *C) { tk.MustExec("CREATE USER 'newuser' IDENTIFIED BY 'mypassword';") for _, testcase := range testcases { comment := Commentf(testcase.name) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`) + tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, true, comment) tk.MustExec(testcase.sql) c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.sameSession, comment) From d641810f1ade5dcb5c29dd09ce0e0e2e87e28127 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Mon, 17 May 2021 21:49:39 +0800 Subject: [PATCH 087/343] session: test that temporary tables will also be retried (#24505) --- session/session.go | 50 ++++++++++++++++++++++++++--------------- session/session_test.go | 43 +++++++++++++++++++++++++++++++++++ 2 files changed, 75 insertions(+), 18 deletions(-) diff --git a/session/session.go b/session/session.go index 84ff7e4eec424..e1581d5ed4074 100644 --- a/session/session.go +++ b/session/session.go @@ -473,6 +473,9 @@ func (s *session) doCommit(ctx context.Context) error { if err != nil { return err } + if err = s.removeTempTableFromBuffer(); err != nil { + return err + } // mockCommitError and mockGetTSErrorInRetry use to test PR #8743. failpoint.Inject("mockCommitError", func(val failpoint.Value) { @@ -526,29 +529,40 @@ func (s *session) doCommit(ctx context.Context) error { s.GetSessionVars().TxnCtx.IsExplicit && s.GetSessionVars().GuaranteeLinearizability) } - // Filter out the temporary table key-values. - if tables := s.sessionVars.TxnCtx.GlobalTemporaryTables; tables != nil { - memBuffer := s.txn.GetMemBuffer() - for tid := range tables { - seekKey := tablecodec.EncodeTablePrefix(tid) - endKey := tablecodec.EncodeTablePrefix(tid + 1) - iter, err := memBuffer.Iter(seekKey, endKey) - if err != nil { + return s.txn.Commit(tikvutil.SetSessionID(ctx, s.GetSessionVars().ConnectionID)) +} + +// removeTempTableFromBuffer filters out the temporary table key-values. +func (s *session) removeTempTableFromBuffer() error { + tables := s.GetSessionVars().TxnCtx.GlobalTemporaryTables + if len(tables) == 0 { + return nil + } + memBuffer := s.txn.GetMemBuffer() + // Reset and new an empty stage buffer. + defer func() { + s.txn.cleanup() + }() + for tid := range tables { + seekKey := tablecodec.EncodeTablePrefix(tid) + endKey := tablecodec.EncodeTablePrefix(tid + 1) + iter, err := memBuffer.Iter(seekKey, endKey) + if err != nil { + return err + } + for iter.Valid() && iter.Key().HasPrefix(seekKey) { + if err = memBuffer.Delete(iter.Key()); err != nil { return err } - for iter.Valid() && iter.Key().HasPrefix(seekKey) { - if err = memBuffer.Delete(iter.Key()); err != nil { - return errors.Trace(err) - } - s.txn.UpdateEntriesCountAndSize() - if err = iter.Next(); err != nil { - return errors.Trace(err) - } + s.txn.UpdateEntriesCountAndSize() + if err = iter.Next(); err != nil { + return err } } } - - return s.txn.Commit(tikvutil.SetSessionID(ctx, s.GetSessionVars().ConnectionID)) + // Flush to the root membuffer. + s.txn.flushStmtBuf() + return nil } // errIsNoisy is used to filter DUPLCATE KEY errors. diff --git a/session/session_test.go b/session/session_test.go index a897cb7db07f3..b7cfecc9c5f4c 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -789,6 +789,49 @@ func (s *testSessionSuite) TestRetryUnion(c *C) { c.Assert(err, ErrorMatches, ".*can not retry select for update statement") } +func (s *testSessionSuite) TestRetryGlobalTempTable(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists normal_table") + tk.MustExec("create table normal_table(a int primary key, b int)") + defer tk.MustExec("drop table if exists normal_table") + tk.MustExec("drop table if exists temp_table") + tk.MustExec("create global temporary table temp_table(a int primary key, b int) on commit delete rows") + defer tk.MustExec("drop table if exists temp_table") + + // insert select + tk.MustExec("set tidb_disable_txn_auto_retry = 0") + tk.MustExec("insert normal_table value(100, 100)") + tk.MustExec("set @@autocommit = 0") + // used to make conflicts + tk.MustExec("update normal_table set b=b+1 where a=100") + tk.MustExec("insert temp_table value(1, 1)") + tk.MustExec("insert normal_table select * from temp_table") + c.Assert(session.GetHistory(tk.Se).Count(), Equals, 3) + + // try to conflict with tk + tk1 := testkit.NewTestKitWithInit(c, s.store) + tk1.MustExec("update normal_table set b=b+1 where a=100") + + // It will retry internally. + tk.MustExec("commit") + tk.MustQuery("select a, b from normal_table order by a").Check(testkit.Rows("1 1", "100 102")) + tk.MustQuery("select a, b from temp_table order by a").Check(testkit.Rows()) + + // update multi-tables + tk.MustExec("update normal_table set b=b+1 where a=100") + tk.MustExec("insert temp_table value(1, 2)") + // before update: normal_table=(1 1) (100 102), temp_table=(1 2) + tk.MustExec("update normal_table, temp_table set normal_table.b=temp_table.b where normal_table.a=temp_table.a") + c.Assert(session.GetHistory(tk.Se).Count(), Equals, 3) + + // try to conflict with tk + tk1.MustExec("update normal_table set b=b+1 where a=100") + + // It will retry internally. + tk.MustExec("commit") + tk.MustQuery("select a, b from normal_table order by a").Check(testkit.Rows("1 2", "100 104")) +} + func (s *testSessionSuite) TestRetryShow(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("set @@autocommit = 0") From 0f10bef470f45bd862b38592f51ecb92a540896a Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 17 May 2021 08:13:39 -0600 Subject: [PATCH 088/343] domain, session: Add new sysvarcache to replace global values cache (#24359) --- cmd/explaintest/main.go | 3 - ddl/db_change_test.go | 2 - domain/domain.go | 73 ++++++++- domain/global_vars_cache.go | 135 ---------------- domain/global_vars_cache_test.go | 221 -------------------------- domain/sysvar_cache.go | 167 +++++++++++++++++++ executor/executor_test.go | 2 - executor/seqtest/seq_executor_test.go | 2 - infoschema/tables_test.go | 6 - metrics/domain.go | 9 ++ metrics/metrics.go | 1 + planner/core/prepare_test.go | 3 - session/session.go | 88 +++++----- session/session_test.go | 6 - sessionctx/variable/session.go | 9 ++ sessionctx/variable/sysvar.go | 54 +++++-- 16 files changed, 343 insertions(+), 438 deletions(-) delete mode 100644 domain/global_vars_cache.go delete mode 100644 domain/global_vars_cache_test.go create mode 100644 domain/sysvar_cache.go diff --git a/cmd/explaintest/main.go b/cmd/explaintest/main.go index fa5265f7af871..a85c8ce82dd3c 100644 --- a/cmd/explaintest/main.go +++ b/cmd/explaintest/main.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/parser/ast" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/logutil" @@ -663,8 +662,6 @@ func main() { log.Fatal(fmt.Sprintf("%s failed", sql), zap.Error(err)) } } - // Wait global variables to reload. - time.Sleep(domain.GlobalVariableCacheExpiry) if _, err = mdb.Exec("set sql_mode='STRICT_TRANS_TABLES'"); err != nil { log.Fatal("set sql_mode='STRICT_TRANS_TABLES' failed", zap.Error(err)) diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 7c3a0f9ad970f..041f35c7734a8 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -1063,7 +1063,6 @@ func (s *testStateChangeSuite) TestParallelAddGeneratedColumnAndAlterModifyColum _, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0") c.Assert(err, IsNil) }() - domain.GetDomain(s.se).GetGlobalVarsCache().Disable() sql1 := "ALTER TABLE t ADD COLUMN f INT GENERATED ALWAYS AS(a+1);" sql2 := "ALTER TABLE t MODIFY COLUMN a tinyint;" @@ -1083,7 +1082,6 @@ func (s *testStateChangeSuite) TestParallelAlterModifyColumnAndAddPK(c *C) { _, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0") c.Assert(err, IsNil) }() - domain.GetDomain(s.se).GetGlobalVarsCache().Disable() sql1 := "ALTER TABLE t ADD PRIMARY KEY (b) NONCLUSTERED;" sql2 := "ALTER TABLE t MODIFY COLUMN b tinyint;" diff --git a/domain/domain.go b/domain/domain.go index e6ea3d1e2d949..44f6df1aa9086 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -79,7 +79,7 @@ type Domain struct { sysSessionPool *sessionPool exit chan struct{} etcdClient *clientv3.Client - gvc GlobalVariableCache + sysVarCache SysVarCache // replaces GlobalVariableCache slowQuery *topNSlowQueries expensiveQueryHandle *expensivequery.Handle wg sync.WaitGroup @@ -900,6 +900,55 @@ func (do *Domain) LoadPrivilegeLoop(ctx sessionctx.Context) error { return nil } +// LoadSysVarCacheLoop create a goroutine loads sysvar cache in a loop, +// it should be called only once in BootstrapSession. +func (do *Domain) LoadSysVarCacheLoop(ctx sessionctx.Context) error { + err := do.sysVarCache.RebuildSysVarCache(ctx) + if err != nil { + return err + } + var watchCh clientv3.WatchChan + duration := 30 * time.Second + if do.etcdClient != nil { + watchCh = do.etcdClient.Watch(context.Background(), sysVarCacheKey) + } + do.wg.Add(1) + go func() { + defer func() { + do.wg.Done() + logutil.BgLogger().Info("LoadSysVarCacheLoop exited.") + util.Recover(metrics.LabelDomain, "LoadSysVarCacheLoop", nil, false) + }() + var count int + for { + ok := true + select { + case <-do.exit: + return + case _, ok = <-watchCh: + case <-time.After(duration): + } + if !ok { + logutil.BgLogger().Error("LoadSysVarCacheLoop loop watch channel closed") + watchCh = do.etcdClient.Watch(context.Background(), sysVarCacheKey) + count++ + if count > 10 { + time.Sleep(time.Duration(count) * time.Second) + } + continue + } + count = 0 + logutil.BgLogger().Debug("Rebuilding sysvar cache from etcd watch event.") + err := do.sysVarCache.RebuildSysVarCache(ctx) + metrics.LoadSysVarCacheCounter.WithLabelValues(metrics.RetLabel(err)).Inc() + if err != nil { + logutil.BgLogger().Error("LoadSysVarCacheLoop failed", zap.Error(err)) + } + } + }() + return nil +} + // PrivilegeHandle returns the MySQLPrivilege. func (do *Domain) PrivilegeHandle() *privileges.Handle { return do.privHandle @@ -1278,7 +1327,10 @@ func (do *Domain) ExpensiveQueryHandle() *expensivequery.Handle { return do.expensiveQueryHandle } -const privilegeKey = "/tidb/privilege" +const ( + privilegeKey = "/tidb/privilege" + sysVarCacheKey = "/tidb/sysvars" +) // NotifyUpdatePrivilege updates privilege key in etcd, TiDB client that watches // the key will get notification. @@ -1300,6 +1352,23 @@ func (do *Domain) NotifyUpdatePrivilege(ctx sessionctx.Context) { } } +// NotifyUpdateSysVarCache updates the sysvar cache key in etcd, which other TiDB +// clients are subscribed to for updates. For the caller, the cache is also built +// synchronously so that the effect is immediate. +func (do *Domain) NotifyUpdateSysVarCache(ctx sessionctx.Context) { + if do.etcdClient != nil { + row := do.etcdClient.KV + _, err := row.Put(context.Background(), sysVarCacheKey, "") + if err != nil { + logutil.BgLogger().Warn("notify update sysvar cache failed", zap.Error(err)) + } + } + // update locally + if err := do.sysVarCache.RebuildSysVarCache(ctx); err != nil { + logutil.BgLogger().Error("rebuilding sysvar cache failed", zap.Error(err)) + } +} + // ServerID gets serverID. func (do *Domain) ServerID() uint64 { return atomic.LoadUint64(&do.serverID) diff --git a/domain/global_vars_cache.go b/domain/global_vars_cache.go deleted file mode 100644 index 52aa12a5ac955..0000000000000 --- a/domain/global_vars_cache.go +++ /dev/null @@ -1,135 +0,0 @@ -// Copyright 2018 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package domain - -import ( - "fmt" - "sync" - "time" - - "github.com/pingcap/parser/ast" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/stmtsummary" - "go.uber.org/zap" - "golang.org/x/sync/singleflight" -) - -// GlobalVariableCache caches global variables. -type GlobalVariableCache struct { - sync.RWMutex - lastModify time.Time - rows []chunk.Row - fields []*ast.ResultField - - // Unit test may like to disable it. - disable bool - SingleFight singleflight.Group -} - -// GlobalVariableCacheExpiry is the global variable cache TTL. -const GlobalVariableCacheExpiry = 2 * time.Second - -// Update updates the global variable cache. -func (gvc *GlobalVariableCache) Update(rows []chunk.Row, fields []*ast.ResultField) { - gvc.Lock() - gvc.lastModify = time.Now() - gvc.rows = rows - gvc.fields = fields - gvc.Unlock() - - checkEnableServerGlobalVar(rows) -} - -// Get gets the global variables from cache. -func (gvc *GlobalVariableCache) Get() (succ bool, rows []chunk.Row, fields []*ast.ResultField) { - gvc.RLock() - defer gvc.RUnlock() - if time.Since(gvc.lastModify) < GlobalVariableCacheExpiry { - succ, rows, fields = !gvc.disable, gvc.rows, gvc.fields - return - } - succ = false - return -} - -type loadResult struct { - rows []chunk.Row - fields []*ast.ResultField -} - -// LoadGlobalVariables will load from global cache first, loadFn will be executed if cache is not valid -func (gvc *GlobalVariableCache) LoadGlobalVariables(loadFn func() ([]chunk.Row, []*ast.ResultField, error)) ([]chunk.Row, []*ast.ResultField, error) { - succ, rows, fields := gvc.Get() - if succ { - return rows, fields, nil - } - fn := func() (interface{}, error) { - resRows, resFields, loadErr := loadFn() - if loadErr != nil { - return nil, loadErr - } - gvc.Update(resRows, resFields) - return &loadResult{resRows, resFields}, nil - } - res, err, _ := gvc.SingleFight.Do("loadGlobalVariable", fn) - if err != nil { - return nil, nil, err - } - loadRes := res.(*loadResult) - return loadRes.rows, loadRes.fields, nil -} - -// Disable disables the global variable cache, used in test only. -func (gvc *GlobalVariableCache) Disable() { - gvc.Lock() - defer gvc.Unlock() - gvc.disable = true -} - -// checkEnableServerGlobalVar processes variables that acts in server and global level. -func checkEnableServerGlobalVar(rows []chunk.Row) { - for _, row := range rows { - sVal := "" - if !row.IsNull(1) { - sVal = row.GetString(1) - } - var err error - switch row.GetString(0) { - case variable.TiDBEnableStmtSummary: - err = stmtsummary.StmtSummaryByDigestMap.SetEnabled(sVal, false) - case variable.TiDBStmtSummaryInternalQuery: - err = stmtsummary.StmtSummaryByDigestMap.SetEnabledInternalQuery(sVal, false) - case variable.TiDBStmtSummaryRefreshInterval: - err = stmtsummary.StmtSummaryByDigestMap.SetRefreshInterval(sVal, false) - case variable.TiDBStmtSummaryHistorySize: - err = stmtsummary.StmtSummaryByDigestMap.SetHistorySize(sVal, false) - case variable.TiDBStmtSummaryMaxStmtCount: - err = stmtsummary.StmtSummaryByDigestMap.SetMaxStmtCount(sVal, false) - case variable.TiDBStmtSummaryMaxSQLLength: - err = stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength(sVal, false) - case variable.TiDBCapturePlanBaseline: - variable.CapturePlanBaseline.Set(sVal, false) - } - if err != nil { - logutil.BgLogger().Error(fmt.Sprintf("load global variable %s error", row.GetString(0)), zap.Error(err)) - } - } -} - -// GetGlobalVarsCache gets the global variable cache. -func (do *Domain) GetGlobalVarsCache() *GlobalVariableCache { - return &do.gvc -} diff --git a/domain/global_vars_cache_test.go b/domain/global_vars_cache_test.go deleted file mode 100644 index 7358d709986af..0000000000000 --- a/domain/global_vars_cache_test.go +++ /dev/null @@ -1,221 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package domain - -import ( - "sync" - "sync/atomic" - "time" - - . "github.com/pingcap/check" - "github.com/pingcap/parser/ast" - "github.com/pingcap/parser/charset" - "github.com/pingcap/parser/model" - "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/stmtsummary" - "github.com/pingcap/tidb/util/testleak" -) - -var _ = SerialSuites(&testGVCSuite{}) - -type testGVCSuite struct{} - -func (gvcSuite *testGVCSuite) TestSimple(c *C) { - defer testleak.AfterTest(c)() - testleak.BeforeTest() - - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - defer func() { - err := store.Close() - c.Assert(err, IsNil) - }() - ddlLease := 50 * time.Millisecond - dom := NewDomain(store, ddlLease, 0, 0, mockFactory) - err = dom.Init(ddlLease, sysMockFactory) - c.Assert(err, IsNil) - defer dom.Close() - - // Get empty global vars cache. - gvc := dom.GetGlobalVarsCache() - succ, rows, fields := gvc.Get() - c.Assert(succ, IsFalse) - c.Assert(rows, IsNil) - c.Assert(fields, IsNil) - // Get a variable from global vars cache. - rf := getResultField("c", 1, 0) - rf1 := getResultField("c1", 2, 1) - ft := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - ft1 := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - ck := chunk.NewChunkWithCapacity([]*types.FieldType{ft, ft1}, 1024) - ck.AppendString(0, "variable1") - ck.AppendString(1, "value1") - row := ck.GetRow(0) - gvc.Update([]chunk.Row{row}, []*ast.ResultField{rf, rf1}) - succ, rows, fields = gvc.Get() - c.Assert(succ, IsTrue) - c.Assert(rows[0], Equals, row) - c.Assert(fields, DeepEquals, []*ast.ResultField{rf, rf1}) - // Disable the cache. - gvc.Disable() - succ, rows, fields = gvc.Get() - c.Assert(succ, IsFalse) - c.Assert(rows[0], Equals, row) - c.Assert(fields, DeepEquals, []*ast.ResultField{rf, rf1}) -} - -func getResultField(colName string, id, offset int) *ast.ResultField { - return &ast.ResultField{ - Column: &model.ColumnInfo{ - Name: model.NewCIStr(colName), - ID: int64(id), - Offset: offset, - FieldType: types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetUTF8, - Collate: charset.CollationUTF8, - }, - }, - TableAsName: model.NewCIStr("tbl"), - DBName: model.NewCIStr("test"), - } -} - -func (gvcSuite *testGVCSuite) TestConcurrentOneFlight(c *C) { - defer testleak.AfterTest(c)() - testleak.BeforeTest() - gvc := &GlobalVariableCache{} - succ, rows, fields := gvc.Get() - c.Assert(succ, IsFalse) - c.Assert(rows, IsNil) - c.Assert(fields, IsNil) - - // Get a variable from global vars cache. - rf := getResultField("c", 1, 0) - rf1 := getResultField("c1", 2, 1) - ft := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - ft1 := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - ckLow := chunk.NewChunkWithCapacity([]*types.FieldType{ft, ft1}, 1024) - val := "fromStorage" - val1 := "fromStorage1" - ckLow.AppendString(0, val) - ckLow.AppendString(1, val1) - - // Let cache become invalid, and try concurrent load - counter := int32(0) - waitToStart := new(sync.WaitGroup) - waitToStart.Add(1) - gvc.lastModify = time.Now().Add(time.Duration(-10) * time.Second) - loadFunc := func() ([]chunk.Row, []*ast.ResultField, error) { - time.Sleep(100 * time.Millisecond) - atomic.AddInt32(&counter, 1) - return []chunk.Row{ckLow.GetRow(0)}, []*ast.ResultField{rf, rf1}, nil - } - wg := new(sync.WaitGroup) - worker := 100 - resArray := make([]loadResult, worker) - for i := 0; i < worker; i++ { - wg.Add(1) - go func(idx int) { - defer wg.Done() - waitToStart.Wait() - resRow, resField, _ := gvc.LoadGlobalVariables(loadFunc) - resArray[idx].rows = resRow - resArray[idx].fields = resField - }(i) - } - waitToStart.Done() - wg.Wait() - succ, rows, fields = gvc.Get() - c.Assert(counter, Equals, int32(1)) - c.Assert(resArray[0].rows[0].GetString(0), Equals, val) - c.Assert(resArray[0].rows[0].GetString(1), Equals, val1) - for i := 0; i < worker; i++ { - c.Assert(resArray[0].rows[0], Equals, resArray[i].rows[0]) - c.Assert(resArray[i].rows[0].GetString(0), Equals, val) - c.Assert(resArray[i].rows[0].GetString(1), Equals, val1) - } - // Validate cache - c.Assert(succ, IsTrue) - c.Assert(rows[0], Equals, resArray[0].rows[0]) - c.Assert(fields, DeepEquals, []*ast.ResultField{rf, rf1}) -} - -func (gvcSuite *testGVCSuite) TestCheckEnableStmtSummary(c *C) { - defer testleak.AfterTest(c)() - testleak.BeforeTest() - - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - defer func() { - err := store.Close() - c.Assert(err, IsNil) - }() - ddlLease := 50 * time.Millisecond - dom := NewDomain(store, ddlLease, 0, 0, mockFactory) - err = dom.Init(ddlLease, sysMockFactory) - c.Assert(err, IsNil) - defer dom.Close() - - gvc := dom.GetGlobalVarsCache() - - rf := getResultField("c", 1, 0) - rf1 := getResultField("c1", 2, 1) - ft := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - ft1 := &types.FieldType{ - Tp: mysql.TypeString, - Charset: charset.CharsetBin, - Collate: charset.CollationBin, - } - - err = stmtsummary.StmtSummaryByDigestMap.SetEnabled("0", false) - c.Assert(err, IsNil) - ck := chunk.NewChunkWithCapacity([]*types.FieldType{ft, ft1}, 1024) - ck.AppendString(0, variable.TiDBEnableStmtSummary) - ck.AppendString(1, "1") - row := ck.GetRow(0) - gvc.Update([]chunk.Row{row}, []*ast.ResultField{rf, rf1}) - c.Assert(stmtsummary.StmtSummaryByDigestMap.Enabled(), Equals, true) - - ck = chunk.NewChunkWithCapacity([]*types.FieldType{ft, ft1}, 1024) - ck.AppendString(0, variable.TiDBEnableStmtSummary) - ck.AppendString(1, "0") - row = ck.GetRow(0) - gvc.Update([]chunk.Row{row}, []*ast.ResultField{rf, rf1}) - c.Assert(stmtsummary.StmtSummaryByDigestMap.Enabled(), Equals, false) -} diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go new file mode 100644 index 0000000000000..23c9688ea2f81 --- /dev/null +++ b/domain/sysvar_cache.go @@ -0,0 +1,167 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package domain + +import ( + "context" + "fmt" + "sync" + + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/stmtsummary" + "go.uber.org/zap" +) + +// The sysvar cache replaces the GlobalVariableCache. +// It is an improvement because it operates similar to privilege cache, +// where it caches for 5 minutes instead of 2 seconds, plus it listens on etcd +// for updates from other servers. + +// SysVarCache represents the cache of system variables broken up into session and global scope. +type SysVarCache struct { + sync.RWMutex + global map[string]string + session map[string]string +} + +// GetSysVarCache gets the global variable cache. +func (do *Domain) GetSysVarCache() *SysVarCache { + return &do.sysVarCache +} + +func (svc *SysVarCache) rebuildCacheIfNeeded(ctx sessionctx.Context) (err error) { + svc.RLock() + cacheNeedsRebuild := len(svc.session) == 0 || len(svc.global) == 0 + svc.RUnlock() + if cacheNeedsRebuild { + logutil.BgLogger().Warn("sysvar cache is empty, triggering rebuild") + if err = svc.RebuildSysVarCache(ctx); err != nil { + logutil.BgLogger().Error("rebuilding sysvar cache failed", zap.Error(err)) + } + } + return err +} + +// GetSessionCache gets a copy of the session sysvar cache. +// The intention is to copy it directly to the systems[] map +// on creating a new session. +func (svc *SysVarCache) GetSessionCache(ctx sessionctx.Context) (map[string]string, error) { + if err := svc.rebuildCacheIfNeeded(ctx); err != nil { + return nil, err + } + svc.RLock() + defer svc.RUnlock() + // Perform a deep copy since this will be assigned directly to the session + newMap := make(map[string]string, len(svc.session)) + for k, v := range svc.session { + newMap[k] = v + } + return newMap, nil +} + +// GetGlobalVar gets an individual global var from the sysvar cache. +func (svc *SysVarCache) GetGlobalVar(ctx sessionctx.Context, name string) (string, error) { + if err := svc.rebuildCacheIfNeeded(ctx); err != nil { + return "", err + } + svc.RLock() + defer svc.RUnlock() + + if val, ok := svc.global[name]; ok { + return val, nil + } + logutil.BgLogger().Warn("could not find key in global cache", zap.String("name", name)) + return "", variable.ErrUnknownSystemVar.GenWithStackByArgs(name) +} + +func (svc *SysVarCache) fetchTableValues(ctx sessionctx.Context) (map[string]string, error) { + tableContents := make(map[string]string) + // Copy all variables from the table to tableContents + exec := ctx.(sqlexec.RestrictedSQLExecutor) + stmt, err := exec.ParseWithParams(context.Background(), `SELECT variable_name, variable_value FROM mysql.global_variables`) + if err != nil { + return tableContents, err + } + rows, _, err := exec.ExecRestrictedStmt(context.TODO(), stmt) + if err != nil { + return nil, err + } + for _, row := range rows { + name := row.GetString(0) + val := row.GetString(1) + tableContents[name] = val + } + return tableContents, nil +} + +// RebuildSysVarCache rebuilds the sysvar cache both globally and for session vars. +// It needs to be called when sysvars are added or removed. +func (svc *SysVarCache) RebuildSysVarCache(ctx sessionctx.Context) error { + newSessionCache := make(map[string]string) + newGlobalCache := make(map[string]string) + tableContents, err := svc.fetchTableValues(ctx) + if err != nil { + return err + } + + for _, sv := range variable.GetSysVars() { + sVal := sv.Value + if _, ok := tableContents[sv.Name]; ok { + sVal = tableContents[sv.Name] + } + if sv.HasSessionScope() { + newSessionCache[sv.Name] = sVal + } + if sv.HasGlobalScope() { + newGlobalCache[sv.Name] = sVal + } + // Propagate any changes to the server scoped variables + checkEnableServerGlobalVar(sv.Name, sVal) + } + + logutil.BgLogger().Debug("rebuilding sysvar cache") + + svc.Lock() + defer svc.Unlock() + svc.session = newSessionCache + svc.global = newGlobalCache + return nil +} + +// checkEnableServerGlobalVar processes variables that acts in server and global level. +func checkEnableServerGlobalVar(name, sVal string) { + var err error + switch name { + case variable.TiDBEnableStmtSummary: + err = stmtsummary.StmtSummaryByDigestMap.SetEnabled(sVal, false) + case variable.TiDBStmtSummaryInternalQuery: + err = stmtsummary.StmtSummaryByDigestMap.SetEnabledInternalQuery(sVal, false) + case variable.TiDBStmtSummaryRefreshInterval: + err = stmtsummary.StmtSummaryByDigestMap.SetRefreshInterval(sVal, false) + case variable.TiDBStmtSummaryHistorySize: + err = stmtsummary.StmtSummaryByDigestMap.SetHistorySize(sVal, false) + case variable.TiDBStmtSummaryMaxStmtCount: + err = stmtsummary.StmtSummaryByDigestMap.SetMaxStmtCount(sVal, false) + case variable.TiDBStmtSummaryMaxSQLLength: + err = stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength(sVal, false) + case variable.TiDBCapturePlanBaseline: + variable.CapturePlanBaseline.Set(sVal, false) + } + if err != nil { + logutil.BgLogger().Error(fmt.Sprintf("load global variable %s error", name), zap.Error(err)) + } +} diff --git a/executor/executor_test.go b/executor/executor_test.go index 5e6b4490f5eb6..7fa0d7b0d10bd 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2267,8 +2267,6 @@ func (s *testSuiteP2) TestSQLMode(c *C) { tk.MustExec("set sql_mode = 'STRICT_TRANS_TABLES'") tk.MustExec("set @@global.sql_mode = ''") - // Disable global variable cache, so load global session variable take effect immediate. - s.domain.GetGlobalVarsCache().Disable() tk2 := testkit.NewTestKit(c, s.store) tk2.MustExec("use test") tk2.MustExec("drop table if exists t2") diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 061e09dcc1315..bcecfc8d52ad4 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -1473,8 +1473,6 @@ func (s *seqTestSuite) TestMaxDeltaSchemaCount(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") c.Assert(variable.GetMaxDeltaSchemaCount(), Equals, int64(variable.DefTiDBMaxDeltaSchemaCount)) - gvc := domain.GetDomain(tk.Se).GetGlobalVarsCache() - gvc.Disable() tk.MustExec("set @@global.tidb_max_delta_schema_count= -1") tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_max_delta_schema_count value: '-1'")) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index ebe4a0620256f..1e5687928f3ad 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -965,8 +965,6 @@ func (s *testTableSuite) TestStmtSummaryTable(c *C) { tk.MustExec("set global tidb_enable_stmt_summary = 1") tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) - // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. - s.dom.GetGlobalVarsCache().Disable() // Disable refreshing summary. tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) @@ -1209,8 +1207,6 @@ func (s *testClusterTableSuite) TestStmtSummaryHistoryTable(c *C) { tk.MustExec("set global tidb_enable_stmt_summary = 1") tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) - // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. - s.dom.GetGlobalVarsCache().Disable() // Disable refreshing summary. tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) @@ -1266,8 +1262,6 @@ func (s *testTableSuite) TestStmtSummaryInternalQuery(c *C) { tk.MustExec("create global binding for select * from t where t.a = 1 using select * from t ignore index(k) where t.a = 1") tk.MustExec("set global tidb_enable_stmt_summary = 1") tk.MustQuery("select @@global.tidb_enable_stmt_summary").Check(testkit.Rows("1")) - // Invalidate the cache manually so that tidb_enable_stmt_summary works immediately. - s.dom.GetGlobalVarsCache().Disable() // Disable refreshing summary. tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) diff --git a/metrics/domain.go b/metrics/domain.go index f30dbd59e5d32..a05b25dd6a46a 100644 --- a/metrics/domain.go +++ b/metrics/domain.go @@ -60,6 +60,15 @@ var ( Help: "Counter of load privilege", }, []string{LblType}) + // LoadSysVarCacheCounter records the counter of loading sysvars + LoadSysVarCacheCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "domain", + Name: "load_sysvarcache_total", + Help: "Counter of load sysvar cache", + }, []string{LblType}) + SchemaValidatorStop = "stop" SchemaValidatorRestart = "restart" SchemaValidatorReset = "reset" diff --git a/metrics/metrics.go b/metrics/metrics.go index 4a879b5d5423c..542398e7bbdee 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -151,6 +151,7 @@ func RegisterMetrics() { prometheus.MustRegister(TiFlashQueryTotalCounter) prometheus.MustRegister(SmallTxnWriteDuration) prometheus.MustRegister(TxnWriteThroughput) + prometheus.MustRegister(LoadSysVarCacheCounter) tikvmetrics.InitMetrics(TiDB, TiKVClient) tikvmetrics.RegisterMetrics() diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index cd43b3964d59b..d6bfe69f82b39 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -278,9 +278,6 @@ func (s *testPrepareSerialSuite) TestPrepareOverMaxPreparedStmtCount(c *C) { tk.MustExec("set @@global.max_prepared_stmt_count = 2") tk.MustQuery("select @@global.max_prepared_stmt_count").Check(testkit.Rows("2")) - // Disable global variable cache, so load global session variable take effect immediate. - dom.GetGlobalVarsCache().Disable() - // test close session to give up all prepared stmt tk.MustExec(`prepare stmt2 from "select 1"`) prePrepared = readGaugeInt(metrics.PreparedStmtGauge) diff --git a/session/session.go b/session/session.go index e1581d5ed4074..e13be2045e941 100644 --- a/session/session.go +++ b/session/session.go @@ -991,6 +991,7 @@ func (s *session) replaceTableValue(ctx context.Context, tblName string, varName return err } _, _, err = s.ExecRestrictedStmt(ctx, stmt) + domain.GetDomain(s).NotifyUpdateSysVarCache(s) return err } @@ -1011,16 +1012,27 @@ func (s *session) GetGlobalSysVar(name string) (string, error) { // When running bootstrap or upgrade, we should not access global storage. return "", nil } - sysVar, err := s.getTableValue(context.TODO(), mysql.GlobalVariablesTable, name) + + sv := variable.GetSysVar(name) + if sv == nil { + // It might be a recently unregistered sysvar. We should return unknown + // since GetSysVar is the canonical version, but we can update the cache + // so the next request doesn't attempt to load this. + logutil.BgLogger().Info("sysvar does not exist. sysvar cache may be stale", zap.String("name", name)) + return "", variable.ErrUnknownSystemVar.GenWithStackByArgs(name) + } + + sysVar, err := domain.GetDomain(s).GetSysVarCache().GetGlobalVar(s, name) if err != nil { - if errResultIsEmpty.Equal(err) { - sv := variable.GetSysVar(name) - if sv != nil { - return sv.Value, nil - } - return "", variable.ErrUnknownSystemVar.GenWithStackByArgs(name) + // The sysvar exists, but there is no cache entry yet. + // This might be because the sysvar was only recently registered. + // In which case it is safe to return the default, but we can also + // update the cache for the future. + logutil.BgLogger().Info("sysvar not in cache yet. sysvar cache may be stale", zap.String("name", name)) + sysVar, err = s.getTableValue(context.TODO(), mysql.GlobalVariablesTable, name) + if err != nil { + return sv.Value, nil } - return "", err } // Fetch mysql.tidb values if required if s.varFromTiDBTable(name) { @@ -1065,12 +1077,7 @@ func (s *session) updateGlobalSysVar(sv *variable.SysVar, value string) error { return err } } - stmt, err := s.ParseWithParams(context.TODO(), "REPLACE %n.%n VALUES (%?, %?)", mysql.SystemDB, mysql.GlobalVariablesTable, sv.Name, value) - if err != nil { - return err - } - _, _, err = s.ExecRestrictedStmt(context.TODO(), stmt) - return err + return s.replaceTableValue(context.TODO(), mysql.GlobalVariablesTable, sv.Name, value) } // setTiDBTableValue handles tikv_* sysvars which need to update mysql.tidb @@ -2330,13 +2337,18 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { } } + // Rebuild sysvar cache in a loop + err = dom.LoadSysVarCacheLoop(se) + if err != nil { + return nil, err + } + if len(cfg.Plugin.Load) > 0 { err := plugin.Init(context.Background(), plugin.Config{EtcdClient: dom.GetEtcdClient()}) if err != nil { return nil, err } } - se4, err := createSession(store) if err != nil { return nil, err @@ -2439,7 +2451,7 @@ func createSessionWithOpt(store kv.Storage, opt *Opt) (*session, error) { // CreateSessionWithDomain creates a new Session and binds it with a Domain. // We need this because when we start DDL in Domain, the DDL need a session // to change some system tables. But at that time, we have been already in -// a lock context, which cause we can't call createSesion directly. +// a lock context, which cause we can't call createSession directly. func CreateSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, error) { s := &session{ store: store, @@ -2647,38 +2659,30 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { return nil } - var err error - // Use GlobalVariableCache if TiDB just loaded global variables within 2 second ago. - // When a lot of connections connect to TiDB simultaneously, it can protect TiKV meta region from overload. - gvc := domain.GetDomain(s).GetGlobalVarsCache() - loadFunc := func() ([]chunk.Row, []*ast.ResultField, error) { - vars := append(make([]string, 0, len(builtinGlobalVariable)+len(variable.PluginVarNames)), builtinGlobalVariable...) - if len(variable.PluginVarNames) > 0 { - vars = append(vars, variable.PluginVarNames...) - } - - stmt, err := s.ParseWithParams(context.TODO(), "select HIGH_PRIORITY * from mysql.global_variables where variable_name in (%?) order by VARIABLE_NAME", vars) - if err != nil { - return nil, nil, errors.Trace(err) - } + vars.CommonGlobalLoaded = true - return s.ExecRestrictedStmt(context.TODO(), stmt) - } - rows, _, err := gvc.LoadGlobalVariables(loadFunc) + // Deep copy sessionvar cache + // Eventually this whole map will be applied to systems[], which is a MySQL behavior. + sessionCache, err := domain.GetDomain(s).GetSysVarCache().GetSessionCache(s) if err != nil { - logutil.BgLogger().Warn("failed to load global variables", - zap.Uint64("conn", s.sessionVars.ConnectionID), zap.Error(err)) return err } - vars.CommonGlobalLoaded = true - - for _, row := range rows { - varName := row.GetString(0) - varVal := row.GetString(1) + for _, varName := range builtinGlobalVariable { + // The item should be in the sessionCache, but due to a strange current behavior there are some Global-only + // vars that are in builtinGlobalVariable. For compatibility we need to fall back to the Global cache on these items. + // TODO: don't load these globals into the session! + var varVal string + var ok bool + if varVal, ok = sessionCache[varName]; !ok { + varVal, err = s.GetGlobalSysVar(varName) + if err != nil { + continue // skip variables that are not loaded. + } + } // `collation_server` is related to `character_set_server`, set `character_set_server` will also set `collation_server`. // We have to make sure we set the `collation_server` with right value. if _, ok := vars.GetSystemVar(varName); !ok || varName == variable.CollationServer { - err = vars.SetSystemVar(varName, varVal) + err = vars.SetSystemVarWithRelaxedValidation(varName, varVal) if err != nil { return err } @@ -2693,8 +2697,6 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { } } } - - vars.CommonGlobalLoaded = true return nil } diff --git a/session/session_test.go b/session/session_test.go index b7cfecc9c5f4c..9845e757470f0 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -213,7 +213,6 @@ func (s *testSessionSuiteBase) SetUpSuite(c *C) { var err error s.dom, err = session.BootstrapSession(s.store) c.Assert(err, IsNil) - s.dom.GetGlobalVarsCache().Disable() } func (s *testSessionSuiteBase) TearDownSuite(c *C) { @@ -639,7 +638,6 @@ func (s *testSessionSuite) TestGlobalVarAccessor(c *C) { c.Assert(v, Equals, varValue2) // For issue 10955, make sure the new session load `max_execution_time` into sessionVars. - s.dom.GetGlobalVarsCache().Disable() tk1.MustExec("set @@global.max_execution_time = 100") tk2 := testkit.NewTestKitWithInit(c, s.store) c.Assert(tk2.Se.GetSessionVars().MaxExecutionTime, Equals, uint64(100)) @@ -2618,8 +2616,6 @@ func (s *testSessionSuite) TestSetGlobalTZ(c *C) { tk.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +08:00")) - // Disable global variable cache, so load global session variable take effect immediate. - s.dom.GetGlobalVarsCache().Disable() tk1 := testkit.NewTestKitWithInit(c, s.store) tk1.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +00:00")) } @@ -2761,8 +2757,6 @@ func (s *testSessionSuite3) TestEnablePartition(c *C) { tk.MustExec("set tidb_enable_list_partition=on") tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) - // Disable global variable cache, so load global session variable take effect immediate. - s.dom.GetGlobalVarsCache().Disable() tk1 := testkit.NewTestKitWithInit(c, s.store) tk1.MustQuery("show variables like 'tidb_enable_table_partition'").Check(testkit.Rows("tidb_enable_table_partition ON")) } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 0c6c74d90a26d..c474e7905fa7b 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1421,6 +1421,15 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { return sv.SetSessionFromHook(s, val) } +// SetSystemVarWithRelaxedValidation sets the value of a system variable for session scope. +// Validation functions are called, but scope validation is skipped. +// Errors are not expected to be returned because this could cause upgrade issues. +func (s *SessionVars) SetSystemVarWithRelaxedValidation(name string, val string) error { + sv := GetSysVar(name) + val = sv.ValidateWithRelaxedValidation(s, val, ScopeSession) + return sv.SetSessionFromHook(s, val) +} + // GetReadableTxnMode returns the session variable TxnMode but rewrites it to "OPTIMISTIC" when it's empty. func (s *SessionVars) GetReadableTxnMode() string { txnMode := s.TxnMode diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 98518fe4af0f0..99c3da8233d68 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -188,6 +188,10 @@ func (sv *SysVar) HasGlobalScope() bool { // Validate checks if system variable satisfies specific restriction. func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (string, error) { + // Check that the scope is correct first. + if err := sv.validateScope(scope); err != nil { + return value, err + } // Normalize the value and apply validation based on type. // i.e. TypeBool converts 1/on/ON to ON. normalizedValue, err := sv.validateFromType(vars, value, scope) @@ -203,17 +207,6 @@ func (sv *SysVar) Validate(vars *SessionVars, value string, scope ScopeFlag) (st // validateFromType provides automatic validation based on the SysVar's type func (sv *SysVar) validateFromType(vars *SessionVars, value string, scope ScopeFlag) (string, error) { - // Check that the scope is correct and return the appropriate error message. - if sv.ReadOnly || sv.Scope == ScopeNone { - return value, ErrIncorrectScope.FastGenByArgs(sv.Name, "read only") - } - if scope == ScopeGlobal && !sv.HasGlobalScope() { - return value, errLocalVariable.FastGenByArgs(sv.Name) - } - if scope == ScopeSession && !sv.HasSessionScope() { - return value, errGlobalVariable.FastGenByArgs(sv.Name) - } - // The string "DEFAULT" is a special keyword in MySQL, which restores // the compiled sysvar value. In which case we can skip further validation. if strings.EqualFold(value, "DEFAULT") { @@ -245,6 +238,37 @@ func (sv *SysVar) validateFromType(vars *SessionVars, value string, scope ScopeF return value, nil // typeString } +func (sv *SysVar) validateScope(scope ScopeFlag) error { + if sv.ReadOnly || sv.Scope == ScopeNone { + return ErrIncorrectScope.FastGenByArgs(sv.Name, "read only") + } + if scope == ScopeGlobal && !sv.HasGlobalScope() { + return errLocalVariable.FastGenByArgs(sv.Name) + } + if scope == ScopeSession && !sv.HasSessionScope() { + return errGlobalVariable.FastGenByArgs(sv.Name) + } + return nil +} + +// ValidateWithRelaxedValidation normalizes values but can not return errors. +// Normalization+validation needs to be applied when reading values because older versions of TiDB +// may be less sophisticated in normalizing values. But errors should be caught and handled, +// because otherwise there will be upgrade issues. +func (sv *SysVar) ValidateWithRelaxedValidation(vars *SessionVars, value string, scope ScopeFlag) string { + normalizedValue, err := sv.validateFromType(vars, value, scope) + if err != nil { + return normalizedValue + } + if sv.Validation != nil { + normalizedValue, err = sv.Validation(vars, normalizedValue, value, scope) + if err != nil { + return normalizedValue + } + } + return normalizedValue +} + const ( localDayTimeFormat = "15:04" // FullDayTimeFormat is the full format of analyze start time and end time. @@ -485,11 +509,15 @@ func SetSysVar(name string, value string) { sysVars[name].Value = value } -// GetSysVars returns the sysVars list under a RWLock +// GetSysVars deep copies the sysVars list under a RWLock func GetSysVars() map[string]*SysVar { sysVarsLock.RLock() defer sysVarsLock.RUnlock() - return sysVars + copy := make(map[string]*SysVar, len(sysVars)) + for name, sv := range sysVars { + copy[name] = sv + } + return copy } // PluginVarNames is global plugin var names set. From 9148ff9f44555d681a401b3f33681f7dc2414b6b Mon Sep 17 00:00:00 2001 From: djshow832 Date: Mon, 17 May 2021 22:31:39 +0800 Subject: [PATCH 089/343] ddl, transaction: DDL on temporary tables won't affect transactions (#24534) --- domain/schema_validator.go | 5 +++-- session/session.go | 6 ++++++ session/session_test.go | 39 ++++++++++++++++++++++++++++++++++++++ 3 files changed, 48 insertions(+), 2 deletions(-) diff --git a/domain/schema_validator.go b/domain/schema_validator.go index b983eff1d6203..a8baa49db93b9 100644 --- a/domain/schema_validator.go +++ b/domain/schema_validator.go @@ -234,8 +234,9 @@ func (s *schemaValidator) Check(txnTS uint64, schemaVer int64, relatedPhysicalTa // Schema changed, result decided by whether related tables change. if schemaVer < s.latestSchemaVer { - // The DDL relatedPhysicalTableIDs is empty. - if len(relatedPhysicalTableIDs) == 0 { + // When a transaction executes a DDL, relatedPhysicalTableIDs is nil. + // When a transaction only contains DML on temporary tables, relatedPhysicalTableIDs is []. + if relatedPhysicalTableIDs == nil { logutil.BgLogger().Info("the related physical table ID is empty", zap.Int64("schemaVer", schemaVer), zap.Int64("latestSchemaVer", s.latestSchemaVer)) return nil, ResultFail diff --git a/session/session.go b/session/session.go index e13be2045e941..78a60a6ebaecf 100644 --- a/session/session.go +++ b/session/session.go @@ -505,8 +505,14 @@ func (s *session) doCommit(ctx context.Context) error { // Get the related table or partition IDs. relatedPhysicalTables := s.GetSessionVars().TxnCtx.TableDeltaMap + // Get accessed global temporary tables in the transaction. + temporaryTables := s.GetSessionVars().TxnCtx.GlobalTemporaryTables physicalTableIDs := make([]int64, 0, len(relatedPhysicalTables)) for id := range relatedPhysicalTables { + // Schema change on global temporary tables doesn't affect transactions. + if _, ok := temporaryTables[id]; ok { + continue + } physicalTableIDs = append(physicalTableIDs, id) } // Set this option for 2 phase commit to validate schema lease. diff --git a/session/session_test.go b/session/session_test.go index 9845e757470f0..f7267e3a13259 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2131,6 +2131,45 @@ func (s *testSchemaSerialSuite) TestSchemaCheckerSQL(c *C) { c.Assert(err, NotNil) } +func (s *testSchemaSerialSuite) TestSchemaCheckerTempTable(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk1 := testkit.NewTestKitWithInit(c, s.store) + + // create table + tk.MustExec(`drop table if exists normal_table`) + tk.MustExec(`create table normal_table (id int, c int);`) + defer tk.MustExec(`drop table if exists normal_table`) + tk.MustExec(`drop table if exists temp_table`) + tk.MustExec(`create global temporary table temp_table (id int, c int) on commit delete rows;`) + defer tk.MustExec(`drop table if exists temp_table`) + + // The schema version is out of date in the first transaction, and the SQL can't be retried. + atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 1) + defer func() { + atomic.StoreUint32(&session.SchemaChangedWithoutRetry, 0) + }() + + // It's fine to change the schema of temporary tables. + tk.MustExec(`begin;`) + tk1.MustExec(`alter table temp_table modify column c bigint;`) + tk.MustExec(`insert into temp_table values(3, 3);`) + tk.MustExec(`commit;`) + + // Truncate will modify table ID. + tk.MustExec(`begin;`) + tk1.MustExec(`truncate table temp_table;`) + tk.MustExec(`insert into temp_table values(3, 3);`) + tk.MustExec(`commit;`) + + // It reports error when also changing the schema of a normal table. + tk.MustExec(`begin;`) + tk1.MustExec(`alter table normal_table modify column c bigint;`) + tk.MustExec(`insert into temp_table values(3, 3);`) + tk.MustExec(`insert into normal_table values(3, 3);`) + _, err := tk.Exec(`commit;`) + c.Assert(terror.ErrorEqual(err, domain.ErrInfoSchemaChanged), IsTrue, Commentf("err %v", err)) +} + func (s *testSchemaSuite) TestPrepareStmtCommitWhenSchemaChanged(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk1 := testkit.NewTestKitWithInit(c, s.store) From e9488ce2ee1d33fcc5533bb928fceb4ef564c830 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Tue, 18 May 2021 14:15:40 +0800 Subject: [PATCH 090/343] *: implement tidb_bounded_staleness built-in function (#24328) --- executor/show_test.go | 5 +- expression/builtin.go | 4 +- expression/builtin_time.go | 95 +++++++++++++++++++++++ expression/builtin_time_test.go | 102 ++++++++++++++++++++++++- expression/builtin_time_vec.go | 64 ++++++++++++++++ expression/builtin_time_vec_test.go | 7 ++ expression/helper.go | 3 +- expression/integration_test.go | 74 ++++++++++++++++++ go.mod | 2 +- go.sum | 4 +- kv/interface_mock_test.go | 4 + kv/kv.go | 2 + sessionctx/stmtctx/stmtctx.go | 41 +++++++--- store/helper/helper.go | 1 + store/mockstore/mockstorage/storage.go | 5 ++ store/tikv/kv.go | 24 +++++- util/mock/store.go | 5 ++ 17 files changed, 421 insertions(+), 21 deletions(-) diff --git a/executor/show_test.go b/executor/show_test.go index a343779245a3f..ea6d6734159b6 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -1102,9 +1102,10 @@ func (s *testSuite5) TestShowBuiltin(c *C) { res := tk.MustQuery("show builtins;") c.Assert(res, NotNil) rows := res.Rows() - c.Assert(268, Equals, len(rows)) + const builtinFuncNum = 269 + c.Assert(builtinFuncNum, Equals, len(rows)) c.Assert("abs", Equals, rows[0][0].(string)) - c.Assert("yearweek", Equals, rows[267][0].(string)) + c.Assert("yearweek", Equals, rows[builtinFuncNum-1][0].(string)) } func (s *testSuite5) TestShowClusterConfig(c *C) { diff --git a/expression/builtin.go b/expression/builtin.go index 9c530f92949e1..a33650eef7b1f 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -687,6 +687,9 @@ var funcs = map[string]functionClass{ ast.Year: &yearFunctionClass{baseFunctionClass{ast.Year, 1, 1}}, ast.YearWeek: &yearWeekFunctionClass{baseFunctionClass{ast.YearWeek, 1, 2}}, ast.LastDay: &lastDayFunctionClass{baseFunctionClass{ast.LastDay, 1, 1}}, + // TSO functions + ast.TiDBBoundedStaleness: &tidbBoundedStalenessFunctionClass{baseFunctionClass{ast.TiDBBoundedStaleness, 2, 2}}, + ast.TiDBParseTso: &tidbParseTsoFunctionClass{baseFunctionClass{ast.TiDBParseTso, 1, 1}}, // string functions ast.ASCII: &asciiFunctionClass{baseFunctionClass{ast.ASCII, 1, 1}}, @@ -881,7 +884,6 @@ var funcs = map[string]functionClass{ // This function is used to show tidb-server version info. ast.TiDBVersion: &tidbVersionFunctionClass{baseFunctionClass{ast.TiDBVersion, 0, 0}}, ast.TiDBIsDDLOwner: &tidbIsDDLOwnerFunctionClass{baseFunctionClass{ast.TiDBIsDDLOwner, 0, 0}}, - ast.TiDBParseTso: &tidbParseTsoFunctionClass{baseFunctionClass{ast.TiDBParseTso, 1, 1}}, ast.TiDBDecodePlan: &tidbDecodePlanFunctionClass{baseFunctionClass{ast.TiDBDecodePlan, 1, 1}}, // TiDB Sequence function. diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 1d52cf6adc2c3..13b3d1eef3def 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -27,6 +27,7 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -7113,3 +7114,97 @@ func handleInvalidZeroTime(ctx sessionctx.Context, t types.Time) (bool, error) { } return true, handleInvalidTimeError(ctx, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, t.String())) } + +// tidbBoundedStalenessFunctionClass reads a time window [a, b] and compares it with the latest SafeTS +// to determine which TS to use in a read only transaction. +type tidbBoundedStalenessFunctionClass struct { + baseFunctionClass +} + +func (c *tidbBoundedStalenessFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { + if err := c.verifyArgs(args); err != nil { + return nil, err + } + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETDatetime, types.ETDatetime, types.ETDatetime) + if err != nil { + return nil, err + } + sig := &builtinTiDBBoundedStalenessSig{bf} + return sig, nil +} + +type builtinTiDBBoundedStalenessSig struct { + baseBuiltinFunc +} + +func (b *builtinTiDBBoundedStalenessSig) Clone() builtinFunc { + newSig := &builtinTidbParseTsoSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinTiDBBoundedStalenessSig) evalTime(row chunk.Row) (types.Time, bool, error) { + leftTime, isNull, err := b.args[0].EvalTime(b.ctx, row) + if isNull || err != nil { + return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err) + } + rightTime, isNull, err := b.args[1].EvalTime(b.ctx, row) + if isNull || err != nil { + return types.ZeroTime, true, handleInvalidTimeError(b.ctx, err) + } + if invalidLeftTime, invalidRightTime := leftTime.InvalidZero(), rightTime.InvalidZero(); invalidLeftTime || invalidRightTime { + if invalidLeftTime { + err = handleInvalidTimeError(b.ctx, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, leftTime.String())) + } + if invalidRightTime { + err = handleInvalidTimeError(b.ctx, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, rightTime.String())) + } + return types.ZeroTime, true, err + } + timeZone := getTimeZone(b.ctx) + minTime, err := leftTime.GoTime(timeZone) + if err != nil { + return types.ZeroTime, true, err + } + maxTime, err := rightTime.GoTime(timeZone) + if err != nil { + return types.ZeroTime, true, err + } + if minTime.After(maxTime) { + return types.ZeroTime, true, nil + } + // Because the minimum unit of a TSO is millisecond, so we only need fsp to be 3. + return types.NewTime(types.FromGoTime(calAppropriateTime(minTime, maxTime, getMinSafeTime(b.ctx, timeZone))), mysql.TypeDatetime, 3), false, nil +} + +func getMinSafeTime(sessionCtx sessionctx.Context, timeZone *time.Location) time.Time { + var minSafeTS uint64 + if store := sessionCtx.GetStore(); store != nil { + minSafeTS = store.GetMinSafeTS(sessionCtx.GetSessionVars().CheckAndGetTxnScope()) + } + // Inject mocked SafeTS for test. + failpoint.Inject("injectSafeTS", func(val failpoint.Value) { + injectTS := val.(int) + minSafeTS = uint64(injectTS) + }) + // Try to get from the stmt cache to make sure this function is deterministic. + stmtCtx := sessionCtx.GetSessionVars().StmtCtx + minSafeTS = stmtCtx.GetOrStoreStmtCache(stmtctx.StmtSafeTSCacheKey, minSafeTS).(uint64) + return oracle.GetTimeFromTS(minSafeTS).In(timeZone) +} + +// For a SafeTS t and a time range [t1, t2]: +// 1. If t < t1, we will use t1 as the result, +// and with it, a read request may fail because it's an unreached SafeTS. +// 2. If t1 <= t <= t2, we will use t as the result, and with it, +// a read request won't fail. +// 2. If t2 < t, we will use t2 as the result, +// and with it, a read request won't fail because it's bigger than the latest SafeTS. +func calAppropriateTime(minTime, maxTime, minSafeTime time.Time) time.Time { + if minSafeTime.Before(minTime) { + return minTime + } else if minSafeTime.After(maxTime) { + return maxTime + } + return minSafeTime +} diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index f82f6fb8f76ea..161912b07e973 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -14,12 +14,14 @@ package expression import ( + "fmt" "math" "strings" "time" . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/mysql" @@ -27,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" @@ -804,7 +807,7 @@ func (s *testEvaluatorSuite) TestTime(c *C) { } func resetStmtContext(ctx sessionctx.Context) { - ctx.GetSessionVars().StmtCtx.ResetNowTs() + ctx.GetSessionVars().StmtCtx.ResetStmtCache() } func (s *testEvaluatorSuite) TestNowAndUTCTimestamp(c *C) { @@ -2854,6 +2857,103 @@ func (s *testEvaluatorSuite) TestTidbParseTso(c *C) { } } +func (s *testEvaluatorSuite) TestTiDBBoundedStaleness(c *C) { + t1, err := time.Parse(types.TimeFormat, "2015-09-21 09:53:04") + c.Assert(err, IsNil) + // time.Parse uses UTC time zone by default, we need to change it to Local manually. + t1 = t1.Local() + t1Str := t1.Format(types.TimeFormat) + t2 := time.Now() + t2Str := t2.Format(types.TimeFormat) + timeZone := time.Local + s.ctx.GetSessionVars().TimeZone = timeZone + tests := []struct { + leftTime interface{} + rightTime interface{} + injectSafeTS uint64 + isNull bool + expect time.Time + }{ + // SafeTS is in the range. + { + leftTime: t1Str, + rightTime: t2Str, + injectSafeTS: oracle.GoTimeToTS(t2.Add(-1 * time.Second)), + isNull: false, + expect: t2.Add(-1 * time.Second), + }, + // SafeTS is less than the left time. + { + leftTime: t1Str, + rightTime: t2Str, + injectSafeTS: oracle.GoTimeToTS(t1.Add(-1 * time.Second)), + isNull: false, + expect: t1, + }, + // SafeTS is bigger than the right time. + { + leftTime: t1Str, + rightTime: t2Str, + injectSafeTS: oracle.GoTimeToTS(t2.Add(time.Second)), + isNull: false, + expect: t2, + }, + // Wrong time order. + { + leftTime: t2Str, + rightTime: t1Str, + injectSafeTS: 0, + isNull: true, + expect: time.Time{}, + }, + } + + fc := funcs[ast.TiDBBoundedStaleness] + for _, test := range tests { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", + fmt.Sprintf("return(%v)", test.injectSafeTS)), IsNil) + f, err := fc.getFunction(s.ctx, s.datumsToConstants([]types.Datum{types.NewDatum(test.leftTime), types.NewDatum(test.rightTime)})) + c.Assert(err, IsNil) + d, err := evalBuiltinFunc(f, chunk.Row{}) + c.Assert(err, IsNil) + if test.isNull { + c.Assert(d.IsNull(), IsTrue) + } else { + goTime, err := d.GetMysqlTime().GoTime(timeZone) + c.Assert(err, IsNil) + c.Assert(goTime.Format(types.TimeFormat), Equals, test.expect.Format(types.TimeFormat)) + } + resetStmtContext(s.ctx) + } + + // Test whether it's deterministic. + safeTime1 := t2.Add(-1 * time.Second) + safeTS1 := oracle.ComposeTS(safeTime1.Unix()*1000, 0) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", + fmt.Sprintf("return(%v)", safeTS1)), IsNil) + f, err := fc.getFunction(s.ctx, s.datumsToConstants([]types.Datum{types.NewDatum(t1Str), types.NewDatum(t2Str)})) + c.Assert(err, IsNil) + d, err := evalBuiltinFunc(f, chunk.Row{}) + c.Assert(err, IsNil) + goTime, err := d.GetMysqlTime().GoTime(timeZone) + c.Assert(err, IsNil) + resultTime := goTime.Format(types.TimeFormat) + c.Assert(resultTime, Equals, safeTime1.Format(types.TimeFormat)) + // SafeTS updated. + safeTime2 := t2.Add(1 * time.Second) + safeTS2 := oracle.ComposeTS(safeTime2.Unix()*1000, 0) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", + fmt.Sprintf("return(%v)", safeTS2)), IsNil) + f, err = fc.getFunction(s.ctx, s.datumsToConstants([]types.Datum{types.NewDatum(t1Str), types.NewDatum(t2Str)})) + c.Assert(err, IsNil) + d, err = evalBuiltinFunc(f, chunk.Row{}) + c.Assert(err, IsNil) + // Still safeTime1 + c.Assert(resultTime, Equals, safeTime1.Format(types.TimeFormat)) + resetStmtContext(s.ctx) + failpoint.Disable("github.com/pingcap/tidb/expression/injectSafeTS") +} + func (s *testEvaluatorSuite) TestGetIntervalFromDecimal(c *C) { du := baseDateArithmitical{} diff --git a/expression/builtin_time_vec.go b/expression/builtin_time_vec.go index 94c1cd8b6f0c4..6f74a8f587e50 100644 --- a/expression/builtin_time_vec.go +++ b/expression/builtin_time_vec.go @@ -854,6 +854,70 @@ func (b *builtinTidbParseTsoSig) vecEvalTime(input *chunk.Chunk, result *chunk.C return nil } +func (b *builtinTiDBBoundedStalenessSig) vectorized() bool { + return true +} + +func (b *builtinTiDBBoundedStalenessSig) vecEvalTime(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + buf0, err := b.bufAllocator.get(types.ETDatetime, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf0) + if err = b.args[0].VecEvalTime(b.ctx, input, buf0); err != nil { + return err + } + buf1, err := b.bufAllocator.get(types.ETDatetime, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf1) + if err = b.args[1].VecEvalTime(b.ctx, input, buf1); err != nil { + return err + } + args0 := buf0.Times() + args1 := buf1.Times() + timeZone := getTimeZone(b.ctx) + minSafeTime := getMinSafeTime(b.ctx, timeZone) + result.ResizeTime(n, false) + result.MergeNulls(buf0, buf1) + times := result.Times() + for i := 0; i < n; i++ { + if result.IsNull(i) { + continue + } + if invalidArg0, invalidArg1 := args0[i].InvalidZero(), args1[i].InvalidZero(); invalidArg0 || invalidArg1 { + if invalidArg0 { + err = handleInvalidTimeError(b.ctx, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, args0[i].String())) + } + if invalidArg1 { + err = handleInvalidTimeError(b.ctx, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, args1[i].String())) + } + if err != nil { + return err + } + result.SetNull(i, true) + continue + } + minTime, err := args0[i].GoTime(timeZone) + if err != nil { + return err + } + maxTime, err := args1[i].GoTime(timeZone) + if err != nil { + return err + } + if minTime.After(maxTime) { + result.SetNull(i, true) + continue + } + // Because the minimum unit of a TSO is millisecond, so we only need fsp to be 3. + times[i] = types.NewTime(types.FromGoTime(calAppropriateTime(minTime, maxTime, minSafeTime)), mysql.TypeDatetime, 3) + } + return nil +} + func (b *builtinFromDaysSig) vectorized() bool { return true } diff --git a/expression/builtin_time_vec_test.go b/expression/builtin_time_vec_test.go index 593cce162d7ff..a757b867b783c 100644 --- a/expression/builtin_time_vec_test.go +++ b/expression/builtin_time_vec_test.go @@ -519,6 +519,13 @@ var vecBuiltinTimeCases = map[string][]vecExprBenchCase{ geners: []dataGenerator{newRangeInt64Gener(0, math.MaxInt64)}, }, }, + // Todo: how to inject the safeTS for better testing. + ast.TiDBBoundedStaleness: { + { + retEvalType: types.ETDatetime, + childrenTypes: []types.EvalType{types.ETDatetime, types.ETDatetime}, + }, + }, ast.LastDay: { {retEvalType: types.ETDatetime, childrenTypes: []types.EvalType{types.ETDatetime}}, }, diff --git a/expression/helper.go b/expression/helper.go index c5f91dbd090b5..d9f1e22610b62 100644 --- a/expression/helper.go +++ b/expression/helper.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" @@ -155,5 +156,5 @@ func getStmtTimestamp(ctx sessionctx.Context) (time.Time, error) { return time.Unix(timestamp, 0), nil } stmtCtx := ctx.GetSessionVars().StmtCtx - return stmtCtx.GetNowTsCached(), nil + return stmtCtx.GetOrStoreStmtCache(stmtctx.StmtNowTsCacheKey, time.Now()).(time.Time), nil } diff --git a/expression/integration_test.go b/expression/integration_test.go index 80e39b76ce746..69142c01c3f35 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -2263,6 +2264,79 @@ func (s *testIntegrationSuite2) TestTimeBuiltin(c *C) { result = tk.MustQuery(`select tidb_parse_tso(-1)`) result.Check(testkit.Rows("")) + // for tidb_bounded_staleness + tk.MustExec("SET time_zone = '+00:00';") + t := time.Now().UTC() + ts := oracle.GoTimeToTS(t) + tidbBoundedStalenessTests := []struct { + sql string + injectSafeTS uint64 + expect string + }{ + { + sql: `select tidb_bounded_staleness(DATE_SUB(NOW(), INTERVAL 600 SECOND), DATE_ADD(NOW(), INTERVAL 600 SECOND))`, + injectSafeTS: ts, + expect: t.Format(types.TimeFSPFormat[:len(types.TimeFSPFormat)-3]), + }, + { + sql: `select tidb_bounded_staleness("2021-04-27 12:00:00.000", "2021-04-27 13:00:00.000")`, + injectSafeTS: func() uint64 { + t, err := time.Parse("2006-01-02 15:04:05.000", "2021-04-27 13:30:04.877") + c.Assert(err, IsNil) + return oracle.GoTimeToTS(t) + }(), + expect: "2021-04-27 13:00:00.000", + }, + { + sql: `select tidb_bounded_staleness("2021-04-27 12:00:00.000", "2021-04-27 13:00:00.000")`, + injectSafeTS: func() uint64 { + t, err := time.Parse("2006-01-02 15:04:05.000", "2021-04-27 11:30:04.877") + c.Assert(err, IsNil) + return oracle.GoTimeToTS(t) + }(), + expect: "2021-04-27 12:00:00.000", + }, + { + sql: `select tidb_bounded_staleness("2021-04-27 12:00:00.000", "2021-04-27 11:00:00.000")`, + injectSafeTS: 0, + expect: "", + }, + // Time is too small. + { + sql: `select tidb_bounded_staleness("0020-04-27 12:00:00.000", "2021-04-27 11:00:00.000")`, + injectSafeTS: 0, + expect: "1970-01-01 00:00:00.000", + }, + // Wrong value. + { + sql: `select tidb_bounded_staleness(1, 2)`, + injectSafeTS: 0, + expect: "", + }, + { + sql: `select tidb_bounded_staleness("invalid_time_1", "invalid_time_2")`, + injectSafeTS: 0, + expect: "", + }, + } + for _, test := range tidbBoundedStalenessTests { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", + fmt.Sprintf("return(%v)", test.injectSafeTS)), IsNil) + result = tk.MustQuery(test.sql) + result.Check(testkit.Rows(test.expect)) + } + failpoint.Disable("github.com/pingcap/tidb/expression/injectSafeTS") + // test whether tidb_bounded_staleness is deterministic + result = tk.MustQuery(`select tidb_bounded_staleness(NOW(), DATE_ADD(NOW(), INTERVAL 600 SECOND)), tidb_bounded_staleness(NOW(), DATE_ADD(NOW(), INTERVAL 600 SECOND))`) + c.Assert(result.Rows()[0], HasLen, 2) + c.Assert(result.Rows()[0][0], Equals, result.Rows()[0][1]) + preResult := result.Rows()[0][0] + time.Sleep(time.Second) + result = tk.MustQuery(`select tidb_bounded_staleness(NOW(), DATE_ADD(NOW(), INTERVAL 600 SECOND)), tidb_bounded_staleness(NOW(), DATE_ADD(NOW(), INTERVAL 600 SECOND))`) + c.Assert(result.Rows()[0], HasLen, 2) + c.Assert(result.Rows()[0][0], Equals, result.Rows()[0][1]) + c.Assert(result.Rows()[0][0], Not(Equals), preResult) + // fix issue 10308 result = tk.MustQuery("select time(\"- -\");") result.Check(testkit.Rows("00:00:00")) diff --git a/go.mod b/go.mod index fe8e08ae42e47..f82a8a187775f 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 - github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b + github.com/pingcap/parser v0.0.0-20210518053259-92fa6fe07eb6 github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 diff --git a/go.sum b/go.sum index 3ee71da011a54..14986c3d1f025 100644 --- a/go.sum +++ b/go.sum @@ -443,8 +443,8 @@ github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 h1:ERrF0fTuIOnwfGbt71Ji3DKbOEaP189tjym50u8gpC8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b h1:eLuDQ6eJCEKCbGwhGrkjzagwev1GJGU2Y2kFkAsBzV0= -github.com/pingcap/parser v0.0.0-20210513020953-ae2c4497c07b/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= +github.com/pingcap/parser v0.0.0-20210518053259-92fa6fe07eb6 h1:wsH3psMH5ksDowsN9VUE9ZqSrX6oF4AYQQfOunkvSfU= +github.com/pingcap/parser v0.0.0-20210518053259-92fa6fe07eb6/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 h1:A9KL9R+lWSVPH8IqUuH1QSTRJ5FGoY1bT2IcfPKsWD8= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index e1d41f1693088..5d85261bc2111 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -213,6 +213,10 @@ func (s *mockStorage) GetMemCache() MemManager { return nil } +func (s *mockStorage) GetMinSafeTS(txnScope string) uint64 { + return 0 +} + // newMockStorage creates a new mockStorage. func newMockStorage() Storage { return &mockStorage{} diff --git a/kv/kv.go b/kv/kv.go index 572fe104024bc..20b0fc84b7144 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -417,6 +417,8 @@ type Storage interface { ShowStatus(ctx context.Context, key string) (interface{}, error) // GetMemCache return memory manager of the storage. GetMemCache() MemManager + // GetMinSafeTS return the minimal SafeTS of the storage with given txnScope. + GetMinSafeTS(txnScope string) uint64 } // EtcdBackend is used for judging a storage is a real TiKV. diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 8df0001427173..d8a75aec48610 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -140,8 +140,6 @@ type StatementContext struct { RuntimeStatsColl *execdetails.RuntimeStatsColl TableIDs []int64 IndexNames []string - nowTs time.Time // use this variable for now/current_timestamp calculation/cache for one stmt - stmtTimeCached bool StmtType string OriginalSQL string digestMemo struct { @@ -164,6 +162,9 @@ type StatementContext struct { TblInfo2UnionScan map[*model.TableInfo]bool TaskID uint64 // unique ID for an execution of a statement TaskMapBakTS uint64 // counter for + + // stmtCache is used to store some statement-related values. + stmtCache map[StmtCacheKey]interface{} } // StmtHints are SessionVars related sql hints. @@ -195,19 +196,35 @@ func (sh *StmtHints) TaskMapNeedBackUp() bool { return sh.ForceNthPlan != -1 } -// GetNowTsCached getter for nowTs, if not set get now time and cache it -func (sc *StatementContext) GetNowTsCached() time.Time { - if !sc.stmtTimeCached { - now := time.Now() - sc.nowTs = now - sc.stmtTimeCached = true +// StmtCacheKey represents the key type in the StmtCache. +type StmtCacheKey int + +const ( + // StmtNowTsCacheKey is a variable for now/current_timestamp calculation/cache of one stmt. + StmtNowTsCacheKey StmtCacheKey = iota + // StmtSafeTSCacheKey is a variable for safeTS calculation/cache of one stmt. + StmtSafeTSCacheKey +) + +// GetOrStoreStmtCache gets the cached value of the given key if it exists, otherwise stores the value. +func (sc *StatementContext) GetOrStoreStmtCache(key StmtCacheKey, value interface{}) interface{} { + if sc.stmtCache == nil { + sc.stmtCache = make(map[StmtCacheKey]interface{}) + } + if _, ok := sc.stmtCache[key]; !ok { + sc.stmtCache[key] = value } - return sc.nowTs + return sc.stmtCache[key] +} + +// ResetInStmtCache resets the cache of given key. +func (sc *StatementContext) ResetInStmtCache(key StmtCacheKey) { + delete(sc.stmtCache, key) } -// ResetNowTs resetter for nowTs, clear cached time flag -func (sc *StatementContext) ResetNowTs() { - sc.stmtTimeCached = false +// ResetStmtCache resets all cached values. +func (sc *StatementContext) ResetStmtCache() { + sc.stmtCache = make(map[StmtCacheKey]interface{}) } // SQLDigest gets normalized and digest for provided sql. diff --git a/store/helper/helper.go b/store/helper/helper.go index e96ad4ae21851..49aa7cf2107e0 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -71,6 +71,7 @@ type Storage interface { SetTiKVClient(client tikv.Client) GetTiKVClient() tikv.Client Closed() <-chan struct{} + GetMinSafeTS(txnScope string) uint64 } // Helper is a middleware to get some information from tikv/pd. It can be used for TiDB's http api or mem table. diff --git a/store/mockstore/mockstorage/storage.go b/store/mockstore/mockstorage/storage.go index 36ded5e434817..6221ef855707d 100644 --- a/store/mockstore/mockstorage/storage.go +++ b/store/mockstore/mockstorage/storage.go @@ -99,6 +99,11 @@ func (s *mockStorage) CurrentVersion(txnScope string) (kv.Version, error) { return kv.NewVersion(ver), err } +// GetMinSafeTS return the minimal SafeTS of the storage with given txnScope. +func (s *mockStorage) GetMinSafeTS(txnScope string) uint64 { + return 0 +} + func newTiKVTxn(txn *tikv.KVTxn, err error) (kv.Transaction, error) { if err != nil { return nil, err diff --git a/store/tikv/kv.go b/store/tikv/kv.go index bbf8517a42a8c..8cec3dfbca964 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/metapb" tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" @@ -84,7 +85,7 @@ type KVStore struct { safePoint uint64 spTime time.Time spMutex sync.RWMutex // this is used to update safePoint and spTime - closed chan struct{} // this is used to nofity when the store is closed + closed chan struct{} // this is used to notify when the store is closed // storeID -> safeTS, stored as map[uint64]uint64 // safeTS here will be used during the Stale Read process, @@ -358,6 +359,27 @@ func (s *KVStore) GetTiKVClient() (client Client) { return s.clientMu.client } +// GetMinSafeTS return the minimal safeTS of the storage with given txnScope. +func (s *KVStore) GetMinSafeTS(txnScope string) uint64 { + stores := make([]*Store, 0) + allStores := s.regionCache.getStoresByType(tikvrpc.TiKV) + if txnScope != oracle.GlobalTxnScope { + for _, store := range allStores { + if store.IsLabelsMatch([]*metapb.StoreLabel{ + { + Key: DCLabelKey, + Value: txnScope, + }, + }) { + stores = append(stores, store) + } + } + } else { + stores = allStores + } + return s.getMinSafeTSByStores(stores) +} + func (s *KVStore) getSafeTS(storeID uint64) uint64 { safeTS, ok := s.safeTSMap.Load(storeID) if !ok { diff --git a/util/mock/store.go b/util/mock/store.go index 804f3d6a3f2d3..7c86de4b3cb72 100644 --- a/util/mock/store.go +++ b/util/mock/store.go @@ -72,3 +72,8 @@ func (s *Store) GetMemCache() kv.MemManager { // ShowStatus implements kv.Storage interface. func (s *Store) ShowStatus(ctx context.Context, key string) (interface{}, error) { return nil, nil } + +// GetMinSafeTS implements kv.Storage interface. +func (s *Store) GetMinSafeTS(txnScope string) uint64 { + return 0 +} From 49d21dd9a82e763854a97627bc0db72b5956a88a Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Tue, 18 May 2021 15:25:40 +0800 Subject: [PATCH 091/343] executor: add correctness tests for partition table with different joins (#24673) --- executor/partition_table_test.go | 235 +++++++++++++++++++++++++++++++ 1 file changed, 235 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index e87e1044278e3..8337be660ec0c 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testutil" ) func (s *partitionTableSuite) TestFourReader(c *C) { @@ -572,6 +573,240 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { tk.MustIndexLookup("select * from tlist where a<1") } +func (s *partitionTableSuite) TestPartitionTableWithDifferentJoin(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_partition_joins") + tk.MustExec("use test_partition_joins") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash and range partition + tk.MustExec("create table thash(a int, b int, key(a)) partition by hash(a) partitions 4") + tk.MustExec("create table tregular1(a int, b int, key(a))") + + tk.MustExec(`create table trange(a int, b int, key(a)) partition by range(a) ( + partition p0 values less than (200), + partition p1 values less than (400), + partition p2 values less than (600), + partition p3 values less than (800), + partition p4 values less than (1001))`) + tk.MustExec("create table tregular2(a int, b int, key(a))") + + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular1 values " + strings.Join(vals, ",")) + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular2 values " + strings.Join(vals, ",")) + + // random params + x1 := rand.Intn(1000) + x2 := rand.Intn(1000) + x3 := rand.Intn(1000) + x4 := rand.Intn(1000) + + // group 1 + // hash_join range partition and hash partition + queryHash := fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.b=thash.b and thash.a = %v and trange.a > %v;", x1, x2) + queryRegular := fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.b=tregular1.b and tregular1.a = %v and tregular2.a > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a > %v;", x1) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a > %v;", x1) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.a=thash.a and trange.b = thash.b and thash.a > %v;", x1) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.b = tregular2.b and tregular1.a > %v;", x1) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ hash_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a = %v;", x1) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a = %v;", x1) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // group 2 + // hash_join range partition and regular table + queryHash = fmt.Sprintf("select /*+ hash_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a >= %v and tregular1.a > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a >= %v and tregular1.a > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ hash_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a in (%v, %v, %v);", x1, x2, x3) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a in (%v, %v, %v);", x1, x2, x3) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ hash_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and tregular1.a >= %v;", x1) + queryRegular = fmt.Sprintf("select /*+ hash_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular1.a >= %v;", x1) + c.Assert(tk.HasPlan(queryHash, "HashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // group 3 + // merge_join range partition and hash partition + queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.b=thash.b and thash.a = %v and trange.a > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.b=tregular1.b and tregular1.a = %v and tregular2.a > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a > %v;", x1) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a > %v;", x1) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.a=thash.a and trange.b = thash.b and thash.a > %v;", x1) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.b = tregular2.b and tregular1.a > %v;", x1) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ merge_join(trange, thash) */ * from trange, thash where trange.a=thash.a and thash.a = %v;", x1) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a=tregular1.a and tregular1.a = %v;", x1) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // group 4 + // merge_join range partition and regular table + queryHash = fmt.Sprintf("select /*+ merge_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a >= %v and tregular1.a > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a >= %v and tregular1.a > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ merge_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and trange.a in (%v, %v, %v);", x1, x2, x3) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular2.a in (%v, %v, %v);", x1, x2, x3) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ merge_join(trange, tregular1) */ * from trange, tregular1 where trange.a = tregular1.a and tregular1.a >= %v;", x1) + queryRegular = fmt.Sprintf("select /*+ merge_join(tregular2, tregular1) */ * from tregular2, tregular1 where tregular2.a = tregular1.a and tregular1.a >= %v;", x1) + c.Assert(tk.HasPlan(queryHash, "MergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // new table instances + tk.MustExec("create table thash2(a int, b int, index idx(a)) partition by hash(a) partitions 4") + tk.MustExec("create table tregular3(a int, b int, index idx(a))") + + tk.MustExec(`create table trange2(a int, b int, index idx(a)) partition by range(a) ( + partition p0 values less than (200), + partition p1 values less than (400), + partition p2 values less than (600), + partition p3 values less than (800), + partition p4 values less than (1001))`) + tk.MustExec("create table tregular4(a int, b int, index idx(a))") + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec("insert into thash2 values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular3 values " + strings.Join(vals, ",")) + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1000), rand.Intn(1000))) + } + tk.MustExec("insert into trange2 values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular4 values " + strings.Join(vals, ",")) + + // group 5 + // index_merge_join range partition and range partition + // Currently don't support index merge join on two partition tables. Only test warning. + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v;", x1) + // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v;", x1) + // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + tk.MustQuery(queryHash) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange2.a > %v;", x1, x2) + // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.a > %v;", x1, x2) + // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + tk.MustQuery(queryHash) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange.b > %v;", x1, x2) + // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular2.b > %v;", x1, x2) + // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + tk.MustQuery(queryHash) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, trange2) */ * from trange, trange2 where trange.a=trange2.a and trange.a > %v and trange2.b > %v;", x1, x2) + // queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.b > %v;", x1, x2) + // c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + // tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + tk.MustQuery(queryHash) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1815|Optimizer Hint /*+ INL_MERGE_JOIN(trange, trange2) */ is inapplicable")) + + // group 6 + // index_merge_join range partition and regualr table + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v;", x1) + queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v;", x1) + c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and tregular4.a > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.a > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and trange.b > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular2.b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_merge_join(trange, tregular4) */ * from trange, tregular4 where trange.a=tregular4.a and trange.a > %v and tregular4.b > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_merge_join(tregular2, tregular4) */ * from tregular2, tregular4 where tregular2.a=tregular4.a and tregular2.a > %v and tregular4.b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexMergeJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // group 7 + // index_hash_join hash partition and hash partition + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, thash2) */ * from thash, thash2 where thash.a = thash2.a and thash.a in (%v, %v);", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v);", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, thash2) */ * from thash, thash2 where thash.a = thash2.a and thash.a in (%v, %v) and thash2.a in (%v, %v);", x1, x2, x3, x4) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v) and tregular3.a in (%v, %v);", x1, x2, x3, x4) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, thash2) */ * from thash, thash2 where thash.a = thash2.a and thash.a > %v and thash2.b > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a > %v and tregular3.b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // group 8 + // index_hash_join hash partition and hash partition + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, tregular3) */ * from thash, tregular3 where thash.a = tregular3.a and thash.a in (%v, %v);", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v);", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, tregular3) */ * from thash, tregular3 where thash.a = tregular3.a and thash.a in (%v, %v) and tregular3.a in (%v, %v);", x1, x2, x3, x4) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a in (%v, %v) and tregular3.a in (%v, %v);", x1, x2, x3, x4) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + queryHash = fmt.Sprintf("select /*+ inl_hash_join(thash, tregular3) */ * from thash, tregular3 where thash.a = tregular3.a and thash.a > %v and tregular3.b > %v;", x1, x2) + queryRegular = fmt.Sprintf("select /*+ inl_hash_join(tregular1, tregular3) */ * from tregular1, tregular3 where tregular1.a = tregular3.a and tregular1.a > %v and tregular3.b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryHash, "IndexHashJoin"), IsTrue) + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) +} + func createTable4DynamicPruneModeTestWithExpression(tk *testkit.TestKit) { tk.MustExec("create table trange(a int, b int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11));") tk.MustExec("create table thash(a int, b int) partition by hash(a) partitions 4;") From 5fd17dd7f4cef3f2e103f12427cb9197d9838b73 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Tue, 18 May 2021 16:27:40 +0800 Subject: [PATCH 092/343] expression: fix the spelling of word arithmetical (#24713) --- expression/builtin_time.go | 238 ++++++++++++++++---------------- expression/builtin_time_test.go | 2 +- 2 files changed, 120 insertions(+), 120 deletions(-) diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 13b3d1eef3def..67413dab11374 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -2769,20 +2769,20 @@ func (b *builtinExtractDurationSig) evalInt(row chunk.Row) (int64, bool, error) return res, err != nil, err } -// baseDateArithmitical is the base class for all "builtinAddDateXXXSig" and "builtinSubDateXXXSig", +// baseDateArithmetical is the base class for all "builtinAddDateXXXSig" and "builtinSubDateXXXSig", // which provides parameter getter and date arithmetical calculate functions. -type baseDateArithmitical struct { +type baseDateArithmetical struct { // intervalRegexp is "*Regexp" used to extract string interval for "DAY" unit. intervalRegexp *regexp.Regexp } -func newDateArighmeticalUtil() baseDateArithmitical { - return baseDateArithmitical{ +func newDateArighmeticalUtil() baseDateArithmetical { + return baseDateArithmetical{ intervalRegexp: regexp.MustCompile(`-?[\d]+`), } } -func (du *baseDateArithmitical) getDateFromString(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (types.Time, bool, error) { +func (du *baseDateArithmetical) getDateFromString(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (types.Time, bool, error) { dateStr, isNull, err := args[0].EvalString(ctx, row) if isNull || err != nil { return types.ZeroTime, true, err @@ -2807,7 +2807,7 @@ func (du *baseDateArithmitical) getDateFromString(ctx sessionctx.Context, args [ return date, false, handleInvalidTimeError(ctx, err) } -func (du *baseDateArithmitical) getDateFromInt(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (types.Time, bool, error) { +func (du *baseDateArithmetical) getDateFromInt(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (types.Time, bool, error) { dateInt, isNull, err := args[0].EvalInt(ctx, row) if isNull || err != nil { return types.ZeroTime, true, err @@ -2827,7 +2827,7 @@ func (du *baseDateArithmitical) getDateFromInt(ctx sessionctx.Context, args []Ex return date, false, nil } -func (du *baseDateArithmitical) getDateFromDatetime(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (types.Time, bool, error) { +func (du *baseDateArithmetical) getDateFromDatetime(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (types.Time, bool, error) { date, isNull, err := args[0].EvalTime(ctx, row) if isNull || err != nil { return types.ZeroTime, true, err @@ -2839,7 +2839,7 @@ func (du *baseDateArithmitical) getDateFromDatetime(ctx sessionctx.Context, args return date, false, nil } -func (du *baseDateArithmitical) getIntervalFromString(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (string, bool, error) { +func (du *baseDateArithmetical) getIntervalFromString(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (string, bool, error) { interval, isNull, err := args[1].EvalString(ctx, row) if isNull || err != nil { return "", true, err @@ -2857,7 +2857,7 @@ func (du *baseDateArithmitical) getIntervalFromString(ctx sessionctx.Context, ar return interval, false, nil } -func (du *baseDateArithmitical) getIntervalFromDecimal(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (string, bool, error) { +func (du *baseDateArithmetical) getIntervalFromDecimal(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (string, bool, error) { decimal, isNull, err := args[1].EvalDecimal(ctx, row) if isNull || err != nil { return "", true, err @@ -2911,7 +2911,7 @@ func (du *baseDateArithmitical) getIntervalFromDecimal(ctx sessionctx.Context, a return interval, false, nil } -func (du *baseDateArithmitical) getIntervalFromInt(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (string, bool, error) { +func (du *baseDateArithmetical) getIntervalFromInt(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (string, bool, error) { interval, isNull, err := args[1].EvalInt(ctx, row) if isNull || err != nil { return "", true, err @@ -2919,7 +2919,7 @@ func (du *baseDateArithmitical) getIntervalFromInt(ctx sessionctx.Context, args return strconv.FormatInt(interval, 10), false, nil } -func (du *baseDateArithmitical) getIntervalFromReal(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (string, bool, error) { +func (du *baseDateArithmetical) getIntervalFromReal(ctx sessionctx.Context, args []Expression, row chunk.Row, unit string) (string, bool, error) { interval, isNull, err := args[1].EvalReal(ctx, row) if isNull || err != nil { return "", true, err @@ -2927,7 +2927,7 @@ func (du *baseDateArithmitical) getIntervalFromReal(ctx sessionctx.Context, args return strconv.FormatFloat(interval, 'f', args[1].GetType().Decimal, 64), false, nil } -func (du *baseDateArithmitical) add(ctx sessionctx.Context, date types.Time, interval string, unit string) (types.Time, bool, error) { +func (du *baseDateArithmetical) add(ctx sessionctx.Context, date types.Time, interval string, unit string) (types.Time, bool, error) { year, month, day, nano, err := types.ParseDurationValue(unit, interval) if err := handleInvalidTimeError(ctx, err); err != nil { return types.ZeroTime, true, err @@ -2935,7 +2935,7 @@ func (du *baseDateArithmitical) add(ctx sessionctx.Context, date types.Time, int return du.addDate(ctx, date, year, month, day, nano) } -func (du *baseDateArithmitical) addDate(ctx sessionctx.Context, date types.Time, year, month, day, nano int64) (types.Time, bool, error) { +func (du *baseDateArithmetical) addDate(ctx sessionctx.Context, date types.Time, year, month, day, nano int64) (types.Time, bool, error) { goTime, err := date.GoTime(time.UTC) if err := handleInvalidTimeError(ctx, err); err != nil { return types.ZeroTime, true, err @@ -2972,7 +2972,7 @@ func (du *baseDateArithmitical) addDate(ctx sessionctx.Context, date types.Time, return date, false, nil } -func (du *baseDateArithmitical) addDuration(ctx sessionctx.Context, d types.Duration, interval string, unit string) (types.Duration, bool, error) { +func (du *baseDateArithmetical) addDuration(ctx sessionctx.Context, d types.Duration, interval string, unit string) (types.Duration, bool, error) { dur, err := types.ExtractDurationValue(unit, interval) if err != nil { return types.ZeroDuration, true, handleInvalidTimeError(ctx, err) @@ -2984,7 +2984,7 @@ func (du *baseDateArithmitical) addDuration(ctx sessionctx.Context, d types.Dura return retDur, false, nil } -func (du *baseDateArithmitical) subDuration(ctx sessionctx.Context, d types.Duration, interval string, unit string) (types.Duration, bool, error) { +func (du *baseDateArithmetical) subDuration(ctx sessionctx.Context, d types.Duration, interval string, unit string) (types.Duration, bool, error) { dur, err := types.ExtractDurationValue(unit, interval) if err != nil { return types.ZeroDuration, true, handleInvalidTimeError(ctx, err) @@ -2996,7 +2996,7 @@ func (du *baseDateArithmitical) subDuration(ctx sessionctx.Context, d types.Dura return retDur, false, nil } -func (du *baseDateArithmitical) sub(ctx sessionctx.Context, date types.Time, interval string, unit string) (types.Time, bool, error) { +func (du *baseDateArithmetical) sub(ctx sessionctx.Context, date types.Time, interval string, unit string) (types.Time, bool, error) { year, month, day, nano, err := types.ParseDurationValue(unit, interval) if err := handleInvalidTimeError(ctx, err); err != nil { return types.ZeroTime, true, err @@ -3004,7 +3004,7 @@ func (du *baseDateArithmitical) sub(ctx sessionctx.Context, date types.Time, int return du.addDate(ctx, date, -year, -month, -day, -nano) } -func (du *baseDateArithmitical) vecGetDateFromInt(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { +func (du *baseDateArithmetical) vecGetDateFromInt(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { n := input.NumRows() buf, err := b.bufAllocator.get(types.ETInt, n) if err != nil { @@ -3046,7 +3046,7 @@ func (du *baseDateArithmitical) vecGetDateFromInt(b *baseBuiltinFunc, input *chu return nil } -func (du *baseDateArithmitical) vecGetDateFromString(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { +func (du *baseDateArithmetical) vecGetDateFromString(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { n := input.NumRows() buf, err := b.bufAllocator.get(types.ETString, n) if err != nil { @@ -3090,7 +3090,7 @@ func (du *baseDateArithmitical) vecGetDateFromString(b *baseBuiltinFunc, input * return nil } -func (du *baseDateArithmitical) vecGetDateFromDatetime(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { +func (du *baseDateArithmetical) vecGetDateFromDatetime(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { n := input.NumRows() result.ResizeTime(n, false) if err := b.args[0].VecEvalTime(b.ctx, input, result); err != nil { @@ -3111,7 +3111,7 @@ func (du *baseDateArithmitical) vecGetDateFromDatetime(b *baseBuiltinFunc, input return nil } -func (du *baseDateArithmitical) vecGetIntervalFromString(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { +func (du *baseDateArithmetical) vecGetIntervalFromString(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { n := input.NumRows() buf, err := b.bufAllocator.get(types.ETString, n) if err != nil { @@ -3148,7 +3148,7 @@ func (du *baseDateArithmitical) vecGetIntervalFromString(b *baseBuiltinFunc, inp return nil } -func (du *baseDateArithmitical) vecGetIntervalFromDecimal(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { +func (du *baseDateArithmetical) vecGetIntervalFromDecimal(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { n := input.NumRows() buf, err := b.bufAllocator.get(types.ETDecimal, n) if err != nil { @@ -3249,7 +3249,7 @@ func (du *baseDateArithmitical) vecGetIntervalFromDecimal(b *baseBuiltinFunc, in return nil } -func (du *baseDateArithmitical) vecGetIntervalFromInt(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { +func (du *baseDateArithmetical) vecGetIntervalFromInt(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { n := input.NumRows() buf, err := b.bufAllocator.get(types.ETInt, n) if err != nil { @@ -3272,7 +3272,7 @@ func (du *baseDateArithmitical) vecGetIntervalFromInt(b *baseBuiltinFunc, input return nil } -func (du *baseDateArithmitical) vecGetIntervalFromReal(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { +func (du *baseDateArithmetical) vecGetIntervalFromReal(b *baseBuiltinFunc, input *chunk.Chunk, unit string, result *chunk.Column) error { n := input.NumRows() buf, err := b.bufAllocator.get(types.ETReal, n) if err != nil { @@ -3356,97 +3356,97 @@ func (c *addDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres case dateEvalTp == types.ETString && intervalEvalTp == types.ETString: sig = &builtinAddDateStringStringSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateStringString) case dateEvalTp == types.ETString && intervalEvalTp == types.ETInt: sig = &builtinAddDateStringIntSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateStringInt) case dateEvalTp == types.ETString && intervalEvalTp == types.ETReal: sig = &builtinAddDateStringRealSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateStringReal) case dateEvalTp == types.ETString && intervalEvalTp == types.ETDecimal: sig = &builtinAddDateStringDecimalSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateStringDecimal) case dateEvalTp == types.ETInt && intervalEvalTp == types.ETString: sig = &builtinAddDateIntStringSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateIntString) case dateEvalTp == types.ETInt && intervalEvalTp == types.ETInt: sig = &builtinAddDateIntIntSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateIntInt) case dateEvalTp == types.ETInt && intervalEvalTp == types.ETReal: sig = &builtinAddDateIntRealSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateIntReal) case dateEvalTp == types.ETInt && intervalEvalTp == types.ETDecimal: sig = &builtinAddDateIntDecimalSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateIntDecimal) case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETString: sig = &builtinAddDateDatetimeStringSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateDatetimeString) case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETInt: sig = &builtinAddDateDatetimeIntSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateDatetimeInt) case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETReal: sig = &builtinAddDateDatetimeRealSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateDatetimeReal) case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETDecimal: sig = &builtinAddDateDatetimeDecimalSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateDatetimeDecimal) case dateEvalTp == types.ETDuration && intervalEvalTp == types.ETString: sig = &builtinAddDateDurationStringSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateDurationString) case dateEvalTp == types.ETDuration && intervalEvalTp == types.ETInt: sig = &builtinAddDateDurationIntSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateDurationInt) case dateEvalTp == types.ETDuration && intervalEvalTp == types.ETReal: sig = &builtinAddDateDurationRealSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateDurationReal) case dateEvalTp == types.ETDuration && intervalEvalTp == types.ETDecimal: sig = &builtinAddDateDurationDecimalSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_AddDateDurationDecimal) } @@ -3455,11 +3455,11 @@ func (c *addDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres type builtinAddDateStringStringSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateStringStringSig) Clone() builtinFunc { - newSig := &builtinAddDateStringStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateStringStringSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3488,11 +3488,11 @@ func (b *builtinAddDateStringStringSig) evalTime(row chunk.Row) (types.Time, boo type builtinAddDateStringIntSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateStringIntSig) Clone() builtinFunc { - newSig := &builtinAddDateStringIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateStringIntSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3521,11 +3521,11 @@ func (b *builtinAddDateStringIntSig) evalTime(row chunk.Row) (types.Time, bool, type builtinAddDateStringRealSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateStringRealSig) Clone() builtinFunc { - newSig := &builtinAddDateStringRealSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateStringRealSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3554,11 +3554,11 @@ func (b *builtinAddDateStringRealSig) evalTime(row chunk.Row) (types.Time, bool, type builtinAddDateStringDecimalSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateStringDecimalSig) Clone() builtinFunc { - newSig := &builtinAddDateStringDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateStringDecimalSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3587,11 +3587,11 @@ func (b *builtinAddDateStringDecimalSig) evalTime(row chunk.Row) (types.Time, bo type builtinAddDateIntStringSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateIntStringSig) Clone() builtinFunc { - newSig := &builtinAddDateIntStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateIntStringSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3620,11 +3620,11 @@ func (b *builtinAddDateIntStringSig) evalTime(row chunk.Row) (types.Time, bool, type builtinAddDateIntIntSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateIntIntSig) Clone() builtinFunc { - newSig := &builtinAddDateIntIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateIntIntSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3653,11 +3653,11 @@ func (b *builtinAddDateIntIntSig) evalTime(row chunk.Row) (types.Time, bool, err type builtinAddDateIntRealSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateIntRealSig) Clone() builtinFunc { - newSig := &builtinAddDateIntRealSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateIntRealSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3686,11 +3686,11 @@ func (b *builtinAddDateIntRealSig) evalTime(row chunk.Row) (types.Time, bool, er type builtinAddDateIntDecimalSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateIntDecimalSig) Clone() builtinFunc { - newSig := &builtinAddDateIntDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateIntDecimalSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3719,11 +3719,11 @@ func (b *builtinAddDateIntDecimalSig) evalTime(row chunk.Row) (types.Time, bool, type builtinAddDateDatetimeStringSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateDatetimeStringSig) Clone() builtinFunc { - newSig := &builtinAddDateDatetimeStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateDatetimeStringSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3752,11 +3752,11 @@ func (b *builtinAddDateDatetimeStringSig) evalTime(row chunk.Row) (types.Time, b type builtinAddDateDatetimeIntSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateDatetimeIntSig) Clone() builtinFunc { - newSig := &builtinAddDateDatetimeIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateDatetimeIntSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3785,11 +3785,11 @@ func (b *builtinAddDateDatetimeIntSig) evalTime(row chunk.Row) (types.Time, bool type builtinAddDateDatetimeRealSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateDatetimeRealSig) Clone() builtinFunc { - newSig := &builtinAddDateDatetimeRealSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateDatetimeRealSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3818,11 +3818,11 @@ func (b *builtinAddDateDatetimeRealSig) evalTime(row chunk.Row) (types.Time, boo type builtinAddDateDatetimeDecimalSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateDatetimeDecimalSig) Clone() builtinFunc { - newSig := &builtinAddDateDatetimeDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateDatetimeDecimalSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3851,11 +3851,11 @@ func (b *builtinAddDateDatetimeDecimalSig) evalTime(row chunk.Row) (types.Time, type builtinAddDateDurationStringSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateDurationStringSig) Clone() builtinFunc { - newSig := &builtinAddDateDurationStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateDurationStringSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3882,11 +3882,11 @@ func (b *builtinAddDateDurationStringSig) evalDuration(row chunk.Row) (types.Dur type builtinAddDateDurationIntSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateDurationIntSig) Clone() builtinFunc { - newSig := &builtinAddDateDurationIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateDurationIntSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3912,11 +3912,11 @@ func (b *builtinAddDateDurationIntSig) evalDuration(row chunk.Row) (types.Durati type builtinAddDateDurationDecimalSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateDurationDecimalSig) Clone() builtinFunc { - newSig := &builtinAddDateDurationDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateDurationDecimalSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -3942,11 +3942,11 @@ func (b *builtinAddDateDurationDecimalSig) evalDuration(row chunk.Row) (types.Du type builtinAddDateDurationRealSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinAddDateDurationRealSig) Clone() builtinFunc { - newSig := &builtinAddDateDurationRealSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinAddDateDurationRealSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4030,97 +4030,97 @@ func (c *subDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres case dateEvalTp == types.ETString && intervalEvalTp == types.ETString: sig = &builtinSubDateStringStringSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateStringString) case dateEvalTp == types.ETString && intervalEvalTp == types.ETInt: sig = &builtinSubDateStringIntSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateStringInt) case dateEvalTp == types.ETString && intervalEvalTp == types.ETReal: sig = &builtinSubDateStringRealSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateStringReal) case dateEvalTp == types.ETString && intervalEvalTp == types.ETDecimal: sig = &builtinSubDateStringDecimalSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateStringDecimal) case dateEvalTp == types.ETInt && intervalEvalTp == types.ETString: sig = &builtinSubDateIntStringSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateIntString) case dateEvalTp == types.ETInt && intervalEvalTp == types.ETInt: sig = &builtinSubDateIntIntSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateIntInt) case dateEvalTp == types.ETInt && intervalEvalTp == types.ETReal: sig = &builtinSubDateIntRealSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateIntReal) case dateEvalTp == types.ETInt && intervalEvalTp == types.ETDecimal: sig = &builtinSubDateIntDecimalSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateIntDecimal) case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETString: sig = &builtinSubDateDatetimeStringSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateDatetimeString) case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETInt: sig = &builtinSubDateDatetimeIntSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateDatetimeInt) case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETReal: sig = &builtinSubDateDatetimeRealSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateDatetimeReal) case dateEvalTp == types.ETDatetime && intervalEvalTp == types.ETDecimal: sig = &builtinSubDateDatetimeDecimalSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateDatetimeDecimal) case dateEvalTp == types.ETDuration && intervalEvalTp == types.ETString: sig = &builtinSubDateDurationStringSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateDurationString) case dateEvalTp == types.ETDuration && intervalEvalTp == types.ETInt: sig = &builtinSubDateDurationIntSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateDurationInt) case dateEvalTp == types.ETDuration && intervalEvalTp == types.ETReal: sig = &builtinSubDateDurationRealSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateDurationReal) case dateEvalTp == types.ETDuration && intervalEvalTp == types.ETDecimal: sig = &builtinSubDateDurationDecimalSig{ baseBuiltinFunc: bf, - baseDateArithmitical: newDateArighmeticalUtil(), + baseDateArithmetical: newDateArighmeticalUtil(), } sig.setPbCode(tipb.ScalarFuncSig_SubDateDurationDecimal) } @@ -4129,11 +4129,11 @@ func (c *subDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres type builtinSubDateStringStringSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateStringStringSig) Clone() builtinFunc { - newSig := &builtinSubDateStringStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateStringStringSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4162,11 +4162,11 @@ func (b *builtinSubDateStringStringSig) evalTime(row chunk.Row) (types.Time, boo type builtinSubDateStringIntSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateStringIntSig) Clone() builtinFunc { - newSig := &builtinSubDateStringIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateStringIntSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4195,11 +4195,11 @@ func (b *builtinSubDateStringIntSig) evalTime(row chunk.Row) (types.Time, bool, type builtinSubDateStringRealSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateStringRealSig) Clone() builtinFunc { - newSig := &builtinSubDateStringRealSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateStringRealSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4228,11 +4228,11 @@ func (b *builtinSubDateStringRealSig) evalTime(row chunk.Row) (types.Time, bool, type builtinSubDateStringDecimalSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateStringDecimalSig) Clone() builtinFunc { - newSig := &builtinSubDateStringDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateStringDecimalSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4259,11 +4259,11 @@ func (b *builtinSubDateStringDecimalSig) evalTime(row chunk.Row) (types.Time, bo type builtinSubDateIntStringSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateIntStringSig) Clone() builtinFunc { - newSig := &builtinSubDateIntStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateIntStringSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4292,11 +4292,11 @@ func (b *builtinSubDateIntStringSig) evalTime(row chunk.Row) (types.Time, bool, type builtinSubDateIntIntSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateIntIntSig) Clone() builtinFunc { - newSig := &builtinSubDateIntIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateIntIntSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4325,11 +4325,11 @@ func (b *builtinSubDateIntIntSig) evalTime(row chunk.Row) (types.Time, bool, err type builtinSubDateIntRealSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateIntRealSig) Clone() builtinFunc { - newSig := &builtinSubDateIntRealSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateIntRealSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4358,16 +4358,16 @@ func (b *builtinSubDateIntRealSig) evalTime(row chunk.Row) (types.Time, bool, er type builtinSubDateDatetimeStringSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } type builtinSubDateIntDecimalSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateIntDecimalSig) Clone() builtinFunc { - newSig := &builtinSubDateIntDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateIntDecimalSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4395,7 +4395,7 @@ func (b *builtinSubDateIntDecimalSig) evalTime(row chunk.Row) (types.Time, bool, } func (b *builtinSubDateDatetimeStringSig) Clone() builtinFunc { - newSig := &builtinSubDateDatetimeStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateDatetimeStringSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4424,11 +4424,11 @@ func (b *builtinSubDateDatetimeStringSig) evalTime(row chunk.Row) (types.Time, b type builtinSubDateDatetimeIntSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateDatetimeIntSig) Clone() builtinFunc { - newSig := &builtinSubDateDatetimeIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateDatetimeIntSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4457,11 +4457,11 @@ func (b *builtinSubDateDatetimeIntSig) evalTime(row chunk.Row) (types.Time, bool type builtinSubDateDatetimeRealSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateDatetimeRealSig) Clone() builtinFunc { - newSig := &builtinSubDateDatetimeRealSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateDatetimeRealSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4490,11 +4490,11 @@ func (b *builtinSubDateDatetimeRealSig) evalTime(row chunk.Row) (types.Time, boo type builtinSubDateDatetimeDecimalSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateDatetimeDecimalSig) Clone() builtinFunc { - newSig := &builtinSubDateDatetimeDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateDatetimeDecimalSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4523,11 +4523,11 @@ func (b *builtinSubDateDatetimeDecimalSig) evalTime(row chunk.Row) (types.Time, type builtinSubDateDurationStringSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateDurationStringSig) Clone() builtinFunc { - newSig := &builtinSubDateDurationStringSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateDurationStringSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4554,11 +4554,11 @@ func (b *builtinSubDateDurationStringSig) evalDuration(row chunk.Row) (types.Dur type builtinSubDateDurationIntSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateDurationIntSig) Clone() builtinFunc { - newSig := &builtinSubDateDurationIntSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateDurationIntSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4585,11 +4585,11 @@ func (b *builtinSubDateDurationIntSig) evalDuration(row chunk.Row) (types.Durati type builtinSubDateDurationDecimalSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateDurationDecimalSig) Clone() builtinFunc { - newSig := &builtinSubDateDurationDecimalSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateDurationDecimalSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } @@ -4616,11 +4616,11 @@ func (b *builtinSubDateDurationDecimalSig) evalDuration(row chunk.Row) (types.Du type builtinSubDateDurationRealSig struct { baseBuiltinFunc - baseDateArithmitical + baseDateArithmetical } func (b *builtinSubDateDurationRealSig) Clone() builtinFunc { - newSig := &builtinSubDateDurationRealSig{baseDateArithmitical: b.baseDateArithmitical} + newSig := &builtinSubDateDurationRealSig{baseDateArithmetical: b.baseDateArithmetical} newSig.cloneFrom(&b.baseBuiltinFunc) return newSig } diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index 161912b07e973..e247e8756ae9a 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -2955,7 +2955,7 @@ func (s *testEvaluatorSuite) TestTiDBBoundedStaleness(c *C) { } func (s *testEvaluatorSuite) TestGetIntervalFromDecimal(c *C) { - du := baseDateArithmitical{} + du := baseDateArithmetical{} tests := []struct { param string From 66c8cd96b0eadd95cebe4bab37cd798422496fdd Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 18 May 2021 18:21:40 +0800 Subject: [PATCH 093/343] store/copr: balance region for batch cop task (#24521) --- store/copr/batch_coprocessor.go | 204 +++++++++++++++++-- store/copr/mpp.go | 14 +- store/{copr => tikv}/batch_request_sender.go | 54 ++--- store/tikv/region_cache.go | 101 +++++++++ 4 files changed, 321 insertions(+), 52 deletions(-) rename store/{copr => tikv}/batch_request_sender.go (54%) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index b0c0ad5c9ea7b..8c73f58fbf892 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -16,6 +16,8 @@ package copr import ( "context" "io" + "math" + "strconv" "sync" "sync/atomic" "time" @@ -25,6 +27,7 @@ import ( "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/log" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" @@ -40,8 +43,9 @@ import ( type batchCopTask struct { storeAddr string cmdType tikvrpc.CmdType + ctx *tikv.RPCContext - copTasks []copTaskAndRPCContext + regionInfos []tikv.RegionInfo } type batchCopResponse struct { @@ -93,9 +97,152 @@ func (rs *batchCopResponse) RespTime() time.Duration { return rs.respTime } -type copTaskAndRPCContext struct { - task *copTask - ctx *tikv.RPCContext +// balanceBatchCopTask balance the regions between available stores, the basic rule is +// 1. the first region of each original batch cop task belongs to its original store because some +// meta data(like the rpc context) in batchCopTask is related to it +// 2. for the remaining regions: +// if there is only 1 available store, then put the region to the related store +// otherwise, use a greedy algorithm to put it into the store with highest weight +func balanceBatchCopTask(originalTasks []*batchCopTask) []*batchCopTask { + if len(originalTasks) <= 1 { + return originalTasks + } + storeTaskMap := make(map[uint64]*batchCopTask) + storeCandidateRegionMap := make(map[uint64]map[string]tikv.RegionInfo) + totalRegionCandidateNum := 0 + totalRemainingRegionNum := 0 + + for _, task := range originalTasks { + taskStoreID := task.regionInfos[0].AllStores[0] + batchTask := &batchCopTask{ + storeAddr: task.storeAddr, + cmdType: task.cmdType, + ctx: task.ctx, + regionInfos: []tikv.RegionInfo{task.regionInfos[0]}, + } + storeTaskMap[taskStoreID] = batchTask + } + + for _, task := range originalTasks { + taskStoreID := task.regionInfos[0].AllStores[0] + for index, ri := range task.regionInfos { + // for each region, figure out the valid store num + validStoreNum := 0 + if index == 0 { + continue + } + if len(ri.AllStores) <= 1 { + validStoreNum = 1 + } else { + for _, storeID := range ri.AllStores { + if _, ok := storeTaskMap[storeID]; ok { + validStoreNum++ + } + } + } + if validStoreNum == 1 { + // if only one store is valid, just put it to storeTaskMap + storeTaskMap[taskStoreID].regionInfos = append(storeTaskMap[taskStoreID].regionInfos, ri) + } else { + // if more than one store is valid, put the region + // to store candidate map + totalRegionCandidateNum += validStoreNum + totalRemainingRegionNum += 1 + taskKey := ri.Region.String() + for _, storeID := range ri.AllStores { + if _, validStore := storeTaskMap[storeID]; !validStore { + continue + } + if _, ok := storeCandidateRegionMap[storeID]; !ok { + candidateMap := make(map[string]tikv.RegionInfo) + storeCandidateRegionMap[storeID] = candidateMap + } + if _, duplicateRegion := storeCandidateRegionMap[storeID][taskKey]; duplicateRegion { + // duplicated region, should not happen, just give up balance + logutil.BgLogger().Warn("Meet duplicated region info during when trying to balance batch cop task, give up balancing") + return originalTasks + } + storeCandidateRegionMap[storeID][taskKey] = ri + } + } + } + } + if totalRemainingRegionNum == 0 { + return originalTasks + } + + avgStorePerRegion := float64(totalRegionCandidateNum) / float64(totalRemainingRegionNum) + findNextStore := func(candidateStores []uint64) uint64 { + store := uint64(math.MaxUint64) + weightedRegionNum := math.MaxFloat64 + if candidateStores != nil { + for _, storeID := range candidateStores { + if _, validStore := storeCandidateRegionMap[storeID]; !validStore { + continue + } + num := float64(len(storeCandidateRegionMap[storeID]))/avgStorePerRegion + float64(len(storeTaskMap[storeID].regionInfos)) + if num < weightedRegionNum { + store = storeID + weightedRegionNum = num + } + } + if store != uint64(math.MaxUint64) { + return store + } + } + for storeID := range storeTaskMap { + if _, validStore := storeCandidateRegionMap[storeID]; !validStore { + continue + } + num := float64(len(storeCandidateRegionMap[storeID]))/avgStorePerRegion + float64(len(storeTaskMap[storeID].regionInfos)) + if num < weightedRegionNum { + store = storeID + weightedRegionNum = num + } + } + return store + } + + store := findNextStore(nil) + for totalRemainingRegionNum > 0 { + if store == uint64(math.MaxUint64) { + break + } + var key string + var ri tikv.RegionInfo + for key, ri = range storeCandidateRegionMap[store] { + // get the first region + break + } + storeTaskMap[store].regionInfos = append(storeTaskMap[store].regionInfos, ri) + totalRemainingRegionNum-- + for _, id := range ri.AllStores { + if _, ok := storeCandidateRegionMap[id]; ok { + delete(storeCandidateRegionMap[id], key) + totalRegionCandidateNum-- + if len(storeCandidateRegionMap[id]) == 0 { + delete(storeCandidateRegionMap, id) + } + } + } + if totalRemainingRegionNum > 0 { + avgStorePerRegion = float64(totalRegionCandidateNum) / float64(totalRemainingRegionNum) + // it is not optimal because we only check the stores that affected by this region, in fact in order + // to find out the store with the lowest weightedRegionNum, all stores should be checked, but I think + // check only the affected stores is more simple and will get a good enough result + store = findNextStore(ri.AllStores) + } + } + if totalRemainingRegionNum > 0 { + logutil.BgLogger().Warn("Some regions are not used when trying to balance batch cop task, give up balancing") + return originalTasks + } + + var ret []*batchCopTask + for _, task := range storeTaskMap { + ret = append(ret, task) + } + return ret } func buildBatchCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { @@ -138,13 +285,15 @@ func buildBatchCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.Key // Then `splitRegion` will reloads these regions. continue } + allStores := cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store) if batchCop, ok := storeTaskMap[rpcCtx.Addr]; ok { - batchCop.copTasks = append(batchCop.copTasks, copTaskAndRPCContext{task: task, ctx: rpcCtx}) + batchCop.regionInfos = append(batchCop.regionInfos, tikv.RegionInfo{Region: task.region, Meta: rpcCtx.Meta, Ranges: task.ranges, AllStores: allStores}) } else { batchTask := &batchCopTask{ - storeAddr: rpcCtx.Addr, - cmdType: cmdType, - copTasks: []copTaskAndRPCContext{{task, rpcCtx}}, + storeAddr: rpcCtx.Addr, + cmdType: cmdType, + ctx: rpcCtx, + regionInfos: []tikv.RegionInfo{{Region: task.region, Meta: rpcCtx.Meta, Ranges: task.ranges, AllStores: allStores}}, } storeTaskMap[rpcCtx.Addr] = batchTask } @@ -159,9 +308,25 @@ func buildBatchCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.Key } continue } + for _, task := range storeTaskMap { batchTasks = append(batchTasks, task) } + if log.GetLevel() <= zap.DebugLevel { + msg := "Before region balance:" + for _, task := range batchTasks { + msg += " store " + task.storeAddr + ": " + strconv.Itoa(len(task.regionInfos)) + " regions," + } + logutil.BgLogger().Debug(msg) + } + batchTasks = balanceBatchCopTask(batchTasks) + if log.GetLevel() <= zap.DebugLevel { + msg := "After region balance:" + for _, task := range batchTasks { + msg += " store " + task.storeAddr + ": " + strconv.Itoa(len(task.regionInfos)) + " regions," + } + logutil.BgLogger().Debug(msg) + } if elapsed := time.Since(start); elapsed > time.Millisecond*500 { logutil.BgLogger().Warn("buildBatchCopTasks takes too much time", @@ -311,8 +476,8 @@ func (b *batchCopIterator) handleTask(ctx context.Context, bo *Backoffer, task * // Merge all ranges and request again. func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *Backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { var ranges []tikvstore.KeyRange - for _, taskCtx := range batchTask.copTasks { - taskCtx.task.ranges.Do(func(ran *tikvstore.KeyRange) { + for _, ri := range batchTask.regionInfos { + ri.Ranges.Do(func(ran *tikvstore.KeyRange) { ranges = append(ranges, *ran) }) } @@ -320,16 +485,16 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *Backoffer, } func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *Backoffer, task *batchCopTask) ([]*batchCopTask, error) { - sender := NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) - var regionInfos = make([]*coprocessor.RegionInfo, 0, len(task.copTasks)) - for _, task := range task.copTasks { + sender := tikv.NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) + var regionInfos = make([]*coprocessor.RegionInfo, 0, len(task.regionInfos)) + for _, ri := range task.regionInfos { regionInfos = append(regionInfos, &coprocessor.RegionInfo{ - RegionId: task.task.region.GetID(), + RegionId: ri.Region.GetID(), RegionEpoch: &metapb.RegionEpoch{ - ConfVer: task.task.region.GetConfVer(), - Version: task.task.region.GetVer(), + ConfVer: ri.Region.GetConfVer(), + Version: ri.Region.GetVer(), }, - Ranges: task.task.ranges.ToPBRanges(), + Ranges: ri.Ranges.ToPBRanges(), }) } @@ -351,13 +516,14 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *Backoffer, ta }) req.StoreTp = tikvrpc.TiFlash - logutil.BgLogger().Debug("send batch request to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.copTasks))) - resp, retry, cancel, err := sender.sendStreamReqToAddr(bo, task.copTasks, req, tikv.ReadTimeoutUltraLong) + logutil.BgLogger().Debug("send batch request to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.regionInfos))) + resp, retry, cancel, err := sender.SendReqToAddr(bo.TiKVBackoffer(), task.ctx, task.regionInfos, req, tikv.ReadTimeoutUltraLong) // If there are store errors, we should retry for all regions. if retry { return b.retryBatchCopTask(ctx, bo, task) } if err != nil { + err = derr.ToTiDBErr(err) return nil, errors.Trace(err) } defer cancel() diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 2aaf4223ed8e5..1941f2b3fbfa4 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -180,14 +180,14 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req var regionInfos []*coprocessor.RegionInfo originalTask, ok := req.Meta.(*batchCopTask) if ok { - for _, task := range originalTask.copTasks { + for _, ri := range originalTask.regionInfos { regionInfos = append(regionInfos, &coprocessor.RegionInfo{ - RegionId: task.task.region.GetID(), + RegionId: ri.Region.GetID(), RegionEpoch: &metapb.RegionEpoch{ - ConfVer: task.task.region.GetConfVer(), - Version: task.task.region.GetVer(), + ConfVer: ri.Region.GetConfVer(), + Version: ri.Region.GetVer(), }, - Ranges: task.task.ranges.ToPBRanges(), + Ranges: ri.Ranges.ToPBRanges(), }) } } @@ -214,8 +214,8 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req // Or else it's the task without region, which always happens in high layer task without table. // In that case if originalTask != nil { - sender := NewRegionBatchRequestSender(m.store.GetRegionCache(), m.store.GetTiKVClient()) - rpcResp, _, _, err = sender.sendStreamReqToAddr(bo, originalTask.copTasks, wrappedReq, tikv.ReadTimeoutMedium) + sender := tikv.NewRegionBatchRequestSender(m.store.GetRegionCache(), m.store.GetTiKVClient()) + rpcResp, _, _, err = sender.SendReqToAddr(bo.TiKVBackoffer(), originalTask.ctx, originalTask.regionInfos, wrappedReq, tikv.ReadTimeoutMedium) // No matter what the rpc error is, we won't retry the mpp dispatch tasks. // TODO: If we want to retry, we must redo the plan fragment cutting and task scheduling. // That's a hard job but we can try it in the future. diff --git a/store/copr/batch_request_sender.go b/store/tikv/batch_request_sender.go similarity index 54% rename from store/copr/batch_request_sender.go rename to store/tikv/batch_request_sender.go index 422306382337d..9aad070b70306 100644 --- a/store/copr/batch_request_sender.go +++ b/store/tikv/batch_request_sender.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package copr +package tikv import ( "context" @@ -19,45 +19,52 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/kvproto/pkg/metapb" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/tikvrpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" ) +// RegionInfo contains region related information for batchCopTask +type RegionInfo struct { + Region RegionVerID + Meta *metapb.Region + Ranges *KeyRanges + AllStores []uint64 +} + // RegionBatchRequestSender sends BatchCop requests to TiFlash server by stream way. type RegionBatchRequestSender struct { - *tikv.RegionRequestSender + *RegionRequestSender } // NewRegionBatchRequestSender creates a RegionBatchRequestSender object. -func NewRegionBatchRequestSender(cache *tikv.RegionCache, client tikv.Client) *RegionBatchRequestSender { +func NewRegionBatchRequestSender(cache *RegionCache, client Client) *RegionBatchRequestSender { return &RegionBatchRequestSender{ - RegionRequestSender: tikv.NewRegionRequestSender(cache, client), + RegionRequestSender: NewRegionRequestSender(cache, client), } } -func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *Backoffer, ctxs []copTaskAndRPCContext, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { - // use the first ctx to send request, because every ctx has same address. +// SendReqToAddr send batch cop request +func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *RPCContext, regionInfos []RegionInfo, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { cancel = func() {} - rpcCtx := ctxs[0].ctx if e := tikvrpc.SetContext(req, rpcCtx.Meta, rpcCtx.Peer); e != nil { return nil, false, cancel, errors.Trace(e) } ctx := bo.GetCtx() - if rawHook := ctx.Value(tikv.RPCCancellerCtxKey{}); rawHook != nil { - ctx, cancel = rawHook.(*tikv.RPCCanceller).WithCancel(ctx) + if rawHook := ctx.Value(RPCCancellerCtxKey{}); rawHook != nil { + ctx, cancel = rawHook.(*RPCCanceller).WithCancel(ctx) } start := time.Now() resp, err = ss.GetClient().SendRequest(ctx, rpcCtx.Addr, req, timout) if ss.Stats != nil { - tikv.RecordRegionRequestRuntimeStats(ss.Stats, req.Type, time.Since(start)) + RecordRegionRequestRuntimeStats(ss.Stats, req.Type, time.Since(start)) } if err != nil { cancel() ss.SetRPCError(err) - e := ss.onSendFail(bo, ctxs, err) + e := ss.onSendFailForBatchRegions(bo, rpcCtx, regionInfos, err) if e != nil { return nil, false, func() {}, errors.Trace(e) } @@ -67,30 +74,25 @@ func (ss *RegionBatchRequestSender) sendStreamReqToAddr(bo *Backoffer, ctxs []co return } -func (ss *RegionBatchRequestSender) onSendFail(bo *Backoffer, ctxs []copTaskAndRPCContext, err error) error { +func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx *RPCContext, regionInfos []RegionInfo, err error) error { // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { return errors.Trace(err) - } else if atomic.LoadUint32(&tikv.ShuttingDown) > 0 { + } else if atomic.LoadUint32(&ShuttingDown) > 0 { return tikverr.ErrTiDBShuttingDown } - for _, failedCtx := range ctxs { - ctx := failedCtx.ctx - if ctx.Meta != nil { - // The reload region param is always true. Because that every time we try, we must - // re-build the range then re-create the batch sender. As a result, the len of "failStores" - // will change. If tiflash's replica is more than two, the "reload region" will always be false. - // Now that the batch cop and mpp has a relative low qps, it's reasonable to reload every time - // when meeting io error. - ss.GetRegionCache().OnSendFail(bo.TiKVBackoffer(), ctx, true, err) - } - } + // The reload region param is always true. Because that every time we try, we must + // re-build the range then re-create the batch sender. As a result, the len of "failStores" + // will change. If tiflash's replica is more than two, the "reload region" will always be false. + // Now that the batch cop and mpp has a relative low qps, it's reasonable to reload every time + // when meeting io error. + ss.GetRegionCache().OnSendFailForBatchRegions(bo, ctx.Store, regionInfos, true, err) // Retry on send request failure when it's not canceled. // When a store is not available, the leader of related region should be elected quickly. // TODO: the number of retry time should be limited:since region may be unavailable // when some unrecoverable disaster happened. - err = bo.Backoff(tikv.BoTiFlashRPC, errors.Errorf("send tikv request error: %v, ctxs: %v, try next peer later", err, ctxs)) + err = bo.Backoff(BoTiFlashRPC, errors.Errorf("send request error: %v, ctx: %v, regionInfos: %v", err, ctx, regionInfos)) return errors.Trace(err) } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index f6225a2724f8e..0d9423a9f5a7e 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -112,6 +112,15 @@ func (r *RegionStore) accessStore(mode AccessMode, idx AccessIndex) (int, *Store return sidx, r.stores[sidx] } +func (r *RegionStore) getAccessIndex(mode AccessMode, store *Store) AccessIndex { + for index, sidx := range r.accessIndex[mode] { + if r.stores[sidx].storeID == store.storeID { + return AccessIndex(index) + } + } + return -1 +} + func (r *RegionStore) accessStoreNum(mode AccessMode) int { return len(r.accessIndex[mode]) } @@ -526,6 +535,40 @@ func (c *RegionCache) GetTiKVRPCContext(bo *Backoffer, id RegionVerID, replicaRe }, nil } +// GetAllValidTiFlashStores returns the store ids of all valid TiFlash stores, the store id of currentStore is always the first one +func (c *RegionCache) GetAllValidTiFlashStores(id RegionVerID, currentStore *Store) []uint64 { + // set the cap to 2 because usually, TiFlash table will have 2 replicas + allStores := make([]uint64, 0, 2) + // make sure currentStore id is always the first in allStores + allStores = append(allStores, currentStore.storeID) + ts := time.Now().Unix() + cachedRegion := c.getCachedRegionWithRLock(id) + if cachedRegion == nil { + return allStores + } + if !cachedRegion.checkRegionCacheTTL(ts) { + return allStores + } + regionStore := cachedRegion.getStore() + currentIndex := regionStore.getAccessIndex(TiFlashOnly, currentStore) + if currentIndex == -1 { + return allStores + } + for startOffset := 1; startOffset < regionStore.accessStoreNum(TiFlashOnly); startOffset++ { + accessIdx := AccessIndex((int(currentIndex) + startOffset) % regionStore.accessStoreNum(TiFlashOnly)) + storeIdx, store := regionStore.accessStore(TiFlashOnly, accessIdx) + if store.getResolveState() == needCheck { + continue + } + storeFailEpoch := atomic.LoadUint32(&store.epoch) + if storeFailEpoch != regionStore.storeEpochs[storeIdx] { + continue + } + allStores = append(allStores, store.storeID) + } + return allStores +} + // GetTiFlashRPCContext returns RPCContext for a region must access flash store. If it returns nil, the region // must be out of date and already dropped from cache or not flash store found. // `loadBalance` is an option. For MPP and batch cop, it is pointless and might cause try the failed store repeatly. @@ -668,6 +711,64 @@ func (c *RegionCache) findRegionByKey(bo *Backoffer, key []byte, isEndKey bool) return r, nil } +// OnSendFailForBatchRegions handles send request fail logic. +func (c *RegionCache) OnSendFailForBatchRegions(bo *Backoffer, store *Store, regionInfos []RegionInfo, scheduleReload bool, err error) { + metrics.RegionCacheCounterWithSendFail.Add(float64(len(regionInfos))) + if store.storeType != tikvrpc.TiFlash { + logutil.Logger(bo.GetCtx()).Info("Should not reach here, OnSendFailForBatchRegions only support TiFlash") + return + } + for _, ri := range regionInfos { + if ri.Meta == nil { + continue + } + r := c.getCachedRegionWithRLock(ri.Region) + if r != nil { + peersNum := len(r.meta.Peers) + if len(ri.Meta.Peers) != peersNum { + logutil.Logger(bo.GetCtx()).Info("retry and refresh current region after send request fail and up/down stores length changed", + zap.Stringer("region", &ri.Region), + zap.Bool("needReload", scheduleReload), + zap.Reflect("oldPeers", ri.Meta.Peers), + zap.Reflect("newPeers", r.meta.Peers), + zap.Error(err)) + continue + } + + rs := r.getStore() + + accessMode := TiFlashOnly + accessIdx := rs.getAccessIndex(accessMode, store) + if accessIdx == -1 { + logutil.Logger(bo.GetCtx()).Warn("can not get access index for region " + ri.Region.String()) + continue + } + if err != nil { + storeIdx, s := rs.accessStore(accessMode, accessIdx) + epoch := rs.storeEpochs[storeIdx] + if atomic.CompareAndSwapUint32(&s.epoch, epoch, epoch+1) { + logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) + metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() + } + // schedule a store addr resolve. + s.markNeedCheck(c.notifyCheckCh) + } + + // try next peer + rs.switchNextFlashPeer(r, accessIdx) + logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", + zap.Stringer("region", &ri.Region), + zap.Bool("needReload", scheduleReload), + zap.Error(err)) + + // force reload region when retry all known peers in region. + if scheduleReload { + r.scheduleReload() + } + } + } +} + // OnSendFail handles send request fail logic. func (c *RegionCache) OnSendFail(bo *Backoffer, ctx *RPCContext, scheduleReload bool, err error) { metrics.RegionCacheCounterWithSendFail.Inc() From fbbf2b49c56cf42bc8a6fbdb2cc2a1eaa0f65e54 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Tue, 18 May 2021 19:19:40 +0800 Subject: [PATCH 094/343] store, metrics: Add metrics for safetTS updating (#24687) --- store/tikv/kv.go | 6 +++++- store/tikv/metrics/metrics.go | 20 ++++++++++++++++++++ 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 8cec3dfbca964..edaef3b4744d7 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -18,6 +18,7 @@ import ( "crypto/tls" "math" "math/rand" + "strconv" "sync" "sync/atomic" "time" @@ -436,17 +437,20 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { storeAddr := store.addr go func(ctx context.Context, wg *sync.WaitGroup, storeID uint64, storeAddr string) { defer wg.Done() - // TODO: add metrics for updateSafeTS resp, err := tikvClient.SendRequest(ctx, storeAddr, tikvrpc.NewRequest(tikvrpc.CmdStoreSafeTS, &kvrpcpb.StoreSafeTSRequest{KeyRange: &kvrpcpb.KeyRange{ StartKey: []byte(""), EndKey: []byte(""), }}), ReadTimeoutShort) + storeIDStr := strconv.Itoa(int(storeID)) if err != nil { + metrics.TiKVSafeTSUpdateCounter.WithLabelValues("fail", storeIDStr).Inc() logutil.BgLogger().Debug("update safeTS failed", zap.Error(err), zap.Uint64("store-id", storeID)) return } safeTSResp := resp.Resp.(*kvrpcpb.StoreSafeTSResponse) s.setSafeTS(storeID, safeTSResp.GetSafeTs()) + metrics.TiKVSafeTSUpdateCounter.WithLabelValues("success", storeIDStr).Inc() + metrics.TiKVSafeTSUpdateStats.WithLabelValues(storeIDStr).Set(float64(safeTSResp.GetSafeTs())) }(ctx, wg, storeID, storeAddr) } wg.Wait() diff --git a/store/tikv/metrics/metrics.go b/store/tikv/metrics/metrics.go index 8d71582fa2522..6b8ea32d456f7 100644 --- a/store/tikv/metrics/metrics.go +++ b/store/tikv/metrics/metrics.go @@ -59,6 +59,8 @@ var ( TiKVPanicCounter *prometheus.CounterVec TiKVForwardRequestCounter *prometheus.CounterVec TiKVTSFutureWaitDuration prometheus.Histogram + TiKVSafeTSUpdateCounter *prometheus.CounterVec + TiKVSafeTSUpdateStats *prometheus.GaugeVec ) // Label constants. @@ -414,6 +416,22 @@ func initMetrics(namespace, subsystem string) { Buckets: prometheus.ExponentialBuckets(0.000005, 2, 30), // 5us ~ 2560s }) + TiKVSafeTSUpdateCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "safets_update_counter", + Help: "Counter of tikv safe_ts being updated.", + }, []string{LblResult, LblStore}) + + TiKVSafeTSUpdateStats = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "safets_update_stats", + Help: "stat of tikv updating safe_ts stats", + }, []string{LblStore}) + initShortcuts() } @@ -468,6 +486,8 @@ func RegisterMetrics() { prometheus.MustRegister(TiKVPanicCounter) prometheus.MustRegister(TiKVForwardRequestCounter) prometheus.MustRegister(TiKVTSFutureWaitDuration) + prometheus.MustRegister(TiKVSafeTSUpdateCounter) + prometheus.MustRegister(TiKVSafeTSUpdateStats) } // readCounter reads the value of a prometheus.Counter. From b515e14d3e62d41a5de87eb176c3cc3ffb146b4c Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Tue, 18 May 2021 06:13:40 -0600 Subject: [PATCH 095/343] sem: add tidbredact log to restricted variables (#24701) --- util/sem/sem.go | 6 ++++-- util/sem/sem_test.go | 2 ++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/util/sem/sem.go b/util/sem/sem.go index d29d29b601559..1aac6d0a9a999 100644 --- a/util/sem/sem.go +++ b/util/sem/sem.go @@ -138,6 +138,7 @@ func IsInvisibleSysVar(varNameInLower string) bool { variable.TiDBCheckMb4ValueInUTF8, variable.TiDBConfig, variable.TiDBEnableSlowLog, + variable.TiDBEnableTelemetry, variable.TiDBExpensiveQueryTimeThreshold, variable.TiDBForcePriority, variable.TiDBGeneralLog, @@ -146,12 +147,13 @@ func IsInvisibleSysVar(varNameInLower string) bool { variable.TiDBOptWriteRowID, variable.TiDBPProfSQLCPU, variable.TiDBRecordPlanInSlowLog, + variable.TiDBRowFormatVersion, variable.TiDBSlowQueryFile, variable.TiDBSlowLogThreshold, variable.TiDBEnableCollectExecutionInfo, variable.TiDBMemoryUsageAlarmRatio, - variable.TiDBEnableTelemetry, - variable.TiDBRowFormatVersion: + variable.TiDBRedactLog, + variable.TiDBSlowLogMasking: return true } return false diff --git a/util/sem/sem_test.go b/util/sem/sem_test.go index 073a195139c37..c2a54170dcf99 100644 --- a/util/sem/sem_test.go +++ b/util/sem/sem_test.go @@ -98,4 +98,6 @@ func (s *testSecurity) TestIsInvisibleSysVar(c *C) { c.Assert(IsInvisibleSysVar(variable.TiDBMemoryUsageAlarmRatio), IsTrue) c.Assert(IsInvisibleSysVar(variable.TiDBEnableTelemetry), IsTrue) c.Assert(IsInvisibleSysVar(variable.TiDBRowFormatVersion), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBRedactLog), IsTrue) + c.Assert(IsInvisibleSysVar(variable.TiDBSlowLogMasking), IsTrue) } From 44830b917465c5d46cd22f617a3e260f09f0b857 Mon Sep 17 00:00:00 2001 From: wjHuang Date: Wed, 19 May 2021 00:25:40 +0800 Subject: [PATCH 096/343] session: fix dml_batch_size doesn't load the global variable (#24710) --- session/session.go | 1 + session/session_test.go | 10 ++++++++++ 2 files changed, 11 insertions(+) diff --git a/session/session.go b/session/session.go index 78a60a6ebaecf..efd6706c4ffb3 100644 --- a/session/session.go +++ b/session/session.go @@ -2652,6 +2652,7 @@ var builtinGlobalVariable = []string{ variable.TiDBAllowFallbackToTiKV, variable.TiDBEnableDynamicPrivileges, variable.CTEMaxRecursionDepth, + variable.TiDBDMLBatchSize, } // loadCommonGlobalVariablesIfNeeded loads and applies commonly used global variables for the session. diff --git a/session/session_test.go b/session/session_test.go index f7267e3a13259..df2a167921e56 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4476,3 +4476,13 @@ func (s *testTxnStateSuite) TestRollbacking(c *C) { c.Assert(tk.Se.TxnInfo().State, Equals, txninfo.TxnRollingBack) <-ch } + +func (s *testSessionSuite) TestReadDMLBatchSize(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set global tidb_dml_batch_size=1000") + se, err := session.CreateSession(s.store) + c.Assert(err, IsNil) + // `select 1` to load the global variables. + _, _ = se.Execute(context.TODO(), "select 1") + c.Assert(se.GetSessionVars().DMLBatchSize, Equals, 1000) +} From 3ec8c8bae798642baf267601a5864238d3367b9e Mon Sep 17 00:00:00 2001 From: Lei Zhao Date: Wed, 19 May 2021 07:39:40 +0800 Subject: [PATCH 097/343] store/tikv: retry TSO RPC (#24682) --- store/tikv/2pc.go | 14 +++++--------- store/tikv/lock_resolver.go | 9 ++------- 2 files changed, 7 insertions(+), 16 deletions(-) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 19f3e4faf40e3..14609f5f77400 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -739,15 +739,11 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { return } bo := retry.NewBackofferWithVars(context.Background(), pessimisticLockMaxBackoff, c.txn.vars) - now, err := c.store.GetOracle().GetTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + now, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) if err != nil { - err1 := bo.Backoff(retry.BoPDRPC, err) - if err1 != nil { - logutil.Logger(bo.GetCtx()).Warn("keepAlive get tso fail", - zap.Error(err)) - return - } - continue + logutil.Logger(bo.GetCtx()).Warn("keepAlive get tso fail", + zap.Error(err)) + return } uptime := uint64(oracle.ExtractPhysical(now) - oracle.ExtractPhysical(c.startTS)) @@ -999,7 +995,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { // from PD and plus one as our MinCommitTS. if commitTSMayBeCalculated && c.needLinearizability() { failpoint.Inject("getMinCommitTSFromTSO", nil) - latestTS, err := c.store.oracle.GetTimestamp(ctx, &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + latestTS, err := c.store.getTimestampWithRetry(retry.NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) // If we fail to get a timestamp from PD, we just propagate the failure // instead of falling back to the normal 2PC because a normal 2PC will // also be likely to fail due to the same timestamp issue. diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index fe50910a896e6..0ed9ecb3fa471 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -229,11 +229,6 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc Regi // locks have been cleaned before GC. expiredLocks := locks - callerStartTS, err := lr.store.GetOracle().GetTimestamp(bo.GetCtx(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) - if err != nil { - return false, errors.Trace(err) - } - txnInfos := make(map[uint64]uint64) startTime := time.Now() for _, l := range expiredLocks { @@ -243,7 +238,7 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc Regi metrics.LockResolverCountWithExpired.Inc() // Use currentTS = math.MaxUint64 means rollback the txn, no matter the lock is expired or not! - status, err := lr.getTxnStatus(bo, l.TxnID, l.Primary, callerStartTS, math.MaxUint64, true, false, l) + status, err := lr.getTxnStatus(bo, l.TxnID, l.Primary, 0, math.MaxUint64, true, false, l) if err != nil { return false, err } @@ -257,7 +252,7 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc Regi continue } if _, ok := errors.Cause(err).(*nonAsyncCommitLock); ok { - status, err = lr.getTxnStatus(bo, l.TxnID, l.Primary, callerStartTS, math.MaxUint64, true, true, l) + status, err = lr.getTxnStatus(bo, l.TxnID, l.Primary, 0, math.MaxUint64, true, true, l) if err != nil { return false, err } From ac7e6a42c50367b55865a587fc4af08f2758f6ae Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Wed, 19 May 2021 07:51:40 +0800 Subject: [PATCH 098/343] expression, planner: push cast down to control function with enum type. (#24542) --- expression/builtin_cast.go | 90 ++++++++++++++++++++++++++++ expression/integration_test.go | 73 ++++++++++++++++++++++ planner/core/logical_plan_builder.go | 13 ++++ 3 files changed, 176 insertions(+) diff --git a/expression/builtin_cast.go b/expression/builtin_cast.go index 0f2d5827c91d9..9d6becab44cbe 100644 --- a/expression/builtin_cast.go +++ b/expression/builtin_cast.go @@ -1804,6 +1804,7 @@ func BuildCastFunction4Union(ctx sessionctx.Context, expr Expression, tp *types. // BuildCastFunction builds a CAST ScalarFunction from the Expression. func BuildCastFunction(ctx sessionctx.Context, expr Expression, tp *types.FieldType) (res Expression) { + expr = TryPushCastIntoControlFunctionForHybridType(ctx, expr, tp) var fc functionClass switch tp.EvalType() { case types.ETInt: @@ -1983,3 +1984,92 @@ func WrapWithCastAsJSON(ctx sessionctx.Context, expr Expression) Expression { } return BuildCastFunction(ctx, expr, tp) } + +// TryPushCastIntoControlFunctionForHybridType try to push cast into control function for Hybrid Type. +// If necessary, it will rebuild control function using changed args. +// When a hybrid type is the output of a control function, the result may be as a numeric type to subsequent calculation +// We should perform the `Cast` operation early to avoid using the wrong type for calculation +// For example, the condition `if(1, e, 'a') = 1`, `if` function will output `e` and compare with `1`. +// If the evaltype is ETString, it will get wrong result. So we can rewrite the condition to +// `IfInt(1, cast(e as int), cast('a' as int)) = 1` to get the correct result. +func TryPushCastIntoControlFunctionForHybridType(ctx sessionctx.Context, expr Expression, tp *types.FieldType) (res Expression) { + sf, ok := expr.(*ScalarFunction) + if !ok { + return expr + } + + var wrapCastFunc func(ctx sessionctx.Context, expr Expression) Expression + switch tp.EvalType() { + case types.ETInt: + wrapCastFunc = WrapWithCastAsInt + case types.ETReal: + wrapCastFunc = WrapWithCastAsReal + default: + return expr + } + + isHybrid := func(ft *types.FieldType) bool { + // todo: compatible with mysql control function using bit type. issue 24725 + return ft.Hybrid() && ft.Tp != mysql.TypeBit + } + + args := sf.GetArgs() + switch sf.FuncName.L { + case ast.If: + if isHybrid(args[1].GetType()) || isHybrid(args[2].GetType()) { + args[1] = wrapCastFunc(ctx, args[1]) + args[2] = wrapCastFunc(ctx, args[2]) + f, err := funcs[ast.If].getFunction(ctx, args) + if err != nil { + return expr + } + sf.RetType, sf.Function = f.getRetTp(), f + return sf + } + case ast.Case: + hasHybrid := false + for i := 0; i < len(args)-1; i += 2 { + hasHybrid = hasHybrid || isHybrid(args[i+1].GetType()) + } + if len(args)%2 == 1 { + hasHybrid = hasHybrid || isHybrid(args[len(args)-1].GetType()) + } + if !hasHybrid { + return expr + } + + for i := 0; i < len(args)-1; i += 2 { + args[i+1] = wrapCastFunc(ctx, args[i+1]) + } + if len(args)%2 == 1 { + args[len(args)-1] = wrapCastFunc(ctx, args[len(args)-1]) + } + f, err := funcs[ast.Case].getFunction(ctx, args) + if err != nil { + return expr + } + sf.RetType, sf.Function = f.getRetTp(), f + return sf + case ast.Elt: + hasHybrid := false + for i := 1; i < len(args); i++ { + hasHybrid = hasHybrid || isHybrid(args[i].GetType()) + } + if !hasHybrid { + return expr + } + + for i := 1; i < len(args); i++ { + args[i] = wrapCastFunc(ctx, args[i]) + } + f, err := funcs[ast.Elt].getFunction(ctx, args) + if err != nil { + return expr + } + sf.RetType, sf.Function = f.getRetTp(), f + return sf + default: + return expr + } + return expr +} diff --git a/expression/integration_test.go b/expression/integration_test.go index 69142c01c3f35..a0fb6fd8b5499 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9435,3 +9435,76 @@ func (s *testIntegrationSuite) TestEnumIndex(c *C) { tk.MustQuery("select /*+ use_index(t,idx) */ col3 from t where col2 = 'b' and col1 is not null;").Check( testkit.Rows("2")) } + +func (s *testIntegrationSuite) TestControlFunctionWithEnumOrSet(c *C) { + defer s.cleanEnv(c) + + // issue 23114 + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists e;") + tk.MustExec("create table e(e enum('c', 'b', 'a'));") + tk.MustExec("insert into e values ('a'),('b'),('a'),('b');") + tk.MustQuery("select e from e where if(e>1, e, e);").Sort().Check( + testkit.Rows("a", "a", "b", "b")) + tk.MustQuery("select e from e where case e when 1 then e else e end;").Sort().Check( + testkit.Rows("a", "a", "b", "b")) + tk.MustQuery("select e from e where case 1 when e then e end;").Check(testkit.Rows()) + + tk.MustQuery("select if(e>1,e,e)='a' from e").Sort().Check( + testkit.Rows("0", "0", "1", "1")) + tk.MustQuery("select if(e>1,e,e)=1 from e").Sort().Check( + testkit.Rows("0", "0", "0", "0")) + // if and if + tk.MustQuery("select if(e>2,e,e) and if(e<=2,e,e) from e;").Sort().Check( + testkit.Rows("1", "1", "1", "1")) + tk.MustQuery("select if(e>2,e,e) and (if(e<3,0,e) or if(e>=2,0,e)) from e;").Sort().Check( + testkit.Rows("0", "0", "1", "1")) + tk.MustQuery("select * from e where if(e>2,e,e) and if(e<=2,e,e);").Sort().Check( + testkit.Rows("a", "a", "b", "b")) + tk.MustQuery("select * from e where if(e>2,e,e) and (if(e<3,0,e) or if(e>=2,0,e));").Sort().Check( + testkit.Rows("a", "a")) + + // issue 24494 + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int,b enum(\"b\",\"y\",\"1\"));") + tk.MustExec("insert into t values(0,\"y\"),(1,\"b\"),(null,null),(2,\"1\");") + tk.MustQuery("SELECT count(*) FROM t where if(a,b ,null);").Check(testkit.Rows("2")) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int,b enum(\"b\"),c enum(\"c\"));") + tk.MustExec("insert into t values(1,1,1),(2,1,1),(1,1,1),(2,1,1);") + tk.MustQuery("select a from t where if(a=1,b,c)=\"b\";").Check(testkit.Rows("1", "1")) + tk.MustQuery("select a from t where if(a=1,b,c)=\"c\";").Check(testkit.Rows("2", "2")) + tk.MustQuery("select a from t where if(a=1,b,c)=1;").Sort().Check(testkit.Rows("1", "1", "2", "2")) + tk.MustQuery("select a from t where if(a=1,b,c);").Sort().Check(testkit.Rows("1", "1", "2", "2")) + + tk.MustExec("drop table if exists e;") + tk.MustExec("create table e(e enum('c', 'b', 'a'));") + tk.MustExec("insert into e values(3)") + tk.MustQuery("select elt(1,e) = 'a' from e").Check(testkit.Rows("1")) + tk.MustQuery("select elt(1,e) = 3 from e").Check(testkit.Rows("1")) + tk.MustQuery("select e from e where elt(1,e)").Check(testkit.Rows("a")) + + // test set type + tk.MustExec("drop table if exists s;") + tk.MustExec("create table s(s set('c', 'b', 'a'));") + tk.MustExec("insert into s values ('a'),('b'),('a'),('b');") + tk.MustQuery("select s from s where if(s>1, s, s);").Sort().Check( + testkit.Rows("a", "a", "b", "b")) + tk.MustQuery("select s from s where case s when 1 then s else s end;").Sort().Check( + testkit.Rows("a", "a", "b", "b")) + tk.MustQuery("select s from s where case 1 when s then s end;").Check(testkit.Rows()) + + tk.MustQuery("select if(s>1,s,s)='a' from s").Sort().Check( + testkit.Rows("0", "0", "1", "1")) + tk.MustQuery("select if(s>1,s,s)=4 from s").Sort().Check( + testkit.Rows("0", "0", "1", "1")) + + tk.MustExec("drop table if exists s;") + tk.MustExec("create table s(s set('c', 'b', 'a'));") + tk.MustExec("insert into s values('a')") + tk.MustQuery("select elt(1,s) = 'a' from s").Check(testkit.Rows("1")) + tk.MustQuery("select elt(1,s) = 4 from s").Check(testkit.Rows("1")) + tk.MustQuery("select s from s where elt(1,s)").Check(testkit.Rows("a")) +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 64bc0c41407e1..fc6fb53dcff44 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -975,6 +975,19 @@ func (b *PlanBuilder) buildSelection(ctx context.Context, p LogicalPlan, where a if len(cnfExpres) == 0 { return p, nil } + // check expr field types. + for i, expr := range cnfExpres { + if expr.GetType().EvalType() == types.ETString { + tp := &types.FieldType{ + Tp: mysql.TypeDouble, + Flag: expr.GetType().Flag, + Flen: mysql.MaxRealWidth, + Decimal: types.UnspecifiedLength, + } + types.SetBinChsClnFlag(tp) + cnfExpres[i] = expression.TryPushCastIntoControlFunctionForHybridType(b.ctx, expr, tp) + } + } selection.Conditions = cnfExpres selection.SetChildren(p) return selection, nil From f5a362d229fd1032231f05432fd93dd0a2b34d36 Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Wed, 19 May 2021 08:03:40 +0800 Subject: [PATCH 099/343] executor: add correctness tests about IndexMerge (#24674) --- executor/partition_table_test.go | 95 ++++++++++++++++++++++++++++++++ 1 file changed, 95 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 8337be660ec0c..529a1da343387 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -1328,6 +1328,101 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { } } +func (s *partitionTableSuite) TestIdexMerge(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_idx_merge") + tk.MustExec("use test_idx_merge") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // list partition table + tk.MustExec(`create table tlist(a int, b int, primary key(a) clustered, index idx_b(b)) partition by list(a)( + partition p0 values in (1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8), + partition p2 values in (9, 10, 11, 12));`) + + // range partition table + tk.MustExec(`create table trange(a int, b int, primary key(a) clustered, index idx_b(b)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than (500), + partition p2 values less than(1100));`) + + // hash partition table + tk.MustExec(`create table thash(a int, b int, primary key(a) clustered, index idx_b(b)) partition by hash(a) partitions 4;`) + + // regular table + tk.MustExec("create table tregular1(a int, b int, primary key(a) clustered)") + tk.MustExec("create table tregular2(a int, b int, primary key(a) clustered)") + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(1100), rand.Intn(2000))) + } + + tk.MustExec("insert ignore into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into tregular1 values " + strings.Join(vals, ",")) + + vals = make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(12)+1, rand.Intn(20))) + } + + tk.MustExec("insert ignore into tlist values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into tregular2 values " + strings.Join(vals, ",")) + + // test range partition + for i := 0; i < 100; i++ { + x1 := rand.Intn(1099) + x2 := rand.Intn(1099) + + queryPartition1 := fmt.Sprintf("select /*+ use_index_merge(trange) */ * from trange where a > %v or b < %v;", x1, x2) + queryRegular1 := fmt.Sprintf("select /*+ use_index_merge(tregular1) */ * from tregular1 where a > %v or b < %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition1, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ use_index_merge(trange) */ * from trange where a > %v or b > %v;", x1, x2) + queryRegular2 := fmt.Sprintf("select /*+ use_index_merge(tregular1) */ * from tregular1 where a > %v or b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition2, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + } + + // test hash partition + for i := 0; i < 100; i++ { + x1 := rand.Intn(1099) + x2 := rand.Intn(1099) + + queryPartition1 := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > %v or b < %v;", x1, x2) + queryRegular1 := fmt.Sprintf("select /*+ use_index_merge(tregualr1) */ * from tregular1 where a > %v or b < %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition1, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ use_index_merge(thash) */ * from thash where a > %v or b > %v;", x1, x2) + queryRegular2 := fmt.Sprintf("select /*+ use_index_merge(tregular1) */ * from tregular1 where a > %v or b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition2, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + } + + // test list partition + for i := 0; i < 100; i++ { + x1 := rand.Intn(12) + 1 + x2 := rand.Intn(12) + 1 + queryPartition1 := fmt.Sprintf("select /*+ use_index_merge(tlist) */ * from tlist where a > %v or b < %v;", x1, x2) + queryRegular1 := fmt.Sprintf("select /*+ use_index_merge(tregular2) */ * from tregular2 where a > %v or b < %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition1, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition1).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryPartition2 := fmt.Sprintf("select /*+ use_index_merge(tlist) */ * from tlist where a > %v or b > %v;", x1, x2) + queryRegular2 := fmt.Sprintf("select /*+ use_index_merge(tregular2) */ * from tregular2 where a > %v or b > %v;", x1, x2) + c.Assert(tk.HasPlan(queryPartition2, "IndexMerge"), IsTrue) // check if IndexLookUp is used + tk.MustQuery(queryPartition2).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + } +} + func (s *globalIndexSuite) TestGlobalIndexScan(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists p") From 0a1c3c0f02ebead5d414c76c71e93b3b64356af7 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Tue, 18 May 2021 18:41:40 -0600 Subject: [PATCH 100/343] variable: change default for DefDMLBatchSize tidbOptInt64 call (#24697) --- sessionctx/variable/sysvar.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 99c3da8233d68..e6f632db6b02d 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -904,7 +904,7 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBDMLBatchSize, Value: strconv.Itoa(DefDMLBatchSize), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.DMLBatchSize = int(tidbOptInt64(val, DefOptCorrelationExpFactor)) + s.DMLBatchSize = int(tidbOptInt64(val, DefDMLBatchSize)) return nil }}, {Scope: ScopeSession, Name: TiDBCurrentTS, Value: strconv.Itoa(DefCurretTS), ReadOnly: true}, From 883c7fb87c988f6d050eb6070b9d70bfd57d674c Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 19 May 2021 04:57:40 +0200 Subject: [PATCH 101/343] planner: add partitioning pruning tests for range partitioning (#24554) --- executor/partition_table_test.go | 77 +++ executor/testdata/executor_suite_in.json | 91 +++ executor/testdata/executor_suite_out.json | 797 ++++++++++++++++++++++ util/testkit/testkit.go | 28 + 4 files changed, 993 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 529a1da343387..cb5414d91815f 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -1452,3 +1452,80 @@ func (s *globalIndexSuite) TestIssue21731(c *C) { tk.MustExec("drop table if exists p, t") tk.MustExec("create table t (a int, b int, unique index idx(a)) partition by list columns(b) (partition p0 values in (1), partition p1 values in (2));") } + +func (s *testSuiteWithData) TestRangePartitionBoundariesEq(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("SET @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("CREATE DATABASE TestRangePartitionBoundaries") + defer tk.MustExec("DROP DATABASE TestRangePartitionBoundaries") + tk.MustExec("USE TestRangePartitionBoundaries") + tk.MustExec("DROP TABLE IF EXISTS t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1000000), + PARTITION p1 VALUES LESS THAN (2000000), + PARTITION p2 VALUES LESS THAN (3000000)); +`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} + +type testOutput struct { + SQL string + Plan []string + Res []string +} + +func (s *testSuiteWithData) TestRangePartitionBoundariesNe(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("SET @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("CREATE DATABASE TestRangePartitionBoundariesNe") + defer tk.MustExec("DROP DATABASE TestRangePartitionBoundariesNe") + tk.MustExec("USE TestRangePartitionBoundariesNe") + tk.MustExec("DROP TABLE IF EXISTS t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1), + PARTITION p1 VALUES LESS THAN (2), + PARTITION p2 VALUES LESS THAN (3), + PARTITION p3 VALUES LESS THAN (4), + PARTITION p4 VALUES LESS THAN (5), + PARTITION p5 VALUES LESS THAN (6), + PARTITION p6 VALUES LESS THAN (7))`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} + +func (s *testSuiteWithData) verifyPartitionResult(tk *testkit.TestKit, input []string, output []testOutput) { + for i, tt := range input { + var isSelect bool = false + if strings.HasPrefix(strings.ToLower(tt), "select ") { + isSelect = true + } + s.testData.OnRecord(func() { + output[i].SQL = tt + if isSelect { + output[i].Plan = s.testData.ConvertRowsToStrings(tk.UsedPartitions(tt).Rows()) + output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + } else { + // to avoid double execution of INSERT (and INSERT does not return anything) + output[i].Res = nil + output[i].Plan = nil + } + }) + if isSelect { + tk.UsedPartitions(tt).Check(testkit.Rows(output[i].Plan...)) + } + tk.MayQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) + } +} diff --git a/executor/testdata/executor_suite_in.json b/executor/testdata/executor_suite_in.json index 6abd20c740a80..fff3187717f0a 100644 --- a/executor/testdata/executor_suite_in.json +++ b/executor/testdata/executor_suite_in.json @@ -51,5 +51,96 @@ "select count(*) from t as t1 left join t as t2 on t1.c1 = t2.c1 where t1.c1 != NULL", "select * from t as t1 left join t as t2 on t1.c1 = t2.c1 where t1.c1 is not NULL" ] + }, + { + "name": "TestRangePartitionBoundariesEq", + "cases": [ + "INSERT INTO t VALUES (999998, '999998 Filler ...'), (999999, '999999 Filler ...'), (1000000, '1000000 Filler ...'), (1000001, '1000001 Filler ...'), (1000002, '1000002 Filler ...')", + "INSERT INTO t VALUES (1999998, '1999998 Filler ...'), (1999999, '1999999 Filler ...'), (2000000, '2000000 Filler ...'), (2000001, '2000001 Filler ...'), (2000002, '2000002 Filler ...')", + "INSERT INTO t VALUES (2999998, '2999998 Filler ...'), (2999999, '2999999 Filler ...')", + "INSERT INTO t VALUES (-2147483648, 'MIN_INT filler...'), (0, '0 Filler...')", + "ANALYZE TABLE t", + "SELECT * FROM t WHERE a = -2147483648", + "SELECT * FROM t WHERE a IN (-2147483648)", + "SELECT * FROM t WHERE a = 0", + "SELECT * FROM t WHERE a IN (0)", + "SELECT * FROM t WHERE a = 999998", + "SELECT * FROM t WHERE a IN (999998)", + "SELECT * FROM t WHERE a = 999999", + "SELECT * FROM t WHERE a IN (999999)", + "SELECT * FROM t WHERE a = 1000000", + "SELECT * FROM t WHERE a IN (1000000)", + "SELECT * FROM t WHERE a = 1000001", + "SELECT * FROM t WHERE a IN (1000001)", + "SELECT * FROM t WHERE a = 1000002", + "SELECT * FROM t WHERE a IN (1000002)", + "SELECT * FROM t WHERE a = 3000000", + "SELECT * FROM t WHERE a IN (3000000)", + "SELECT * FROM t WHERE a = 3000001", + "SELECT * FROM t WHERE a IN (3000001)", + "SELECT * FROM t WHERE a IN (-2147483648, -2147483647)", + "SELECT * FROM t WHERE a IN (-2147483647, -2147483646)", + "SELECT * FROM t WHERE a IN (999997, 999998, 999999)", + "SELECT * FROM t WHERE a IN (999998, 999999, 1000000)", + "SELECT * FROM t WHERE a IN (999999, 1000000, 1000001)", + "SELECT * FROM t WHERE a IN (1000000, 1000001, 1000002)", + "SELECT * FROM t WHERE a IN (1999997, 1999998, 1999999)", + "SELECT * FROM t WHERE a IN (1999998, 1999999, 2000000)", + "SELECT * FROM t WHERE a IN (1999999, 2000000, 2000001)", + "SELECT * FROM t WHERE a IN (2000000, 2000001, 2000002)", + "SELECT * FROM t WHERE a IN (2999997, 2999998, 2999999)", + "SELECT * FROM t WHERE a IN (2999998, 2999999, 3000000)", + "SELECT * FROM t WHERE a IN (2999999, 3000000, 3000001)", + "SELECT * FROM t WHERE a IN (3000000, 3000001, 3000002)" + ] + }, + { + "name": "TestRangePartitionBoundariesNe", + "cases": [ + "INSERT INTO t VALUES (0, '0 Filler...')", + "INSERT INTO t VALUES (1, '1 Filler...')", + "INSERT INTO t VALUES (2, '2 Filler...')", + "INSERT INTO t VALUES (3, '3 Filler...')", + "INSERT INTO t VALUES (4, '4 Filler...')", + "INSERT INTO t VALUES (5, '5 Filler...')", + "INSERT INTO t VALUES (6, '6 Filler...')", + "ANALYZE TABLE t", + "SELECT * FROM t WHERE a != -1", + "SELECT * FROM t WHERE 1 = 1 AND a != -1", + "SELECT * FROM t WHERE a NOT IN (-2, -1)", + "SELECT * FROM t WHERE 1 = 0 OR a = -1", + "SELECT * FROM t WHERE a != 0", + "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0", + "SELECT * FROM t WHERE a NOT IN (-2, -1, 0)", + "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0", + "SELECT * FROM t WHERE a != 1", + "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1", + "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1)", + "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1", + "SELECT * FROM t WHERE a != 2", + "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2", + "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2)", + "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2", + "SELECT * FROM t WHERE a != 3", + "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3", + "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3)", + "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3", + "SELECT * FROM t WHERE a != 4", + "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4", + "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4)", + "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4", + "SELECT * FROM t WHERE a != 5", + "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5", + "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5)", + "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5", + "SELECT * FROM t WHERE a != 6", + "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5 AND a != 6", + "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5, 6)", + "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5 OR a = 6", + "SELECT * FROM t WHERE a != 7", + "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5 AND a != 6 AND a != 7", + "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5, 6, 7)", + "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5 OR a = 6 OR a = 7" + ] } ] diff --git a/executor/testdata/executor_suite_out.json b/executor/testdata/executor_suite_out.json index 2be3c8ea4894f..caa5c4f948966 100644 --- a/executor/testdata/executor_suite_out.json +++ b/executor/testdata/executor_suite_out.json @@ -598,5 +598,802 @@ ] } ] + }, + { + "Name": "TestRangePartitionBoundariesEq", + "Cases": [ + { + "SQL": "INSERT INTO t VALUES (999998, '999998 Filler ...'), (999999, '999999 Filler ...'), (1000000, '1000000 Filler ...'), (1000001, '1000001 Filler ...'), (1000002, '1000002 Filler ...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (1999998, '1999998 Filler ...'), (1999999, '1999999 Filler ...'), (2000000, '2000000 Filler ...'), (2000001, '2000001 Filler ...'), (2000002, '2000002 Filler ...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (2999998, '2999998 Filler ...'), (2999999, '2999999 Filler ...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (-2147483648, 'MIN_INT filler...'), (0, '0 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "ANALYZE TABLE t", + "Plan": null, + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a = -2147483648", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (-2147483648)", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a = 0", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (0)", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a = 999998", + "Plan": [ + "p0" + ], + "Res": [ + "999998 999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (999998)", + "Plan": [ + "p0" + ], + "Res": [ + "999998 999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a = 999999", + "Plan": [ + "p0" + ], + "Res": [ + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (999999)", + "Plan": [ + "p0" + ], + "Res": [ + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a = 1000000", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (1000000)", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a = 1000001", + "Plan": [ + "p1" + ], + "Res": [ + "1000001 1000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (1000001)", + "Plan": [ + "p1" + ], + "Res": [ + "1000001 1000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a = 1000002", + "Plan": [ + "p1" + ], + "Res": [ + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (1000002)", + "Plan": [ + "p1" + ], + "Res": [ + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a = 3000000", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a IN (3000000)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a = 3000001", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a IN (3000001)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a IN (-2147483648, -2147483647)", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (-2147483647, -2147483646)", + "Plan": [ + "p0" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a IN (999997, 999998, 999999)", + "Plan": [ + "p0" + ], + "Res": [ + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (999998, 999999, 1000000)", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (999999, 1000000, 1000001)", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (1000000, 1000001, 1000002)", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (1999997, 1999998, 1999999)", + "Plan": [ + "p1" + ], + "Res": [ + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (1999998, 1999999, 2000000)", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (1999999, 2000000, 2000001)", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (2000000, 2000001, 2000002)", + "Plan": [ + "p2" + ], + "Res": [ + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (2999997, 2999998, 2999999)", + "Plan": [ + "p2" + ], + "Res": [ + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (2999998, 2999999, 3000000)", + "Plan": [ + "p2" + ], + "Res": [ + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (2999999, 3000000, 3000001)", + "Plan": [ + "p2" + ], + "Res": [ + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a IN (3000000, 3000001, 3000002)", + "Plan": [ + "dual" + ], + "Res": null + } + ] + }, + { + "Name": "TestRangePartitionBoundariesNe", + "Cases": [ + { + "SQL": "INSERT INTO t VALUES (0, '0 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (1, '1 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (2, '2 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (3, '3 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (4, '4 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (5, '5 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (6, '6 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "ANALYZE TABLE t", + "Plan": null, + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a != -1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 1 AND a != -1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a NOT IN (-2, -1)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 0 OR a = -1", + "Plan": [ + "p0" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a != 0", + "Plan": [ + "all" + ], + "Res": [ + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0", + "Plan": [ + "all" + ], + "Res": [ + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a NOT IN (-2, -1, 0)", + "Plan": [ + "all" + ], + "Res": [ + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a != 1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1", + "Plan": [ + "all" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1)", + "Plan": [ + "all" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a != 2", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2", + "Plan": [ + "all" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2)", + "Plan": [ + "all" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2", + "Plan": [ + "p0 p1 p2" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a != 3", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3", + "Plan": [ + "all" + ], + "Res": [ + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3)", + "Plan": [ + "all" + ], + "Res": [ + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3", + "Plan": [ + "p0 p1 p2 p3" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a != 4", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4", + "Plan": [ + "all" + ], + "Res": [ + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4)", + "Plan": [ + "all" + ], + "Res": [ + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4", + "Plan": [ + "p0 p1 p2 p3 p4" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a != 5", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5", + "Plan": [ + "all" + ], + "Res": [ + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5)", + "Plan": [ + "all" + ], + "Res": [ + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5", + "Plan": [ + "p0 p1 p2 p3 p4 p5" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a != 6", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5 AND a != 6", + "Plan": [ + "all" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5, 6)", + "Plan": [ + "all" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5 OR a = 6", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a != 7", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 1 AND a != -1 AND a != 0 AND a != 1 AND a != 2 AND a != 3 AND a != 4 AND a != 5 AND a != 6 AND a != 7", + "Plan": [ + "all" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5, 6, 7)", + "Plan": [ + "all" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5 OR a = 6 OR a = 7", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + } + ] } ] diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 7cacaf211375e..ef0a0858f76cf 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -271,6 +271,21 @@ func (tk *TestKit) MustPartition(sql string, partitions string, args ...interfac return tk.MustQuery(sql, args...) } +// UsedPartitions returns the partition names that will be used or all/dual. +func (tk *TestKit) UsedPartitions(sql string, args ...interface{}) *Result { + rs := tk.MustQuery("explain "+sql, args...) + var usedPartitions [][]string + for i := range rs.rows { + index := strings.Index(rs.rows[i][3], "partition:") + if index != -1 { + p := rs.rows[i][3][index+len("partition:"):] + partitions := strings.Split(strings.SplitN(p, " ", 2)[0], ",") + usedPartitions = append(usedPartitions, partitions) + } + } + return &Result{rows: usedPartitions, c: tk.c, comment: check.Commentf("sql:%s, args:%v", sql, args)} +} + // MustUseIndex checks if the result execution plan contains specific index(es). func (tk *TestKit) MustUseIndex(sql string, index string, args ...interface{}) bool { rs := tk.MustQuery("explain "+sql, args...) @@ -312,6 +327,19 @@ func (tk *TestKit) MustQuery(sql string, args ...interface{}) *Result { return tk.ResultSetToResult(rs, comment) } +// MayQuery query the statements and returns result rows if result set is returned. +// If expected result is set it asserts the query result equals expected result. +func (tk *TestKit) MayQuery(sql string, args ...interface{}) *Result { + comment := check.Commentf("sql:%s, args:%v", sql, args) + rs, err := tk.Exec(sql, args...) + tk.c.Assert(errors.ErrorStack(err), check.Equals, "", comment) + if rs == nil { + var emptyStringAoA [][]string + return &Result{rows: emptyStringAoA, c: tk.c, comment: comment} + } + return tk.ResultSetToResult(rs, comment) +} + // QueryToErr executes a sql statement and discard results. func (tk *TestKit) QueryToErr(sql string, args ...interface{}) error { comment := check.Commentf("sql:%s, args:%v", sql, args) From 3e41d1473f540a09ea96c8adb7193fff25ccbd7c Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Wed, 19 May 2021 14:15:41 +0800 Subject: [PATCH 102/343] *: add option for enum push down (#24685) --- executor/executor_test.go | 76 +++++++++++++++++++++++++++++++++++++++ expression/expr_to_pb.go | 4 +++ 2 files changed, 80 insertions(+) diff --git a/executor/executor_test.go b/executor/executor_test.go index 7fa0d7b0d10bd..730de6dd70bde 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8153,3 +8153,79 @@ func (s *testSerialSuite) TestIssue24210(c *C) { c.Assert(err, IsNil) } + +func (s testSerialSuite) TestExprBlackListForEnum(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a enum('a','b','c'), b enum('a','b','c'), c int, index idx(b,a));") + tk.MustExec("insert into t values(1,1,1),(2,2,2),(3,3,3);") + + checkFuncPushDown := func(rows [][]interface{}, keyWord string) bool { + for _, line := range rows { + // Agg/Expr push down + if line[2].(string) == "cop[tikv]" && strings.Contains(line[4].(string), keyWord) { + return true + } + // access index + if line[2].(string) == "cop[tikv]" && strings.Contains(line[3].(string), keyWord) { + return true + } + } + return false + } + + // Test agg(enum) push down + tk.MustExec("insert into mysql.expr_pushdown_blacklist(name) values('enum');") + tk.MustExec("admin reload expr_pushdown_blacklist;") + rows := tk.MustQuery("desc format='brief' select /*+ HASH_AGG() */ max(a) from t;").Rows() + c.Assert(checkFuncPushDown(rows, "max"), IsFalse) + rows = tk.MustQuery("desc format='brief' select /*+ STREAM_AGG() */ max(a) from t;").Rows() + c.Assert(checkFuncPushDown(rows, "max"), IsFalse) + + tk.MustExec("delete from mysql.expr_pushdown_blacklist;") + tk.MustExec("admin reload expr_pushdown_blacklist;") + rows = tk.MustQuery("desc format='brief' select /*+ HASH_AGG() */ max(a) from t;").Rows() + c.Assert(checkFuncPushDown(rows, "max"), IsTrue) + rows = tk.MustQuery("desc format='brief' select /*+ STREAM_AGG() */ max(a) from t;").Rows() + c.Assert(checkFuncPushDown(rows, "max"), IsTrue) + + // Test expr(enum) push down + tk.MustExec("insert into mysql.expr_pushdown_blacklist(name) values('enum');") + tk.MustExec("admin reload expr_pushdown_blacklist;") + rows = tk.MustQuery("desc format='brief' select * from t where a + b;").Rows() + c.Assert(checkFuncPushDown(rows, "plus"), IsFalse) + rows = tk.MustQuery("desc format='brief' select * from t where a + b;").Rows() + c.Assert(checkFuncPushDown(rows, "plus"), IsFalse) + + tk.MustExec("delete from mysql.expr_pushdown_blacklist;") + tk.MustExec("admin reload expr_pushdown_blacklist;") + rows = tk.MustQuery("desc format='brief' select * from t where a + b;").Rows() + c.Assert(checkFuncPushDown(rows, "plus"), IsTrue) + rows = tk.MustQuery("desc format='brief' select * from t where a + b;").Rows() + c.Assert(checkFuncPushDown(rows, "plus"), IsTrue) + + // Test enum index + tk.MustExec("insert into mysql.expr_pushdown_blacklist(name) values('enum');") + tk.MustExec("admin reload expr_pushdown_blacklist;") + rows = tk.MustQuery("desc format='brief' select * from t where b = 1;").Rows() + c.Assert(checkFuncPushDown(rows, "index:idx(b)"), IsFalse) + rows = tk.MustQuery("desc format='brief' select * from t where b = 'a';").Rows() + c.Assert(checkFuncPushDown(rows, "index:idx(b)"), IsFalse) + rows = tk.MustQuery("desc format='brief' select * from t where b > 1;").Rows() + c.Assert(checkFuncPushDown(rows, "index:idx(b)"), IsFalse) + rows = tk.MustQuery("desc format='brief' select * from t where b > 'a';").Rows() + c.Assert(checkFuncPushDown(rows, "index:idx(b)"), IsFalse) + + tk.MustExec("delete from mysql.expr_pushdown_blacklist;") + tk.MustExec("admin reload expr_pushdown_blacklist;") + rows = tk.MustQuery("desc format='brief' select * from t where b = 1 and a = 1;").Rows() + c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue) + rows = tk.MustQuery("desc format='brief' select * from t where b = 'a' and a = 'a';").Rows() + c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue) + rows = tk.MustQuery("desc format='brief' select * from t where b = 1 and a > 1;").Rows() + c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue) + rows = tk.MustQuery("desc format='brief' select * from t where b = 1 and a > 'a'").Rows() + c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue) +} diff --git a/expression/expr_to_pb.go b/expression/expr_to_pb.go index 59ff01b73b67c..dc031145d0d95 100644 --- a/expression/expr_to_pb.go +++ b/expression/expr_to_pb.go @@ -211,6 +211,10 @@ func (pc PbConverter) columnToPBExpr(column *Column) *tipb.Expr { switch column.GetType().Tp { case mysql.TypeBit, mysql.TypeSet, mysql.TypeGeometry, mysql.TypeUnspecified: return nil + case mysql.TypeEnum: + if !IsPushDownEnabled("enum", kv.UnSpecified) { + return nil + } } if pc.client.IsRequestTypeSupported(kv.ReqTypeDAG, kv.ReqSubTypeBasic) { From 49d1eaa4096e3ffe9981d1734b5a271911fd9bf2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Wed, 19 May 2021 17:39:42 +0800 Subject: [PATCH 103/343] txn: break dependency from store/tikv to tidb/kv cause by TransactionOption (#24656) --- kv/fault_injection.go | 4 +- kv/fault_injection_test.go | 3 +- kv/interface_mock_test.go | 3 +- kv/kv.go | 49 +----------- session/session.go | 12 +-- session/txn.go | 5 +- store/driver/tikv_driver.go | 2 +- store/helper/helper.go | 2 +- store/mockstore/mockstorage/storage.go | 2 +- store/tikv/kv.go | 8 +- store/tikv/test_probe.go | 18 +++++ store/tikv/tests/2pc_test.go | 5 +- .../tikv/{ => tests}/extract_start_ts_test.go | 72 ++++++++--------- store/tikv/txn.go | 80 +++++++++++++++---- util/mock/context.go | 3 +- util/mock/store.go | 3 +- 16 files changed, 145 insertions(+), 126 deletions(-) rename store/tikv/{ => tests}/extract_start_ts_test.go (54%) diff --git a/kv/fault_injection.go b/kv/fault_injection.go index 218ca9cbd6966..d61685a7f8a71 100644 --- a/kv/fault_injection.go +++ b/kv/fault_injection.go @@ -16,6 +16,8 @@ package kv import ( "context" "sync" + + "github.com/pingcap/tidb/store/tikv" ) // InjectionConfig is used for fault injections for KV components. @@ -64,7 +66,7 @@ func (s *InjectedStore) Begin() (Transaction, error) { } // BeginWithOption creates an injected Transaction with given option. -func (s *InjectedStore) BeginWithOption(option TransactionOption) (Transaction, error) { +func (s *InjectedStore) BeginWithOption(option tikv.StartTSOption) (Transaction, error) { txn, err := s.Storage.BeginWithOption(option) return &InjectedTransaction{ Transaction: txn, diff --git a/kv/fault_injection_test.go b/kv/fault_injection_test.go index 33b6535214b2c..4979dbf4268cd 100644 --- a/kv/fault_injection_test.go +++ b/kv/fault_injection_test.go @@ -19,6 +19,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" ) @@ -35,7 +36,7 @@ func (s testFaultInjectionSuite) TestFaultInjectionBasic(c *C) { storage := NewInjectedStore(newMockStorage(), &cfg) txn, err := storage.Begin() c.Assert(err, IsNil) - _, err = storage.BeginWithOption(TransactionOption{}.SetTxnScope(oracle.GlobalTxnScope).SetStartTs(0)) + _, err = storage.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(oracle.GlobalTxnScope).SetStartTs(0)) c.Assert(err, IsNil) ver := Version{Ver: 1} snap := storage.GetSnapshot(ver) diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index 5d85261bc2111..9e41832678294 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -17,6 +17,7 @@ import ( "context" "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" ) @@ -154,7 +155,7 @@ func (s *mockStorage) Begin() (Transaction, error) { return newMockTxn(), nil } -func (s *mockStorage) BeginWithOption(option TransactionOption) (Transaction, error) { +func (s *mockStorage) BeginWithOption(option tikv.StartTSOption) (Transaction, error) { return newMockTxn(), nil } diff --git a/kv/kv.go b/kv/kv.go index 20b0fc84b7144..471dfe09a110b 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/memory" @@ -339,59 +340,13 @@ type Driver interface { Open(path string) (Storage, error) } -// TransactionOption indicates the option when beginning a transaction -// `TxnScope` must be set for each object -// Every other fields are optional, but currently at most one of them can be set -type TransactionOption struct { - TxnScope string - StartTS *uint64 - PrevSec *uint64 - MinStartTS *uint64 - MaxPrevSec *uint64 -} - -// DefaultTransactionOption creates a default TransactionOption, ie. Work in GlobalTxnScope and get start ts when got used -func DefaultTransactionOption() TransactionOption { - return TransactionOption{TxnScope: oracle.GlobalTxnScope} -} - -// SetMaxPrevSec set maxPrevSec -func (to TransactionOption) SetMaxPrevSec(maxPrevSec uint64) TransactionOption { - to.MaxPrevSec = &maxPrevSec - return to -} - -// SetMinStartTS set minStartTS -func (to TransactionOption) SetMinStartTS(minStartTS uint64) TransactionOption { - to.MinStartTS = &minStartTS - return to -} - -// SetStartTs set startTS -func (to TransactionOption) SetStartTs(startTS uint64) TransactionOption { - to.StartTS = &startTS - return to -} - -// SetPrevSec set prevSec -func (to TransactionOption) SetPrevSec(prevSec uint64) TransactionOption { - to.PrevSec = &prevSec - return to -} - -// SetTxnScope set txnScope -func (to TransactionOption) SetTxnScope(txnScope string) TransactionOption { - to.TxnScope = txnScope - return to -} - // Storage defines the interface for storage. // Isolation should be at least SI(SNAPSHOT ISOLATION) type Storage interface { // Begin a global transaction Begin() (Transaction, error) // Begin a transaction with given option - BeginWithOption(option TransactionOption) (Transaction, error) + BeginWithOption(option tikv.StartTSOption) (Transaction, error) // GetSnapshot gets a snapshot that is able to read any data which data is <= ver. // if ver is MaxVersion or > current max committed version, we will use current version for this snapshot. GetSnapshot(ver Version) Snapshot diff --git a/session/session.go b/session/session.go index efd6706c4ffb3..c9b3f8f7a8abd 100644 --- a/session/session.go +++ b/session/session.go @@ -1975,7 +1975,7 @@ func (s *session) NewTxn(ctx context.Context) error { zap.String("txnScope", txnScope)) } - txn, err := s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(s.sessionVars.CheckAndGetTxnScope())) + txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(s.sessionVars.CheckAndGetTxnScope())) if err != nil { return err } @@ -2768,7 +2768,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { } // no need to get txn from txnFutureCh since txn should init with startTs - txn, err := s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTs(startTS)) + txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTs(startTS)) if err != nil { return err } @@ -2801,22 +2801,22 @@ func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionc txnScope := s.GetSessionVars().CheckAndGetTxnScope() switch option.Mode { case ast.TimestampBoundReadTimestamp: - txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetStartTs(option.StartTS)) + txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetStartTs(option.StartTS)) if err != nil { return err } case ast.TimestampBoundExactStaleness: - txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetPrevSec(option.PrevSec)) + txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetPrevSec(option.PrevSec)) if err != nil { return err } case ast.TimestampBoundMaxStaleness: - txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetMaxPrevSec(option.PrevSec)) + txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetMaxPrevSec(option.PrevSec)) if err != nil { return err } case ast.TimestampBoundMinReadTimestamp: - txn, err = s.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(txnScope).SetMinStartTS(option.StartTS)) + txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetMinStartTS(option.StartTS)) if err != nil { return err } diff --git a/session/txn.go b/session/txn.go index 133cafb976aae..294725f8efaa0 100644 --- a/session/txn.go +++ b/session/txn.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" + "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" @@ -436,14 +437,14 @@ type txnFuture struct { func (tf *txnFuture) wait() (kv.Transaction, error) { startTS, err := tf.future.Wait() if err == nil { - return tf.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(tf.txnScope).SetStartTs(startTS)) + return tf.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(tf.txnScope).SetStartTs(startTS)) } else if config.GetGlobalConfig().Store == "unistore" { return nil, err } logutil.BgLogger().Warn("wait tso failed", zap.Error(err)) // It would retry get timestamp. - return tf.store.BeginWithOption(kv.DefaultTransactionOption().SetTxnScope(tf.txnScope)) + return tf.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(tf.txnScope)) } func (s *session) getTxnFuture(ctx context.Context) *txnFuture { diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index cc0f217280f31..2d93b7eda4abb 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -307,7 +307,7 @@ func (s *tikvStore) Begin() (kv.Transaction, error) { } // BeginWithOption begins a transaction with given option -func (s *tikvStore) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { +func (s *tikvStore) BeginWithOption(option tikv.StartTSOption) (kv.Transaction, error) { txn, err := s.KVStore.BeginWithOption(option) if err != nil { return nil, derr.ToTiDBErr(err) diff --git a/store/helper/helper.go b/store/helper/helper.go index 49aa7cf2107e0..8eb9b9d7db828 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -48,7 +48,7 @@ import ( // Methods copied from kv.Storage and tikv.Storage due to limitation of go1.13. type Storage interface { Begin() (kv.Transaction, error) - BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) + BeginWithOption(option tikv.StartTSOption) (kv.Transaction, error) GetSnapshot(ver kv.Version) kv.Snapshot GetClient() kv.Client GetMPPClient() kv.MPPClient diff --git a/store/mockstore/mockstorage/storage.go b/store/mockstore/mockstorage/storage.go index 6221ef855707d..7d78d1a9b7418 100644 --- a/store/mockstore/mockstorage/storage.go +++ b/store/mockstore/mockstorage/storage.go @@ -83,7 +83,7 @@ func (s *mockStorage) ShowStatus(ctx context.Context, key string) (interface{}, } // BeginWithOption begins a transaction with given option -func (s *mockStorage) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { +func (s *mockStorage) BeginWithOption(option tikv.StartTSOption) (kv.Transaction, error) { return newTiKVTxn(s.KVStore.BeginWithOption(option)) } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index edaef3b4744d7..622313f382abd 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" - tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" @@ -189,11 +188,11 @@ func (s *KVStore) runSafePointChecker() { // Begin a global transaction. func (s *KVStore) Begin() (*KVTxn, error) { - return s.BeginWithOption(tidbkv.DefaultTransactionOption()) + return s.BeginWithOption(DefaultStartTSOption()) } -// BeginWithOption begins a transaction with the given TransactionOption -func (s *KVStore) BeginWithOption(options tidbkv.TransactionOption) (*KVTxn, error) { +// BeginWithOption begins a transaction with the given StartTSOption +func (s *KVStore) BeginWithOption(options StartTSOption) (*KVTxn, error) { return newTiKVTxnWithOptions(s, options) } @@ -389,6 +388,7 @@ func (s *KVStore) getSafeTS(storeID uint64) uint64 { return safeTS.(uint64) } +// setSafeTS sets safeTs for store storeID, export for testing func (s *KVStore) setSafeTS(storeID, safeTS uint64) { s.safeTSMap.Store(storeID, safeTS) } diff --git a/store/tikv/test_probe.go b/store/tikv/test_probe.go index 1a8dc5062218d..3e80e6310fe4b 100644 --- a/store/tikv/test_probe.go +++ b/store/tikv/test_probe.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" @@ -81,6 +82,23 @@ func (s StoreProbe) SaveSafePoint(v uint64) error { return saveSafePoint(s.GetSafePointKV(), v) } +// SetRegionCacheStore is used to set a store in region cache, for testing only +func (s StoreProbe) SetRegionCacheStore(id uint64, storeType tikvrpc.EndpointType, state uint64, labels []*metapb.StoreLabel) { + s.regionCache.storeMu.Lock() + defer s.regionCache.storeMu.Unlock() + s.regionCache.storeMu.stores[id] = &Store{ + storeID: id, + storeType: storeType, + state: state, + labels: labels, + } +} + +// SetSafeTS is used to set safeTS for the store with `storeID` +func (s StoreProbe) SetSafeTS(storeID, safeTS uint64) { + s.setSafeTS(storeID, safeTS) +} + // TxnProbe wraps a txn and exports internal states for testing purpose. type TxnProbe struct { *KVTxn diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 5589752043b2b..43b682160c514 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" - tidbkv "github.com/pingcap/tidb/kv" drivertxn "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" @@ -603,12 +602,12 @@ func (s *testCommitterSuite) TestRejectCommitTS(c *C) { // Use max.Uint64 to read the data and success. // That means the final commitTS > startTS+2, it's not the one we provide. // So we cover the rety commitTS logic. - txn1, err := s.store.BeginWithOption(tidbkv.DefaultTransactionOption().SetStartTs(committer.GetStartTS() + 2)) + txn1, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTs(committer.GetStartTS() + 2)) c.Assert(err, IsNil) _, err = txn1.Get(bo.GetCtx(), []byte("x")) c.Assert(tikverr.IsErrNotFound(err), IsTrue) - txn2, err := s.store.BeginWithOption(tidbkv.DefaultTransactionOption().SetStartTs(math.MaxUint64)) + txn2, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTs(math.MaxUint64)) c.Assert(err, IsNil) val, err := txn2.Get(bo.GetCtx(), []byte("x")) c.Assert(err, IsNil) diff --git a/store/tikv/extract_start_ts_test.go b/store/tikv/tests/extract_start_ts_test.go similarity index 54% rename from store/tikv/extract_start_ts_test.go rename to store/tikv/tests/extract_start_ts_test.go index b392ca365cde8..566211006b66c 100644 --- a/store/tikv/extract_start_ts_test.go +++ b/store/tikv/tests/extract_start_ts_test.go @@ -11,20 +11,20 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tikv +package tikv_test import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore/unistore" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" ) type extractStartTsSuite struct { - store *KVStore + store *tikv.KVStore } var _ = SerialSuites(&extractStartTsSuite{}) @@ -33,31 +33,24 @@ func (s *extractStartTsSuite) SetUpTest(c *C) { client, pdClient, cluster, err := unistore.New("") c.Assert(err, IsNil) unistore.BootstrapWithSingleStore(cluster) - store, err := NewTestTiKVStore(client, pdClient, nil, nil, 0) + store, err := tikv.NewTestTiKVStore(client, pdClient, nil, nil, 0) c.Assert(err, IsNil) - store.regionCache.storeMu.stores[2] = &Store{ - storeID: 2, - storeType: tikvrpc.TiKV, - state: uint64(resolved), - labels: []*metapb.StoreLabel{ - { - Key: DCLabelKey, - Value: oracle.LocalTxnScope, - }, + probe := tikv.StoreProbe{KVStore: store} + probe.SetRegionCacheStore(2, tikvrpc.TiKV, 1, []*metapb.StoreLabel{ + { + Key: tikv.DCLabelKey, + Value: oracle.LocalTxnScope, }, - } - store.regionCache.storeMu.stores[3] = &Store{ - storeID: 3, - storeType: tikvrpc.TiKV, - state: uint64(resolved), - labels: []*metapb.StoreLabel{{ - Key: DCLabelKey, + }) + probe.SetRegionCacheStore(3, tikvrpc.TiKV, 1, []*metapb.StoreLabel{ + { + Key: tikv.DCLabelKey, Value: "Some Random Label", - }}, - } - store.setSafeTS(2, 102) - store.setSafeTS(3, 101) - s.store = store + }, + }) + probe.SetSafeTS(2, 102) + probe.SetSafeTS(3, 101) + s.store = probe.KVStore } func (s *extractStartTsSuite) TestExtractStartTs(c *C) { @@ -69,26 +62,26 @@ func (s *extractStartTsSuite) TestExtractStartTs(c *C) { cases := []struct { expectedTS uint64 - option kv.TransactionOption + option tikv.StartTSOption }{ // StartTS setted - {100, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, + {100, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, // PrevSec setted - {200, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}}, + {200, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}}, // MinStartTS setted, global - {101, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, + {101, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, // MinStartTS setted, local - {102, kv.TransactionOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, + {102, tikv.StartTSOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, // MaxPrevSec setted // however we need to add more cases to check the behavior when it fall backs to MinStartTS setted // see `TestMaxPrevSecFallback` - {200, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, + {200, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, // nothing setted - {300, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, + {300, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, } for _, cs := range cases { expected := cs.expectedTS - result, _ := extractStartTs(s.store, cs.option) + result, _ := tikv.ExtractStartTs(s.store, cs.option) c.Assert(result, Equals, expected) } @@ -97,18 +90,19 @@ func (s *extractStartTsSuite) TestExtractStartTs(c *C) { } func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { - s.store.setSafeTS(2, 0x8000000000000002) - s.store.setSafeTS(3, 0x8000000000000001) + probe := tikv.StoreProbe{KVStore: s.store} + probe.SetSafeTS(2, 0x8000000000000002) + probe.SetSafeTS(3, 0x8000000000000001) i := uint64(100) cases := []struct { expectedTS uint64 - option kv.TransactionOption + option tikv.StartTSOption }{ - {0x8000000000000001, kv.TransactionOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, - {0x8000000000000002, kv.TransactionOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, + {0x8000000000000001, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, + {0x8000000000000002, tikv.StartTSOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, } for _, cs := range cases { - result, _ := extractStartTs(s.store, cs.option) + result, _ := tikv.ExtractStartTs(s.store, cs.option) c.Assert(result, Equals, cs.expectedTS) } } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index beeeafe66a063..d228b834e00dc 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/kv" tikverr "github.com/pingcap/tidb/store/tikv/error" tikv "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -55,6 +54,52 @@ type SchemaAmender interface { AmendTxn(ctx context.Context, startInfoSchema SchemaVer, change *RelatedSchemaChange, mutations CommitterMutations) (CommitterMutations, error) } +// StartTSOption indicates the option when beginning a transaction +// `TxnScope` must be set for each object +// Every other fields are optional, but currently at most one of them can be set +type StartTSOption struct { + TxnScope string + StartTS *uint64 + PrevSec *uint64 + MinStartTS *uint64 + MaxPrevSec *uint64 +} + +// DefaultStartTSOption creates a default StartTSOption, ie. Work in GlobalTxnScope and get start ts when got used +func DefaultStartTSOption() StartTSOption { + return StartTSOption{TxnScope: oracle.GlobalTxnScope} +} + +// SetMaxPrevSec returns a new StartTSOption with MaxPrevSec set to maxPrevSec +func (to StartTSOption) SetMaxPrevSec(maxPrevSec uint64) StartTSOption { + to.MaxPrevSec = &maxPrevSec + return to +} + +// SetMinStartTS returns a new StartTSOption with MinStartTS set to minStartTS +func (to StartTSOption) SetMinStartTS(minStartTS uint64) StartTSOption { + to.MinStartTS = &minStartTS + return to +} + +// SetStartTs returns a new StartTSOption with StartTS set to startTS +func (to StartTSOption) SetStartTs(startTS uint64) StartTSOption { + to.StartTS = &startTS + return to +} + +// SetPrevSec returns a new StartTSOption with PrevSec set to prevSec +func (to StartTSOption) SetPrevSec(prevSec uint64) StartTSOption { + to.PrevSec = &prevSec + return to +} + +// SetTxnScope returns a new StartTSOption with TxnScope set to txnScope +func (to StartTSOption) SetTxnScope(txnScope string) StartTSOption { + to.TxnScope = txnScope + return to +} + // KVTxn contains methods to interact with a TiKV transaction. type KVTxn struct { snapshot *KVSnapshot @@ -90,23 +135,24 @@ type KVTxn struct { kvFilter KVFilter } -func extractStartTs(store *KVStore, options kv.TransactionOption) (uint64, error) { +// ExtractStartTs use `option` to get the proper startTS for a transaction +func ExtractStartTs(store *KVStore, option StartTSOption) (uint64, error) { var startTs uint64 var err error - if options.StartTS != nil { - startTs = *options.StartTS - } else if options.PrevSec != nil { + if option.StartTS != nil { + startTs = *option.StartTS + } else if option.PrevSec != nil { bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTs, err = store.getStalenessTimestamp(bo, options.TxnScope, *options.PrevSec) - } else if options.MinStartTS != nil { + startTs, err = store.getStalenessTimestamp(bo, option.TxnScope, *option.PrevSec) + } else if option.MinStartTS != nil { stores := make([]*Store, 0) allStores := store.regionCache.getStoresByType(tikvrpc.TiKV) - if options.TxnScope != oracle.GlobalTxnScope { + if option.TxnScope != oracle.GlobalTxnScope { for _, store := range allStores { if store.IsLabelsMatch([]*metapb.StoreLabel{ { Key: DCLabelKey, - Value: options.TxnScope, + Value: option.TxnScope, }, }) { stores = append(stores, store) @@ -116,32 +162,32 @@ func extractStartTs(store *KVStore, options kv.TransactionOption) (uint64, error stores = allStores } safeTS := store.getMinSafeTSByStores(stores) - startTs = *options.MinStartTS + startTs = *option.MinStartTS // If the safeTS is larger than the minStartTS, we will use safeTS as StartTS, otherwise we will use // minStartTS directly. if startTs < safeTS { startTs = safeTS } - } else if options.MaxPrevSec != nil { + } else if option.MaxPrevSec != nil { bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - minStartTS, err := store.getStalenessTimestamp(bo, options.TxnScope, *options.MaxPrevSec) + minStartTS, err := store.getStalenessTimestamp(bo, option.TxnScope, *option.MaxPrevSec) if err != nil { return 0, errors.Trace(err) } - options.MinStartTS = &minStartTS - return extractStartTs(store, options) + option.MinStartTS = &minStartTS + return ExtractStartTs(store, option) } else { bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTs, err = store.getTimestampWithRetry(bo, options.TxnScope) + startTs, err = store.getTimestampWithRetry(bo, option.TxnScope) } return startTs, err } -func newTiKVTxnWithOptions(store *KVStore, options kv.TransactionOption) (*KVTxn, error) { +func newTiKVTxnWithOptions(store *KVStore, options StartTSOption) (*KVTxn, error) { if options.TxnScope == "" { options.TxnScope = oracle.GlobalTxnScope } - startTs, err := extractStartTs(store, options) + startTs, err := ExtractStartTs(store, options) if err != nil { return nil, errors.Trace(err) } diff --git a/util/mock/context.go b/util/mock/context.go index d6a5f1d913902..6df2c9c10d356 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/disk" @@ -204,7 +205,7 @@ func (c *Context) InitTxnWithStartTS(startTS uint64) error { return nil } if c.Store != nil { - txn, err := c.Store.BeginWithOption(kv.TransactionOption{}.SetTxnScope(oracle.GlobalTxnScope).SetStartTs(startTS)) + txn, err := c.Store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(oracle.GlobalTxnScope).SetStartTs(startTS)) if err != nil { return errors.Trace(err) } diff --git a/util/mock/store.go b/util/mock/store.go index 7c86de4b3cb72..3adba59e115e5 100644 --- a/util/mock/store.go +++ b/util/mock/store.go @@ -17,6 +17,7 @@ import ( "context" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" ) @@ -38,7 +39,7 @@ func (s *Store) GetOracle() oracle.Oracle { return nil } func (s *Store) Begin() (kv.Transaction, error) { return nil, nil } // BeginWithOption implements kv.Storage interface. -func (s *Store) BeginWithOption(option kv.TransactionOption) (kv.Transaction, error) { +func (s *Store) BeginWithOption(option tikv.StartTSOption) (kv.Transaction, error) { return s.Begin() } From 424a5a880c219a8523d27fe3665b34959272b985 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 19 May 2021 19:15:41 +0800 Subject: [PATCH 104/343] executor: enhancement for ListInDisk(support writing after reading) (#24379) --- util/checksum/checksum.go | 22 +++- util/checksum/checksum_test.go | 72 +++++++++++ util/chunk/disk.go | 63 +++++++++- util/chunk/disk_test.go | 131 ++++++++++++++++++++ util/chunk/row_container_test.go | 22 ++++ util/encrypt/{ase_layer.go => aes_layer.go} | 22 +++- 6 files changed, 317 insertions(+), 15 deletions(-) rename util/encrypt/{ase_layer.go => aes_layer.go} (91%) diff --git a/util/checksum/checksum.go b/util/checksum/checksum.go index 843440547100c..ef90f44e7cb00 100644 --- a/util/checksum/checksum.go +++ b/util/checksum/checksum.go @@ -42,11 +42,12 @@ var checksumReaderBufPool = sync.Pool{ // | -- 4B -- | -- 1020B -- || -- 4B -- | -- 1020B -- || -- 4B -- | -- 60B -- | // | -- checksum -- | -- payload -- || -- checksum -- | -- payload -- || -- checksum -- | -- payload -- | type Writer struct { - err error - w io.WriteCloser - buf []byte - payload []byte - payloadUsed int + err error + w io.WriteCloser + buf []byte + payload []byte + payloadUsed int + flushedUserDataCnt int64 } // NewWriter returns a new Writer which calculates and stores a CRC-32 checksum for the payload before @@ -104,10 +105,21 @@ func (w *Writer) Flush() error { w.err = err return err } + w.flushedUserDataCnt += int64(w.payloadUsed) w.payloadUsed = 0 return nil } +// GetCache returns the byte slice that holds the data not flushed to disk. +func (w *Writer) GetCache() []byte { + return w.payload[:w.payloadUsed] +} + +// GetCacheDataOffset return the user data offset in cache. +func (w *Writer) GetCacheDataOffset() int64 { + return w.flushedUserDataCnt +} + // Close implements the io.Closer interface. func (w *Writer) Close() (err error) { err = w.Flush() diff --git a/util/checksum/checksum_test.go b/util/checksum/checksum_test.go index b0de5b90586c9..1473903fbe080 100644 --- a/util/checksum/checksum_test.go +++ b/util/checksum/checksum_test.go @@ -651,3 +651,75 @@ func (s *testChecksumSuite) testTiCase3651and3652(c *check.C, encrypt bool) { assertReadAt(0, make([]byte, 10200), nil, 10200, strings.Repeat("0123456789", 1020), f1) assertReadAt(0, make([]byte, 10200), nil, 10200, strings.Repeat("0123456789", 1020), f2) } + +var checkFlushedData = func(c *check.C, f io.ReaderAt, off int64, readBufLen int, assertN int, assertErr interface{}, assertRes []byte) { + readBuf := make([]byte, readBufLen) + r := NewReader(f) + n, err := r.ReadAt(readBuf, off) + c.Assert(err, check.Equals, assertErr) + c.Assert(n, check.Equals, assertN) + c.Assert(bytes.Compare(readBuf, assertRes), check.Equals, 0) +} + +func (s *testChecksumSuite) TestChecksumWriter(c *check.C) { + path := "checksum_TestChecksumWriter" + f, err := os.Create(path) + c.Assert(err, check.IsNil) + defer func() { + err = f.Close() + c.Assert(err, check.IsNil) + err = os.Remove(path) + c.Assert(err, check.IsNil) + }() + + buf := bytes.NewBuffer(nil) + testData := "0123456789" + for i := 0; i < 100; i++ { + buf.WriteString(testData) + } + + // Write 1000 bytes and flush. + w := NewWriter(f) + n, err := w.Write(buf.Bytes()) + c.Assert(err, check.IsNil) + c.Assert(n, check.Equals, 1000) + + err = w.Flush() + c.Assert(err, check.IsNil) + checkFlushedData(c, f, 0, 1000, 1000, nil, buf.Bytes()) + + // All data flushed, so no data in cache. + cacheOff := w.GetCacheDataOffset() + c.Assert(cacheOff, check.Equals, int64(1000)) +} + +func (s *testChecksumSuite) TestChecksumWriterAutoFlush(c *check.C) { + path := "checksum_TestChecksumWriterAutoFlush" + f, err := os.Create(path) + c.Assert(err, check.IsNil) + defer func() { + err = f.Close() + c.Assert(err, check.IsNil) + err = os.Remove(path) + c.Assert(err, check.IsNil) + }() + + w := NewWriter(f) + + buf := bytes.NewBuffer(nil) + testData := "0123456789" + for i := 0; i < 102; i++ { + buf.WriteString(testData) + } + n, err := w.Write(buf.Bytes()) + c.Assert(err, check.IsNil) + c.Assert(n, check.Equals, len(buf.Bytes())) + + // This write will trigger flush. + n, err = w.Write([]byte("0")) + c.Assert(err, check.IsNil) + c.Assert(n, check.Equals, 1) + checkFlushedData(c, f, 0, 1020, 1020, nil, buf.Bytes()) + cacheOff := w.GetCacheDataOffset() + c.Assert(cacheOff, check.Equals, int64(len(buf.Bytes()))) +} diff --git a/util/chunk/disk.go b/util/chunk/disk.go index c7962c9aa9e9d..ef269213e9d0d 100644 --- a/util/chunk/disk.go +++ b/util/chunk/disk.go @@ -46,6 +46,9 @@ type ListInDisk struct { diskTracker *disk.Tracker // track disk usage. numRowsInDisk int + checksumWriter *checksum.Writer + cipherWriter *encrypt.Writer + // ctrCipher stores the key and nonce using by aes encrypt io layer ctrCipher *encrypt.CtrCipher } @@ -78,9 +81,11 @@ func (l *ListInDisk) initDiskFile() (err error) { if err != nil { return } - underlying = encrypt.NewWriter(l.disk, l.ctrCipher) + l.cipherWriter = encrypt.NewWriter(l.disk, l.ctrCipher) + underlying = l.cipherWriter } - l.w = checksum.NewWriter(underlying) + l.checksumWriter = checksum.NewWriter(underlying) + l.w = l.checksumWriter l.bufFlushMutex = sync.RWMutex{} return } @@ -164,16 +169,16 @@ func (l *ListInDisk) GetChunk(chkIdx int) (*Chunk, error) { // GetRow gets a Row from the ListInDisk by RowPtr. func (l *ListInDisk) GetRow(ptr RowPtr) (row Row, err error) { - err = l.flush() if err != nil { return } off := l.offsets[ptr.ChkIdx][ptr.RowIdx] var underlying io.ReaderAt = l.disk if l.ctrCipher != nil { - underlying = encrypt.NewReader(l.disk, l.ctrCipher) + underlying = NewReaderWithCache(encrypt.NewReader(l.disk, l.ctrCipher), l.cipherWriter.GetCache(), l.cipherWriter.GetCacheDataOffset()) } - r := io.NewSectionReader(checksum.NewReader(underlying), off, l.offWrite-off) + checksumReader := NewReaderWithCache(checksum.NewReader(underlying), l.checksumWriter.GetCache(), l.checksumWriter.GetCacheDataOffset()) + r := io.NewSectionReader(checksumReader, off, l.offWrite-off) format := rowInDisk{numCol: len(l.fieldTypes)} _, err = format.ReadFrom(r) if err != nil { @@ -367,3 +372,51 @@ func (format *diskFormatRow) toMutRow(fields []*types.FieldType) MutRow { } return MutRow{c: chk} } + +// ReaderWithCache helps to read data that has not be flushed to underlying layer. +// By using ReaderWithCache, user can still write data into ListInDisk even after reading. +type ReaderWithCache struct { + r io.ReaderAt + cacheOff int64 + cache []byte +} + +// NewReaderWithCache returns a ReaderWithCache. +func NewReaderWithCache(r io.ReaderAt, cache []byte, cacheOff int64) *ReaderWithCache { + return &ReaderWithCache{ + r: r, + cacheOff: cacheOff, + cache: cache, + } +} + +// ReadAt implements the ReadAt interface. +func (r *ReaderWithCache) ReadAt(p []byte, off int64) (readCnt int, err error) { + readCnt, err = r.r.ReadAt(p, off) + if err != io.EOF { + return readCnt, err + } + + if len(p) == readCnt { + return readCnt, err + } else if len(p) < readCnt { + return readCnt, errors2.Trace(errors2.Errorf("cannot read more data than user requested"+ + "(readCnt: %v, len(p): %v", readCnt, len(p))) + } + + // When got here, user input is not filled fully, so we need read data from cache. + err = nil + p = p[readCnt:] + beg := off - r.cacheOff + if beg < 0 { + // This happens when only partial data of user requested resides in r.cache. + beg = 0 + } + end := int(beg) + len(p) + if end > len(r.cache) { + err = io.EOF + end = len(r.cache) + } + readCnt += copy(p, r.cache[beg:end]) + return readCnt, err +} diff --git a/util/chunk/disk_test.go b/util/chunk/disk_test.go index 86461de5659c7..36750aa898244 100644 --- a/util/chunk/disk_test.go +++ b/util/chunk/disk_test.go @@ -14,12 +14,14 @@ package chunk import ( + "bytes" "fmt" "io" "io/ioutil" "math/rand" "os" "path/filepath" + "reflect" "strconv" "strings" "testing" @@ -30,6 +32,8 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" + "github.com/pingcap/tidb/util/checksum" + "github.com/pingcap/tidb/util/encrypt" ) func initChunks(numChk, numRow int) ([]*Chunk, []*types.FieldType) { @@ -219,6 +223,8 @@ func (s *testChunkSuite) TestListInDiskWithChecksum(c *check.C) { }) testListInDisk(c) + testReaderWithCache(c) + testReaderWithCacheNoFlush(c) } func (s *testChunkSuite) TestListInDiskWithChecksumAndEncrypt(c *check.C) { @@ -227,4 +233,129 @@ func (s *testChunkSuite) TestListInDiskWithChecksumAndEncrypt(c *check.C) { conf.Security.SpilledFileEncryptionMethod = config.SpilledFileEncryptionMethodAES128CTR }) testListInDisk(c) + + testReaderWithCache(c) + testReaderWithCacheNoFlush(c) +} + +// Following diagram describes the testdata we use to test: +// 4 B: checksum of this segment. +// 8 B: all columns' length, in the following example, we will only have one column. +// 1012 B: data in file. because max length of each segment is 1024, so we only have 1020B for user payload. +// +// Data in File Data in mem cache +// +------+------------------------------------------+ +-----------------------------+ +// | | 1020B payload | | | +// |4Bytes| +---------+----------------------------+ | | | +// |checksum|8B collen| 1012B user data | | | 12B remained user data | +// | | +---------+----------------------------+ | | | +// | | | | | +// +------+------------------------------------------+ +-----------------------------+ +func testReaderWithCache(c *check.C) { + testData := "0123456789" + buf := bytes.NewBuffer(nil) + for i := 0; i < 102; i++ { + buf.WriteString(testData) + } + buf.WriteString("0123") + + field := []*types.FieldType{types.NewFieldType(mysql.TypeString)} + chk := NewChunkWithCapacity(field, 1) + chk.AppendString(0, buf.String()) + l := NewListInDisk(field) + err := l.Add(chk) + c.Assert(err, check.IsNil) + + // Basic test for GetRow(). + row, err := l.GetRow(RowPtr{0, 0}) + c.Assert(err, check.IsNil) + c.Assert(row.GetDatumRow(field), check.DeepEquals, chk.GetRow(0).GetDatumRow(field)) + + var underlying io.ReaderAt = l.disk + if l.ctrCipher != nil { + underlying = NewReaderWithCache(encrypt.NewReader(l.disk, l.ctrCipher), l.cipherWriter.GetCache(), l.cipherWriter.GetCacheDataOffset()) + } + checksumReader := NewReaderWithCache(checksum.NewReader(underlying), l.checksumWriter.GetCache(), l.checksumWriter.GetCacheDataOffset()) + + // Read all data. + data := make([]byte, 1024) + // Offset is 8, because we want to ignore col length. + readCnt, err := checksumReader.ReadAt(data, 8) + c.Assert(err, check.IsNil) + c.Assert(readCnt, check.Equals, 1024) + c.Assert(reflect.DeepEqual(data, buf.Bytes()), check.IsTrue) + + // Only read data of mem cache. + data = make([]byte, 1024) + readCnt, err = checksumReader.ReadAt(data, 1020) + c.Assert(err, check.Equals, io.EOF) + c.Assert(readCnt, check.Equals, 12) + c.Assert(reflect.DeepEqual(data[:12], buf.Bytes()[1012:]), check.IsTrue) + + // Read partial data of mem cache. + data = make([]byte, 1024) + readCnt, err = checksumReader.ReadAt(data, 1025) + c.Assert(err, check.Equals, io.EOF) + c.Assert(readCnt, check.Equals, 7) + c.Assert(reflect.DeepEqual(data[:7], buf.Bytes()[1017:]), check.IsTrue) + + // Read partial data from both file and mem cache. + data = make([]byte, 1024) + readCnt, err = checksumReader.ReadAt(data, 1010) + c.Assert(err, check.Equals, io.EOF) + c.Assert(readCnt, check.Equals, 22) + c.Assert(reflect.DeepEqual(data[:22], buf.Bytes()[1002:]), check.IsTrue) + + // Offset is too large, so no data is read. + data = make([]byte, 1024) + readCnt, err = checksumReader.ReadAt(data, 1032) + c.Assert(err, check.Equals, io.EOF) + c.Assert(readCnt, check.Equals, 0) + c.Assert(reflect.DeepEqual(data, make([]byte, 1024)), check.IsTrue) + + // Only read 1 byte from mem cache. + data = make([]byte, 1024) + readCnt, err = checksumReader.ReadAt(data, 1031) + c.Assert(err, check.Equals, io.EOF) + c.Assert(readCnt, check.Equals, 1) + c.Assert(reflect.DeepEqual(data[:1], buf.Bytes()[1023:]), check.IsTrue) + + // Test user requested data is small. + // Only request 10 bytes. + data = make([]byte, 10) + readCnt, err = checksumReader.ReadAt(data, 1010) + c.Assert(err, check.IsNil) + c.Assert(readCnt, check.Equals, 10) + c.Assert(reflect.DeepEqual(data, buf.Bytes()[1002:1012]), check.IsTrue) +} + +// Here we test situations where size of data is small, so no data is flushed to disk. +func testReaderWithCacheNoFlush(c *check.C) { + testData := "0123456789" + + field := []*types.FieldType{types.NewFieldType(mysql.TypeString)} + chk := NewChunkWithCapacity(field, 1) + chk.AppendString(0, testData) + l := NewListInDisk(field) + err := l.Add(chk) + c.Assert(err, check.IsNil) + + // Basic test for GetRow(). + row, err := l.GetRow(RowPtr{0, 0}) + c.Assert(err, check.IsNil) + c.Assert(row.GetDatumRow(field), check.DeepEquals, chk.GetRow(0).GetDatumRow(field)) + + var underlying io.ReaderAt = l.disk + if l.ctrCipher != nil { + underlying = NewReaderWithCache(encrypt.NewReader(l.disk, l.ctrCipher), l.cipherWriter.GetCache(), l.cipherWriter.GetCacheDataOffset()) + } + checksumReader := NewReaderWithCache(checksum.NewReader(underlying), l.checksumWriter.GetCache(), l.checksumWriter.GetCacheDataOffset()) + + // Read all data. + data := make([]byte, 1024) + // Offset is 8, because we want to ignore col length. + readCnt, err := checksumReader.ReadAt(data, 8) + c.Assert(err, check.Equals, io.EOF) + c.Assert(readCnt, check.Equals, len(testData)) + c.Assert(reflect.DeepEqual(data[:10], []byte(testData)), check.IsTrue) } diff --git a/util/chunk/row_container_test.go b/util/chunk/row_container_test.go index feed2290f38b6..a39346e34ff80 100644 --- a/util/chunk/row_container_test.go +++ b/util/chunk/row_container_test.go @@ -113,6 +113,28 @@ func (r *rowContainerTestSuite) TestSpillAction(c *check.C) { rc.actionSpill.WaitForTest() c.Assert(err, check.IsNil) c.Assert(rc.AlreadySpilledSafeForTest(), check.Equals, true) + + // Read + resChk, err := rc.GetChunk(0) + c.Assert(err, check.IsNil) + c.Assert(resChk.NumRows(), check.Equals, chk.NumRows()) + for rowIdx := 0; rowIdx < resChk.NumRows(); rowIdx++ { + c.Assert(resChk.GetRow(rowIdx).GetDatumRow(fields), check.DeepEquals, chk.GetRow(rowIdx).GetDatumRow(fields)) + } + // Write again + err = rc.Add(chk) + rc.actionSpill.WaitForTest() + c.Assert(err, check.IsNil) + c.Assert(rc.AlreadySpilledSafeForTest(), check.Equals, true) + + // Read + resChk, err = rc.GetChunk(2) + c.Assert(err, check.IsNil) + c.Assert(resChk.NumRows(), check.Equals, chk.NumRows()) + for rowIdx := 0; rowIdx < resChk.NumRows(); rowIdx++ { + c.Assert(resChk.GetRow(rowIdx).GetDatumRow(fields), check.DeepEquals, chk.GetRow(rowIdx).GetDatumRow(fields)) + } + err = rc.Reset() c.Assert(err, check.IsNil) } diff --git a/util/encrypt/ase_layer.go b/util/encrypt/aes_layer.go similarity index 91% rename from util/encrypt/ase_layer.go rename to util/encrypt/aes_layer.go index 2bcea4373073f..a27d23da90fa6 100644 --- a/util/encrypt/ase_layer.go +++ b/util/encrypt/aes_layer.go @@ -71,11 +71,12 @@ func (ctr *CtrCipher) stream(counter uint64) cipher.Stream { // Writer implements an io.WriteCloser, it encrypt data using AES before writing to the underlying object. type Writer struct { - err error - w io.WriteCloser - n int - buf []byte - cipherStream cipher.Stream + err error + w io.WriteCloser + n int + buf []byte + cipherStream cipher.Stream + flushedUserDataCnt int64 } // NewWriter returns a new Writer which encrypt data using AES before writing to the underlying object. @@ -123,6 +124,7 @@ func (w *Writer) Flush() error { } w.cipherStream.XORKeyStream(w.buf[:w.n], w.buf[:w.n]) n, err := w.w.Write(w.buf[:w.n]) + w.flushedUserDataCnt += int64(n) if n < w.n && err == nil { err = io.ErrShortWrite } @@ -134,6 +136,16 @@ func (w *Writer) Flush() error { return nil } +// GetCache returns the byte slice that holds the data not flushed to disk. +func (w *Writer) GetCache() []byte { + return w.buf[:w.n] +} + +// GetCacheDataOffset return the user data offset in cache. +func (w *Writer) GetCacheDataOffset() int64 { + return w.flushedUserDataCnt +} + // Close implements the io.Closer interface. func (w *Writer) Close() (err error) { err = w.Flush() From 8fb29eb17e88cabd3968d6034d1e976823f2f79e Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 19 May 2021 20:01:42 +0800 Subject: [PATCH 105/343] kv: move TxnScope into kv (#24715) --- ddl/db_integration_test.go | 3 +- ddl/delete_range.go | 3 +- ddl/reorg.go | 3 +- distsql/request_builder.go | 9 ++- domain/domain.go | 3 +- domain/domain_test.go | 4 +- domain/infosync/info.go | 2 +- executor/batch_point_get.go | 3 +- executor/executor_test.go | 4 +- executor/point_get.go | 5 +- executor/stale_txn_test.go | 3 +- kv/fault_injection_test.go | 3 +- kv/mock_test.go | 3 +- kv/txn_scope_var.go | 73 +++++++++++++++++++++++ meta/meta_test.go | 3 +- server/server.go | 6 +- session/pessimistic_test.go | 4 +- session/schema_amender_test.go | 3 +- session/session.go | 5 +- session/session_test.go | 25 ++++---- sessionctx/variable/session.go | 10 ++-- sessionctx/variable/sysvar.go | 13 ++-- store/gcworker/gc_worker.go | 6 +- store/mockstore/mockcopr/executor_test.go | 2 +- store/store_test.go | 3 +- store/tikv/oracle/oracle.go | 47 --------------- store/tikv/tests/extract_start_ts_test.go | 6 +- util/mock/context.go | 3 +- 28 files changed, 134 insertions(+), 123 deletions(-) create mode 100644 kv/txn_scope_var.go diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 405ada57f15ec..0c2f851da4413 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -41,7 +41,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -1343,7 +1342,7 @@ func getMaxTableHandle(ctx *testMaxTableRowIDContext, store kv.Storage) (kv.Hand c := ctx.c d := ctx.d tbl := ctx.tbl - curVer, err := store.CurrentVersion(oracle.GlobalTxnScope) + curVer, err := store.CurrentVersion(kv.GlobalTxnScope) c.Assert(err, IsNil) maxHandle, emptyTable, err := d.GetTableMaxHandle(curVer.Ver, tbl.(table.PhysicalTable)) c.Assert(err, IsNil) diff --git a/ddl/delete_range.go b/ddl/delete_range.go index e64c122d41e4d..1aeb5ab0354da 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" @@ -451,7 +450,7 @@ func doBatchInsert(s sqlexec.SQLExecutor, jobID int64, tableIDs []int64, ts uint // getNowTS gets the current timestamp, in TSO. func getNowTSO(ctx sessionctx.Context) (uint64, error) { - currVer, err := ctx.GetStore().CurrentVersion(oracle.GlobalTxnScope) + currVer, err := ctx.GetStore().CurrentVersion(kv.GlobalTxnScope) if err != nil { return 0, errors.Trace(err) } diff --git a/ddl/reorg.go b/ddl/reorg.go index fbe42573dbbf7..37bfe82d4ce3d 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -534,7 +533,7 @@ func getTableRange(d *ddlCtx, tbl table.PhysicalTable, snapshotVer uint64, prior } func getValidCurrentVersion(store kv.Storage) (ver kv.Version, err error) { - ver, err = store.CurrentVersion(oracle.GlobalTxnScope) + ver, err = store.CurrentVersion(kv.GlobalTxnScope) if err != nil { return ver, errors.Trace(err) } else if ver.Ver <= 0 { diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 69a6da548ec60..aced1a71aaa7b 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" @@ -236,7 +235,7 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req } builder.txnScope = sv.TxnCtx.TxnScope builder.IsStaleness = sv.TxnCtx.IsStaleness - if builder.IsStaleness && builder.txnScope != oracle.GlobalTxnScope { + if builder.IsStaleness && builder.txnScope != kv.GlobalTxnScope { builder.MatchStoreLabels = []*metapb.StoreLabel{ { Key: placement.DCLabelKey, @@ -279,9 +278,9 @@ func (builder *RequestBuilder) SetFromInfoSchema(is infoschema.InfoSchema) *Requ func (builder *RequestBuilder) verifyTxnScope() error { if builder.txnScope == "" { - builder.txnScope = oracle.GlobalTxnScope + builder.txnScope = kv.GlobalTxnScope } - if builder.txnScope == oracle.GlobalTxnScope || builder.is == nil { + if builder.txnScope == kv.GlobalTxnScope || builder.is == nil { return nil } visitPhysicalTableID := make(map[int64]struct{}) @@ -600,7 +599,7 @@ func CommonHandleRangesToKVRanges(sc *stmtctx.StatementContext, tids []int64, ra // VerifyTxnScope verify whether the txnScope and visited physical table break the leader rule's dcLocation. func VerifyTxnScope(txnScope string, physicalTableID int64, is infoschema.InfoSchema) bool { - if txnScope == "" || txnScope == oracle.GlobalTxnScope { + if txnScope == "" || txnScope == kv.GlobalTxnScope { return true } bundle, ok := is.BundleByName(placement.GroupID(physicalTableID)) diff --git a/domain/domain.go b/domain/domain.go index 44f6df1aa9086..a22d647066ea1 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -48,7 +48,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/telemetry" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" @@ -336,7 +335,7 @@ func (do *Domain) Reload() error { defer do.m.Unlock() startTime := time.Now() - ver, err := do.store.CurrentVersion(oracle.GlobalTxnScope) + ver, err := do.store.CurrentVersion(kv.GlobalTxnScope) if err != nil { return err } diff --git a/domain/domain_test.go b/domain/domain_test.go index 82a583866aad3..51e0948d30715 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -347,7 +347,7 @@ func (*testSuite) TestT(c *C) { // for schemaValidator schemaVer := dom.SchemaValidator.(*schemaValidator).LatestSchemaVersion() - ver, err := store.CurrentVersion(oracle.GlobalTxnScope) + ver, err := store.CurrentVersion(kv.GlobalTxnScope) c.Assert(err, IsNil) ts := ver.Ver @@ -360,7 +360,7 @@ func (*testSuite) TestT(c *C) { c.Assert(succ, Equals, ResultSucc) time.Sleep(ddlLease) - ver, err = store.CurrentVersion(oracle.GlobalTxnScope) + ver, err = store.CurrentVersion(kv.GlobalTxnScope) c.Assert(err, IsNil) ts = ver.Ver _, succ = dom.SchemaValidator.Check(ts, schemaVer, nil) diff --git a/domain/infosync/info.go b/domain/infosync/info.go index cbd76d4e6f266..be8d80246e96b 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -554,7 +554,7 @@ func (is *InfoSyncer) ReportMinStartTS(store kv.Storage) { pl := is.manager.ShowProcessList() // Calculate the lower limit of the start timestamp to avoid extremely old transaction delaying GC. - currentVer, err := store.CurrentVersion(oracle.GlobalTxnScope) + currentVer, err := store.CurrentVersion(kv.GlobalTxnScope) if err != nil { logutil.BgLogger().Error("update minStartTS failed", zap.Error(err)) return diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 2137884c69745..726603a0ff88f 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -30,7 +30,6 @@ import ( driver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -122,7 +121,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness snapshot.SetOption(kv.IsStalenessReadOnly, isStaleness) - if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != oracle.GlobalTxnScope { + if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != kv.GlobalTxnScope { snapshot.SetOption(kv.MatchStoreLabels, []*metapb.StoreLabel{ { Key: placement.DCLabelKey, diff --git a/executor/executor_test.go b/executor/executor_test.go index 730de6dd70bde..dde9511a8410d 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2693,11 +2693,11 @@ func (s *testSuiteP2) TestHistoryRead(c *C) { // SnapshotTS Is not updated if check failed. c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(0)) - curVer1, _ := s.store.CurrentVersion(oracle.GlobalTxnScope) + curVer1, _ := s.store.CurrentVersion(kv.GlobalTxnScope) time.Sleep(time.Millisecond) snapshotTime := time.Now() time.Sleep(time.Millisecond) - curVer2, _ := s.store.CurrentVersion(oracle.GlobalTxnScope) + curVer2, _ := s.store.CurrentVersion(kv.GlobalTxnScope) tk.MustExec("insert history_read values (2)") tk.MustQuery("select * from history_read").Check(testkit.Rows("1", "2")) tk.MustExec("set @@tidb_snapshot = '" + snapshotTime.Format("2006-01-02 15:04:05.999999") + "'") diff --git a/executor/point_get.go b/executor/point_get.go index fc8326555bf01..dccb72bdebb17 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/tikv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -153,7 +152,7 @@ func (e *PointGetExecutor) Open(context.Context) error { e.snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness e.snapshot.SetOption(kv.IsStalenessReadOnly, isStaleness) - if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != oracle.GlobalTxnScope { + if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != kv.GlobalTxnScope { e.snapshot.SetOption(kv.MatchStoreLabels, []*metapb.StoreLabel{ { Key: placement.DCLabelKey, @@ -392,7 +391,7 @@ func (e *PointGetExecutor) get(ctx context.Context, key kv.Key) ([]byte, error) func (e *PointGetExecutor) verifyTxnScope() error { txnScope := e.txn.GetOption(kv.TxnScope).(string) - if txnScope == "" || txnScope == oracle.GlobalTxnScope { + if txnScope == "" || txnScope == kv.GlobalTxnScope { return nil } var tblID int64 diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 7cf235bd3c0f7..64b334b15bf94 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -20,6 +20,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl/placement" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/testkit" ) @@ -76,7 +77,7 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { preSQL: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`, sql: "begin", IsStaleness: false, - txnScope: oracle.GlobalTxnScope, + txnScope: kv.GlobalTxnScope, zone: "", }, } diff --git a/kv/fault_injection_test.go b/kv/fault_injection_test.go index 4979dbf4268cd..c5e203151fe63 100644 --- a/kv/fault_injection_test.go +++ b/kv/fault_injection_test.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/oracle" ) type testFaultInjectionSuite struct{} @@ -36,7 +35,7 @@ func (s testFaultInjectionSuite) TestFaultInjectionBasic(c *C) { storage := NewInjectedStore(newMockStorage(), &cfg) txn, err := storage.Begin() c.Assert(err, IsNil) - _, err = storage.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(oracle.GlobalTxnScope).SetStartTs(0)) + _, err = storage.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(GlobalTxnScope).SetStartTs(0)) c.Assert(err, IsNil) ver := Version{Ver: 1} snap := storage.GetSnapshot(ver) diff --git a/kv/mock_test.go b/kv/mock_test.go index eba059e763f82..e09c291d5de95 100644 --- a/kv/mock_test.go +++ b/kv/mock_test.go @@ -17,7 +17,6 @@ import ( "context" . "github.com/pingcap/check" - "github.com/pingcap/tidb/store/tikv/oracle" ) var _ = Suite(testMockSuite{}) @@ -29,7 +28,7 @@ func (s testMockSuite) TestInterface(c *C) { storage := newMockStorage() storage.GetClient() storage.UUID() - version, err := storage.CurrentVersion(oracle.GlobalTxnScope) + version, err := storage.CurrentVersion(GlobalTxnScope) c.Check(err, IsNil) snapshot := storage.GetSnapshot(version) _, err = snapshot.BatchGet(context.Background(), []Key{Key("abc"), Key("def")}) diff --git a/kv/txn_scope_var.go b/kv/txn_scope_var.go new file mode 100644 index 0000000000000..941fdaff5f26f --- /dev/null +++ b/kv/txn_scope_var.go @@ -0,0 +1,73 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package kv + +import ( + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/store/tikv/oracle" +) + +// TxnScopeVar indicates the used txnScope for oracle +type TxnScopeVar struct { + // varValue indicates the value of @@txn_scope, which can only be `global` or `local` + varValue string + // txnScope indicates the value which the tidb-server holds to request tso to pd + txnScope string +} + +// GetTxnScopeVar gets TxnScopeVar from config +func GetTxnScopeVar() TxnScopeVar { + isGlobal, location := config.GetTxnScopeFromConfig() + if isGlobal { + return NewGlobalTxnScopeVar() + } + return NewLocalTxnScopeVar(location) +} + +// NewGlobalTxnScopeVar creates a Global TxnScopeVar +func NewGlobalTxnScopeVar() TxnScopeVar { + return newTxnScopeVar(GlobalTxnScope, GlobalTxnScope) +} + +// NewLocalTxnScopeVar creates a Local TxnScopeVar with given real txnScope value. +func NewLocalTxnScopeVar(txnScope string) TxnScopeVar { + return newTxnScopeVar(LocalTxnScope, txnScope) +} + +// GetVarValue returns the value of @@txn_scope which can only be `global` or `local` +func (t TxnScopeVar) GetVarValue() string { + return t.varValue +} + +// GetTxnScope returns the value of the tidb-server holds to request tso to pd. +func (t TxnScopeVar) GetTxnScope() string { + return t.txnScope +} + +func newTxnScopeVar(varValue string, txnScope string) TxnScopeVar { + return TxnScopeVar{ + varValue: varValue, + txnScope: txnScope, + } +} + +// Transaction scopes constants. +const ( + // GlobalTxnScope is synced with PD's define of global scope. + // If we want to remove the dependency on store/tikv here, we need to map + // the two GlobalTxnScopes in the driver layer. + GlobalTxnScope = oracle.GlobalTxnScope + // LocalTxnScope indicates the transaction should use local ts. + LocalTxnScope = "local" +) diff --git a/meta/meta_test.go b/meta/meta_test.go index 590e85fc2a21e..4ba54f1935a3a 100644 --- a/meta/meta_test.go +++ b/meta/meta_test.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/testleak" . "github.com/pingcap/tidb/util/testutil" ) @@ -291,7 +290,7 @@ func (s *testSuite) TestSnapshot(c *C) { err = txn.Commit(context.Background()) c.Assert(err, IsNil) - ver1, _ := store.CurrentVersion(oracle.GlobalTxnScope) + ver1, _ := store.CurrentVersion(kv.GlobalTxnScope) time.Sleep(time.Millisecond) txn, _ = store.Begin() m = meta.NewMeta(txn) diff --git a/server/server.go b/server/server.go index 29f5307895cc2..935abbb7bc693 100644 --- a/server/server.go +++ b/server/server.go @@ -54,11 +54,11 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/plugin" "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/fastrand" @@ -311,9 +311,9 @@ func setSSLVariable(ca, key, cert string) { func setTxnScope() { variable.SetSysVar("txn_scope", func() string { if isGlobal, _ := config.GetTxnScopeFromConfig(); isGlobal { - return oracle.GlobalTxnScope + return kv.GlobalTxnScope } - return oracle.LocalTxnScope + return kv.LocalTxnScope }()) } diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 72853d86208a9..2e8c01c75577b 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -2029,14 +2029,14 @@ func (s *testPessimisticSuite) TestSelectForUpdateConflictRetry(c *C) { tsCh := make(chan uint64) go func() { tk3.MustExec("update tk set c2 = c2 + 1 where c1 = 1") - lastTS, err := s.store.GetOracle().GetLowResolutionTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + lastTS, err := s.store.GetOracle().GetLowResolutionTimestamp(context.Background(), &oracle.Option{TxnScope: kv.GlobalTxnScope}) c.Assert(err, IsNil) tsCh <- lastTS tk3.MustExec("commit") tsCh <- lastTS }() // tk2LastTS should be its forUpdateTS - tk2LastTS, err := s.store.GetOracle().GetLowResolutionTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + tk2LastTS, err := s.store.GetOracle().GetLowResolutionTimestamp(context.Background(), &oracle.Option{TxnScope: kv.GlobalTxnScope}) c.Assert(err, IsNil) tk2.MustExec("commit") diff --git a/session/schema_amender_test.go b/session/schema_amender_test.go index ca05f4a74dbff..eda0e5e387e05 100644 --- a/session/schema_amender_test.go +++ b/session/schema_amender_test.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -426,7 +425,7 @@ func (s *testSchemaAmenderSuite) TestAmendCollectAndGenMutations(c *C) { } c.Assert(err, IsNil) } - curVer, err := se.store.CurrentVersion(oracle.GlobalTxnScope) + curVer, err := se.store.CurrentVersion(kv.GlobalTxnScope) c.Assert(err, IsNil) se.sessionVars.TxnCtx.SetForUpdateTS(curVer.Ver + 1) mutationVals, err := txn.BatchGet(ctx, checkKeys) diff --git a/session/session.go b/session/session.go index c9b3f8f7a8abd..43bd8be8cf4c3 100644 --- a/session/session.go +++ b/session/session.go @@ -69,7 +69,6 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/telemetry" @@ -2929,9 +2928,9 @@ func (s *session) checkPlacementPolicyBeforeCommit() error { // Get the txnScope of the transaction we're going to commit. txnScope := s.GetSessionVars().TxnCtx.TxnScope if txnScope == "" { - txnScope = oracle.GlobalTxnScope + txnScope = kv.GlobalTxnScope } - if txnScope != oracle.GlobalTxnScope { + if txnScope != kv.GlobalTxnScope { is := s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) deltaMap := s.GetSessionVars().TxnCtx.TableDeltaMap for physicalTableID := range deltaMap { diff --git a/session/session_test.go b/session/session_test.go index df2a167921e56..a47557642602d 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -52,7 +52,6 @@ import ( "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" - "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -2075,7 +2074,7 @@ func (s *testSchemaSerialSuite) TestLoadSchemaFailed(c *C) { _, err = tk1.Exec("commit") c.Check(err, NotNil) - ver, err := s.store.CurrentVersion(oracle.GlobalTxnScope) + ver, err := s.store.CurrentVersion(kv.GlobalTxnScope) c.Assert(err, IsNil) c.Assert(ver, NotNil) @@ -3338,26 +3337,26 @@ func (s *testSessionSerialSuite) TestSetTxnScope(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) // assert default value result := tk.MustQuery("select @@txn_scope;") - result.Check(testkit.Rows(oracle.GlobalTxnScope)) - c.Assert(tk.Se.GetSessionVars().CheckAndGetTxnScope(), Equals, oracle.GlobalTxnScope) + result.Check(testkit.Rows(kv.GlobalTxnScope)) + c.Assert(tk.Se.GetSessionVars().CheckAndGetTxnScope(), Equals, kv.GlobalTxnScope) // assert set sys variable tk.MustExec("set @@session.txn_scope = 'local';") result = tk.MustQuery("select @@txn_scope;") - result.Check(testkit.Rows(oracle.GlobalTxnScope)) - c.Assert(tk.Se.GetSessionVars().CheckAndGetTxnScope(), Equals, oracle.GlobalTxnScope) + result.Check(testkit.Rows(kv.GlobalTxnScope)) + c.Assert(tk.Se.GetSessionVars().CheckAndGetTxnScope(), Equals, kv.GlobalTxnScope) failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") failpoint.Enable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope", `return("bj")`) defer failpoint.Disable("github.com/pingcap/tidb/store/tikv/config/injectTxnScope") tk = testkit.NewTestKitWithInit(c, s.store) // assert default value result = tk.MustQuery("select @@txn_scope;") - result.Check(testkit.Rows(oracle.LocalTxnScope)) + result.Check(testkit.Rows(kv.LocalTxnScope)) c.Assert(tk.Se.GetSessionVars().CheckAndGetTxnScope(), Equals, "bj") // assert set sys variable tk.MustExec("set @@session.txn_scope = 'global';") result = tk.MustQuery("select @@txn_scope;") - result.Check(testkit.Rows(oracle.GlobalTxnScope)) - c.Assert(tk.Se.GetSessionVars().CheckAndGetTxnScope(), Equals, oracle.GlobalTxnScope) + result.Check(testkit.Rows(kv.GlobalTxnScope)) + c.Assert(tk.Se.GetSessionVars().CheckAndGetTxnScope(), Equals, kv.GlobalTxnScope) // assert set invalid txn_scope err := tk.ExecToErr("set @@txn_scope='foo'") @@ -3414,9 +3413,9 @@ PARTITION BY RANGE (c) ( setBundle("p1", "dc-2") // set txn_scope to global - tk.MustExec(fmt.Sprintf("set @@session.txn_scope = '%s';", oracle.GlobalTxnScope)) + tk.MustExec(fmt.Sprintf("set @@session.txn_scope = '%s';", kv.GlobalTxnScope)) result := tk.MustQuery("select @@txn_scope;") - result.Check(testkit.Rows(oracle.GlobalTxnScope)) + result.Check(testkit.Rows(kv.GlobalTxnScope)) // test global txn auto commit tk.MustExec("insert into t1 (c) values (1)") // write dc-1 with global scope @@ -3427,7 +3426,7 @@ PARTITION BY RANGE (c) ( tk.MustExec("begin") txn, err := tk.Se.Txn(true) c.Assert(err, IsNil) - c.Assert(tk.Se.GetSessionVars().TxnCtx.TxnScope, Equals, oracle.GlobalTxnScope) + c.Assert(tk.Se.GetSessionVars().TxnCtx.TxnScope, Equals, kv.GlobalTxnScope) c.Assert(txn.Valid(), IsTrue) tk.MustExec("insert into t1 (c) values (1)") // write dc-1 with global scope result = tk.MustQuery("select * from t1") // read dc-1 and dc-2 with global scope @@ -3441,7 +3440,7 @@ PARTITION BY RANGE (c) ( tk.MustExec("begin") txn, err = tk.Se.Txn(true) c.Assert(err, IsNil) - c.Assert(tk.Se.GetSessionVars().TxnCtx.TxnScope, Equals, oracle.GlobalTxnScope) + c.Assert(tk.Se.GetSessionVars().TxnCtx.TxnScope, Equals, kv.GlobalTxnScope) c.Assert(txn.Valid(), IsTrue) tk.MustExec("insert into t1 (c) values (101)") // write dc-2 with global scope result = tk.MustQuery("select * from t1") // read dc-1 and dc-2 with global scope diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index c474e7905fa7b..d38881a8ddcfa 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -800,7 +800,7 @@ type SessionVars struct { PartitionPruneMode atomic2.String // TxnScope indicates the scope of the transactions. It should be `global` or equal to `dc-location` in configuration. - TxnScope oracle.TxnScope + TxnScope kv.TxnScopeVar // EnabledRateLimitAction indicates whether enabled ratelimit action during coprocessor EnabledRateLimitAction bool @@ -863,12 +863,12 @@ func (s *SessionVars) IsMPPEnforced() bool { // CheckAndGetTxnScope will return the transaction scope we should use in the current session. func (s *SessionVars) CheckAndGetTxnScope() string { if s.InRestrictedSQL { - return oracle.GlobalTxnScope + return kv.GlobalTxnScope } - if s.TxnScope.GetVarValue() == oracle.LocalTxnScope { + if s.TxnScope.GetVarValue() == kv.LocalTxnScope { return s.TxnScope.GetTxnScope() } - return oracle.GlobalTxnScope + return kv.GlobalTxnScope } // UseDynamicPartitionPrune indicates whether use new dynamic partition prune. @@ -1055,7 +1055,7 @@ func NewSessionVars() *SessionVars { EnableAlterPlacement: DefTiDBEnableAlterPlacement, EnableAmendPessimisticTxn: DefTiDBEnableAmendPessimisticTxn, PartitionPruneMode: *atomic2.NewString(DefTiDBPartitionPruneMode), - TxnScope: oracle.GetTxnScope(), + TxnScope: kv.GetTxnScopeVar(), EnabledRateLimitAction: DefTiDBEnableRateLimitAction, EnableAsyncCommit: DefTiDBEnableAsyncCommit, Enable1PC: DefTiDBEnable1PC, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index e6f632db6b02d..99ff4a09f7af6 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" tikvstore "github.com/pingcap/tidb/store/tikv/kv" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" @@ -734,15 +733,15 @@ var defaultSysVars = []*SysVar{ /* TiDB specific variables */ {Scope: ScopeSession, Name: TiDBTxnScope, Value: func() string { if isGlobal, _ := config.GetTxnScopeFromConfig(); isGlobal { - return oracle.GlobalTxnScope + return kv.GlobalTxnScope } - return oracle.LocalTxnScope + return kv.LocalTxnScope }(), SetSession: func(s *SessionVars, val string) error { switch val { - case oracle.GlobalTxnScope: - s.TxnScope = oracle.NewGlobalTxnScope() - case oracle.LocalTxnScope: - s.TxnScope = oracle.GetTxnScope() + case kv.GlobalTxnScope: + s.TxnScope = kv.NewGlobalTxnScopeVar() + case kv.LocalTxnScope: + s.TxnScope = kv.GetTxnScopeVar() default: return ErrWrongValueForVar.GenWithStack("@@txn_scope value should be global or local") } diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index b408f279be98a..b5b42df1838b9 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -74,7 +74,7 @@ type GCWorker struct { // NewGCWorker creates a GCWorker instance. func NewGCWorker(store kv.Storage, pdClient pd.Client) (*GCWorker, error) { - ver, err := store.CurrentVersion(oracle.GlobalTxnScope) + ver, err := store.CurrentVersion(kv.GlobalTxnScope) if err != nil { return nil, errors.Trace(err) } @@ -429,7 +429,7 @@ func (w *GCWorker) calcSafePointByMinStartTS(ctx context.Context, safePoint uint } func (w *GCWorker) getOracleTime() (time.Time, error) { - currentVer, err := w.store.CurrentVersion(oracle.GlobalTxnScope) + currentVer, err := w.store.CurrentVersion(kv.GlobalTxnScope) if err != nil { return time.Time{}, errors.Trace(err) } @@ -1984,7 +1984,7 @@ type MockGCWorker struct { // NewMockGCWorker creates a MockGCWorker instance ONLY for test. func NewMockGCWorker(store kv.Storage) (*MockGCWorker, error) { - ver, err := store.CurrentVersion(oracle.GlobalTxnScope) + ver, err := store.CurrentVersion(kv.GlobalTxnScope) if err != nil { return nil, errors.Trace(err) } diff --git a/store/mockstore/mockcopr/executor_test.go b/store/mockstore/mockcopr/executor_test.go index af9ac45beae96..7437b8d995997 100644 --- a/store/mockstore/mockcopr/executor_test.go +++ b/store/mockstore/mockcopr/executor_test.go @@ -83,7 +83,7 @@ func (s *testExecutorSuite) TestResolvedLargeTxnLocks(c *C) { tk.MustExec("insert into t values (1, 1)") o := s.store.GetOracle() - tso, err := o.GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + tso, err := o.GetTimestamp(context.Background(), &oracle.Option{TxnScope: kv.GlobalTxnScope}) c.Assert(err, IsNil) key := tablecodec.EncodeRowKeyWithHandle(tbl.Meta().ID, kv.IntHandle(1)) diff --git a/store/store_test.go b/store/store_test.go index 627a214badee7..3f4a44cecc189 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -26,7 +26,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/testleak" ) @@ -543,7 +542,7 @@ func (s *testKVSuite) TestDBClose(c *C) { err = txn.Commit(context.Background()) c.Assert(err, IsNil) - ver, err := store.CurrentVersion(oracle.GlobalTxnScope) + ver, err := store.CurrentVersion(kv.GlobalTxnScope) c.Assert(err, IsNil) c.Assert(kv.MaxVersion.Cmp(ver), Equals, 1) diff --git a/store/tikv/oracle/oracle.go b/store/tikv/oracle/oracle.go index daf00c66814ca..1b08129d412aa 100644 --- a/store/tikv/oracle/oracle.go +++ b/store/tikv/oracle/oracle.go @@ -18,7 +18,6 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/logutil" "go.uber.org/zap" ) @@ -45,57 +44,11 @@ type Future interface { Wait() (uint64, error) } -// TxnScope indicates the used txnScope for oracle -type TxnScope struct { - // varValue indicates the value of @@txn_scope, which can only be `global` or `local` - varValue string - // txnScope indicates the value which the tidb-server holds to request tso to pd - txnScope string -} - -// GetTxnScope gets oracle.TxnScope from config -func GetTxnScope() TxnScope { - isGlobal, location := config.GetTxnScopeFromConfig() - if isGlobal { - return NewGlobalTxnScope() - } - return NewLocalTxnScope(location) -} - -// NewGlobalTxnScope creates a Global TxnScope -func NewGlobalTxnScope() TxnScope { - return newTxnScope(GlobalTxnScope, GlobalTxnScope) -} - -// NewLocalTxnScope creates a Local TxnScope with given real txnScope value. -func NewLocalTxnScope(txnScope string) TxnScope { - return newTxnScope(LocalTxnScope, txnScope) -} - -// GetVarValue returns the value of @@txn_scope which can only be `global` or `local` -func (t TxnScope) GetVarValue() string { - return t.varValue -} - -// GetTxnScope returns the value of the tidb-server holds to request tso to pd. -func (t TxnScope) GetTxnScope() string { - return t.txnScope -} - -func newTxnScope(varValue string, txnScope string) TxnScope { - return TxnScope{ - varValue: varValue, - txnScope: txnScope, - } -} - const ( physicalShiftBits = 18 logicalBits = (1 << physicalShiftBits) - 1 // GlobalTxnScope is the default transaction scope for a Oracle service. GlobalTxnScope = "global" - // LocalTxnScope indicates the local txn scope for a Oracle service. - LocalTxnScope = "local" ) // ComposeTS creates a ts from physical and logical parts. diff --git a/store/tikv/tests/extract_start_ts_test.go b/store/tikv/tests/extract_start_ts_test.go index 566211006b66c..82f37796dce8a 100644 --- a/store/tikv/tests/extract_start_ts_test.go +++ b/store/tikv/tests/extract_start_ts_test.go @@ -39,7 +39,7 @@ func (s *extractStartTsSuite) SetUpTest(c *C) { probe.SetRegionCacheStore(2, tikvrpc.TiKV, 1, []*metapb.StoreLabel{ { Key: tikv.DCLabelKey, - Value: oracle.LocalTxnScope, + Value: "local1", }, }) probe.SetRegionCacheStore(3, tikvrpc.TiKV, 1, []*metapb.StoreLabel{ @@ -71,7 +71,7 @@ func (s *extractStartTsSuite) TestExtractStartTs(c *C) { // MinStartTS setted, global {101, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, // MinStartTS setted, local - {102, tikv.StartTSOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, + {102, tikv.StartTSOption{TxnScope: "local1", StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, // MaxPrevSec setted // however we need to add more cases to check the behavior when it fall backs to MinStartTS setted // see `TestMaxPrevSecFallback` @@ -99,7 +99,7 @@ func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { option tikv.StartTSOption }{ {0x8000000000000001, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, - {0x8000000000000002, tikv.StartTSOption{TxnScope: oracle.LocalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, + {0x8000000000000002, tikv.StartTSOption{TxnScope: "local1", StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, } for _, cs := range cases { result, _ := tikv.ExtractStartTs(s.store, cs.option) diff --git a/util/mock/context.go b/util/mock/context.go index 6df2c9c10d356..d23124e555ea2 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/disk" "github.com/pingcap/tidb/util/kvcache" @@ -205,7 +204,7 @@ func (c *Context) InitTxnWithStartTS(startTS uint64) error { return nil } if c.Store != nil { - txn, err := c.Store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(oracle.GlobalTxnScope).SetStartTs(startTS)) + txn, err := c.Store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(kv.GlobalTxnScope).SetStartTs(startTS)) if err != nil { return errors.Trace(err) } From f71fbdb179f6adc6d87ee435894e14f2c1ddcc71 Mon Sep 17 00:00:00 2001 From: cfzjywxk Date: Wed, 19 May 2021 21:01:42 +0800 Subject: [PATCH 106/343] execution: fix the incorrect use of cached plan for point get (#24749) --- session/session.go | 2 +- session/session_test.go | 58 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 59 insertions(+), 1 deletion(-) diff --git a/session/session.go b/session/session.go index 43bd8be8cf4c3..902f35ca28e79 100644 --- a/session/session.go +++ b/session/session.go @@ -1820,7 +1820,7 @@ func (s *session) IsCachedExecOk(ctx context.Context, preparedStmt *plannercore. return false, nil } // check auto commit - if !s.GetSessionVars().IsAutocommit() { + if !plannercore.IsAutoCommitTxn(s) { return false, nil } // check schema version diff --git a/session/session_test.go b/session/session_test.go index a47557642602d..a6c7908237bca 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4485,3 +4485,61 @@ func (s *testSessionSuite) TestReadDMLBatchSize(c *C) { _, _ = se.Execute(context.TODO(), "select 1") c.Assert(se.GetSessionVars().DMLBatchSize, Equals, 1000) } + +func (s *testSessionSuite) TestInTxnPSProtoPointGet(c *C) { + ctx := context.Background() + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t1(c1 int primary key, c2 int, c3 int)") + tk.MustExec("insert into t1 values(1, 10, 100)") + + // Generate the ps statement and make the prepared plan cached for point get. + id, _, _, err := tk.Se.PrepareStmt("select c1, c2 from t1 where c1 = ?") + c.Assert(err, IsNil) + idForUpdate, _, _, err := tk.Se.PrepareStmt("select c1, c2 from t1 where c1 = ? for update") + c.Assert(err, IsNil) + params := []types.Datum{types.NewDatum(1)} + rs, err := tk.Se.ExecutePreparedStmt(ctx, id, params) + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) + rs, err = tk.Se.ExecutePreparedStmt(ctx, idForUpdate, params) + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) + + // Query again the cached plan will be used. + rs, err = tk.Se.ExecutePreparedStmt(ctx, id, params) + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) + rs, err = tk.Se.ExecutePreparedStmt(ctx, idForUpdate, params) + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) + + // Start a transaction, now the in txn flag will be added to the session vars. + _, err = tk.Se.Execute(ctx, "start transaction") + c.Assert(err, IsNil) + rs, err = tk.Se.ExecutePreparedStmt(ctx, id, params) + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) + txn, err := tk.Se.Txn(false) + c.Assert(err, IsNil) + c.Assert(txn.Valid(), IsTrue) + rs, err = tk.Se.ExecutePreparedStmt(ctx, idForUpdate, params) + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) + txn, err = tk.Se.Txn(false) + c.Assert(err, IsNil) + c.Assert(txn.Valid(), IsTrue) + _, err = tk.Se.Execute(ctx, "update t1 set c2 = c2 + 1") + c.Assert(err, IsNil) + // Check the read result after in-transaction update. + rs, err = tk.Se.ExecutePreparedStmt(ctx, id, params) + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 11")) + rs, err = tk.Se.ExecutePreparedStmt(ctx, idForUpdate, params) + c.Assert(err, IsNil) + tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 11")) + txn, err = tk.Se.Txn(false) + c.Assert(err, IsNil) + c.Assert(txn.Valid(), IsTrue) + tk.MustExec("commit") +} From 89cf9702024c9990c88cb4398d3279f33e1a86cd Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Wed, 19 May 2021 22:03:42 +0800 Subject: [PATCH 107/343] executor: add correctness tests about direct reading with indexJoin (#24497) --- executor/partition_table_test.go | 114 +++++++++++++++++++++++++++++++ 1 file changed, 114 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index cb5414d91815f..cc0e4074b39a0 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -436,12 +436,126 @@ func (s *partitionTableSuite) TestView(c *C) { } } +func (s *partitionTableSuite) TestDirectReadingwithIndexJoin(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_dr_join") + tk.MustExec("use test_dr_join") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // hash and range partition + tk.MustExec("create table thash (a int, b int, c int, primary key(a), index idx_b(b)) partition by hash(a) partitions 4;") + tk.MustExec(`create table trange (a int, b int, c int, primary key(a), index idx_b(b)) partition by range(a) ( +   partition p0 values less than(1000), +   partition p1 values less than(2000), +   partition p2 values less than(3000), +   partition p3 values less than(4000));`) + + // regualr table + tk.MustExec(`create table tnormal (a int, b int, c int, primary key(a), index idx_b(b));`) + tk.MustExec(`create table touter (a int, b int, c int);`) + + // generate some random data to be inserted + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v, %v)", rand.Intn(4000), rand.Intn(4000), rand.Intn(4000))) + } + tk.MustExec("insert ignore into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into tnormal values " + strings.Join(vals, ",")) + tk.MustExec("insert ignore into touter values " + strings.Join(vals, ",")) + + // test indexLookUp + hash + queryPartition := fmt.Sprintf("select /*+ INL_JOIN(touter, thash) */ * from touter join thash use index(idx_b) on touter.b = thash.b") + queryRegular := fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal use index(idx_b) on touter.b = tnormal.b") + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:IndexLookUp, outer key:test_dr_join.touter.b, inner key:test_dr_join.thash.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.thash.b)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root partition:all ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test_dr_join.thash.b))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:thash, index:idx_b(b) range: decided by [eq(test_dr_join.thash.b, test_dr_join.touter.b)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 cop[tikv] table:thash keep order:false, stats:pseudo")) // check if IndexLookUp is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // test tableReader + hash + queryPartition = fmt.Sprintf("select /*+ INL_JOIN(touter, thash) */ * from touter join thash on touter.a = thash.a") + queryRegular = fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal on touter.a = tnormal.a") + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test_dr_join.touter.a, inner key:test_dr_join.thash.a, equal cond:eq(test_dr_join.touter.a, test_dr_join.thash.a)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─TableReader(Probe) 1.00 root partition:all data:TableRangeScan", + " └─TableRangeScan 1.00 cop[tikv] table:thash range: decided by [test_dr_join.touter.a], keep order:false, stats:pseudo")) // check if tableReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // test indexReader + hash + queryPartition = fmt.Sprintf("select /*+ INL_JOIN(touter, thash) */ thash.b from touter join thash use index(idx_b) on touter.b = thash.b;") + queryRegular = fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b;") + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test_dr_join.touter.b, inner key:test_dr_join.thash.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.thash.b)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─IndexReader(Probe) 1.25 root partition:all index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test_dr_join.thash.b))", + " └─IndexRangeScan 1.25 cop[tikv] table:thash, index:idx_b(b) range: decided by [eq(test_dr_join.thash.b, test_dr_join.touter.b)], keep order:false, stats:pseudo")) // check if indexReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // test indexLookUp + range + // explain select /*+ INL_JOIN(touter, tinner) */ * from touter join tinner use index(a) on touter.a = tinner.a; + queryPartition = fmt.Sprintf("select /*+ INL_JOIN(touter, trange) */ * from touter join trange use index(idx_b) on touter.b = trange.b;") + queryRegular = fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal use index(idx_b) on touter.b = tnormal.b;") + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:IndexLookUp, outer key:test_dr_join.touter.b, inner key:test_dr_join.trange.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.trange.b)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─IndexLookUp(Probe) 1.25 root partition:all ", + " ├─Selection(Build) 1.25 cop[tikv] not(isnull(test_dr_join.trange.b))", + " │ └─IndexRangeScan 1.25 cop[tikv] table:trange, index:idx_b(b) range: decided by [eq(test_dr_join.trange.b, test_dr_join.touter.b)], keep order:false, stats:pseudo", + " └─TableRowIDScan(Probe) 1.25 cop[tikv] table:trange keep order:false, stats:pseudo")) // check if IndexLookUp is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // test tableReader + range + queryPartition = fmt.Sprintf("select /*+ INL_JOIN(touter, trange) */ * from touter join trange on touter.a = trange.a;") + queryRegular = fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal on touter.a = tnormal.a;") + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test_dr_join.touter.a, inner key:test_dr_join.trange.a, equal cond:eq(test_dr_join.touter.a, test_dr_join.trange.a)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─TableReader(Probe) 1.00 root partition:all data:TableRangeScan", + " └─TableRangeScan 1.00 cop[tikv] table:trange range: decided by [test_dr_join.touter.a], keep order:false, stats:pseudo")) // check if tableReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) + + // test indexReader + range + // explain select /*+ INL_JOIN(touter, tinner) */ tinner.a from touter join tinner on touter.a = tinner.a; + queryPartition = fmt.Sprintf("select /*+ INL_JOIN(touter, trange) */ trange.b from touter join trange use index(idx_b) on touter.b = trange.b;") + queryRegular = fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b;") + tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( + "IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test_dr_join.touter.b, inner key:test_dr_join.trange.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.trange.b)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test_dr_join.touter.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo", + "└─IndexReader(Probe) 1.25 root partition:all index:Selection", + " └─Selection 1.25 cop[tikv] not(isnull(test_dr_join.trange.b))", + " └─IndexRangeScan 1.25 cop[tikv] table:trange, index:idx_b(b) range: decided by [eq(test_dr_join.trange.b, test_dr_join.touter.b)], keep order:false, stats:pseudo")) // check if indexReader is used + tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) +} + func (s *partitionTableSuite) TestDynamicPruningUnderIndexJoin(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") } tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database pruing_under_index_join") tk.MustExec("use pruing_under_index_join") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") From 113612699103a4d4cd5b1622c666a608f5fc63c0 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Wed, 19 May 2021 09:55:42 -0600 Subject: [PATCH 108/343] variable: fix sysvar datarace with deep copy (#24732) --- sessionctx/variable/sysvar.go | 13 +++++++++++-- sessionctx/variable/sysvar_test.go | 27 +++++++++++++++++++++++++++ 2 files changed, 38 insertions(+), 2 deletions(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 99ff4a09f7af6..1df9e3c0f582c 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -491,12 +491,21 @@ func UnregisterSysVar(name string) { sysVarsLock.Unlock() } +// Clone deep copies the sysvar struct to avoid a race +func (sv *SysVar) Clone() *SysVar { + dst := *sv + return &dst +} + // GetSysVar returns sys var info for name as key. func GetSysVar(name string) *SysVar { name = strings.ToLower(name) sysVarsLock.RLock() defer sysVarsLock.RUnlock() - return sysVars[name] + if sysVars[name] == nil { + return nil + } + return sysVars[name].Clone() } // SetSysVar sets a sysvar. This will not propagate to the cluster, so it should only be @@ -514,7 +523,7 @@ func GetSysVars() map[string]*SysVar { defer sysVarsLock.RUnlock() copy := make(map[string]*SysVar, len(sysVars)) for name, sv := range sysVars { - copy[name] = sv + copy[name] = sv.Clone() } return copy } diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index 71979a57b7eef..d236dc142f1dd 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -555,3 +555,30 @@ func (*testSysVarSuite) TestInstanceScopedVars(c *C) { c.Assert(err, IsNil) c.Assert(val, Equals, vars.TxnScope.GetVarValue()) } + +// Calling GetSysVars/GetSysVar needs to return a deep copy, otherwise there will be data races. +// This is a bit unfortunate, since the only time the race occurs is in the testsuite (Enabling/Disabling SEM) and +// during startup (setting the .Value of ScopeNone variables). In future it might also be able +// to fix this by delaying the LoadSysVarCacheLoop start time until after the server is fully initialized. +func (*testSysVarSuite) TestDeepCopyGetSysVars(c *C) { + // Check GetSysVar + sv := SysVar{Scope: ScopeGlobal | ScopeSession, Name: "datarace", Value: On, Type: TypeBool} + RegisterSysVar(&sv) + svcopy := GetSysVar("datarace") + svcopy.Name = "datarace2" + c.Assert(sv.Name, Equals, "datarace") + c.Assert(GetSysVar("datarace").Name, Equals, "datarace") + UnregisterSysVar("datarace") + + // Check GetSysVars + sv = SysVar{Scope: ScopeGlobal | ScopeSession, Name: "datarace", Value: On, Type: TypeBool} + RegisterSysVar(&sv) + for name, svcopy := range GetSysVars() { + if name == "datarace" { + svcopy.Name = "datarace2" + } + } + c.Assert(sv.Name, Equals, "datarace") + c.Assert(GetSysVar("datarace").Name, Equals, "datarace") + UnregisterSysVar("datarace") +} From 15dfd7bdd3990eb7590c94817c0112cca79f8761 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 19 May 2021 18:09:41 +0200 Subject: [PATCH 109/343] *: Implementing RENAME USER (#24413) --- executor/simple.go | 142 +++++++++++++++++++++++- planner/core/logical_plan_test.go | 6 + planner/core/planbuilder.go | 5 +- privilege/privileges/privileges_test.go | 48 ++++++++ session/session.go | 3 +- session/tidb.go | 2 +- 6 files changed, 201 insertions(+), 5 deletions(-) diff --git a/executor/simple.go b/executor/simple.go index 7270f12aecdd0..d1e5181e2f439 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -136,6 +136,8 @@ func (e *SimpleExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { err = e.executeAlterUser(x) case *ast.DropUserStmt: err = e.executeDropUser(x) + case *ast.RenameUserStmt: + err = e.executeRenameUser(x) case *ast.SetPwdStmt: err = e.executeSetPwd(x) case *ast.KillStmt: @@ -1026,6 +1028,123 @@ func (e *SimpleExec) executeGrantRole(s *ast.GrantRoleStmt) error { return nil } +// Should cover same internal mysql.* tables as DROP USER, so this function is very similar +func (e *SimpleExec) executeRenameUser(s *ast.RenameUserStmt) error { + + var failedUser string + sysSession, err := e.getSysSession() + defer e.releaseSysSession(sysSession) + if err != nil { + return err + } + sqlExecutor := sysSession.(sqlexec.SQLExecutor) + + if _, err := sqlExecutor.ExecuteInternal(context.TODO(), "begin"); err != nil { + return err + } + + for _, userToUser := range s.UserToUsers { + oldUser, newUser := userToUser.OldUser, userToUser.NewUser + exists, err := userExistsInternal(sqlExecutor, oldUser.Username, oldUser.Hostname) + if err != nil { + return err + } + if !exists { + failedUser = oldUser.String() + " TO " + newUser.String() + " old did not exist" + break + } + + exists, err = userExistsInternal(sqlExecutor, newUser.Username, newUser.Hostname) + if err != nil { + return err + } + if exists { + // MySQL reports the old user, even when the issue is the new user. + failedUser = oldUser.String() + " TO " + newUser.String() + " new did exist" + break + } + + if err = renameUserHostInSystemTable(sqlExecutor, mysql.UserTable, "User", "Host", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.UserTable + " error" + break + } + + // rename privileges from mysql.global_priv + if err = renameUserHostInSystemTable(sqlExecutor, mysql.GlobalPrivTable, "User", "Host", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.GlobalPrivTable + " error" + break + } + + // rename privileges from mysql.db + if err = renameUserHostInSystemTable(sqlExecutor, mysql.DBTable, "User", "Host", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.DBTable + " error" + break + } + + // rename privileges from mysql.tables_priv + if err = renameUserHostInSystemTable(sqlExecutor, mysql.TablePrivTable, "User", "Host", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.TablePrivTable + " error" + break + } + + // rename relationship from mysql.role_edges + if err = renameUserHostInSystemTable(sqlExecutor, mysql.RoleEdgeTable, "TO_USER", "TO_HOST", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.RoleEdgeTable + " (to) error" + break + } + + if err = renameUserHostInSystemTable(sqlExecutor, mysql.RoleEdgeTable, "FROM_USER", "FROM_HOST", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.RoleEdgeTable + " (from) error" + break + } + + // rename relationship from mysql.default_roles + if err = renameUserHostInSystemTable(sqlExecutor, mysql.DefaultRoleTable, "DEFAULT_ROLE_USER", "DEFAULT_ROLE_HOST", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.DefaultRoleTable + " (default role user) error" + break + } + + if err = renameUserHostInSystemTable(sqlExecutor, mysql.DefaultRoleTable, "USER", "HOST", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " " + mysql.DefaultRoleTable + " error" + break + } + + // rename relationship from mysql.global_grants + // TODO: add global_grants into the parser + if err = renameUserHostInSystemTable(sqlExecutor, "global_grants", "User", "Host", userToUser); err != nil { + failedUser = oldUser.String() + " TO " + newUser.String() + " mysql.global_grants error" + break + } + + //TODO: need update columns_priv once we implement columns_priv functionality. + // When that is added, please refactor both executeRenameUser and executeDropUser to use an array of tables + // to loop over, so it is easier to maintain. + } + + if failedUser == "" { + if _, err := sqlExecutor.ExecuteInternal(context.TODO(), "commit"); err != nil { + return err + } + } else { + if _, err := sqlExecutor.ExecuteInternal(context.TODO(), "rollback"); err != nil { + return err + } + return ErrCannotUser.GenWithStackByArgs("RENAME USER", failedUser) + } + domain.GetDomain(e.ctx).NotifyUpdatePrivilege(e.ctx) + return nil +} + +func renameUserHostInSystemTable(sqlExecutor sqlexec.SQLExecutor, tableName, usernameColumn, hostColumn string, users *ast.UserToUser) error { + sql := new(strings.Builder) + sqlexec.MustFormatSQL(sql, `UPDATE %n.%n SET %n = %?, %n = %? WHERE %n = %? and %n = %?;`, + mysql.SystemDB, tableName, + usernameColumn, users.NewUser.Username, hostColumn, users.NewUser.Hostname, + usernameColumn, users.OldUser.Username, hostColumn, users.OldUser.Hostname) + _, err := sqlExecutor.ExecuteInternal(context.TODO(), sql.String()) + return err +} + func (e *SimpleExec) executeDropUser(s *ast.DropUserStmt) error { // Check privileges. // Check `CREATE USER` privilege. @@ -1181,6 +1300,27 @@ func userExists(ctx sessionctx.Context, name string, host string) (bool, error) return len(rows) > 0, nil } +// use the same internal executor to read within the same transaction, otherwise same as userExists +func userExistsInternal(sqlExecutor sqlexec.SQLExecutor, name string, host string) (bool, error) { + sql := new(strings.Builder) + sqlexec.MustFormatSQL(sql, `SELECT * FROM %n.%n WHERE User=%? AND Host=%?;`, mysql.SystemDB, mysql.UserTable, name, host) + recordSet, err := sqlExecutor.ExecuteInternal(context.TODO(), sql.String()) + if err != nil { + return false, err + } + req := recordSet.NewChunk() + err = recordSet.Next(context.TODO(), req) + var rows int = 0 + if err == nil { + rows = req.NumRows() + } + errClose := recordSet.Close() + if errClose != nil { + return false, errClose + } + return rows > 0, err +} + func (e *SimpleExec) executeSetPwd(s *ast.SetPwdStmt) error { var u, h string if s.User == nil { @@ -1389,7 +1529,7 @@ func (e *SimpleExec) executeDropStats(s *ast.DropStatsStmt) (err error) { func (e *SimpleExec) autoNewTxn() bool { switch e.Statement.(type) { - case *ast.CreateUserStmt, *ast.AlterUserStmt, *ast.DropUserStmt: + case *ast.CreateUserStmt, *ast.AlterUserStmt, *ast.DropUserStmt, *ast.RenameUserStmt: return true } return false diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 921f1c99b34ec..56652983ff8f8 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1296,6 +1296,12 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", true}, }, }, + { + sql: "RENAME USER user1 to user1_tmp", + ans: []visitInfo{ + {mysql.CreateUserPriv, "", "", "", ErrSpecificAccessDenied, false, "", false}, + }, + }, } for _, tt := range tests { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 7dc2459dace33..ccab0a28cc863 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -641,7 +641,8 @@ func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (Plan, error) { case *ast.BinlogStmt, *ast.FlushStmt, *ast.UseStmt, *ast.BRIEStmt, *ast.BeginStmt, *ast.CommitStmt, *ast.RollbackStmt, *ast.CreateUserStmt, *ast.SetPwdStmt, *ast.AlterInstanceStmt, *ast.GrantStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.RevokeStmt, *ast.KillStmt, *ast.DropStatsStmt, - *ast.GrantRoleStmt, *ast.RevokeRoleStmt, *ast.SetRoleStmt, *ast.SetDefaultRoleStmt, *ast.ShutdownStmt: + *ast.GrantRoleStmt, *ast.RevokeRoleStmt, *ast.SetRoleStmt, *ast.SetDefaultRoleStmt, *ast.ShutdownStmt, + *ast.RenameUserStmt: return b.buildSimple(node.(ast.StmtNode)) case ast.DDLNode: return b.buildDDL(ctx, x) @@ -2268,7 +2269,7 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { case *ast.AlterInstanceStmt: err := ErrSpecificAccessDenied.GenWithStack("SUPER") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SuperPriv, "", "", "", err) - case *ast.AlterUserStmt: + case *ast.AlterUserStmt, *ast.RenameUserStmt: err := ErrSpecificAccessDenied.GenWithStackByArgs("CREATE USER") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreateUserPriv, "", "", "", err) case *ast.GrantStmt: diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 2f6cbef8af2cf..3917d822aa403 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1400,6 +1400,54 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeStatusVars(c *C) { }, nil, nil) } +func (s *testPrivilegeSuite) TestRenameUser(c *C) { + rootSe := newSession(c, s.store, s.dbName) + mustExec(c, rootSe, "DROP USER IF EXISTS 'ru1'@'localhost'") + mustExec(c, rootSe, "DROP USER IF EXISTS ru3") + mustExec(c, rootSe, "DROP USER IF EXISTS ru6@localhost") + mustExec(c, rootSe, "CREATE USER 'ru1'@'localhost'") + mustExec(c, rootSe, "CREATE USER ru3") + mustExec(c, rootSe, "CREATE USER ru6@localhost") + se1 := newSession(c, s.store, s.dbName) + c.Assert(se1.Auth(&auth.UserIdentity{Username: "ru1", Hostname: "localhost"}, nil, nil), IsTrue) + + // Check privileges (need CREATE USER) + _, err := se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru4") + c.Assert(err, ErrorMatches, ".*Access denied; you need .at least one of. the CREATE USER privilege.s. for this operation") + mustExec(c, rootSe, "GRANT UPDATE ON mysql.user TO 'ru1'@'localhost'") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru4") + c.Assert(err, ErrorMatches, ".*Access denied; you need .at least one of. the CREATE USER privilege.s. for this operation") + mustExec(c, rootSe, "GRANT CREATE USER ON *.* TO 'ru1'@'localhost'") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru4") + c.Assert(err, IsNil) + + // Test a few single rename (both Username and Hostname) + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER 'ru4'@'%' TO 'ru3'@'localhost'") + c.Assert(err, IsNil) + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER 'ru3'@'localhost' TO 'ru3'@'%'") + c.Assert(err, IsNil) + // Including negative tests, i.e. non existing from user and existing to user + _, err = rootSe.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru1@localhost") + c.Assert(err, ErrorMatches, ".*Operation RENAME USER failed for ru3@%.*") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru4 TO ru5@localhost") + c.Assert(err, ErrorMatches, ".*Operation RENAME USER failed for ru4@%.*") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru3") + c.Assert(err, ErrorMatches, ".*Operation RENAME USER failed for ru3@%.*") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru5@localhost, ru4 TO ru7") + c.Assert(err, ErrorMatches, ".*Operation RENAME USER failed for ru4@%.*") + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER ru3 TO ru5@localhost, ru6@localhost TO ru1@localhost") + c.Assert(err, ErrorMatches, ".*Operation RENAME USER failed for ru6@localhost.*") + + // Test multi rename, this is a full swap of ru3 and ru6, i.e. need to read its previous state in the same transaction. + _, err = se1.ExecuteInternal(context.Background(), "RENAME USER 'ru3' TO 'ru3_tmp', ru6@localhost TO ru3, 'ru3_tmp' to ru6@localhost") + c.Assert(err, IsNil) + + // Cleanup + mustExec(c, rootSe, "DROP USER ru6@localhost") + mustExec(c, rootSe, "DROP USER ru3") + mustExec(c, rootSe, "DROP USER 'ru1'@'localhost'") +} + func (s *testPrivilegeSuite) TestSecurityEnhancedModeSysVars(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("CREATE USER svroot1, svroot2") diff --git a/session/session.go b/session/session.go index 902f35ca28e79..f116daf96dd04 100644 --- a/session/session.go +++ b/session/session.go @@ -2862,7 +2862,8 @@ func logStmt(execStmt *executor.ExecStmt, vars *variable.SessionVars) { switch stmt := execStmt.StmtNode.(type) { case *ast.CreateUserStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.SetPwdStmt, *ast.GrantStmt, *ast.RevokeStmt, *ast.AlterTableStmt, *ast.CreateDatabaseStmt, *ast.CreateIndexStmt, *ast.CreateTableStmt, - *ast.DropDatabaseStmt, *ast.DropIndexStmt, *ast.DropTableStmt, *ast.RenameTableStmt, *ast.TruncateTableStmt: + *ast.DropDatabaseStmt, *ast.DropIndexStmt, *ast.DropTableStmt, *ast.RenameTableStmt, *ast.TruncateTableStmt, + *ast.RenameUserStmt: user := vars.User schemaVersion := vars.GetInfoSchema().SchemaMetaVersion() if ss, ok := execStmt.StmtNode.(ast.SensitiveStmtNode); ok { diff --git a/session/tidb.go b/session/tidb.go index 85732b457f7a6..583c5074e6805 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -239,7 +239,7 @@ func autoCommitAfterStmt(ctx context.Context, se *session, meetsErr error, sql s sessVars := se.sessionVars if meetsErr != nil { if !sessVars.InTxn() { - logutil.BgLogger().Info("rollbackTxn for ddl/autocommit failed") + logutil.BgLogger().Info("rollbackTxn called due to ddl/autocommit failure") se.RollbackTxn(ctx) recordAbortTxnDuration(sessVars) } else if se.txn.Valid() && se.txn.IsPessimistic() && executor.ErrDeadlock.Equal(meetsErr) { From 3904ca3e4eb993bde1fabe164398d0706b917a0e Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Thu, 20 May 2021 00:21:42 +0800 Subject: [PATCH 110/343] infoschema, executor: Add the deadlock table (#24524) --- executor/builder.go | 4 +- executor/executor.go | 9 + executor/executor_test.go | 65 ++++ executor/infoschema_reader.go | 32 ++ infoschema/cluster.go | 3 + infoschema/tables.go | 16 + infoschema/tables_test.go | 21 ++ session/pessimistic_test.go | 48 ++- store/tikv/kv/kv.go | 2 + store/tikv/txn.go | 7 +- util/deadlockhistory/deadlock_history.go | 198 +++++++++++++ util/deadlockhistory/deadlock_history_test.go | 277 ++++++++++++++++++ 12 files changed, 668 insertions(+), 14 deletions(-) create mode 100644 util/deadlockhistory/deadlock_history.go create mode 100644 util/deadlockhistory/deadlock_history_test.go diff --git a/executor/builder.go b/executor/builder.go index 54e2dfb93012a..b04dfe45f4bab 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1534,7 +1534,9 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableClientErrorsSummaryByUser), strings.ToLower(infoschema.TableClientErrorsSummaryByHost), strings.ToLower(infoschema.TableTiDBTrx), - strings.ToLower(infoschema.ClusterTableTiDBTrx): + strings.ToLower(infoschema.ClusterTableTiDBTrx), + strings.ToLower(infoschema.TableDeadlocks), + strings.ToLower(infoschema.ClusterTableDeadlocks): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), table: v.Table, diff --git a/executor/executor.go b/executor/executor.go index 1666f6955bba9..2b9b8f0f52954 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -49,6 +49,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" + tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" tikvutil "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/table" @@ -58,6 +59,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/disk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" @@ -983,6 +985,13 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.Loc LockKeysCount: &seVars.StmtCtx.LockKeysCount, LockExpired: &seVars.TxnCtx.LockExpire, ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(sqlDigest), + OnDeadlock: func(deadlock *tikverr.ErrDeadlock) { + // TODO: Support collecting retryable deadlocks according to the config. + if !deadlock.IsRetryable { + rec := deadlockhistory.ErrDeadlockToDeadlockRecord(deadlock) + deadlockhistory.GlobalDeadlockHistory.Push(rec) + } + }, } } diff --git a/executor/executor_test.go b/executor/executor_test.go index dde9511a8410d..ab6dc6f274712 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -70,6 +70,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" + "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" @@ -8151,7 +8152,71 @@ func (s *testSerialSuite) TestIssue24210(c *C) { c.Assert(err.Error(), Equals, "mock SelectionExec.baseExecutor.Open returned error") err = failpoint.Disable("github.com/pingcap/tidb/executor/mockSelectionExecBaseExecutorOpenReturnedError") c.Assert(err, IsNil) +} + +func (s *testSerialSuite) TestDeadlockTable(c *C) { + deadlockhistory.GlobalDeadlockHistory.Clear() + occurTime := time.Date(2021, 5, 10, 1, 2, 3, 456789000, time.UTC) + rec := &deadlockhistory.DeadlockRecord{ + OccurTime: occurTime, + IsRetryable: false, + WaitChain: []deadlockhistory.WaitChainItem{ + { + TryLockTxn: 101, + SQLDigest: "aabbccdd", + Key: []byte("k1"), + AllSQLs: nil, + TxnHoldingLock: 102, + }, + { + TryLockTxn: 102, + SQLDigest: "ddccbbaa", + Key: []byte("k2"), + AllSQLs: []string{"sql1"}, + TxnHoldingLock: 101, + }, + }, + } + deadlockhistory.GlobalDeadlockHistory.Push(rec) + + occurTime2 := time.Date(2022, 6, 11, 2, 3, 4, 987654000, time.UTC) + rec2 := &deadlockhistory.DeadlockRecord{ + OccurTime: occurTime2, + IsRetryable: true, + WaitChain: []deadlockhistory.WaitChainItem{ + { + TryLockTxn: 201, + AllSQLs: []string{}, + TxnHoldingLock: 202, + }, + { + TryLockTxn: 202, + AllSQLs: []string{"sql1", "sql2, sql3"}, + TxnHoldingLock: 203, + }, + { + TryLockTxn: 203, + TxnHoldingLock: 201, + }, + }, + } + deadlockhistory.GlobalDeadlockHistory.Push(rec2) + + // `Push` sets the record's ID, and ID in a single DeadlockHistory is monotonically increasing. We must get it here + // to know what it is. + id1 := strconv.FormatUint(rec.ID, 10) + id2 := strconv.FormatUint(rec2.ID, 10) + + tk := testkit.NewTestKit(c, s.store) + tk.MustQuery("select * from information_schema.deadlocks").Check( + testutil.RowsWithSep("/", + id1+"/2021-05-10 01:02:03.456789/0/101/aabbccdd/6B31//102", + id1+"/2021-05-10 01:02:03.456789/0/102/ddccbbaa/6B32/[sql1]/101", + id2+"/2022-06-11 02:03:04.987654/1/201///[]/202", + id2+"/2022-06-11 02:03:04.987654/1/202///[sql1, sql2, sql3]/203", + id2+"/2022-06-11 02:03:04.987654/1/203////201", + )) } func (s testSerialSuite) TestExprBlackListForEnum(c *C) { diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 4f788a3d7bd1d..491eb3a3fe26f 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -52,6 +52,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/set" @@ -153,6 +154,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex e.setDataForTiDBTrx(sctx) case infoschema.ClusterTableTiDBTrx: err = e.setDataForClusterTiDBTrx(sctx) + case infoschema.TableDeadlocks: + err = e.setDataForDeadlock(sctx) + case infoschema.ClusterTableDeadlocks: + err = e.setDataForClusterDeadlock(sctx) } if err != nil { return nil, err @@ -2048,6 +2053,33 @@ func (e *memtableRetriever) setDataForClusterTiDBTrx(ctx sessionctx.Context) err return nil } +func (e *memtableRetriever) setDataForDeadlock(ctx sessionctx.Context) error { + hasPriv := false + if pm := privilege.GetPrivilegeManager(ctx); pm != nil { + hasPriv = pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) + } + + if !hasPriv { + return plannercore.ErrSpecificAccessDenied.GenWithStackByArgs("PROCESS") + } + + e.rows = deadlockhistory.GlobalDeadlockHistory.GetAllDatum() + return nil +} + +func (e *memtableRetriever) setDataForClusterDeadlock(ctx sessionctx.Context) error { + err := e.setDataForDeadlock(ctx) + if err != nil { + return err + } + rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows) + if err != nil { + return err + } + e.rows = rows + return nil +} + type hugeMemTableRetriever struct { dummyCloser table *model.TableInfo diff --git a/infoschema/cluster.go b/infoschema/cluster.go index 2d196fe5b0023..20589ad7a0c67 100644 --- a/infoschema/cluster.go +++ b/infoschema/cluster.go @@ -39,6 +39,8 @@ const ( ClusterTableStatementsSummaryHistory = "CLUSTER_STATEMENTS_SUMMARY_HISTORY" // ClusterTableTiDBTrx is the string constant of cluster transaction running table. ClusterTableTiDBTrx = "CLUSTER_TIDB_TRX" + // ClusterTableDeadlocks is the string constant of cluster dead lock table. + ClusterTableDeadlocks = "CLUSTER_DEADLOCKS" ) // memTableToClusterTables means add memory table to cluster table. @@ -48,6 +50,7 @@ var memTableToClusterTables = map[string]string{ TableStatementsSummary: ClusterTableStatementsSummary, TableStatementsSummaryHistory: ClusterTableStatementsSummaryHistory, TableTiDBTrx: ClusterTableTiDBTrx, + TableDeadlocks: ClusterTableDeadlocks, } func init() { diff --git a/infoschema/tables.go b/infoschema/tables.go index 2d5112ada05c0..40451046fe8ec 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -165,6 +165,8 @@ const ( TableClientErrorsSummaryByHost = "CLIENT_ERRORS_SUMMARY_BY_HOST" // TableTiDBTrx is current running transaction status table. TableTiDBTrx = "TIDB_TRX" + // TableDeadlocks is the string constatnt of deadlock table. + TableDeadlocks = "DEADLOCKS" ) var tableIDMap = map[string]int64{ @@ -239,6 +241,8 @@ var tableIDMap = map[string]int64{ TableClientErrorsSummaryByHost: autoid.InformationSchemaDBID + 69, TableTiDBTrx: autoid.InformationSchemaDBID + 70, ClusterTableTiDBTrx: autoid.InformationSchemaDBID + 71, + TableDeadlocks: autoid.InformationSchemaDBID + 72, + ClusterTableDeadlocks: autoid.InformationSchemaDBID + 73, } type columnInfo struct { @@ -1353,6 +1357,17 @@ var tableTiDBTrxCols = []columnInfo{ {name: "DB", tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"}, } +var tableDeadlocksCols = []columnInfo{ + {name: "DEADLOCK_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The ID to dinstinguish different deadlock events"}, + {name: "OCCUR_TIME", tp: mysql.TypeTimestamp, decimal: 6, size: 26, comment: "The physical time when the deadlock occurs"}, + {name: "RETRYABLE", tp: mysql.TypeTiny, size: 1, flag: mysql.NotNullFlag, comment: "Whether the deadlock is retryable. Retryable deadlocks are usually not reported to the client"}, + {name: "TRY_LOCK_TRX_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The transaction ID (start ts) of the transaction that's trying to acquire the lock"}, + {name: "CURRENT_SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "The digest of the SQL that's being blocked"}, + {name: "KEY", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "The key on which a transaction is waiting for another"}, + {name: "ALL_SQLS", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the digests of SQL statements that the transaction has executed"}, + {name: "TRX_HOLDING_LOCK", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The transaction ID (start ts) of the transaction that's currently holding the lock"}, +} + // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. // The returned description string may be: // - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. @@ -1723,6 +1738,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableClientErrorsSummaryByUser: tableClientErrorsSummaryByUserCols, TableClientErrorsSummaryByHost: tableClientErrorsSummaryByHostCols, TableTiDBTrx: tableTiDBTrxCols, + TableDeadlocks: tableDeadlocksCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 1e5687928f3ad..2d6506b56d5f4 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1531,3 +1531,24 @@ func (s *testTableSuite) TestTrx(c *C) { testkit.Rows("424768545227014155 2021-05-07 12:56:48 " + digest + " Normal 1 19 2 root test"), ) } + +func (s *testTableSuite) TestInfoschemaDeadlockPrivilege(c *C) { + tk := s.newTestKitWithRoot(c) + tk.MustExec("create user 'testuser'@'localhost'") + c.Assert(tk.Se.Auth(&auth.UserIdentity{ + Username: "testuser", + Hostname: "localhost", + }, nil, nil), IsTrue) + err := tk.QueryToErr("select * from information_schema.deadlocks") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation") + + tk = s.newTestKitWithRoot(c) + tk.MustExec("create user 'testuser2'@'localhost'") + tk.MustExec("grant process on *.* to 'testuser2'@'localhost'") + c.Assert(tk.Se.Auth(&auth.UserIdentity{ + Username: "testuser2", + Hostname: "localhost", + }, nil, nil), IsTrue) + _ = tk.MustQuery("select * from information_schema.deadlocks") +} diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 2e8c01c75577b..58cf8a624fe54 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -16,6 +16,7 @@ package session_test import ( "context" "fmt" + "strconv" "strings" "sync" "sync/atomic" @@ -24,6 +25,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/parser" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" @@ -37,6 +39,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/testkit" ) @@ -171,27 +174,33 @@ func (s *testPessimisticSuite) TestTxnMode(c *C) { } func (s *testPessimisticSuite) TestDeadlock(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("drop table if exists deadlock") - tk.MustExec("create table deadlock (k int primary key, v int)") - tk.MustExec("insert into deadlock values (1, 1), (2, 1)") + deadlockhistory.GlobalDeadlockHistory.Clear() + + tk1 := testkit.NewTestKitWithInit(c, s.store) + tk1.MustExec("drop table if exists deadlock") + tk1.MustExec("create table deadlock (k int primary key, v int)") + tk1.MustExec("insert into deadlock values (1, 1), (2, 1)") + tk1.MustExec("begin pessimistic") + tk1.MustExec("update deadlock set v = v + 1 where k = 1") + ts1, err := strconv.ParseUint(tk1.MustQuery("select @@tidb_current_ts").Rows()[0][0].(string), 10, 64) + c.Assert(err, IsNil) + + tk2 := testkit.NewTestKitWithInit(c, s.store) + tk2.MustExec("begin pessimistic") + ts2, err := strconv.ParseUint(tk2.MustQuery("select @@tidb_current_ts").Rows()[0][0].(string), 10, 64) + c.Assert(err, IsNil) syncCh := make(chan error) go func() { - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk1.MustExec("begin pessimistic") - tk1.MustExec("update deadlock set v = v + 1 where k = 2") + tk2.MustExec("update deadlock set v = v + 1 where k = 2") syncCh <- nil - _, err := tk1.Exec("update deadlock set v = v + 1 where k = 1") + _, err := tk2.Exec("update deadlock set v = v + 1 where k = 1") syncCh <- err }() - tk.MustExec("begin pessimistic") - tk.MustExec("update deadlock set v = v + 1 where k = 1") <-syncCh - _, err1 := tk.Exec("update deadlock set v = v + 1 where k = 2") + _, err1 := tk1.Exec("update deadlock set v = v + 1 where k = 2") err2 := <-syncCh // Either err1 or err2 is deadlock error. - var err error if err1 != nil { c.Assert(err2, IsNil) err = err1 @@ -201,6 +210,21 @@ func (s *testPessimisticSuite) TestDeadlock(c *C) { e, ok := errors.Cause(err).(*terror.Error) c.Assert(ok, IsTrue) c.Assert(int(e.Code()), Equals, mysql.ErrLockDeadlock) + + _, digest := parser.NormalizeDigest("update deadlock set v = v + 1 where k = 1") + + expectedDeadlockInfo := []string{ + fmt.Sprintf("%v %v %v", ts1, ts2, digest), + fmt.Sprintf("%v %v %v", ts2, ts1, digest), + } + // The last one is the transaction that encountered the deadlock error. + if err1 != nil { + // Swap the two to match the correct order. + expectedDeadlockInfo[0], expectedDeadlockInfo[1] = expectedDeadlockInfo[1], expectedDeadlockInfo[0] + } + res := tk1.MustQuery("select deadlock_id, try_lock_trx_id, trx_holding_lock, current_sql_digest from information_schema.deadlocks") + res.CheckAt([]int{1, 2, 3}, testkit.Rows(expectedDeadlockInfo...)) + c.Assert(res.Rows()[0][0], Equals, res.Rows()[1][0]) } func (s *testPessimisticSuite) TestSingleStatementRollback(c *C) { diff --git a/store/tikv/kv/kv.go b/store/tikv/kv/kv.go index 8ba36a749db4f..980b95842a361 100644 --- a/store/tikv/kv/kv.go +++ b/store/tikv/kv/kv.go @@ -4,6 +4,7 @@ import ( "sync" "time" + tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/util" ) @@ -28,4 +29,5 @@ type LockCtx struct { LockExpired *uint32 Stats *util.LockKeysDetails ResourceGroupTag []byte + OnDeadlock func(*tikverr.ErrDeadlock) } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index d228b834e00dc..988f6501be553 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -642,8 +642,13 @@ func (txn *KVTxn) LockKeys(ctx context.Context, lockCtx *tikv.LockCtx, keysInput keyMayBeLocked := !(tikverr.IsErrWriteConflict(err) || tikverr.IsErrKeyExist(err)) // If there is only 1 key and lock fails, no need to do pessimistic rollback. if len(keys) > 1 || keyMayBeLocked { + dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock) + if ok && lockCtx.OnDeadlock != nil { + // Call OnDeadlock before pessimistic rollback. + lockCtx.OnDeadlock(dl) + } wg := txn.asyncPessimisticRollback(ctx, keys) - if dl, ok := errors.Cause(err).(*tikverr.ErrDeadlock); ok { + if ok { logutil.Logger(ctx).Debug("deadlock error received", zap.Uint64("startTS", txn.startTS), zap.Stringer("deadlockInfo", dl)) if hashInKeys(dl.DeadlockKeyHash, keys) { dl.IsRetryable = true diff --git a/util/deadlockhistory/deadlock_history.go b/util/deadlockhistory/deadlock_history.go new file mode 100644 index 0000000000000..ddb78067ffe7c --- /dev/null +++ b/util/deadlockhistory/deadlock_history.go @@ -0,0 +1,198 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package deadlockhistory + +import ( + "encoding/hex" + "strings" + "sync" + "time" + + "github.com/pingcap/parser/mysql" + tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/resourcegrouptag" + "go.uber.org/zap" +) + +// WaitChainItem represents an entry in a deadlock's wait chain. +type WaitChainItem struct { + TryLockTxn uint64 + SQLDigest string + Key []byte + AllSQLs []string + TxnHoldingLock uint64 +} + +// DeadlockRecord represents a deadlock events, and contains multiple transactions' information. +type DeadlockRecord struct { + // The ID doesn't need to be set manually and it's set when it's added into the DeadlockHistory by invoking its Push + // method. + ID uint64 + OccurTime time.Time + IsRetryable bool + WaitChain []WaitChainItem +} + +// DeadlockHistory is a collection for maintaining recent several deadlock events. +type DeadlockHistory struct { + sync.RWMutex + + deadlocks []*DeadlockRecord + + // The `head` and `size` makes the `deadlocks` array behaves like a deque. The valid elements are + // deadlocks[head:head+size], or deadlocks[head:] + deadlocks[:head+size-len] if `head+size` exceeds the array's + // length. + head int + size int + + // currentID is used to allocate IDs for deadlock records pushed to the queue that's unique in the deadlock + // history queue instance. + currentID uint64 +} + +// NewDeadlockHistory creates an instance of DeadlockHistory +func NewDeadlockHistory(capacity int) *DeadlockHistory { + return &DeadlockHistory{ + deadlocks: make([]*DeadlockRecord, capacity), + currentID: 1, + } +} + +// GlobalDeadlockHistory is the global instance of DeadlockHistory, which is used to maintain recent several recent +// deadlock events globally. +// TODO: Make the capacity configurable +var GlobalDeadlockHistory = NewDeadlockHistory(10) + +// Push pushes an element into the queue. It will set the `ID` field of the record, and add the pointer directly to +// the collection. Be aware that do not modify the record's content after pushing. +func (d *DeadlockHistory) Push(record *DeadlockRecord) { + d.Lock() + defer d.Unlock() + + capacity := len(d.deadlocks) + if capacity == 0 { + return + } + + record.ID = d.currentID + d.currentID++ + + if d.size == capacity { + // The current head is popped and it's cell becomes the latest pushed item. + d.deadlocks[d.head] = record + d.head = (d.head + 1) % capacity + } else if d.size < capacity { + d.deadlocks[(d.head+d.size)%capacity] = record + d.size++ + } else { + panic("unreachable") + } +} + +// GetAll gets all collected deadlock events. +func (d *DeadlockHistory) GetAll() []*DeadlockRecord { + d.RLock() + defer d.RUnlock() + + res := make([]*DeadlockRecord, 0, d.size) + capacity := len(d.deadlocks) + if d.head+d.size <= capacity { + res = append(res, d.deadlocks[d.head:d.head+d.size]...) + } else { + res = append(res, d.deadlocks[d.head:]...) + res = append(res, d.deadlocks[:(d.head+d.size)%capacity]...) + } + return res +} + +// GetAllDatum gets all collected deadlock events, and make it into datum that matches the definition of the table +// `INFORMATION_SCHEMA.DEADLOCKS`. +func (d *DeadlockHistory) GetAllDatum() [][]types.Datum { + records := d.GetAll() + rowsCount := 0 + for _, rec := range records { + rowsCount += len(rec.WaitChain) + } + + rows := make([][]types.Datum, 0, rowsCount) + + row := make([]interface{}, 8) + for _, rec := range records { + row[0] = rec.ID + row[1] = types.NewTime(types.FromGoTime(rec.OccurTime), mysql.TypeTimestamp, types.MaxFsp) + row[2] = rec.IsRetryable + + for _, item := range rec.WaitChain { + row[3] = item.TryLockTxn + + row[4] = nil + if len(item.SQLDigest) > 0 { + row[4] = item.SQLDigest + } + + row[5] = nil + if len(item.Key) > 0 { + row[5] = strings.ToUpper(hex.EncodeToString(item.Key)) + } + + row[6] = nil + if item.AllSQLs != nil { + row[6] = "[" + strings.Join(item.AllSQLs, ", ") + "]" + } + + row[7] = item.TxnHoldingLock + + rows = append(rows, types.MakeDatums(row...)) + } + } + + return rows +} + +// Clear clears content from deadlock histories +func (d *DeadlockHistory) Clear() { + d.Lock() + defer d.Unlock() + for i := 0; i < len(d.deadlocks); i++ { + d.deadlocks[i] = nil + } + d.head = 0 + d.size = 0 +} + +// ErrDeadlockToDeadlockRecord generates a DeadlockRecord from the information in a `tikverr.ErrDeadlock` error. +func ErrDeadlockToDeadlockRecord(dl *tikverr.ErrDeadlock) *DeadlockRecord { + waitChain := make([]WaitChainItem, 0, len(dl.WaitChain)) + for _, rawItem := range dl.WaitChain { + sqlDigest, err := resourcegrouptag.DecodeResourceGroupTag(rawItem.ResourceGroupTag) + if err != nil { + logutil.BgLogger().Warn("decoding resource group tag encounters error", zap.Error(err)) + } + waitChain = append(waitChain, WaitChainItem{ + TryLockTxn: rawItem.Txn, + SQLDigest: sqlDigest, + Key: rawItem.Key, + AllSQLs: nil, + TxnHoldingLock: rawItem.WaitForTxn, + }) + } + rec := &DeadlockRecord{ + OccurTime: time.Now(), + IsRetryable: dl.IsRetryable, + WaitChain: waitChain, + } + return rec +} diff --git a/util/deadlockhistory/deadlock_history_test.go b/util/deadlockhistory/deadlock_history_test.go new file mode 100644 index 0000000000000..35cbb6c8513cd --- /dev/null +++ b/util/deadlockhistory/deadlock_history_test.go @@ -0,0 +1,277 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package deadlockhistory + +import ( + "testing" + "time" + + . "github.com/pingcap/check" + "github.com/pingcap/kvproto/pkg/deadlock" + "github.com/pingcap/kvproto/pkg/kvrpcpb" + tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/resourcegrouptag" +) + +type testDeadlockHistorySuite struct{} + +var _ = Suite(&testDeadlockHistorySuite{}) + +func TestT(t *testing.T) { + TestingT(t) +} + +func (s *testDeadlockHistorySuite) TestDeadlockHistoryCollection(c *C) { + h := NewDeadlockHistory(1) + c.Assert(len(h.GetAll()), Equals, 0) + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 0) + + rec1 := &DeadlockRecord{ + OccurTime: time.Now(), + } + h.Push(rec1) + res := h.GetAll() + c.Assert(len(res), Equals, 1) + c.Assert(res[0], Equals, rec1) // Checking pointer equals is ok. + c.Assert(res[0].ID, Equals, uint64(1)) + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 1) + + rec2 := &DeadlockRecord{ + OccurTime: time.Now(), + } + h.Push(rec2) + res = h.GetAll() + c.Assert(len(res), Equals, 1) + c.Assert(res[0], Equals, rec2) + c.Assert(res[0].ID, Equals, uint64(2)) + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 1) + + h.Clear() + c.Assert(len(h.GetAll()), Equals, 0) + + h = NewDeadlockHistory(3) + rec1 = &DeadlockRecord{ + OccurTime: time.Now(), + } + h.Push(rec1) + res = h.GetAll() + c.Assert(len(res), Equals, 1) + c.Assert(res[0], Equals, rec1) // Checking pointer equals is ok. + c.Assert(res[0].ID, Equals, uint64(1)) + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 1) + + rec2 = &DeadlockRecord{ + OccurTime: time.Now(), + } + h.Push(rec2) + res = h.GetAll() + c.Assert(len(res), Equals, 2) + c.Assert(res[0], Equals, rec1) + c.Assert(res[0].ID, Equals, uint64(1)) + c.Assert(res[1], Equals, rec2) + c.Assert(res[1].ID, Equals, uint64(2)) + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 2) + + rec3 := &DeadlockRecord{ + OccurTime: time.Now(), + } + h.Push(rec3) + res = h.GetAll() + c.Assert(len(res), Equals, 3) + c.Assert(res[0], Equals, rec1) + c.Assert(res[0].ID, Equals, uint64(1)) + c.Assert(res[1], Equals, rec2) + c.Assert(res[1].ID, Equals, uint64(2)) + c.Assert(res[2], Equals, rec3) + c.Assert(res[2].ID, Equals, uint64(3)) + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 3) + + // Continuously pushing items to check the correctness of the deque + expectedItems := []*DeadlockRecord{rec1, rec2, rec3} + expectedIDs := []uint64{1, 2, 3} + expectedDequeHead := 0 + for i := 0; i < 6; i++ { + newRec := &DeadlockRecord{ + OccurTime: time.Now(), + } + h.Push(newRec) + + expectedItems = append(expectedItems[1:], newRec) + for idx := range expectedIDs { + expectedIDs[idx]++ + } + expectedDequeHead = (expectedDequeHead + 1) % 3 + + res = h.GetAll() + c.Assert(len(res), Equals, 3) + for idx, item := range res { + c.Assert(item, Equals, expectedItems[idx]) + c.Assert(item.ID, Equals, expectedIDs[idx]) + } + c.Assert(h.head, Equals, expectedDequeHead) + c.Assert(h.size, Equals, 3) + } + + h.Clear() + c.Assert(len(h.GetAll()), Equals, 0) +} + +func (s *testDeadlockHistorySuite) TestGetDatum(c *C) { + time1 := time.Date(2021, 05, 14, 15, 28, 30, 123456000, time.UTC) + time2 := time.Date(2022, 06, 15, 16, 29, 31, 123457000, time.UTC) + + h := NewDeadlockHistory(10) + h.Push(&DeadlockRecord{ + OccurTime: time1, + IsRetryable: false, + WaitChain: []WaitChainItem{ + { + TryLockTxn: 101, + SQLDigest: "sql1", + Key: []byte("k1"), + AllSQLs: []string{"sql1", "sql2"}, + TxnHoldingLock: 102, + }, + // It should work even some information are missing. + { + TryLockTxn: 102, + TxnHoldingLock: 101, + }, + }, + }) + h.Push(&DeadlockRecord{ + OccurTime: time2, + IsRetryable: true, + WaitChain: []WaitChainItem{ + { + TryLockTxn: 201, + AllSQLs: []string{}, + TxnHoldingLock: 202, + }, + { + TryLockTxn: 202, + AllSQLs: []string{"sql1"}, + TxnHoldingLock: 201, + }, + }, + }) + // A deadlock error without wait chain shows nothing in the query result. + h.Push(&DeadlockRecord{ + OccurTime: time.Now(), + IsRetryable: false, + WaitChain: nil, + }) + + res := h.GetAllDatum() + c.Assert(len(res), Equals, 4) + for _, row := range res { + c.Assert(len(row), Equals, 8) + } + + toGoTime := func(d types.Datum) time.Time { + v, ok := d.GetValue().(types.Time) + c.Assert(ok, IsTrue) + t, err := v.GoTime(time.UTC) + c.Assert(err, IsNil) + return t + } + + c.Assert(res[0][0].GetValue(), Equals, uint64(1)) // ID + c.Assert(toGoTime(res[0][1]), Equals, time1) // OCCUR_TIME + c.Assert(res[0][2].GetValue(), Equals, int64(0)) // RETRYABLE + c.Assert(res[0][3].GetValue(), Equals, uint64(101)) // TRY_LOCK_TRX_ID + c.Assert(res[0][4].GetValue(), Equals, "sql1") // SQL_DIGEST + c.Assert(res[0][5].GetValue(), Equals, "6B31") // KEY + c.Assert(res[0][6].GetValue(), Equals, "[sql1, sql2]") // ALL_SQLS + c.Assert(res[0][7].GetValue(), Equals, uint64(102)) // TRX_HOLDING_LOCK + + c.Assert(res[1][0].GetValue(), Equals, uint64(1)) // ID + c.Assert(toGoTime(res[1][1]), Equals, time1) // OCCUR_TIME + c.Assert(res[1][2].GetValue(), Equals, int64(0)) // RETRYABLE + c.Assert(res[1][3].GetValue(), Equals, uint64(102)) // TRY_LOCK_TRX_ID + c.Assert(res[1][4].GetValue(), Equals, nil) // SQL_DIGEST + c.Assert(res[1][5].GetValue(), Equals, nil) // KEY + c.Assert(res[1][6].GetValue(), Equals, nil) // ALL_SQLS + c.Assert(res[1][7].GetValue(), Equals, uint64(101)) // TRX_HOLDING_LOCK + + c.Assert(res[2][0].GetValue(), Equals, uint64(2)) // ID + c.Assert(toGoTime(res[2][1]), Equals, time2) // OCCUR_TIME + c.Assert(res[2][2].GetValue(), Equals, int64(1)) // RETRYABLE + c.Assert(res[2][3].GetValue(), Equals, uint64(201)) // TRY_LOCK_TRX_ID + c.Assert(res[2][6].GetValue(), Equals, "[]") // ALL_SQLS + c.Assert(res[2][7].GetValue(), Equals, uint64(202)) // TRX_HOLDING_LOCK + + c.Assert(res[3][0].GetValue(), Equals, uint64(2)) // ID + c.Assert(toGoTime(res[3][1]), Equals, time2) // OCCUR_TIME + c.Assert(res[3][2].GetValue(), Equals, int64(1)) // RETRYABLE + c.Assert(res[3][3].GetValue(), Equals, uint64(202)) // TRY_LOCK_TRX_ID + c.Assert(res[3][6].GetValue(), Equals, "[sql1]") // ALL_SQLS + c.Assert(res[3][7].GetValue(), Equals, uint64(201)) // TRX_HOLDING_LOCK +} + +func (s *testDeadlockHistorySuite) TestErrDeadlockToDeadlockRecord(c *C) { + err := &tikverr.ErrDeadlock{ + Deadlock: &kvrpcpb.Deadlock{ + LockTs: 101, + LockKey: []byte("k1"), + DeadlockKeyHash: 1234567, + WaitChain: []*deadlock.WaitForEntry{ + { + Txn: 100, + WaitForTxn: 101, + Key: []byte("k2"), + ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag("aabbccdd"), + }, + { + Txn: 101, + WaitForTxn: 100, + Key: []byte("k1"), + ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag("ddccbbaa"), + }, + }, + }, + IsRetryable: true, + } + + expectedRecord := &DeadlockRecord{ + IsRetryable: true, + WaitChain: []WaitChainItem{ + { + TryLockTxn: 100, + SQLDigest: "aabbccdd", + Key: []byte("k2"), + TxnHoldingLock: 101, + }, + { + TryLockTxn: 101, + SQLDigest: "ddccbbaa", + Key: []byte("k1"), + TxnHoldingLock: 100, + }, + }, + } + + record := ErrDeadlockToDeadlockRecord(err) + // The OccurTime is set to time.Now + c.Assert(time.Since(record.OccurTime), Less, time.Millisecond*5) + expectedRecord.OccurTime = record.OccurTime + c.Assert(record, DeepEquals, expectedRecord) +} From 9fe83e759fe5e4820318092c3400e5f1280128cf Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Thu, 20 May 2021 01:53:41 +0800 Subject: [PATCH 111/343] docs: Some proposal for renaming and configurations for Lock View (#24718) --- docs/design/2021-04-26-lock-view.md | 46 +++++++++++++++++++++-------- 1 file changed, 33 insertions(+), 13 deletions(-) diff --git a/docs/design/2021-04-26-lock-view.md b/docs/design/2021-04-26-lock-view.md index 56d16e8e86194..3ed0e5902c146 100644 --- a/docs/design/2021-04-26-lock-view.md +++ b/docs/design/2021-04-26-lock-view.md @@ -1,7 +1,7 @@ # TiDB Design Documents - Author(s): [longfangsong](https://github.com/longfangsong), [MyonKeminta](http://github.com/MyonKeminta) -- Last updated: May 6, 2021 +- Last updated: May 18, 2021 - Discussion PR: N/A - Tracking Issue: https://github.com/pingcap/tidb/issues/24199 @@ -35,14 +35,14 @@ Several tables will be provided in `information_schema`. Some tables has both lo | Field | Type | Comment | |------------|------------|---------| -|`TRX_ID` | `unsigned bigint` | The transaction ID (aka. start ts) | -|`TRX_STARTED`|`time`| Human readable start time of the transaction | -|`DIGEST`|`text`| The digest of the current executing SQL statement | -|`SQLS` | `text` | A list of all executed SQL statements' digests | -|`STATE`| `enum('Running', 'Lock waiting', 'Committing', 'RollingBack')`| The state of the transaction | +| `TRX_ID` | `unsigned bigint` | The transaction ID (aka. start ts) | +| `TRX_STARTED`|`time`| Human readable start time of the transaction | +| `DIGEST`|`text`| The digest of the current executing SQL statement | +| `ALL_SQLS` | `text` | A list of all executed SQL statements' digests | +| `STATE`| `enum('Running', 'Lock waiting', 'Committing', 'RollingBack')`| The state of the transaction | | `WAITING_START_TIME` | `time` | The elapsed time since the start of the current lock waiting (if any) | | `SCOPE` | `enum('Global', 'Local')` | The scope of the transaction | -| `ISOLATION_LEVEL` | `enum('RR', 'RC')` | | +| `ISOLATION_LEVEL` | `enum('REPEATABLE-READ', 'READ-COMMITTED')` | | | `AUTOCOMMIT` | `bool` | | | `SESSION_ID` | `unsigned bigint` | | | `USER` | `varchar` | | @@ -79,24 +79,28 @@ Several tables will be provided in `information_schema`. Some tables has both lo * Permission: * `PROCESS` privilege is needed to access this table. -### Table `(CLUSTER_)DEAD_LOCK` +### Table `(CLUSTER_)DEADLOCKS` | Field | Type | Comment | |------------|------------|---------| | `DEADLOCK_ID` | `int` | There needs multiple rows to represent information of a single deadlock event. This field is used to distinguish different events. | | `OCCUR_TIME` | `time` | The physical time when the deadlock occurs | +| `RETRYABLE` | `bool` | Is the deadlock retryable. TiDB tries to determine if the current statement is (indirectly) waiting for a lock locked by the current statement. | | `TRY_LOCK_TRX_ID` | `unsigned bigint` | The transaction ID (start ts) of the transaction that's trying to acquire the lock | | `CURRENT_SQL_DIGEST` | `text` | The SQL that's being blocked | | `KEY` | `varchar` | The key that's being locked, but locked by another transaction in the deadlock event | -| `SQLS` | `text` | A list of the digest of SQL statements that the transaction has executed | +| `ALL_SQLS` | `text` | A list of the digest of SQL statements that the transaction has executed | | `TRX_HOLDING_LOCK` | `unsigned bigint` | The transaction that's currently holding the lock. There will be another record in the table with the same `DEADLOCK_ID` for that transaction. | * Life span of rows: * Create after TiDB receive a deadlock error * FIFO,clean the oldest after buffer is full * Collecting, storing and querying: - * All of these information can be collected on TiDB side. It just need to add the information to the table when receives deadlock error from TiKV. The information of other transactions involved in the deadlock circle needed to be fetched from elsewhere (the `TIDB_TRX` table) when handling the deadlock error. - * Currently there are no much information in the deadlock error (it doesn't has the SQLs and keys' information), which needs to be improved. + * All of these information can be collected on TiDB side. It just need to add the information to the table when receives deadlock error from TiKV. The information of other transactions involved in the deadlock circle needed to be fetched from elsewhere (the `CLUSTER_TIDB_TRX` table) when handling the deadlock error. + * TiKV needs to report more rich information in the deadlock error for collecting. + * There are two types of deadlock errors internally: retryable or non-retryable. The transaction will internally retry on retryable deadlocks and won't report error to the client. Therefore, the user are typically more interested in the non-retryable deadlocks. + * Retryable deadlock errors are by default not collected, and can be enabled with configuration. + * Collecting `CLUSTER_TIDB_TRX` for more rich information for retryable deadlock is possible to make the performance worse. Whether it will be collected for retryable deadlock will be decided after some tests. * Permission: * `PROCESS` privilege is needed to access this table. @@ -151,9 +155,25 @@ The locking key and `resource_group_tag` that comes from the `Context` of the pe The wait chain will be added to the `Deadlock` error which is returned by the `PessimisticLock` request, so that when deadlock happens, the full wait chain information can be passed to TiDB. +### Configurations + +#### TiDB Config File `pessimistic-txn.tidb_deadlock_history_capacity` + +Specifies how many recent deadlock events each TiDB node should keep. +Dynamically changeable via HTTP API. +Value: 0 to 10000 +Default: 10 + +#### TiDB Config File `pessimistic-txn.tidb_deadlock_history_collect_retryable` + +Specifies whether to collect retryable deadlock errors to the `(CLUSTER_)DEADLOCKS` table. +Dynamically changeable via HTTP API. +Value: 0 (do not collect) or 1 (collect) +Default: 0 + ## Compatibility -This feature is not expected to be incompatible with other features. During upgrading, when there are different versions of TiDB nodes exists at the same time, it's possible that the `CLUSTER_` prefixed tables may encounter errors. But since this feature is typically used by user manually, this shouldn't be a severe problem. So we don't need to care much about that. +This feature is not expected to be incompatible with other features. During upgrading, when there are different versions of TiDB nodes exists at the same time, it's possible that the `CLUSTER_` prefixed tables may encounter errors. However, since this feature is typically used by user manually, this shouldn't be a severe problem. So we don't need to care much about that. ## Test Design @@ -190,7 +210,7 @@ This feature is not expected to be incompatible with other features. During upgr * Since lock waiting on TiKV may timeout and retry, it's possible that in a single query to `DATA_LOCK_WAIT` table doesn't shows all (logical) lock waiting. * Information about internal transactions may not be collected in our first version of implementation. -* Since TiDB need to query transaction information after it receives the deadlock error, the transactions' status may be changed during that time. As a result the information in `(CLUSTER_)DEAD_LOCK` table can't be promised to be accurate and complete. +* Since TiDB need to query transaction information after it receives the deadlock error, the transactions' status may be changed during that time. As a result the information in `(CLUSTER_)DEADLOCKS` table can't be promised to be accurate and complete. * Statistics about transaction conflicts is still not enough. * Historical information of `TIDB_TRX` and `DATA_LOCK_WAITS` is not kept, which possibly makes it still difficult to investigate some kind of problems. * The SQL digest that's holding lock and blocking the current transaction is hard to retrieve and is not included in the current design. From d1a5fa3ac9a402d67ba92705a5cb6965fbf9d189 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 19 May 2021 20:05:41 +0200 Subject: [PATCH 112/343] planner: add range partition boundaries tests with BETWEEN expression (#24598) --- executor/partition_table_test.go | 102 +- executor/testdata/executor_suite_in.json | 131 +++ executor/testdata/executor_suite_out.json | 1167 +++++++++++++++++++++ 3 files changed, 1372 insertions(+), 28 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index cc0e4074b39a0..101f429475748 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -1567,6 +1567,38 @@ func (s *globalIndexSuite) TestIssue21731(c *C) { tk.MustExec("create table t (a int, b int, unique index idx(a)) partition by list columns(b) (partition p0 values in (1), partition p1 values in (2));") } +type testOutput struct { + SQL string + Plan []string + Res []string +} + +func (s *testSuiteWithData) verifyPartitionResult(tk *testkit.TestKit, input []string, output []testOutput) { + for i, tt := range input { + var isSelect bool = false + if strings.HasPrefix(strings.ToLower(tt), "select ") { + isSelect = true + } + s.testData.OnRecord(func() { + output[i].SQL = tt + if isSelect { + output[i].Plan = s.testData.ConvertRowsToStrings(tk.UsedPartitions(tt).Rows()) + output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) + } else { + // Just verify SELECT (also avoid double INSERTs during record) + output[i].Res = nil + output[i].Plan = nil + } + }) + if isSelect { + tk.UsedPartitions(tt).Check(testkit.Rows(output[i].Plan...)) + tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) + } else { + tk.MustExec(tt) + } + } +} + func (s *testSuiteWithData) TestRangePartitionBoundariesEq(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -1589,12 +1621,6 @@ PARTITION BY RANGE (a) ( s.verifyPartitionResult(tk, input, output) } -type testOutput struct { - SQL string - Plan []string - Res []string -} - func (s *testSuiteWithData) TestRangePartitionBoundariesNe(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -1620,26 +1646,46 @@ PARTITION BY RANGE (a) ( s.verifyPartitionResult(tk, input, output) } -func (s *testSuiteWithData) verifyPartitionResult(tk *testkit.TestKit, input []string, output []testOutput) { - for i, tt := range input { - var isSelect bool = false - if strings.HasPrefix(strings.ToLower(tt), "select ") { - isSelect = true - } - s.testData.OnRecord(func() { - output[i].SQL = tt - if isSelect { - output[i].Plan = s.testData.ConvertRowsToStrings(tk.UsedPartitions(tt).Rows()) - output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Sort().Rows()) - } else { - // to avoid double execution of INSERT (and INSERT does not return anything) - output[i].Res = nil - output[i].Plan = nil - } - }) - if isSelect { - tk.UsedPartitions(tt).Check(testkit.Rows(output[i].Plan...)) - } - tk.MayQuery(tt).Sort().Check(testkit.Rows(output[i].Res...)) - } +func (s *testSuiteWithData) TestRangePartitionBoundariesBetweenM(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("CREATE DATABASE IF NOT EXISTS TestRangePartitionBoundariesBetweenM") + defer tk.MustExec("DROP DATABASE TestRangePartitionBoundariesBetweenM") + tk.MustExec("USE TestRangePartitionBoundariesBetweenM") + tk.MustExec("DROP TABLE IF EXISTS t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1000000), + PARTITION p1 VALUES LESS THAN (2000000), + PARTITION p2 VALUES LESS THAN (3000000))`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} + +func (s *testSuiteWithData) TestRangePartitionBoundariesBetweenS(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("CREATE DATABASE IF NOT EXISTS TestRangePartitionBoundariesBetweenS") + defer tk.MustExec("DROP DATABASE TestRangePartitionBoundariesBetweenS") + tk.MustExec("USE TestRangePartitionBoundariesBetweenS") + tk.MustExec("DROP TABLE IF EXISTS t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1), + PARTITION p1 VALUES LESS THAN (2), + PARTITION p2 VALUES LESS THAN (3), + PARTITION p3 VALUES LESS THAN (4), + PARTITION p4 VALUES LESS THAN (5), + PARTITION p5 VALUES LESS THAN (6), + PARTITION p6 VALUES LESS THAN (7))`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) } diff --git a/executor/testdata/executor_suite_in.json b/executor/testdata/executor_suite_in.json index fff3187717f0a..a8db9425a7078 100644 --- a/executor/testdata/executor_suite_in.json +++ b/executor/testdata/executor_suite_in.json @@ -142,5 +142,136 @@ "SELECT * FROM t WHERE a NOT IN (-2, -1, 0, 1, 2, 3, 4, 5, 6, 7)", "SELECT * FROM t WHERE 1 = 0 OR a = -1 OR a = 0 OR a = 1 OR a = 2 OR a = 3 OR a = 4 OR a = 5 OR a = 6 OR a = 7" ] + }, + { + "name": "TestRangePartitionBoundariesBetweenM", + "cases": [ + "INSERT INTO t VALUES (999998, '999998 Filler ...'), (999999, '999999 Filler ...'), (1000000, '1000000 Filler ...'), (1000001, '1000001 Filler ...'), (1000002, '1000002 Filler ...')", + "INSERT INTO t VALUES (1999998, '1999998 Filler ...'), (1999999, '1999999 Filler ...'), (2000000, '2000000 Filler ...'), (2000001, '2000001 Filler ...'), (2000002, '2000002 Filler ...')", + "INSERT INTO t VALUES (2999998, '2999998 Filler ...'), (2999999, '2999999 Filler ...')", + "INSERT INTO t VALUES (-2147483648, 'MIN_INT filler...'), (0, '0 Filler...')", + "ANALYZE TABLE t", + "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483649", + "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483648", + "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483647", + "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483646", + "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483638", + "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483650", + "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483649", + "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483648", + "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483647", + "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483646", + "SELECT * FROM t WHERE a BETWEEN 0 AND -1", + "SELECT * FROM t WHERE a BETWEEN 0 AND 0", + "SELECT * FROM t WHERE a BETWEEN 0 AND 1", + "SELECT * FROM t WHERE a BETWEEN 0 AND 2", + "SELECT * FROM t WHERE a BETWEEN 0 AND 10", + "SELECT * FROM t WHERE a BETWEEN 0 AND 999998", + "SELECT * FROM t WHERE a BETWEEN 0 AND 999999", + "SELECT * FROM t WHERE a BETWEEN 0 AND 1000000", + "SELECT * FROM t WHERE a BETWEEN 0 AND 1000001", + "SELECT * FROM t WHERE a BETWEEN 0 AND 1000002", + "SELECT * FROM t WHERE a BETWEEN 999998 AND 999997", + "SELECT * FROM t WHERE a BETWEEN 999998 AND 999998", + "SELECT * FROM t WHERE a BETWEEN 999998 AND 999999", + "SELECT * FROM t WHERE a BETWEEN 999998 AND 1000000", + "SELECT * FROM t WHERE a BETWEEN 999998 AND 1000008", + "SELECT * FROM t WHERE a BETWEEN 999998 AND 1999996", + "SELECT * FROM t WHERE a BETWEEN 999998 AND 1999997", + "SELECT * FROM t WHERE a BETWEEN 999998 AND 1999998", + "SELECT * FROM t WHERE a BETWEEN 999998 AND 1999999", + "SELECT * FROM t WHERE a BETWEEN 999998 AND 2000000", + "SELECT * FROM t WHERE a BETWEEN 999999 AND 999998", + "SELECT * FROM t WHERE a BETWEEN 999999 AND 999999", + "SELECT * FROM t WHERE a BETWEEN 999999 AND 1000000", + "SELECT * FROM t WHERE a BETWEEN 999999 AND 1000001", + "SELECT * FROM t WHERE a BETWEEN 999999 AND 1000009", + "SELECT * FROM t WHERE a BETWEEN 999999 AND 1999997", + "SELECT * FROM t WHERE a BETWEEN 999999 AND 1999998", + "SELECT * FROM t WHERE a BETWEEN 999999 AND 1999999", + "SELECT * FROM t WHERE a BETWEEN 999999 AND 2000000", + "SELECT * FROM t WHERE a BETWEEN 999999 AND 2000001", + "SELECT * FROM t WHERE a BETWEEN 1000000 AND 999999", + "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000000", + "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000001", + "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000002", + "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000010", + "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1999998", + "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1999999", + "SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000000", + "SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000001", + "SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000002", + "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000000", + "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000001", + "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000002", + "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000003", + "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000011", + "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1999999", + "SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000000", + "SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000001", + "SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000002", + "SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000003", + "SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000001", + "SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000002", + "SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000003", + "SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000004", + "SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000012", + "SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000000", + "SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000001", + "SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000002", + "SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000003", + "SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000004", + "SELECT * FROM t WHERE a BETWEEN 3000000 AND 2999999", + "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000000", + "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000001", + "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000002", + "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000010", + "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3999998", + "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3999999", + "SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000000", + "SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000001", + "SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000002", + "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000000", + "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000001", + "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000002", + "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000003", + "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000011", + "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3999999", + "SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000000", + "SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000001", + "SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000002", + "SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000003" + ] + }, + { + "name": "TestRangePartitionBoundariesBetweenS", + "cases": [ + "INSERT INTO t VALUES (0, '0 Filler...')", + "INSERT INTO t VALUES (1, '1 Filler...')", + "INSERT INTO t VALUES (2, '2 Filler...')", + "INSERT INTO t VALUES (3, '3 Filler...')", + "INSERT INTO t VALUES (4, '4 Filler...')", + "INSERT INTO t VALUES (5, '5 Filler...')", + "INSERT INTO t VALUES (6, '6 Filler...')", + "ANALYZE TABLE t", + "SELECT * FROM t WHERE a BETWEEN 2 AND -1", + "SELECT * FROM t WHERE a BETWEEN -1 AND 4", + "SELECT * FROM t WHERE a BETWEEN 2 AND 0", + "SELECT * FROM t WHERE a BETWEEN 0 AND 4", + "SELECT * FROM t WHERE a BETWEEN 2 AND 1", + "SELECT * FROM t WHERE a BETWEEN 1 AND 4", + "SELECT * FROM t WHERE a BETWEEN 2 AND 2", + "SELECT * FROM t WHERE a BETWEEN 2 AND 4", + "SELECT * FROM t WHERE a BETWEEN 2 AND 3", + "SELECT * FROM t WHERE a BETWEEN 3 AND 4", + "SELECT * FROM t WHERE a BETWEEN 2 AND 4", + "SELECT * FROM t WHERE a BETWEEN 4 AND 4", + "SELECT * FROM t WHERE a BETWEEN 2 AND 5", + "SELECT * FROM t WHERE a BETWEEN 5 AND 4", + "SELECT * FROM t WHERE a BETWEEN 2 AND 6", + "SELECT * FROM t WHERE a BETWEEN 6 AND 4", + "SELECT * FROM t WHERE a BETWEEN 2 AND 7", + "SELECT * FROM t WHERE a BETWEEN 7 AND 4" + ] } ] diff --git a/executor/testdata/executor_suite_out.json b/executor/testdata/executor_suite_out.json index caa5c4f948966..bd5fbdb486ac0 100644 --- a/executor/testdata/executor_suite_out.json +++ b/executor/testdata/executor_suite_out.json @@ -1395,5 +1395,1172 @@ ] } ] + }, + { + "Name": "TestRangePartitionBoundariesBetweenM", + "Cases": [ + { + "SQL": "INSERT INTO t VALUES (999998, '999998 Filler ...'), (999999, '999999 Filler ...'), (1000000, '1000000 Filler ...'), (1000001, '1000001 Filler ...'), (1000002, '1000002 Filler ...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (1999998, '1999998 Filler ...'), (1999999, '1999999 Filler ...'), (2000000, '2000000 Filler ...'), (2000001, '2000001 Filler ...'), (2000002, '2000002 Filler ...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (2999998, '2999998 Filler ...'), (2999999, '2999999 Filler ...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (-2147483648, 'MIN_INT filler...'), (0, '0 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "ANALYZE TABLE t", + "Plan": null, + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483649", + "Plan": [ + "p0" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483648", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483647", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483646", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2147483638", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483650", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483649", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483648", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483647", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -2147483648 AND -2146483646", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND -1", + "Plan": [ + "p0" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND 0", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND 1", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND 2", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND 10", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND 999998", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler...", + "999998 999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND 999999", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND 1000000", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1000000 1000000 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND 1000001", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND 1000002", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999998 AND 999997", + "Plan": [ + "p0" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999998 AND 999998", + "Plan": [ + "p0" + ], + "Res": [ + "999998 999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999998 AND 999999", + "Plan": [ + "p0" + ], + "Res": [ + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999998 AND 1000000", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999998 AND 1000008", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999998 AND 1999996", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999998 AND 1999997", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999998 AND 1999998", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999998 AND 1999999", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999998 AND 2000000", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999999 AND 999998", + "Plan": [ + "p0" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999999 AND 999999", + "Plan": [ + "p0" + ], + "Res": [ + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999999 AND 1000000", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999999 AND 1000001", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999999 AND 1000009", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999999 AND 1999997", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999999 AND 1999998", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999999 AND 1999999", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999999 AND 2000000", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 999999 AND 2000001", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000000 AND 999999", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000000", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000001", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000002", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1000010", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1999998", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000000 AND 1999999", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000000", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000001", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000000 AND 2000002", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000000", + "Plan": [ + "p1" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000001", + "Plan": [ + "p1" + ], + "Res": [ + "1000001 1000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000002", + "Plan": [ + "p1" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000003", + "Plan": [ + "p1" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1000011", + "Plan": [ + "p1" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000001 AND 1999999", + "Plan": [ + "p1" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000000", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000001", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000002", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000001 AND 2000003", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000001", + "Plan": [ + "p1" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000002", + "Plan": [ + "p1" + ], + "Res": [ + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000003", + "Plan": [ + "p1" + ], + "Res": [ + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000004", + "Plan": [ + "p1" + ], + "Res": [ + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000002 AND 1000012", + "Plan": [ + "p1" + ], + "Res": [ + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000000", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000001", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000002", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000003", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1000002 AND 2000004", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000000 AND 2999999", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000000", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000001", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000002", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3000010", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3999998", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000000 AND 3999999", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000000", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000001", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000000 AND 4000002", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000000", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000001", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000002", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000003", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3000011", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000001 AND 3999999", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000000", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000001", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000002", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3000001 AND 4000003", + "Plan": [ + "dual" + ], + "Res": null + } + ] + }, + { + "Name": "TestRangePartitionBoundariesBetweenS", + "Cases": [ + { + "SQL": "INSERT INTO t VALUES (0, '0 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (1, '1 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (2, '2 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (3, '3 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (4, '4 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (5, '5 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (6, '6 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "ANALYZE TABLE t", + "Plan": null, + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 2 AND -1", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN -1 AND 4", + "Plan": [ + "p0 p1 p2 p3 p4" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 2 AND 0", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 0 AND 4", + "Plan": [ + "p0 p1 p2 p3 p4" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 2 AND 1", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 1 AND 4", + "Plan": [ + "p1 p2 p3 p4" + ], + "Res": [ + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 2 AND 2", + "Plan": [ + "p2" + ], + "Res": [ + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 2 AND 4", + "Plan": [ + "p2 p3 p4" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 2 AND 3", + "Plan": [ + "p2 p3" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 3 AND 4", + "Plan": [ + "p3 p4" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 2 AND 4", + "Plan": [ + "p2 p3 p4" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 4 AND 4", + "Plan": [ + "p4" + ], + "Res": [ + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 2 AND 5", + "Plan": [ + "p2 p3 p4 p5" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 5 AND 4", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 2 AND 6", + "Plan": [ + "p2 p3 p4 p5 p6" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 6 AND 4", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 2 AND 7", + "Plan": [ + "p2 p3 p4 p5 p6" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a BETWEEN 7 AND 4", + "Plan": [ + "dual" + ], + "Res": null + } + ] } ] From a2278dfdf55f28ec394383bcd2d23cccb9992724 Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 20 May 2021 02:53:41 +0800 Subject: [PATCH 113/343] oracle: simplify timestamp utilities (#24688) --- ddl/index.go | 5 ++--- domain/domain_test.go | 4 ++-- domain/infosync/info.go | 2 +- executor/executor_test.go | 2 +- executor/show_stats.go | 3 +-- executor/simple.go | 4 ++-- executor/stale_txn_test.go | 12 ++++-------- statistics/handle/update_test.go | 4 ++-- store/gcworker/gc_worker_test.go | 6 +++--- store/tikv/2pc.go | 4 ++-- store/tikv/latch/latch_test.go | 4 ++-- store/tikv/oracle/oracle.go | 22 ---------------------- store/tikv/oracle/oracles/local.go | 10 ++++------ store/tikv/oracle/oracles/mock.go | 16 +++++++--------- store/tikv/oracle/oracles/pd.go | 4 ++-- store/tikv/oracle/oracles/pd_test.go | 8 ++++---- store/tikv/tests/2pc_test.go | 2 +- store/tikv/tests/store_test.go | 18 ------------------ 18 files changed, 40 insertions(+), 90 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index fdba6c65008f6..622e93213929c 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1217,10 +1217,9 @@ func (w *worker) updateReorgInfo(t table.PartitionedTable, reorg *reorgInfo) (bo failpoint.Inject("mockUpdateCachedSafePoint", func(val failpoint.Value) { if val.(bool) { - // 18 is for the logical time. - ts := oracle.GetPhysical(time.Now()) << 18 + ts := oracle.GoTimeToTS(time.Now()) s := reorg.d.store.(tikv.Storage) - s.UpdateSPCache(uint64(ts), time.Now()) + s.UpdateSPCache(ts, time.Now()) time.Sleep(time.Millisecond * 3) } }) diff --git a/domain/domain_test.go b/domain/domain_test.go index 51e0948d30715..099e3234c8279 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -287,7 +287,7 @@ func (*testSuite) TestT(c *C) { c.Assert(dd, NotNil) c.Assert(dd.GetLease(), Equals, 80*time.Millisecond) - snapTS := oracle.EncodeTSO(oracle.GetPhysical(time.Now())) + snapTS := oracle.GoTimeToTS(time.Now()) cs := &ast.CharsetOpt{ Chs: "utf8", Col: "utf8_bin", @@ -317,7 +317,7 @@ func (*testSuite) TestT(c *C) { c.Assert(err, IsNil) // for GetSnapshotInfoSchema - currSnapTS := oracle.EncodeTSO(oracle.GetPhysical(time.Now())) + currSnapTS := oracle.GoTimeToTS(time.Now()) currSnapIs, err := dom.GetSnapshotInfoSchema(currSnapTS) c.Assert(err, IsNil) c.Assert(currSnapIs, NotNil) diff --git a/domain/infosync/info.go b/domain/infosync/info.go index be8d80246e96b..a94a15fb7e212 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -559,7 +559,7 @@ func (is *InfoSyncer) ReportMinStartTS(store kv.Storage) { logutil.BgLogger().Error("update minStartTS failed", zap.Error(err)) return } - now := time.Unix(0, oracle.ExtractPhysical(currentVer.Ver)*1e6) + now := oracle.GetTimeFromTS(currentVer.Ver) startTSLowerLimit := oracle.GoTimeToLowerLimitStartTS(now, tikv.MaxTxnTimeUse) minStartTS := oracle.GoTimeToTS(now) diff --git a/executor/executor_test.go b/executor/executor_test.go index ab6dc6f274712..8836b4f59fbd3 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2727,7 +2727,7 @@ func (s *testSuiteP2) TestHistoryRead(c *C) { tk.MustQuery("select * from history_read order by a").Check(testkit.Rows("2 ", "4 ", "8 8", "9 9")) tk.MustExec("set @@tidb_snapshot = '" + snapshotTime.Format("2006-01-02 15:04:05.999999") + "'") tk.MustQuery("select * from history_read order by a").Check(testkit.Rows("2", "4")) - tsoStr := strconv.FormatUint(oracle.EncodeTSO(snapshotTime.UnixNano()/int64(time.Millisecond)), 10) + tsoStr := strconv.FormatUint(oracle.GoTimeToTS(snapshotTime), 10) tk.MustExec("set @@tidb_snapshot = '" + tsoStr + "'") tk.MustQuery("select * from history_read order by a").Check(testkit.Rows("2", "4")) diff --git a/executor/show_stats.go b/executor/show_stats.go index 3a1cf0cb48adc..b6449863b0c1f 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -17,7 +17,6 @@ import ( "fmt" "sort" "strings" - "time" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" @@ -203,7 +202,7 @@ func (e *ShowExec) histogramToRow(dbName, tblName, partitionName, colName string } func (e *ShowExec) versionToTime(version uint64) types.Time { - t := time.Unix(0, oracle.ExtractPhysical(version)*int64(time.Millisecond)) + t := oracle.GetTimeFromTS(version) return types.NewTime(types.FromGoTime(t), mysql.TypeDatetime, 0) } diff --git a/executor/simple.go b/executor/simple.go index d1e5181e2f439..3c45ee05ada4e 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -632,7 +632,7 @@ func (e *SimpleExec) executeStartTransactionReadOnlyWithTimestampBound(ctx conte if err != nil { return err } - startTS := oracle.ComposeTS(gt.Unix()*1000, 0) + startTS := oracle.GoTimeToTS(gt) opt.StartTS = startTS case ast.TimestampBoundExactStaleness: // TODO: support funcCallExpr in future @@ -668,7 +668,7 @@ func (e *SimpleExec) executeStartTransactionReadOnlyWithTimestampBound(ctx conte if err != nil { return err } - startTS := oracle.ComposeTS(gt.Unix()*1000, 0) + startTS := oracle.GoTimeToTS(gt) opt.StartTS = startTS } err := e.ctx.NewTxnWithStalenessOption(ctx, opt) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 64b334b15bf94..db2b55a9a1637 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -196,8 +196,7 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { name: "max 20 seconds ago, safeTS 10 secs ago", sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:20'`, injectSafeTS: func() uint64 { - phy := time.Now().Add(-10*time.Second).Unix() * 1000 - return oracle.ComposeTS(phy, 0) + return oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)) }(), useSafeTS: true, }, @@ -205,8 +204,7 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { name: "max 10 seconds ago, safeTS 20 secs ago", sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`, injectSafeTS: func() uint64 { - phy := time.Now().Add(-20*time.Second).Unix() * 1000 - return oracle.ComposeTS(phy, 0) + return oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)) }(), useSafeTS: false, }, @@ -217,8 +215,7 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { time.Now().Add(-20*time.Second).Format("2006-01-02 15:04:05")) }(), injectSafeTS: func() uint64 { - phy := time.Now().Add(-10*time.Second).Unix() * 1000 - return oracle.ComposeTS(phy, 0) + return oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)) }(), useSafeTS: true, }, @@ -229,8 +226,7 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { time.Now().Add(-10*time.Second).Format("2006-01-02 15:04:05")) }(), injectSafeTS: func() uint64 { - phy := time.Now().Add(-20*time.Second).Unix() * 1000 - return oracle.ComposeTS(phy, 0) + return oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)) }(), useSafeTS: false, }, diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index a1de28e78eeef..a8924021c5846 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -1453,7 +1453,7 @@ func (s *testStatsSuite) TestNeedAnalyzeTable(c *C) { }{ // table was never analyzed and has reach the limit { - tbl: &statistics.Table{Version: oracle.EncodeTSO(oracle.GetPhysical(time.Now()))}, + tbl: &statistics.Table{Version: oracle.GoTimeToTS(time.Now())}, limit: 0, ratio: 0, start: "00:00 +0800", @@ -1464,7 +1464,7 @@ func (s *testStatsSuite) TestNeedAnalyzeTable(c *C) { }, // table was never analyzed but has not reach the limit { - tbl: &statistics.Table{Version: oracle.EncodeTSO(oracle.GetPhysical(time.Now()))}, + tbl: &statistics.Table{Version: oracle.GoTimeToTS(time.Now())}, limit: time.Hour, ratio: 0, start: "00:00 +0800", diff --git a/store/gcworker/gc_worker_test.go b/store/gcworker/gc_worker_test.go index 39abe369f82fb..2beef12da62c0 100644 --- a/store/gcworker/gc_worker_test.go +++ b/store/gcworker/gc_worker_test.go @@ -259,10 +259,10 @@ func (s *testGCWorkerSuite) TestMinStartTS(c *C) { strconv.FormatUint(now, 10)) c.Assert(err, IsNil) err = spkv.Put(fmt.Sprintf("%s/%s", infosync.ServerMinStartTSPath, "b"), - strconv.FormatUint(now-oracle.EncodeTSO(20000), 10)) + strconv.FormatUint(now-oracle.ComposeTS(20000, 0), 10)) c.Assert(err, IsNil) - sp = s.gcWorker.calcSafePointByMinStartTS(ctx, now-oracle.EncodeTSO(10000)) - c.Assert(sp, Equals, now-oracle.EncodeTSO(20000)-1) + sp = s.gcWorker.calcSafePointByMinStartTS(ctx, now-oracle.ComposeTS(10000, 0)) + c.Assert(sp, Equals, now-oracle.ComposeTS(20000, 0)-1) } func (s *testGCWorkerSuite) TestPrepareGC(c *C) { diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 14609f5f77400..c01d97981dd09 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -1428,7 +1428,7 @@ func (c *twoPhaseCommitter) checkSchemaValid(ctx context.Context, checkTS uint64 func (c *twoPhaseCommitter) calculateMaxCommitTS(ctx context.Context) error { // Amend txn with current time first, then we can make sure we have another SafeWindow time to commit - currentTS := oracle.EncodeTSO(int64(time.Since(c.txn.startTime)/time.Millisecond)) + c.startTS + currentTS := oracle.ComposeTS(int64(time.Since(c.txn.startTime)/time.Millisecond), 0) + c.startTS _, _, err := c.checkSchemaValid(ctx, currentTS, c.txn.schemaVer, true) if err != nil { logutil.Logger(ctx).Info("Schema changed for async commit txn", @@ -1438,7 +1438,7 @@ func (c *twoPhaseCommitter) calculateMaxCommitTS(ctx context.Context) error { } safeWindow := config.GetGlobalConfig().TiKVClient.AsyncCommit.SafeWindow - maxCommitTS := oracle.EncodeTSO(int64(safeWindow/time.Millisecond)) + currentTS + maxCommitTS := oracle.ComposeTS(int64(safeWindow/time.Millisecond), 0) + currentTS logutil.BgLogger().Debug("calculate MaxCommitTS", zap.Time("startTime", c.txn.startTime), zap.Duration("safeWindow", safeWindow), diff --git a/store/tikv/latch/latch_test.go b/store/tikv/latch/latch_test.go index 4b10c118883a6..ce53794d44f12 100644 --- a/store/tikv/latch/latch_test.go +++ b/store/tikv/latch/latch_test.go @@ -110,7 +110,7 @@ func (s *testLatchSuite) TestFirstAcquireFailedWithStale(c *C) { func (s *testLatchSuite) TestRecycle(c *C) { latches := NewLatches(8) now := time.Now() - startTS := oracle.ComposeTS(oracle.GetPhysical(now), 0) + startTS := oracle.GoTimeToTS(now) lock := latches.genLock(startTS, [][]byte{ []byte("a"), []byte("b"), }) @@ -142,7 +142,7 @@ func (s *testLatchSuite) TestRecycle(c *C) { } c.Assert(allEmpty, IsFalse) - currentTS := oracle.ComposeTS(oracle.GetPhysical(now.Add(expireDuration)), 3) + currentTS := oracle.GoTimeToTS(now.Add(expireDuration)) + 3 latches.recycle(currentTS) for i := 0; i < len(latches.slots); i++ { diff --git a/store/tikv/oracle/oracle.go b/store/tikv/oracle/oracle.go index 1b08129d412aa..fd95f1b357013 100644 --- a/store/tikv/oracle/oracle.go +++ b/store/tikv/oracle/oracle.go @@ -16,10 +16,6 @@ package oracle import ( "context" "time" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/store/tikv/logutil" - "go.uber.org/zap" ) // Option represents available options for the oracle.Oracle. @@ -53,19 +49,6 @@ const ( // ComposeTS creates a ts from physical and logical parts. func ComposeTS(physical, logical int64) uint64 { - failpoint.Inject("changeTSFromPD", func(val failpoint.Value) { - valInt, ok := val.(int) - if ok { - origPhyTS := physical - logical := logical - newPhyTs := origPhyTS + int64(valInt) - origTS := uint64((physical << physicalShiftBits) + logical) - newTS := uint64((newPhyTs << physicalShiftBits) + logical) - logutil.BgLogger().Warn("ComposeTS failpoint", zap.Uint64("origTS", origTS), - zap.Int("valInt", valInt), zap.Uint64("ts", newTS)) - failpoint.Return(newTS) - } - }) return uint64((physical << physicalShiftBits) + logical) } @@ -84,11 +67,6 @@ func GetPhysical(t time.Time) int64 { return t.UnixNano() / int64(time.Millisecond) } -// EncodeTSO encodes a millisecond into tso. -func EncodeTSO(ts int64) uint64 { - return uint64(ts) << physicalShiftBits -} - // GetTimeFromTS extracts time.Time from a timestamp. func GetTimeFromTS(ts uint64) time.Time { ms := ExtractPhysical(ts) diff --git a/store/tikv/oracle/oracles/local.go b/store/tikv/oracle/oracles/local.go index 4fcd7cbc51d78..8c7f8a30de645 100644 --- a/store/tikv/oracle/oracles/local.go +++ b/store/tikv/oracle/oracles/local.go @@ -42,7 +42,8 @@ func (l *localOracle) IsExpired(lockTS, TTL uint64, _ *oracle.Option) bool { if l.hook != nil { now = l.hook.currentTime } - return oracle.GetPhysical(now) >= oracle.ExtractPhysical(lockTS)+int64(TTL) + expire := oracle.GetTimeFromTS(lockTS).Add(time.Duration(TTL) * time.Millisecond) + return !now.Before(expire) } func (l *localOracle) GetTimestamp(ctx context.Context, _ *oracle.Option) (uint64, error) { @@ -52,8 +53,7 @@ func (l *localOracle) GetTimestamp(ctx context.Context, _ *oracle.Option) (uint6 if l.hook != nil { now = l.hook.currentTime } - physical := oracle.GetPhysical(now) - ts := oracle.ComposeTS(physical, 0) + ts := oracle.GoTimeToTS(now) if l.lastTimeStampTS == ts { l.n++ return ts + l.n, nil @@ -80,9 +80,7 @@ func (l *localOracle) GetLowResolutionTimestampAsync(ctx context.Context, opt *o // GetStaleTimestamp return physical func (l *localOracle) GetStaleTimestamp(ctx context.Context, txnScope string, prevSecond uint64) (ts uint64, err error) { - physical := oracle.GetPhysical(time.Now().Add(-time.Second * time.Duration(prevSecond))) - ts = oracle.ComposeTS(physical, 0) - return ts, nil + return oracle.GoTimeToTS(time.Now().Add(-time.Second * time.Duration(prevSecond))), nil } type future struct { diff --git a/store/tikv/oracle/oracles/mock.go b/store/tikv/oracle/oracles/mock.go index 2afd962fb5c42..b1eabe57feb37 100644 --- a/store/tikv/oracle/oracles/mock.go +++ b/store/tikv/oracle/oracles/mock.go @@ -62,9 +62,8 @@ func (o *MockOracle) GetTimestamp(ctx context.Context, _ *oracle.Option) (uint64 if o.stop { return 0, errors.Trace(errStopped) } - physical := oracle.GetPhysical(time.Now().Add(o.offset)) - ts := oracle.ComposeTS(physical, 0) - if oracle.ExtractPhysical(o.lastTS) == physical { + ts := oracle.GoTimeToTS(time.Now().Add(o.offset)) + if oracle.ExtractPhysical(o.lastTS) == oracle.ExtractPhysical(ts) { ts = o.lastTS + 1 } o.lastTS = ts @@ -73,9 +72,7 @@ func (o *MockOracle) GetTimestamp(ctx context.Context, _ *oracle.Option) (uint64 // GetStaleTimestamp implements oracle.Oracle interface. func (o *MockOracle) GetStaleTimestamp(ctx context.Context, txnScope string, prevSecond uint64) (ts uint64, err error) { - physical := oracle.GetPhysical(time.Now().Add(-time.Second * time.Duration(prevSecond))) - ts = oracle.ComposeTS(physical, 0) - return ts, nil + return oracle.GoTimeToTS(time.Now().Add(-time.Second * time.Duration(prevSecond))), nil } type mockOracleFuture struct { @@ -106,15 +103,16 @@ func (o *MockOracle) GetLowResolutionTimestampAsync(ctx context.Context, opt *or func (o *MockOracle) IsExpired(lockTimestamp, TTL uint64, _ *oracle.Option) bool { o.RLock() defer o.RUnlock() - - return oracle.GetPhysical(time.Now().Add(o.offset)) >= oracle.ExtractPhysical(lockTimestamp)+int64(TTL) + expire := oracle.GetTimeFromTS(lockTimestamp).Add(time.Duration(TTL) * time.Millisecond) + return !time.Now().Add(o.offset).Before(expire) } // UntilExpired implement oracle.Oracle interface. func (o *MockOracle) UntilExpired(lockTimeStamp, TTL uint64, _ *oracle.Option) int64 { o.RLock() defer o.RUnlock() - return oracle.ExtractPhysical(lockTimeStamp) + int64(TTL) - oracle.GetPhysical(time.Now().Add(o.offset)) + expire := oracle.GetTimeFromTS(lockTimeStamp).Add(time.Duration(TTL) * time.Millisecond) + return expire.Sub(time.Now().Add(o.offset)).Milliseconds() } // Close implements oracle.Oracle interface. diff --git a/store/tikv/oracle/oracles/pd.go b/store/tikv/oracle/oracles/pd.go index 063f73e343ce0..907dc278d71cb 100644 --- a/store/tikv/oracle/oracles/pd.go +++ b/store/tikv/oracle/oracles/pd.go @@ -135,7 +135,7 @@ func (o *pdOracle) getTimestamp(ctx context.Context, txnScope string) (uint64, e } func (o *pdOracle) getArrivalTimestamp() uint64 { - return oracle.ComposeTS(oracle.GetPhysical(time.Now()), 0) + return oracle.GoTimeToTS(time.Now()) } func (o *pdOracle) setLastTS(ts uint64, txnScope string) { @@ -288,7 +288,7 @@ func (o *pdOracle) getStaleTimestamp(txnScope string, prevSecond uint64) (uint64 staleTime := physicalTime.Add(-arrivalTime.Sub(time.Now().Add(-time.Duration(prevSecond) * time.Second))) - return oracle.ComposeTS(oracle.GetPhysical(staleTime), 0), nil + return oracle.GoTimeToTS(staleTime), nil } // GetStaleTimestamp generate a TSO which represents for the TSO prevSecond secs ago. diff --git a/store/tikv/oracle/oracles/pd_test.go b/store/tikv/oracle/oracles/pd_test.go index 4e881a82126b5..2894a782e505f 100644 --- a/store/tikv/oracle/oracles/pd_test.go +++ b/store/tikv/oracle/oracles/pd_test.go @@ -36,8 +36,8 @@ func (s *testPDSuite) TestPDOracle_UntilExpired(c *C) { lockAfter, lockExp := 10, 15 o := oracles.NewEmptyPDOracle() start := time.Now() - oracles.SetEmptyPDOracleLastTs(o, oracle.ComposeTS(oracle.GetPhysical(start), 0)) - lockTs := oracle.ComposeTS(oracle.GetPhysical(start.Add(time.Duration(lockAfter)*time.Millisecond)), 1) + oracles.SetEmptyPDOracleLastTs(o, oracle.GoTimeToTS(start)) + lockTs := oracle.GoTimeToTS((start.Add(time.Duration(lockAfter) * time.Millisecond))) + 1 waitTs := o.UntilExpired(lockTs, uint64(lockExp), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) c.Assert(waitTs, Equals, int64(lockAfter+lockExp), Commentf("waitTs shoulb be %d but got %d", int64(lockAfter+lockExp), waitTs)) } @@ -45,7 +45,7 @@ func (s *testPDSuite) TestPDOracle_UntilExpired(c *C) { func (s *testPDSuite) TestPdOracle_GetStaleTimestamp(c *C) { o := oracles.NewEmptyPDOracle() start := time.Now() - oracles.SetEmptyPDOracleLastTs(o, oracle.ComposeTS(oracle.GetPhysical(start), 0)) + oracles.SetEmptyPDOracleLastTs(o, oracle.GoTimeToTS(start)) ts, err := o.GetStaleTimestamp(context.Background(), oracle.GlobalTxnScope, 10) c.Assert(err, IsNil) @@ -75,7 +75,7 @@ func (s *testPDSuite) TestPdOracle_GetStaleTimestamp(c *C) { for _, testcase := range testcases { comment := Commentf("%s", testcase.name) start = time.Now() - oracles.SetEmptyPDOracleLastTs(o, oracle.ComposeTS(oracle.GetPhysical(start), 0)) + oracles.SetEmptyPDOracleLastTs(o, oracle.GoTimeToTS(start)) ts, err = o.GetStaleTimestamp(context.Background(), oracle.GlobalTxnScope, testcase.preSec) if testcase.expectErr == "" { c.Assert(err, IsNil, comment) diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 43b682160c514..12aa9466a3837 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -724,7 +724,7 @@ func (s *testCommitterSuite) TestPessimisticLockReturnValues(c *C) { func (s *testCommitterSuite) TestElapsedTTL(c *C) { key := []byte("key") txn := s.begin(c) - txn.SetStartTS(oracle.ComposeTS(oracle.GetPhysical(time.Now().Add(time.Second*10)), 1)) + txn.SetStartTS(oracle.GoTimeToTS(time.Now().Add(time.Second*10)) + 1) txn.SetPessimistic(true) time.Sleep(time.Millisecond * 100) lockCtx := &kv.LockCtx{ diff --git a/store/tikv/tests/store_test.go b/store/tikv/tests/store_test.go index 659dc6ea8f226..f8055a96e4fb9 100644 --- a/store/tikv/tests/store_test.go +++ b/store/tikv/tests/store_test.go @@ -19,7 +19,6 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" @@ -154,20 +153,3 @@ func (s *testStoreSuite) TestRequestPriority(c *C) { } iter.Close() } - -func (s *testStoreSerialSuite) TestOracleChangeByFailpoint(c *C) { - defer func() { - failpoint.Disable("github.com/pingcap/tidb/store/tikv/oracle/changeTSFromPD") - }() - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/oracle/changeTSFromPD", - "return(10000)"), IsNil) - o := &oracles.MockOracle{} - s.store.SetOracle(o) - ctx := context.Background() - t1, err := s.store.GetTimestampWithRetry(tikv.NewBackofferWithVars(ctx, 100, nil), oracle.GlobalTxnScope) - c.Assert(err, IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/oracle/changeTSFromPD"), IsNil) - t2, err := s.store.GetTimestampWithRetry(tikv.NewBackofferWithVars(ctx, 100, nil), oracle.GlobalTxnScope) - c.Assert(err, IsNil) - c.Assert(t1, Greater, t2) -} From 28f4d79a0ba5d2963ac767d0d38fe32f2b0a851d Mon Sep 17 00:00:00 2001 From: Zhuomin Liu Date: Thu, 20 May 2021 03:05:41 +0800 Subject: [PATCH 114/343] executor: fix wrong enum key in point get (#24618) --- executor/batch_point_get_test.go | 10 ++++++++++ executor/point_get.go | 12 ++++++++++++ 2 files changed, 22 insertions(+) diff --git a/executor/batch_point_get_test.go b/executor/batch_point_get_test.go index 926834dc9281b..8f8c39d4b0eed 100644 --- a/executor/batch_point_get_test.go +++ b/executor/batch_point_get_test.go @@ -156,6 +156,16 @@ func (s *testBatchPointGetSuite) TestIssue18843(c *C) { tk.MustQuery("select * from t18843 where f is null").Check(testkit.Rows("2 ")) } +func (s *testBatchPointGetSuite) TestIssue24562(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists ttt") + tk.MustExec("create table ttt(a enum(\"a\",\"b\",\"c\",\"d\"), primary key(a));") + tk.MustExec("insert into ttt values(1)") + tk.MustQuery("select * from ttt where ttt.a in (\"1\",\"b\")").Check(testkit.Rows()) + tk.MustQuery("select * from ttt where ttt.a in (1,\"b\")").Check(testkit.Rows("a")) +} + func (s *testBatchPointGetSuite) TestBatchPointGetUnsignedHandleWithSort(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/point_get.go b/executor/point_get.go index dccb72bdebb17..c5ff4b98fa2ba 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -441,6 +441,18 @@ func EncodeUniqueIndexValuesForKey(ctx sessionctx.Context, tblInfo *model.TableI var str string str, err = idxVals[i].ToString() idxVals[i].SetString(str, colInfo.FieldType.Collate) + } else if colInfo.Tp == mysql.TypeEnum && (idxVals[i].Kind() == types.KindString || idxVals[i].Kind() == types.KindBytes || idxVals[i].Kind() == types.KindBinaryLiteral) { + var str string + var e types.Enum + str, err = idxVals[i].ToString() + if err != nil { + return nil, kv.ErrNotExist + } + e, err = types.ParseEnumName(colInfo.FieldType.Elems, str, colInfo.FieldType.Collate) + if err != nil { + return nil, kv.ErrNotExist + } + idxVals[i].SetMysqlEnum(e, colInfo.FieldType.Collate) } else { // If a truncated error or an overflow error is thrown when converting the type of `idxVal[i]` to // the type of `colInfo`, the `idxVal` does not exist in the `idxInfo` for sure. From 651e041201fe2796e9cce3f5ee3cc541f150df27 Mon Sep 17 00:00:00 2001 From: Zhuomin Liu Date: Thu, 20 May 2021 03:17:41 +0800 Subject: [PATCH 115/343] ranger: fix incorrect enum range for xxx_ci collation (#24661) --- util/ranger/points.go | 2 +- util/ranger/ranger_test.go | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/util/ranger/points.go b/util/ranger/points.go index 46a4283dd3222..3931a2c42b9fe 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -460,7 +460,7 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val for i := range ft.Elems { tmpEnum.Name = ft.Elems[i] tmpEnum.Value = uint64(i) + 1 - d := types.NewMysqlEnumDatum(tmpEnum) + d := types.NewCollateMysqlEnumDatum(tmpEnum, ft.Collate) if v, err := d.CompareDatum(sc, &val); err == nil { switch op { case ast.LT: diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index ed4722566033a..b0b66e8a469c1 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -347,12 +347,14 @@ create table t( d varchar(10), e binary(10), f varchar(10) collate utf8mb4_general_ci, + g enum('A','B','C') collate utf8mb4_general_ci, index idx_ab(a(50), b), index idx_cb(c, a), index idx_d(d(2)), index idx_e(e(2)), index idx_f(f), - index idx_de(d(2), e) + index idx_de(d(2), e), + index idx_g(g) )`) tests := []struct { @@ -628,6 +630,13 @@ create table t( filterConds: "[in(test.t.d, aab, aac)]", resultStr: "[[\"aa\" 0x61,\"aa\" 0x61]]", }, + { + indexPos: 6, + exprStr: "g = 'a'", + accessConds: "[eq(test.t.g, a)]", + filterConds: "[]", + resultStr: "[[\"A\",\"A\"]]", + }, } collate.SetNewCollationEnabledForTest(true) From 20e467a0469af63cf253be2e8fccfcd8e8851c5f Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Thu, 20 May 2021 03:29:41 +0800 Subject: [PATCH 116/343] executor: add some test cases about dynamic-mode and apply operator (#24683) --- executor/partition_table_test.go | 144 ++++++++++++++++++++++++++++++- 1 file changed, 142 insertions(+), 2 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 101f429475748..446e689184086 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -240,8 +240,8 @@ func (s *partitionTableSuite) TestOrderByandLimit(c *C) { // range partition table tk.MustExec(`create table trange(a int, b int, index idx_a(a)) partition by range(a) ( - partition p0 values less than(300), - partition p1 values less than (500), + partition p0 values less than(300), + partition p1 values less than (500), partition p2 values less than(1100));`) // hash partition table @@ -1298,6 +1298,146 @@ func (s *partitionTableSuite) TestSplitRegion(c *C) { tk.MustPartition(`select * from thash where a in (1, 10001, 20001)`, "p1").Sort().Check(result) } +func (s *partitionTableSuite) TestParallelApply(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_parallel_apply") + tk.MustExec("use test_parallel_apply") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("set tidb_enable_parallel_apply=true") + + tk.MustExec(`create table touter (a int, b int)`) + tk.MustExec(`create table tinner (a int, b int, key(a))`) + tk.MustExec(`create table thash (a int, b int, key(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, key(a)) partition by range(a) ( + partition p0 values less than(10000), + partition p1 values less than(20000), + partition p2 values less than(30000), + partition p3 values less than(40000))`) + + vouter := make([]string, 0, 100) + for i := 0; i < 100; i++ { + vouter = append(vouter, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec("insert into touter values " + strings.Join(vouter, ", ")) + + vals := make([]string, 0, 2000) + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec("insert into tinner values " + strings.Join(vals, ", ")) + tk.MustExec("insert into thash values " + strings.Join(vals, ", ")) + tk.MustExec("insert into trange values " + strings.Join(vals, ", ")) + + // parallel apply + hash partition + IndexReader as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(thash.a) from thash use index(a) where thash.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─StreamAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─IndexReader 1.00 root partition:all index:StreamAgg`, // IndexReader is a inner child of Apply + ` └─StreamAgg 1.00 cop[tikv] funcs:sum(test_parallel_apply.thash.a)->Column#9`, + ` └─Selection 8000.00 cop[tikv] gt(test_parallel_apply.thash.a, test_parallel_apply.touter.b)`, + ` └─IndexFullScan 10000.00 cop[tikv] table:thash, index:a(a) keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(thash.a) from thash use index(a) where thash.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.a) from tinner use index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // parallel apply + hash partition + TableReader as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(thash.b) from thash ignore index(a) where thash.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─StreamAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─TableReader 1.00 root partition:all data:StreamAgg`, // TableReader is a inner child of Apply + ` └─StreamAgg 1.00 cop[tikv] funcs:sum(test_parallel_apply.thash.b)->Column#9`, + ` └─Selection 8000.00 cop[tikv] gt(test_parallel_apply.thash.a, test_parallel_apply.touter.b)`, + ` └─TableFullScan 10000.00 cop[tikv] table:thash keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(thash.b) from thash ignore index(a) where thash.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.b) from tinner ignore index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // parallel apply + hash partition + IndexLookUp as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(tinner.b) from tinner use index(a) where tinner.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─HashAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─IndexLookUp 1.00 root `, // IndexLookUp is a inner child of Apply + ` ├─Selection(Build) 8000.00 cop[tikv] gt(test_parallel_apply.tinner.a, test_parallel_apply.touter.b)`, + ` │ └─IndexFullScan 10000.00 cop[tikv] table:tinner, index:a(a) keep order:false, stats:pseudo`, + ` └─HashAgg(Probe) 1.00 cop[tikv] funcs:sum(test_parallel_apply.tinner.b)->Column#9`, + ` └─TableRowIDScan 8000.00 cop[tikv] table:tinner keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(thash.b) from thash use index(a) where thash.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.b) from tinner use index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // parallel apply + range partition + IndexReader as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(trange.a) from trange use index(a) where trange.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─StreamAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─IndexReader 1.00 root partition:all index:StreamAgg`, // IndexReader is a inner child of Apply + ` └─StreamAgg 1.00 cop[tikv] funcs:sum(test_parallel_apply.trange.a)->Column#9`, + ` └─Selection 8000.00 cop[tikv] gt(test_parallel_apply.trange.a, test_parallel_apply.touter.b)`, + ` └─IndexFullScan 10000.00 cop[tikv] table:trange, index:a(a) keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(trange.a) from trange use index(a) where trange.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.a) from tinner use index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // parallel apply + range partition + TableReader as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(trange.b) from trange ignore index(a) where trange.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─StreamAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─TableReader 1.00 root partition:all data:StreamAgg`, // TableReader is a inner child of Apply + ` └─StreamAgg 1.00 cop[tikv] funcs:sum(test_parallel_apply.trange.b)->Column#9`, + ` └─Selection 8000.00 cop[tikv] gt(test_parallel_apply.trange.a, test_parallel_apply.touter.b)`, + ` └─TableFullScan 10000.00 cop[tikv] table:trange keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(trange.b) from trange ignore index(a) where trange.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.b) from tinner ignore index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // parallel apply + range partition + IndexLookUp as its inner child + tk.MustQuery(`explain format='brief' select * from touter where touter.a > (select sum(tinner.b) from tinner use index(a) where tinner.a>touter.b)`).Check(testkit.Rows( + `Projection 10000.00 root test_parallel_apply.touter.a, test_parallel_apply.touter.b`, + `└─Apply 10000.00 root CARTESIAN inner join, other cond:gt(cast(test_parallel_apply.touter.a, decimal(20,0) BINARY), Column#7)`, + ` ├─TableReader(Build) 10000.00 root data:TableFullScan`, + ` │ └─TableFullScan 10000.00 cop[tikv] table:touter keep order:false, stats:pseudo`, + ` └─HashAgg(Probe) 1.00 root funcs:sum(Column#9)->Column#7`, + ` └─IndexLookUp 1.00 root `, // IndexLookUp is a inner child of Apply + ` ├─Selection(Build) 8000.00 cop[tikv] gt(test_parallel_apply.tinner.a, test_parallel_apply.touter.b)`, + ` │ └─IndexFullScan 10000.00 cop[tikv] table:tinner, index:a(a) keep order:false, stats:pseudo`, + ` └─HashAgg(Probe) 1.00 cop[tikv] funcs:sum(test_parallel_apply.tinner.b)->Column#9`, + ` └─TableRowIDScan 8000.00 cop[tikv] table:tinner keep order:false, stats:pseudo`)) + tk.MustQuery(`select * from touter where touter.a > (select sum(trange.b) from trange use index(a) where trange.a>touter.b)`).Sort().Check( + tk.MustQuery(`select * from touter where touter.a > (select sum(tinner.b) from tinner use index(a) where tinner.a>touter.b)`).Sort().Rows()) + + // random queries + ops := []string{"!=", ">", "<", ">=", "<="} + aggFuncs := []string{"sum", "count", "max", "min"} + tbls := []string{"tinner", "thash", "trange"} + for i := 0; i < 50; i++ { + var r [][]interface{} + op := ops[rand.Intn(len(ops))] + agg := aggFuncs[rand.Intn(len(aggFuncs))] + x := rand.Intn(10000) + for _, tbl := range tbls { + q := fmt.Sprintf(`select * from touter where touter.a > (select %v(%v.b) from %v where %v.a%vtouter.b-%v)`, agg, tbl, tbl, tbl, op, x) + if r == nil { + r = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Sort().Check(r) + } + } + } +} + func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From b54603637c49cd2290fd0b49a47faf7be8d3163f Mon Sep 17 00:00:00 2001 From: disksing Date: Thu, 20 May 2021 04:49:40 +0800 Subject: [PATCH 117/343] store/tikv: remove Variables.Hook (#24758) --- session/session_fail_test.go | 23 +++++++++++------------ store/tikv/kv/variables.go | 3 --- store/tikv/region_request.go | 2 -- store/tikv/retry/backoff.go | 10 ---------- 4 files changed, 11 insertions(+), 27 deletions(-) diff --git a/session/session_fail_test.go b/session/session_fail_test.go index 12f49e0ed1abf..3488592051b9f 100644 --- a/session/session_fail_test.go +++ b/session/session_fail_test.go @@ -15,11 +15,11 @@ package session_test import ( "context" - "sync/atomic" + "strings" . "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/util/testkit" ) @@ -79,20 +79,19 @@ func (s *testSessionSerialSuite) TestGetTSFailDirtyStateInretry(c *C) { func (s *testSessionSerialSuite) TestKillFlagInBackoff(c *C) { // This test checks the `killed` flag is passed down to the backoffer through - // session.KVVars. It works by setting the `killed = 3` first, then using - // failpoint to run backoff() and check the vars.Killed using the Hook() function. + // session.KVVars. tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table kill_backoff (id int)") - var killValue uint32 - tk.Se.GetSessionVars().KVVars.Hook = func(name string, vars *tikv.Variables) { - killValue = atomic.LoadUint32(vars.Killed) - } - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult", `return("callBackofferHook")`), IsNil) + // Inject 1 time timeout. If `Killed` is not successfully passed, it will retry and complete query. + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult", `return("timeout")->return("")`), IsNil) defer failpoint.Disable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult") // Set kill flag and check its passed to backoffer. - tk.Se.GetSessionVars().Killed = 3 - tk.MustQuery("select * from kill_backoff") - c.Assert(killValue, Equals, uint32(3)) + tk.Se.GetSessionVars().Killed = 1 + rs, err := tk.Exec("select * from kill_backoff") + c.Assert(err, IsNil) + _, err = session.ResultSetToStringSlice(context.TODO(), tk.Se, rs) + // `interrupted` is returned when `Killed` is set. + c.Assert(strings.Contains(err.Error(), "Query execution was interrupted"), IsTrue) } func (s *testSessionSerialSuite) TestClusterTableSendError(c *C) { diff --git a/store/tikv/kv/variables.go b/store/tikv/kv/variables.go index b722023bcae08..5e7a4c83b669a 100644 --- a/store/tikv/kv/variables.go +++ b/store/tikv/kv/variables.go @@ -21,9 +21,6 @@ type Variables struct { // BackOffWeight specifies the weight of the max back off time duration. BackOffWeight int - // Hook is used for test to verify the variable take effect. - Hook func(name string, vars *Variables) - // Pointer to SessionVars.Killed // Killed is a flag to indicate that this query is killed. Killed *uint32 diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index b8b61aac05fc8..f42f7add092db 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -248,8 +248,6 @@ func (s *RegionRequestSender) SendReqCtx( Resp: &kvrpcpb.GCResponse{RegionError: &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}}}, }, nil, nil) } - case "callBackofferHook": - bo.SetVarsHook("callBackofferHook", bo.GetVars()) case "requestTiDBStoreError": if et == tikvrpc.TiDB { failpoint.Return(nil, nil, tikverr.ErrTiKVServerTimeout) diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go index a563ec7359d22..9e7a527c69caa 100644 --- a/store/tikv/retry/backoff.go +++ b/store/tikv/retry/backoff.go @@ -135,9 +135,6 @@ const ( ) func (t BackoffType) createFn(vars *kv.Variables) func(context.Context, int) int { - if vars.Hook != nil { - vars.Hook(t.String(), vars) - } switch t { case boTiKVRPC, BoTiFlashRPC: return NewBackoffFn(100, 2000, EqualJitter) @@ -431,10 +428,3 @@ func (b *Backoffer) GetBackoffSleepMS() map[BackoffType]int { func (b *Backoffer) ErrorsNum() int { return len(b.errors) } - -// SetVarsHook sets the vars.Hook is used for test to verify the variable take effect. -func (b *Backoffer) SetVarsHook(name string, vars *kv.Variables) { - if b.vars != nil && b.vars.Hook != nil { - b.vars.Hook(name, vars) - } -} From c8c2726f3e7bae9f24758bb590fb2d08e142988f Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 20 May 2021 05:51:40 +0800 Subject: [PATCH 118/343] ddl: speed up the execution time of `TestBackwardCompatibility`. (#24704) --- ddl/db_integration_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 0c2f851da4413..8a49182e92f08 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -1233,7 +1233,7 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { // Split the table. tableStart := tablecodec.GenTableRecordPrefix(tbl.Meta().ID) - s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 100) + s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 10) unique := false indexName := model.NewCIStr("idx_b") @@ -1275,7 +1275,6 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { historyJob, err := getHistoryDDLJob(s.store, job.ID) c.Assert(err, IsNil) if historyJob == nil { - continue } c.Assert(historyJob.Error, IsNil) From 542da5d60bb6d7e63e694e908d11f8bde3bce920 Mon Sep 17 00:00:00 2001 From: wjHuang Date: Thu, 20 May 2021 06:03:40 +0800 Subject: [PATCH 119/343] *: prepare errors for CTE (#24763) --- errno/errcode.go | 6 ++ errno/errname.go | 8 ++- errors.toml | 32 +++++++++- executor/errors.go | 9 +-- planner/core/errors.go | 141 +++++++++++++++++++++-------------------- 5 files changed, 122 insertions(+), 74 deletions(-) diff --git a/errno/errcode.go b/errno/errcode.go index 2ed488242dd10..1cb4889eccbc1 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -855,6 +855,11 @@ const ( ErrGrantRole = 3523 ErrRoleNotGranted = 3530 ErrLockAcquireFailAndNoWaitSet = 3572 + ErrCTERecursiveRequiresUnion = 3573 + ErrCTERecursiveRequiresNonRecursiveFirst = 3574 + ErrCTERecursiveForbidsAggregation = 3575 + ErrCTERecursiveForbiddenJoinOrder = 3576 + ErrInvalidRequiresSingleReference = 3577 ErrWindowNoSuchWindow = 3579 ErrWindowCircularityInWindowGraph = 3580 ErrWindowNoChildPartitioning = 3581 @@ -877,6 +882,7 @@ const ( ErrWindowExplainJSON = 3598 ErrWindowFunctionIgnoresFrame = 3599 ErrIllegalPrivilegeLevel = 3619 + ErrCTEMaxRecursionDepth = 3636 ErrNotHintUpdatable = 3637 ErrDataTruncatedFunctionalIndex = 3751 ErrDataOutOfRangeFunctionalIndex = 3752 diff --git a/errno/errname.go b/errno/errname.go index 62662ce5ac934..98cbb17cd6b25 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -355,7 +355,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrViewSelectClause: mysql.Message("View's SELECT contains a '%s' clause", nil), ErrViewSelectVariable: mysql.Message("View's SELECT contains a variable or parameter", nil), ErrViewSelectTmptable: mysql.Message("View's SELECT refers to a temporary table '%-.192s'", nil), - ErrViewWrongList: mysql.Message("View's SELECT and view's field list have different column counts", nil), + ErrViewWrongList: mysql.Message("In definition of view, derived table or common table expression, SELECT list and column names list have different column counts", nil), ErrWarnViewMerge: mysql.Message("View merge algorithm can't be used here for now (assumed undefined algorithm)", nil), ErrWarnViewWithoutKey: mysql.Message("View being updated does not have complete key of underlying table in it", nil), ErrViewInvalid: mysql.Message("View '%-.192s.%-.192s' references invalid table(s) or column(s) or function(s) or definer/invoker of view lack rights to use them", nil), @@ -902,6 +902,12 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrUnsupportedConstraintCheck: mysql.Message("%s is not supported", nil), ErrDynamicPrivilegeNotRegistered: mysql.Message("Dynamic privilege '%s' is not registered with the server.", nil), ErrIllegalPrivilegeLevel: mysql.Message("Illegal privilege level specified for %s", nil), + ErrCTERecursiveRequiresUnion: mysql.Message("Recursive Common Table Expression '%s' should contain a UNION", nil), + ErrCTERecursiveRequiresNonRecursiveFirst: mysql.Message("Recursive Common Table Expression '%s' should have one or more non-recursive query blocks followed by one or more recursive ones", nil), + ErrCTERecursiveForbidsAggregation: mysql.Message("Recursive Common Table Expression '%s' can contain neither aggregation nor window functions in recursive query block", nil), + ErrCTERecursiveForbiddenJoinOrder: mysql.Message("In recursive query block of Recursive Common Table Expression '%s', the recursive table must neither be in the right argument of a LEFT JOIN, nor be forced to be non-first with join order hints", nil), + ErrInvalidRequiresSingleReference: mysql.Message("In recursive query block of Recursive Common Table Expression '%s', the recursive table must be referenced only once, and not in any subquery", nil), + ErrCTEMaxRecursionDepth: mysql.Message("Recursive query aborted after %d iterations. Try increasing @@cte_max_recursion_depth to a larger value", nil), // MariaDB errors. ErrOnlyOneDefaultPartionAllowed: mysql.Message("Only one DEFAULT partition allowed", nil), ErrWrongPartitionTypeExpectedSystemTime: mysql.Message("Wrong partitioning type, expected type: `SYSTEM_TIME`", nil), diff --git a/errors.toml b/errors.toml index 926823909f96e..3d8b98f2368bb 100644 --- a/errors.toml +++ b/errors.toml @@ -193,7 +193,7 @@ View's SELECT contains a '%s' clause ["ddl:1353"] error = ''' -View's SELECT and view's field list have different column counts +In definition of view, derived table or common table expression, SELECT list and column names list have different column counts ''' ["ddl:1481"] @@ -561,6 +561,11 @@ error = ''' Illegal privilege level specified for %s ''' +["executor:3636"] +error = ''' +Recursive query aborted after %d iterations. Try increasing @@cte_max_recursion_depth to a larger value +''' + ["executor:3929"] error = ''' Dynamic privilege '%s' is not registered with the server. @@ -1016,6 +1021,31 @@ error = ''' Unresolved name '%s' for %s hint ''' +["planner:3573"] +error = ''' +Recursive Common Table Expression '%s' should contain a UNION +''' + +["planner:3574"] +error = ''' +Recursive Common Table Expression '%s' should have one or more non-recursive query blocks followed by one or more recursive ones +''' + +["planner:3575"] +error = ''' +Recursive Common Table Expression '%s' can contain neither aggregation nor window functions in recursive query block +''' + +["planner:3576"] +error = ''' +In recursive query block of Recursive Common Table Expression '%s', the recursive table must neither be in the right argument of a LEFT JOIN, nor be forced to be non-first with join order hints +''' + +["planner:3577"] +error = ''' +In recursive query block of Recursive Common Table Expression '%s', the recursive table must be referenced only once, and not in any subquery +''' + ["planner:3579"] error = ''' Window name '%s' is not defined. diff --git a/executor/errors.go b/executor/errors.go index ad8104a96e7ee..94237808d1562 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -50,8 +50,9 @@ var ( ErrIllegalPrivilegeLevel = dbterror.ClassExecutor.NewStd(mysql.ErrIllegalPrivilegeLevel) ErrInvalidSplitRegionRanges = dbterror.ClassExecutor.NewStd(mysql.ErrInvalidSplitRegionRanges) - ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed) - ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed) - ErrBRIEImportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEImportFailed) - ErrBRIEExportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEExportFailed) + ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed) + ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed) + ErrBRIEImportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEImportFailed) + ErrBRIEExportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEExportFailed) + ErrCTEMaxRecursionDepth = dbterror.ClassExecutor.NewStd(mysql.ErrCTEMaxRecursionDepth) ) diff --git a/planner/core/errors.go b/planner/core/errors.go index c713aab6367c1..66d7c17e8a7a7 100644 --- a/planner/core/errors.go +++ b/planner/core/errors.go @@ -20,74 +20,79 @@ import ( // error definitions. var ( - ErrUnsupportedType = dbterror.ClassOptimizer.NewStd(mysql.ErrUnsupportedType) - ErrAnalyzeMissIndex = dbterror.ClassOptimizer.NewStd(mysql.ErrAnalyzeMissIndex) - ErrWrongParamCount = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongParamCount) - ErrSchemaChanged = dbterror.ClassOptimizer.NewStd(mysql.ErrSchemaChanged) - ErrTablenameNotAllowedHere = dbterror.ClassOptimizer.NewStd(mysql.ErrTablenameNotAllowedHere) - ErrNotSupportedYet = dbterror.ClassOptimizer.NewStd(mysql.ErrNotSupportedYet) - ErrWrongUsage = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongUsage) - ErrUnknown = dbterror.ClassOptimizer.NewStd(mysql.ErrUnknown) - ErrUnknownTable = dbterror.ClassOptimizer.NewStd(mysql.ErrUnknownTable) - ErrNoSuchTable = dbterror.ClassOptimizer.NewStd(mysql.ErrNoSuchTable) - ErrViewRecursive = dbterror.ClassOptimizer.NewStd(mysql.ErrViewRecursive) - ErrWrongArguments = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongArguments) - ErrWrongNumberOfColumnsInSelect = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongNumberOfColumnsInSelect) - ErrBadGeneratedColumn = dbterror.ClassOptimizer.NewStd(mysql.ErrBadGeneratedColumn) - ErrFieldNotInGroupBy = dbterror.ClassOptimizer.NewStd(mysql.ErrFieldNotInGroupBy) - ErrAggregateOrderNonAggQuery = dbterror.ClassOptimizer.NewStd(mysql.ErrAggregateOrderNonAggQuery) - ErrFieldInOrderNotSelect = dbterror.ClassOptimizer.NewStd(mysql.ErrFieldInOrderNotSelect) - ErrAggregateInOrderNotSelect = dbterror.ClassOptimizer.NewStd(mysql.ErrAggregateInOrderNotSelect) - ErrBadTable = dbterror.ClassOptimizer.NewStd(mysql.ErrBadTable) - ErrKeyDoesNotExist = dbterror.ClassOptimizer.NewStd(mysql.ErrKeyDoesNotExist) - ErrOperandColumns = dbterror.ClassOptimizer.NewStd(mysql.ErrOperandColumns) - ErrInvalidGroupFuncUse = dbterror.ClassOptimizer.NewStd(mysql.ErrInvalidGroupFuncUse) - ErrIllegalReference = dbterror.ClassOptimizer.NewStd(mysql.ErrIllegalReference) - ErrNoDB = dbterror.ClassOptimizer.NewStd(mysql.ErrNoDB) - ErrUnknownExplainFormat = dbterror.ClassOptimizer.NewStd(mysql.ErrUnknownExplainFormat) - ErrWrongGroupField = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongGroupField) - ErrDupFieldName = dbterror.ClassOptimizer.NewStd(mysql.ErrDupFieldName) - ErrNonUpdatableTable = dbterror.ClassOptimizer.NewStd(mysql.ErrNonUpdatableTable) - ErrMultiUpdateKeyConflict = dbterror.ClassOptimizer.NewStd(mysql.ErrMultiUpdateKeyConflict) - ErrInternal = dbterror.ClassOptimizer.NewStd(mysql.ErrInternal) - ErrNonUniqTable = dbterror.ClassOptimizer.NewStd(mysql.ErrNonuniqTable) - ErrWindowInvalidWindowFuncUse = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowInvalidWindowFuncUse) - ErrWindowInvalidWindowFuncAliasUse = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowInvalidWindowFuncAliasUse) - ErrWindowNoSuchWindow = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowNoSuchWindow) - ErrWindowCircularityInWindowGraph = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowCircularityInWindowGraph) - ErrWindowNoChildPartitioning = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowNoChildPartitioning) - ErrWindowNoInherentFrame = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowNoInherentFrame) - ErrWindowNoRedefineOrderBy = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowNoRedefineOrderBy) - ErrWindowDuplicateName = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowDuplicateName) - ErrPartitionClauseOnNonpartitioned = dbterror.ClassOptimizer.NewStd(mysql.ErrPartitionClauseOnNonpartitioned) - ErrWindowFrameStartIllegal = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowFrameStartIllegal) - ErrWindowFrameEndIllegal = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowFrameEndIllegal) - ErrWindowFrameIllegal = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowFrameIllegal) - ErrWindowRangeFrameOrderType = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowRangeFrameOrderType) - ErrWindowRangeFrameTemporalType = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowRangeFrameTemporalType) - ErrWindowRangeFrameNumericType = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowRangeFrameNumericType) - ErrWindowRangeBoundNotConstant = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowRangeBoundNotConstant) - ErrWindowRowsIntervalUse = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowRowsIntervalUse) - ErrWindowFunctionIgnoresFrame = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowFunctionIgnoresFrame) - ErrUnsupportedOnGeneratedColumn = dbterror.ClassOptimizer.NewStd(mysql.ErrUnsupportedOnGeneratedColumn) - ErrPrivilegeCheckFail = dbterror.ClassOptimizer.NewStd(mysql.ErrPrivilegeCheckFail) - ErrInvalidWildCard = dbterror.ClassOptimizer.NewStd(mysql.ErrInvalidWildCard) - ErrMixOfGroupFuncAndFields = dbterror.ClassOptimizer.NewStd(mysql.ErrMixOfGroupFuncAndFieldsIncompatible) - errTooBigPrecision = dbterror.ClassExpression.NewStd(mysql.ErrTooBigPrecision) - ErrDBaccessDenied = dbterror.ClassOptimizer.NewStd(mysql.ErrDBaccessDenied) - ErrTableaccessDenied = dbterror.ClassOptimizer.NewStd(mysql.ErrTableaccessDenied) - ErrSpecificAccessDenied = dbterror.ClassOptimizer.NewStd(mysql.ErrSpecificAccessDenied) - ErrViewNoExplain = dbterror.ClassOptimizer.NewStd(mysql.ErrViewNoExplain) - ErrWrongValueCountOnRow = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongValueCountOnRow) - ErrViewInvalid = dbterror.ClassOptimizer.NewStd(mysql.ErrViewInvalid) - ErrNoSuchThread = dbterror.ClassOptimizer.NewStd(mysql.ErrNoSuchThread) - ErrUnknownColumn = dbterror.ClassOptimizer.NewStd(mysql.ErrBadField) - ErrCartesianProductUnsupported = dbterror.ClassOptimizer.NewStd(mysql.ErrCartesianProductUnsupported) - ErrStmtNotFound = dbterror.ClassOptimizer.NewStd(mysql.ErrPreparedStmtNotFound) - ErrAmbiguous = dbterror.ClassOptimizer.NewStd(mysql.ErrNonUniq) - ErrUnresolvedHintName = dbterror.ClassOptimizer.NewStd(mysql.ErrUnresolvedHintName) - ErrNotHintUpdatable = dbterror.ClassOptimizer.NewStd(mysql.ErrNotHintUpdatable) - ErrWarnConflictingHint = dbterror.ClassOptimizer.NewStd(mysql.ErrWarnConflictingHint) + ErrUnsupportedType = dbterror.ClassOptimizer.NewStd(mysql.ErrUnsupportedType) + ErrAnalyzeMissIndex = dbterror.ClassOptimizer.NewStd(mysql.ErrAnalyzeMissIndex) + ErrWrongParamCount = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongParamCount) + ErrSchemaChanged = dbterror.ClassOptimizer.NewStd(mysql.ErrSchemaChanged) + ErrTablenameNotAllowedHere = dbterror.ClassOptimizer.NewStd(mysql.ErrTablenameNotAllowedHere) + ErrNotSupportedYet = dbterror.ClassOptimizer.NewStd(mysql.ErrNotSupportedYet) + ErrWrongUsage = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongUsage) + ErrUnknown = dbterror.ClassOptimizer.NewStd(mysql.ErrUnknown) + ErrUnknownTable = dbterror.ClassOptimizer.NewStd(mysql.ErrUnknownTable) + ErrNoSuchTable = dbterror.ClassOptimizer.NewStd(mysql.ErrNoSuchTable) + ErrViewRecursive = dbterror.ClassOptimizer.NewStd(mysql.ErrViewRecursive) + ErrWrongArguments = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongArguments) + ErrWrongNumberOfColumnsInSelect = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongNumberOfColumnsInSelect) + ErrBadGeneratedColumn = dbterror.ClassOptimizer.NewStd(mysql.ErrBadGeneratedColumn) + ErrFieldNotInGroupBy = dbterror.ClassOptimizer.NewStd(mysql.ErrFieldNotInGroupBy) + ErrAggregateOrderNonAggQuery = dbterror.ClassOptimizer.NewStd(mysql.ErrAggregateOrderNonAggQuery) + ErrFieldInOrderNotSelect = dbterror.ClassOptimizer.NewStd(mysql.ErrFieldInOrderNotSelect) + ErrAggregateInOrderNotSelect = dbterror.ClassOptimizer.NewStd(mysql.ErrAggregateInOrderNotSelect) + ErrBadTable = dbterror.ClassOptimizer.NewStd(mysql.ErrBadTable) + ErrKeyDoesNotExist = dbterror.ClassOptimizer.NewStd(mysql.ErrKeyDoesNotExist) + ErrOperandColumns = dbterror.ClassOptimizer.NewStd(mysql.ErrOperandColumns) + ErrInvalidGroupFuncUse = dbterror.ClassOptimizer.NewStd(mysql.ErrInvalidGroupFuncUse) + ErrIllegalReference = dbterror.ClassOptimizer.NewStd(mysql.ErrIllegalReference) + ErrNoDB = dbterror.ClassOptimizer.NewStd(mysql.ErrNoDB) + ErrUnknownExplainFormat = dbterror.ClassOptimizer.NewStd(mysql.ErrUnknownExplainFormat) + ErrWrongGroupField = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongGroupField) + ErrDupFieldName = dbterror.ClassOptimizer.NewStd(mysql.ErrDupFieldName) + ErrNonUpdatableTable = dbterror.ClassOptimizer.NewStd(mysql.ErrNonUpdatableTable) + ErrMultiUpdateKeyConflict = dbterror.ClassOptimizer.NewStd(mysql.ErrMultiUpdateKeyConflict) + ErrInternal = dbterror.ClassOptimizer.NewStd(mysql.ErrInternal) + ErrNonUniqTable = dbterror.ClassOptimizer.NewStd(mysql.ErrNonuniqTable) + ErrWindowInvalidWindowFuncUse = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowInvalidWindowFuncUse) + ErrWindowInvalidWindowFuncAliasUse = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowInvalidWindowFuncAliasUse) + ErrWindowNoSuchWindow = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowNoSuchWindow) + ErrWindowCircularityInWindowGraph = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowCircularityInWindowGraph) + ErrWindowNoChildPartitioning = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowNoChildPartitioning) + ErrWindowNoInherentFrame = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowNoInherentFrame) + ErrWindowNoRedefineOrderBy = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowNoRedefineOrderBy) + ErrWindowDuplicateName = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowDuplicateName) + ErrPartitionClauseOnNonpartitioned = dbterror.ClassOptimizer.NewStd(mysql.ErrPartitionClauseOnNonpartitioned) + ErrWindowFrameStartIllegal = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowFrameStartIllegal) + ErrWindowFrameEndIllegal = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowFrameEndIllegal) + ErrWindowFrameIllegal = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowFrameIllegal) + ErrWindowRangeFrameOrderType = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowRangeFrameOrderType) + ErrWindowRangeFrameTemporalType = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowRangeFrameTemporalType) + ErrWindowRangeFrameNumericType = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowRangeFrameNumericType) + ErrWindowRangeBoundNotConstant = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowRangeBoundNotConstant) + ErrWindowRowsIntervalUse = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowRowsIntervalUse) + ErrWindowFunctionIgnoresFrame = dbterror.ClassOptimizer.NewStd(mysql.ErrWindowFunctionIgnoresFrame) + ErrUnsupportedOnGeneratedColumn = dbterror.ClassOptimizer.NewStd(mysql.ErrUnsupportedOnGeneratedColumn) + ErrPrivilegeCheckFail = dbterror.ClassOptimizer.NewStd(mysql.ErrPrivilegeCheckFail) + ErrInvalidWildCard = dbterror.ClassOptimizer.NewStd(mysql.ErrInvalidWildCard) + ErrMixOfGroupFuncAndFields = dbterror.ClassOptimizer.NewStd(mysql.ErrMixOfGroupFuncAndFieldsIncompatible) + errTooBigPrecision = dbterror.ClassExpression.NewStd(mysql.ErrTooBigPrecision) + ErrDBaccessDenied = dbterror.ClassOptimizer.NewStd(mysql.ErrDBaccessDenied) + ErrTableaccessDenied = dbterror.ClassOptimizer.NewStd(mysql.ErrTableaccessDenied) + ErrSpecificAccessDenied = dbterror.ClassOptimizer.NewStd(mysql.ErrSpecificAccessDenied) + ErrViewNoExplain = dbterror.ClassOptimizer.NewStd(mysql.ErrViewNoExplain) + ErrWrongValueCountOnRow = dbterror.ClassOptimizer.NewStd(mysql.ErrWrongValueCountOnRow) + ErrViewInvalid = dbterror.ClassOptimizer.NewStd(mysql.ErrViewInvalid) + ErrNoSuchThread = dbterror.ClassOptimizer.NewStd(mysql.ErrNoSuchThread) + ErrUnknownColumn = dbterror.ClassOptimizer.NewStd(mysql.ErrBadField) + ErrCartesianProductUnsupported = dbterror.ClassOptimizer.NewStd(mysql.ErrCartesianProductUnsupported) + ErrStmtNotFound = dbterror.ClassOptimizer.NewStd(mysql.ErrPreparedStmtNotFound) + ErrAmbiguous = dbterror.ClassOptimizer.NewStd(mysql.ErrNonUniq) + ErrUnresolvedHintName = dbterror.ClassOptimizer.NewStd(mysql.ErrUnresolvedHintName) + ErrNotHintUpdatable = dbterror.ClassOptimizer.NewStd(mysql.ErrNotHintUpdatable) + ErrWarnConflictingHint = dbterror.ClassOptimizer.NewStd(mysql.ErrWarnConflictingHint) + ErrCTERecursiveRequiresUnion = dbterror.ClassOptimizer.NewStd(mysql.ErrCTERecursiveRequiresUnion) + ErrCTERecursiveRequiresNonRecursiveFirst = dbterror.ClassOptimizer.NewStd(mysql.ErrCTERecursiveRequiresNonRecursiveFirst) + ErrCTERecursiveForbidsAggregation = dbterror.ClassOptimizer.NewStd(mysql.ErrCTERecursiveForbidsAggregation) + ErrCTERecursiveForbiddenJoinOrder = dbterror.ClassOptimizer.NewStd(mysql.ErrCTERecursiveForbiddenJoinOrder) + ErrInvalidRequiresSingleReference = dbterror.ClassOptimizer.NewStd(mysql.ErrInvalidRequiresSingleReference) // Since we cannot know if user logged in with a password, use message of ErrAccessDeniedNoPassword instead ErrAccessDenied = dbterror.ClassOptimizer.NewStdErr(mysql.ErrAccessDenied, mysql.MySQLErrName[mysql.ErrAccessDeniedNoPassword]) ErrBadNull = dbterror.ClassOptimizer.NewStd(mysql.ErrBadNull) From 5be8a6563399e7aaf0833cc64c9599611a3252b0 Mon Sep 17 00:00:00 2001 From: Zhuhe Fang Date: Thu, 20 May 2021 18:00:22 +0800 Subject: [PATCH 120/343] expression: support cast real/int as real (#24670) --- expression/expression.go | 2 +- planner/core/testdata/integration_serial_suite_out.json | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/expression/expression.go b/expression/expression.go index de6bb1cab5e54..20178919a89b4 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1018,7 +1018,7 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { tipb.ScalarFuncSig_CastRealAsInt, tipb.ScalarFuncSig_CastRealAsDecimal, tipb.ScalarFuncSig_CastRealAsString, tipb.ScalarFuncSig_CastRealAsTime, tipb.ScalarFuncSig_CastStringAsInt, tipb.ScalarFuncSig_CastStringAsDecimal, tipb.ScalarFuncSig_CastStringAsString, tipb.ScalarFuncSig_CastStringAsTime, tipb.ScalarFuncSig_CastDecimalAsInt, tipb.ScalarFuncSig_CastDecimalAsDecimal, tipb.ScalarFuncSig_CastDecimalAsString, tipb.ScalarFuncSig_CastDecimalAsTime, - tipb.ScalarFuncSig_CastTimeAsInt, tipb.ScalarFuncSig_CastTimeAsDecimal, tipb.ScalarFuncSig_CastTimeAsTime: + tipb.ScalarFuncSig_CastTimeAsInt, tipb.ScalarFuncSig_CastTimeAsDecimal, tipb.ScalarFuncSig_CastTimeAsTime, tipb.ScalarFuncSig_CastIntAsReal, tipb.ScalarFuncSig_CastRealAsReal: return true } case ast.DateAdd: diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 1f25f899a68d9..026ef6ea1bce7 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -13,8 +13,8 @@ { "SQL": "explain format = 'brief' select * from t where cast(t.a as float) + 3 = 5.1", "Plan": [ - "Selection 10000.00 root eq(plus(cast(test.t.a, float BINARY), 3), 5.1)", - "└─TableReader 10000.00 root data:TableFullScan", + "TableReader 8000.00 root data:Selection", + "└─Selection 8000.00 cop[tiflash] eq(plus(cast(test.t.a, float BINARY), 3), 5.1)", " └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ] }, From 013b3f1f498d04305941dcde94a7add35375a050 Mon Sep 17 00:00:00 2001 From: Cheese Date: Thu, 20 May 2021 18:34:22 +0800 Subject: [PATCH 121/343] executor: add table name in log (#24666) --- executor/distsql.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/executor/distsql.go b/executor/distsql.go index bd422a0458ef1..5e83a63d8f9b1 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -1212,7 +1212,9 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er obtainedHandlesMap.Set(handle, true) } - logutil.Logger(ctx).Error("inconsistent index handles", zap.String("index", w.idxLookup.index.Name.O), + logutil.Logger(ctx).Error("inconsistent index handles", + zap.String("table_name", w.idxLookup.index.Table.O), + zap.String("index", w.idxLookup.index.Name.O), zap.Int("index_cnt", handleCnt), zap.Int("table_cnt", len(task.rows)), zap.String("missing_handles", fmt.Sprint(GetLackHandles(task.handles, obtainedHandlesMap))), zap.String("total_handles", fmt.Sprint(task.handles))) From ca23f55d8975423930f3f7b09ad6b3de0cc90977 Mon Sep 17 00:00:00 2001 From: jianzhiyao <739319867@qq.com> Date: Thu, 20 May 2021 19:36:22 +0800 Subject: [PATCH 122/343] expression: add builtin function ``json_pretty`` (#24675) --- expression/builtin_json.go | 44 ++++++++++++++++++-- expression/builtin_json_test.go | 73 +++++++++++++++++++++++++++++++++ expression/builtin_json_vec.go | 38 +++++++++++++++++ expression/integration_test.go | 68 ++++++++++++++++++++++++++++++ 4 files changed, 220 insertions(+), 3 deletions(-) diff --git a/expression/builtin_json.go b/expression/builtin_json.go index bb2eee747606b..287e4383bf165 100644 --- a/expression/builtin_json.go +++ b/expression/builtin_json.go @@ -14,7 +14,8 @@ package expression import ( - json2 "encoding/json" + "bytes" + goJSON "encoding/json" "strconv" "strings" @@ -841,7 +842,7 @@ func (b *builtinJSONValidStringSig) evalInt(row chunk.Row) (res int64, isNull bo } data := hack.Slice(val) - if json2.Valid(data) { + if goJSON.Valid(data) { res = 1 } else { res = 0 @@ -1072,8 +1073,45 @@ type jsonPrettyFunctionClass struct { baseFunctionClass } +type builtinJSONSPrettySig struct { + baseBuiltinFunc +} + +func (b *builtinJSONSPrettySig) Clone() builtinFunc { + newSig := &builtinJSONSPrettySig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + func (c *jsonPrettyFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { - return nil, errFunctionNotExists.GenWithStackByArgs("FUNCTION", "JSON_PRETTY") + if err := c.verifyArgs(args); err != nil { + return nil, err + } + + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETString, types.ETJson) + if err != nil { + return nil, err + } + sig := &builtinJSONSPrettySig{bf} + sig.setPbCode(tipb.ScalarFuncSig_JsonPrettySig) + return sig, nil +} + +func (b *builtinJSONSPrettySig) evalString(row chunk.Row) (res string, isNull bool, err error) { + obj, isNull, err := b.args[0].EvalJSON(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + + buf, err := obj.MarshalJSON() + if err != nil { + return res, isNull, err + } + var resBuf bytes.Buffer + if err = goJSON.Indent(&resBuf, buf, "", " "); err != nil { + return res, isNull, err + } + return resBuf.String(), false, nil } type jsonQuoteFunctionClass struct { diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index 0302e4f3c2d3d..666c1a1eb5b00 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -995,3 +995,76 @@ func (s *testEvaluatorSuite) TestJSONStorageSize(c *C) { } } } + +func (s *testEvaluatorSuite) TestJSONPretty(c *C) { + fc := funcs[ast.JSONPretty] + tbl := []struct { + input []interface{} + expected interface{} + success bool + }{ + // Tests scalar arguments + {[]interface{}{nil}, nil, true}, + {[]interface{}{`true`}, "true", true}, + {[]interface{}{`false`}, "false", true}, + {[]interface{}{`2223`}, "2223", true}, + // Tests simple json + {[]interface{}{`{"a":1}`}, `{ + "a": 1 +}`, true}, + {[]interface{}{`[1]`}, `[ + 1 +]`, true}, + // Test complex json + {[]interface{}{`{"a":1,"b":[{"d":1},{"e":2},{"f":3}],"c":"eee"}`}, `{ + "a": 1, + "b": [ + { + "d": 1 + }, + { + "e": 2 + }, + { + "f": 3 + } + ], + "c": "eee" +}`, true}, + {[]interface{}{`{"a":1,"b":"qwe","c":[1,2,3,"123",null],"d":{"d1":1,"d2":2}}`}, `{ + "a": 1, + "b": "qwe", + "c": [ + 1, + 2, + 3, + "123", + null + ], + "d": { + "d1": 1, + "d2": 2 + } +}`, true}, + // Tests invalid json data + {[]interface{}{`{1}`}, nil, false}, + {[]interface{}{`[1,3,4,5]]`}, nil, false}, + } + for _, t := range tbl { + args := types.MakeDatums(t.input...) + f, err := fc.getFunction(s.ctx, s.datumsToConstants(args)) + c.Assert(err, IsNil) + d, err := evalBuiltinFunc(f, chunk.Row{}) + if t.success { + c.Assert(err, IsNil) + + if t.expected == nil { + c.Assert(d.IsNull(), IsTrue) + } else { + c.Assert(d.GetString(), Equals, t.expected.(string)) + } + } else { + c.Assert(err, NotNil) + } + } +} diff --git a/expression/builtin_json_vec.go b/expression/builtin_json_vec.go index 953da67458040..86e1d64b09902 100644 --- a/expression/builtin_json_vec.go +++ b/expression/builtin_json_vec.go @@ -14,6 +14,8 @@ package expression import ( + "bytes" + goJSON "encoding/json" "strconv" "strings" @@ -1176,3 +1178,39 @@ func (b *builtinJSONUnquoteSig) vecEvalString(input *chunk.Chunk, result *chunk. } return nil } + +func (b *builtinJSONSPrettySig) vectorized() bool { + return true +} + +func (b *builtinJSONSPrettySig) vecEvalString(input *chunk.Chunk, result *chunk.Column) error { + n := input.NumRows() + buf, err := b.bufAllocator.get(types.ETJson, n) + if err != nil { + return err + } + defer b.bufAllocator.put(buf) + if err := b.args[0].VecEvalJSON(b.ctx, input, buf); err != nil { + return err + } + result.ReserveString(n) + for i := 0; i < n; i++ { + if buf.IsNull(i) { + result.AppendNull() + continue + } + + jb, err := buf.GetJSON(i).MarshalJSON() + if err != nil { + return err + } + + var resBuf bytes.Buffer + if err = goJSON.Indent(&resBuf, jb, "", " "); err != nil { + return err + } + + result.AppendString(resBuf.String()) + } + return nil +} diff --git a/expression/integration_test.go b/expression/integration_test.go index a0fb6fd8b5499..c6baa9bcea350 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -213,6 +213,74 @@ func (s *testIntegrationSuite) TestFuncLpadAndRpad(c *C) { result.Check(testkit.Rows(" ")) } +func (s *testIntegrationSuite) TestBuiltinFuncJsonPretty(c *C) { + ctx := context.Background() + tk := testkit.NewTestKit(c, s.store) + defer s.cleanEnv(c) + + tk.MustExec(`use test;`) + tk.MustExec(`drop table if exists t;`) + tk.MustExec("CREATE TABLE t (`id` int NOT NULL AUTO_INCREMENT, `j` json,vc VARCHAR(500) , PRIMARY KEY (`id`));") + tk.MustExec(`INSERT INTO t ( id, j, vc ) VALUES + ( 1, '{"a":1,"b":"qwe","c":[1,2,3,"123",null],"d":{"d1":1,"d2":2}}', '{"a":1,"b":"qwe","c":[1,2,3,"123",null],"d":{"d1":1,"d2":2}}' ), + ( 2, '[1,2,34]', '{' );`) + + // valid json format in json and varchar + checkResult := []string{ + `{ + "a": 1, + "b": "qwe", + "c": [ + 1, + 2, + 3, + "123", + null + ], + "d": { + "d1": 1, + "d2": 2 + } +}`, + `{ + "a": 1, + "b": "qwe", + "c": [ + 1, + 2, + 3, + "123", + null + ], + "d": { + "d1": 1, + "d2": 2 + } +}`, + } + tk. + MustQuery("select JSON_PRETTY(t.j),JSON_PRETTY(vc) from t where id = 1;"). + Check(testkit.Rows(strings.Join(checkResult, " "))) + + // invalid json format in varchar + rs, _ := tk.Exec("select JSON_PRETTY(t.j),JSON_PRETTY(vc) from t where id = 2;") + _, err := session.GetRows4Test(ctx, tk.Se, rs) + terr := errors.Cause(err).(*terror.Error) + c.Assert(terr.Code(), Equals, errors.ErrCode(mysql.ErrInvalidJSONText)) + + // invalid json format in one row + rs, _ = tk.Exec("select JSON_PRETTY(t.j),JSON_PRETTY(vc) from t where id in (1,2);") + _, err = session.GetRows4Test(ctx, tk.Se, rs) + terr = errors.Cause(err).(*terror.Error) + c.Assert(terr.Code(), Equals, errors.ErrCode(mysql.ErrInvalidJSONText)) + + // invalid json string + rs, _ = tk.Exec(`select JSON_PRETTY("[1,2,3]}");`) + _, err = session.GetRows4Test(ctx, tk.Se, rs) + terr = errors.Cause(err).(*terror.Error) + c.Assert(terr.Code(), Equals, errors.ErrCode(mysql.ErrInvalidJSONText)) +} + func (s *testIntegrationSuite) TestMiscellaneousBuiltin(c *C) { ctx := context.Background() defer s.cleanEnv(c) From 40a8ae771130e0e4c56f2c580b3d21d01e989a18 Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 20 May 2021 20:42:17 +0800 Subject: [PATCH 123/343] ddl: make `TestDropLastVisibleColumns` stable (#24790) * ddl: make `TestDropLastVisibleColumns` stable --- ddl/db_integration_test.go | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 8a49182e92f08..a39322c2f5b9c 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2614,24 +2614,24 @@ func (s *testIntegrationSuite5) TestDropColumnsWithMultiIndex(c *C) { tk.MustQuery(query).Check(testkit.Rows()) } -func (s *testIntegrationSuite5) TestDropLastVisibleColumn(c *C) { +func (s *testSerialDBSuite) TestDropLastVisibleColumnOrColumns(c *C) { + defer config.RestoreFunc() + config.UpdateGlobal(func(conf *config.Config) { + conf.Experimental.AllowsExpressionIndex = true + }) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test_db") tk.MustExec("create table t_drop_last_column(x int, key((1+1)))") - defer tk.MustExec("drop table if exists t_drop_last_column") _, err := tk.Exec("alter table t_drop_last_column drop column x") c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "[ddl:1113]A table must have at least 1 column") -} - -func (s *testIntegrationSuite5) TestDropLastVisibleColumns(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test_db") + // for visible columns tk.MustExec("create table t_drop_last_columns(x int, y int, key((1+1)))") - defer tk.MustExec("drop table if exists t_drop_last_columns") - _, err := tk.Exec("alter table t_drop_last_columns drop column x, drop column y") + _, err = tk.Exec("alter table t_drop_last_columns drop column x, drop column y") c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "[ddl:1113]A table must have at least 1 column") + + tk.MustExec("drop table if exists t_drop_last_column, t_drop_last_columns") } func (s *testIntegrationSuite7) TestAutoIncrementTableOption(c *C) { From 6bfc5f8b44005f2c988d383c9cdf65cf98e7742d Mon Sep 17 00:00:00 2001 From: JmPotato Date: Thu, 20 May 2021 20:54:22 +0800 Subject: [PATCH 124/343] *: support AS OF TIMESTAMP read-only begin statement (#24740) --- executor/builder.go | 9 +-- executor/simple.go | 32 +++++++-- executor/stale_txn_test.go | 118 ++++++++++++++++++++++++-------- expression/builtin_time_test.go | 4 +- planner/core/common_plans.go | 3 + planner/core/planbuilder.go | 36 +++++++++- session/session.go | 54 +++++++-------- session/session_test.go | 4 +- sessionctx/context.go | 2 +- 9 files changed, 188 insertions(+), 74 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index b04dfe45f4bab..c8a128e505562 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -706,10 +706,11 @@ func (b *executorBuilder) buildSimple(v *plannercore.Simple) Executor { base := newBaseExecutor(b.ctx, v.Schema(), v.ID()) base.initCap = chunk.ZeroCapacity e := &SimpleExec{ - baseExecutor: base, - Statement: v.Statement, - IsFromRemote: v.IsFromRemote, - is: b.is, + baseExecutor: base, + Statement: v.Statement, + IsFromRemote: v.IsFromRemote, + is: b.is, + StalenessTxnOption: v.StalenessTxnOption, } return e } diff --git a/executor/simple.go b/executor/simple.go index 3c45ee05ada4e..193949eda3fdd 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -74,6 +74,9 @@ type SimpleExec struct { IsFromRemote bool done bool is infoschema.InfoSchema + + // StalenessTxnOption is used to execute the staleness txn during a read-only begin statement. + StalenessTxnOption *sessionctx.StalenessTxnOption } func (e *baseExecutor) getSysSession() (sessionctx.Context, error) { @@ -566,13 +569,16 @@ func (e *SimpleExec) executeUse(s *ast.UseStmt) error { } func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { - // If `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND` is the first statement in TxnCtx, we should + // If `START TRANSACTION READ ONLY` is the first statement in TxnCtx, we should // always create a new Txn instead of reusing it. if s.ReadOnly { enableNoopFuncs := e.ctx.GetSessionVars().EnableNoopFuncs - if !enableNoopFuncs && s.Bound == nil { + if !enableNoopFuncs && s.AsOf == nil && s.Bound == nil { return expression.ErrFunctionsNoopImpl.GenWithStackByArgs("READ ONLY") } + if s.AsOf != nil { + return e.executeStartTransactionReadOnlyWithBoundedStaleness(ctx, s) + } if s.Bound != nil { return e.executeStartTransactionReadOnlyWithTimestampBound(ctx, s) } @@ -614,6 +620,22 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { return nil } +func (e *SimpleExec) executeStartTransactionReadOnlyWithBoundedStaleness(ctx context.Context, s *ast.BeginStmt) error { + if e.StalenessTxnOption == nil { + return errors.New("Failed to get timestamp during start transaction read only as of timestamp") + } + if err := e.ctx.NewTxnWithStalenessOption(ctx, *e.StalenessTxnOption); err != nil { + return err + } + + // With START TRANSACTION, autocommit remains disabled until you end + // the transaction with COMMIT or ROLLBACK. The autocommit mode then + // reverts to its previous state. + e.ctx.GetSessionVars().SetInTxn(true) + return nil +} + +// TODO: deprecate this syntax and only keep `AS OF TIMESTAMP` statement. func (e *SimpleExec) executeStartTransactionReadOnlyWithTimestampBound(ctx context.Context, s *ast.BeginStmt) error { opt := sessionctx.StalenessTxnOption{} opt.Mode = s.Bound.Mode @@ -632,8 +654,7 @@ func (e *SimpleExec) executeStartTransactionReadOnlyWithTimestampBound(ctx conte if err != nil { return err } - startTS := oracle.GoTimeToTS(gt) - opt.StartTS = startTS + opt.StartTS = oracle.GoTimeToTS(gt) case ast.TimestampBoundExactStaleness: // TODO: support funcCallExpr in future v, ok := s.Bound.Timestamp.(*driver.ValueExpr) @@ -668,8 +689,7 @@ func (e *SimpleExec) executeStartTransactionReadOnlyWithTimestampBound(ctx conte if err != nil { return err } - startTS := oracle.GoTimeToTS(gt) - opt.StartTS = startTS + opt.StartTS = oracle.GoTimeToTS(gt) } err := e.ctx.NewTxnWithStalenessOption(ctx, opt) if err != nil { diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index db2b55a9a1637..dc5ddd0785208 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -73,13 +73,30 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { zone: "sz", }, { - name: "begin", + name: "begin after TimestampBoundReadTimestamp", preSQL: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`, sql: "begin", IsStaleness: false, txnScope: kv.GlobalTxnScope, zone: "", }, + { + name: "AsOfTimestamp", + preSQL: "begin", + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`, + IsStaleness: true, + expectPhysicalTS: 1599321600000, + txnScope: "local", + zone: "sh", + }, + { + name: "begin after AsOfTimestamp", + preSQL: `START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`, + sql: "begin", + IsStaleness: false, + txnScope: oracle.GlobalTxnScope, + zone: "", + }, } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -106,8 +123,8 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { } c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.IsStaleness) tk.MustExec("commit") - failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") } + failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") } func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { @@ -147,13 +164,17 @@ func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { failpoint.Enable("github.com/pingcap/tidb/config/injectTxnScope", fmt.Sprintf(`return("%v")`, testcase.zone)) failpoint.Enable("github.com/pingcap/tidb/store/tikv/assertStoreLabels", fmt.Sprintf(`return("%v_%v")`, placement.DCLabelKey, testcase.txnScope)) failpoint.Enable("github.com/pingcap/tidb/store/tikv/assertStaleReadFlag", `return(true)`) + // Using NOW() will cause the loss of fsp precision, so we use NOW(3) to be accurate to the millisecond. + tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) + tk.MustQuery(testcase.sql) + tk.MustExec(`commit`) tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) tk.MustQuery(testcase.sql) tk.MustExec(`commit`) - failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/assertStoreLabels") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/assertStaleReadFlag") } + failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/assertStoreLabels") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/assertStaleReadFlag") } func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { @@ -169,6 +190,17 @@ func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { tk.MustExec(updateSafePoint) // set @@tidb_snapshot before staleness txn tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) + tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`) + // 1599321600000 == 2020-09-06 00:00:00 + c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) + tk.MustExec("commit") + // set @@tidb_snapshot during staleness txn + tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`) + tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) + c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) + tk.MustExec("commit") + // set @@tidb_snapshot before staleness txn + tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`) c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) tk.MustExec("commit") @@ -190,23 +222,20 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { name string sql string injectSafeTS uint64 - useSafeTS bool + // compareWithSafeTS will be 0 if StartTS==SafeTS, -1 if StartTS < SafeTS, and +1 if StartTS > SafeTS. + compareWithSafeTS int }{ { - name: "max 20 seconds ago, safeTS 10 secs ago", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:20'`, - injectSafeTS: func() uint64 { - return oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)) - }(), - useSafeTS: true, + name: "max 20 seconds ago, safeTS 10 secs ago", + sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:20'`, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)), + compareWithSafeTS: 0, }, { - name: "max 10 seconds ago, safeTS 20 secs ago", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`, - injectSafeTS: func() uint64 { - return oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)) - }(), - useSafeTS: false, + name: "max 10 seconds ago, safeTS 20 secs ago", + sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)), + compareWithSafeTS: 1, }, { name: "max 20 seconds ago, safeTS 10 secs ago", @@ -214,10 +243,8 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { return fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MIN READ TIMESTAMP '%v'`, time.Now().Add(-20*time.Second).Format("2006-01-02 15:04:05")) }(), - injectSafeTS: func() uint64 { - return oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)) - }(), - useSafeTS: true, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)), + compareWithSafeTS: 0, }, { name: "max 10 seconds ago, safeTS 20 secs ago", @@ -225,25 +252,46 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { return fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MIN READ TIMESTAMP '%v'`, time.Now().Add(-10*time.Second).Format("2006-01-02 15:04:05")) }(), - injectSafeTS: func() uint64 { - return oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)) - }(), - useSafeTS: false, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)), + compareWithSafeTS: 1, + }, + { + name: "20 seconds ago to now, safeTS 10 secs ago", + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP tidb_bounded_staleness(NOW() - INTERVAL 20 SECOND, NOW())`, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)), + compareWithSafeTS: 0, + }, + { + name: "10 seconds ago to now, safeTS 20 secs ago", + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP tidb_bounded_staleness(NOW() - INTERVAL 10 SECOND, NOW())`, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)), + compareWithSafeTS: 1, + }, + { + name: "20 seconds ago to 10 seconds ago, safeTS 5 secs ago", + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP tidb_bounded_staleness(NOW() - INTERVAL 20 SECOND, NOW() - INTERVAL 10 SECOND)`, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-5 * time.Second)), + compareWithSafeTS: -1, }, } for _, testcase := range testcases { c.Log(testcase.name) c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/injectSafeTS", fmt.Sprintf("return(%v)", testcase.injectSafeTS)), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", + fmt.Sprintf("return(%v)", testcase.injectSafeTS)), IsNil) tk.MustExec(testcase.sql) - if testcase.useSafeTS { + if testcase.compareWithSafeTS == 1 { + c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Greater, testcase.injectSafeTS) + } else if testcase.compareWithSafeTS == 0 { c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.injectSafeTS) } else { - c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Greater, testcase.injectSafeTS) + c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Less, testcase.injectSafeTS) } tk.MustExec("commit") - failpoint.Disable("github.com/pingcap/tidb/store/tikv/injectSafeTS") } + failpoint.Disable("github.com/pingcap/tidb/expression/injectSafeTS") + failpoint.Disable("github.com/pingcap/tidb/store/tikv/injectSafeTS") } func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { @@ -263,4 +311,16 @@ func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { schemaVer3 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() // got an old infoSchema c.Assert(schemaVer3, Equals, schemaVer1) + + schemaVer4 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() + time.Sleep(time.Second) + tk.MustExec("create table t (id int primary key);") + schemaVer5 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() + // confirm schema changed + c.Assert(schemaVer4, Less, schemaVer5) + + tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW() - INTERVAL 1 SECOND`) + schemaVer6 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() + // got an old infoSchema + c.Assert(schemaVer6, Equals, schemaVer4) } diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index e247e8756ae9a..4015794377486 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -2928,7 +2928,7 @@ func (s *testEvaluatorSuite) TestTiDBBoundedStaleness(c *C) { // Test whether it's deterministic. safeTime1 := t2.Add(-1 * time.Second) - safeTS1 := oracle.ComposeTS(safeTime1.Unix()*1000, 0) + safeTS1 := oracle.GoTimeToTS(safeTime1) c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", fmt.Sprintf("return(%v)", safeTS1)), IsNil) f, err := fc.getFunction(s.ctx, s.datumsToConstants([]types.Datum{types.NewDatum(t1Str), types.NewDatum(t2Str)})) @@ -2941,7 +2941,7 @@ func (s *testEvaluatorSuite) TestTiDBBoundedStaleness(c *C) { c.Assert(resultTime, Equals, safeTime1.Format(types.TimeFormat)) // SafeTS updated. safeTime2 := t2.Add(1 * time.Second) - safeTS2 := oracle.ComposeTS(safeTime2.Unix()*1000, 0) + safeTS2 := oracle.GoTimeToTS(safeTime2) c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", fmt.Sprintf("return(%v)", safeTS2)), IsNil) f, err = fc.getFunction(s.ctx, s.datumsToConstants([]types.Datum{types.NewDatum(t1Str), types.NewDatum(t2Str)})) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 3818486955646..b42a84d926eeb 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -723,6 +723,9 @@ type Simple struct { // and executing in co-processor. // Used for `global kill`. See https://github.com/pingcap/tidb/blob/master/docs/design/2020-06-01-global-kill.md. IsFromRemote bool + + // StalenessTxnOption is the transaction option that will be built when planner builder calls buildSimple. + StalenessTxnOption *sessionctx.StalenessTxnOption } // PhysicalSimpleWrapper is a wrapper of `Simple` to implement physical plan interface. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index ccab0a28cc863..2ae66a61602a5 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" @@ -643,7 +644,7 @@ func (b *PlanBuilder) Build(ctx context.Context, node ast.Node) (Plan, error) { *ast.GrantStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.RevokeStmt, *ast.KillStmt, *ast.DropStatsStmt, *ast.GrantRoleStmt, *ast.RevokeRoleStmt, *ast.SetRoleStmt, *ast.SetDefaultRoleStmt, *ast.ShutdownStmt, *ast.RenameUserStmt: - return b.buildSimple(node.(ast.StmtNode)) + return b.buildSimple(ctx, node.(ast.StmtNode)) case ast.DDLNode: return b.buildDDL(ctx, x) case *ast.CreateBindingStmt: @@ -2259,7 +2260,7 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, return np, nil } -func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { +func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, error) { p := &Simple{Statement: node} switch raw := node.(type) { @@ -2325,10 +2326,41 @@ func (b *PlanBuilder) buildSimple(node ast.StmtNode) (Plan, error) { } case *ast.ShutdownStmt: b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShutdownPriv, "", "", "", nil) + case *ast.BeginStmt: + if raw.AsOf != nil { + startTS, err := b.calculateTsExpr(raw.AsOf) + if err != nil { + return nil, err + } + p.StalenessTxnOption = &sessionctx.StalenessTxnOption{ + Mode: ast.TimestampBoundReadTimestamp, + StartTS: startTS, + } + } } return p, nil } +// calculateTsExpr calculates the TsExpr of AsOfClause to get a StartTS. +func (b *PlanBuilder) calculateTsExpr(asOfClause *ast.AsOfClause) (uint64, error) { + tsVal, err := evalAstExpr(b.ctx, asOfClause.TsExpr) + if err != nil { + return 0, err + } + toTypeTimestamp := types.NewFieldType(mysql.TypeTimestamp) + // We need at least the millionsecond here, so set fsp to 3. + toTypeTimestamp.Decimal = 3 + tsTimestamp, err := tsVal.ConvertTo(b.ctx.GetSessionVars().StmtCtx, toTypeTimestamp) + if err != nil { + return 0, err + } + tsTime, err := tsTimestamp.GetMysqlTime().GoTime(b.ctx.GetSessionVars().TimeZone) + if err != nil { + return 0, err + } + return oracle.GoTimeToTS(tsTime), nil +} + func collectVisitInfoFromRevokeStmt(sctx sessionctx.Context, vi []visitInfo, stmt *ast.RevokeStmt) []visitInfo { // To use REVOKE, you must have the GRANT OPTION privilege, // and you must have the privileges that you are granting. diff --git a/session/session.go b/session/session.go index f116daf96dd04..2874330a02fd3 100644 --- a/session/session.go +++ b/session/session.go @@ -1960,20 +1960,9 @@ func (s *session) isTxnRetryable() bool { } func (s *session) NewTxn(ctx context.Context) error { - if s.txn.Valid() { - txnStartTS := s.txn.StartTS() - txnScope := s.GetSessionVars().TxnCtx.TxnScope - err := s.CommitTxn(ctx) - if err != nil { - return err - } - vars := s.GetSessionVars() - logutil.Logger(ctx).Info("NewTxn() inside a transaction auto commit", - zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), - zap.Uint64("txnStartTS", txnStartTS), - zap.String("txnScope", txnScope)) + if err := s.checkBeforeNewTxn(ctx); err != nil { + return err } - txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(s.sessionVars.CheckAndGetTxnScope())) if err != nil { return err @@ -1995,6 +1984,23 @@ func (s *session) NewTxn(ctx context.Context) error { return nil } +func (s *session) checkBeforeNewTxn(ctx context.Context) error { + if s.txn.Valid() { + txnStartTS := s.txn.StartTS() + txnScope := s.GetSessionVars().TxnCtx.TxnScope + err := s.CommitTxn(ctx) + if err != nil { + return err + } + vars := s.GetSessionVars() + logutil.Logger(ctx).Info("Try to create a new txn inside a transaction auto commit", + zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), + zap.Uint64("txnStartTS", txnStartTS), + zap.String("txnScope", txnScope)) + } + return nil +} + func (s *session) SetValue(key fmt.Stringer, value interface{}) { s.mu.Lock() s.mu.values[key] = value @@ -2782,22 +2788,14 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { // NewTxnWithStalenessOption create a transaction with Staleness option func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionctx.StalenessTxnOption) error { - if s.txn.Valid() { - txnID := s.txn.StartTS() - txnScope := s.txn.GetOption(kv.TxnScope).(string) - err := s.CommitTxn(ctx) - if err != nil { - return err - } - vars := s.GetSessionVars() - logutil.Logger(ctx).Info("InitTxnWithExactStaleness() inside a transaction auto commit", - zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), - zap.Uint64("txnStartTS", txnID), - zap.String("txnScope", txnScope)) + err := s.checkBeforeNewTxn(ctx) + if err != nil { + return err } - var txn kv.Transaction - var err error - txnScope := s.GetSessionVars().CheckAndGetTxnScope() + var ( + txn kv.Transaction + txnScope = s.GetSessionVars().CheckAndGetTxnScope() + ) switch option.Mode { case ast.TimestampBoundReadTimestamp: txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetStartTs(option.StartTS)) diff --git a/session/session_test.go b/session/session_test.go index a6c7908237bca..9d2d63cb02804 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4109,7 +4109,7 @@ func (s *testSessionSerialSuite) TestValidateReadOnlyInStalenessTransaction(c *C tk.MustExec(`set @@tidb_enable_noop_functions=1;`) for _, testcase := range testcases { c.Log(testcase.name) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) + tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) if testcase.isValidate { _, err := tk.Exec(testcase.sql) c.Assert(err, IsNil) @@ -4169,7 +4169,7 @@ func (s *testSessionSerialSuite) TestSpecialSQLInStalenessTxn(c *C) { tk.MustExec("CREATE USER 'newuser' IDENTIFIED BY 'mypassword';") for _, testcase := range testcases { comment := Commentf(testcase.name) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) + tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, true, comment) tk.MustExec(testcase.sql) c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.sameSession, comment) diff --git a/sessionctx/context.go b/sessionctx/context.go index 2aeda663a038d..59a917f86a9bc 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -73,7 +73,7 @@ type Context interface { // It should be called right before we builds an executor. InitTxnWithStartTS(startTS uint64) error - // NewTxnWithStalenessOption initializes a transaction with StalenessTxnOption + // NewTxnWithStalenessOption initializes a transaction with StalenessTxnOption. NewTxnWithStalenessOption(ctx context.Context, option StalenessTxnOption) error // GetStore returns the store of session. From bcfe952921eafc8c0fb90c6544cf0ddf3086ca09 Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Thu, 20 May 2021 21:22:22 +0800 Subject: [PATCH 125/343] executor: Fix unstable TestTiDBLastTxnInfoCommitMode (#24779) --- executor/executor_test.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 8836b4f59fbd3..a796a27326a70 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3067,9 +3067,10 @@ func (s *testSerialSuite) TestTiDBLastTxnInfoCommitMode(c *C) { c.Assert(rows[0][1], Equals, "false") c.Assert(rows[0][2], Equals, "false") - config.UpdateGlobal(func(conf *config.Config) { - conf.TiKVClient.AsyncCommit.SafeWindow = 0 - }) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/invalidMaxCommitTS", "return"), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/invalidMaxCommitTS"), IsNil) + }() tk.MustExec("set @@tidb_enable_async_commit = 1") tk.MustExec("set @@tidb_enable_1pc = 0") From b91aeedab707481472237a0504646b2659d17204 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Thu, 20 May 2021 15:34:22 +0200 Subject: [PATCH 126/343] planner: add tests for partition range boundaries for LT/GT (#24574) --- executor/partition_table_test.go | 46 + executor/testdata/executor_suite_in.json | 335 ++ executor/testdata/executor_suite_out.json | 3811 +++++++++++++++++++++ 3 files changed, 4192 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 446e689184086..7398789eadba5 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -1829,3 +1829,49 @@ PARTITION BY RANGE (a) ( s.testData.GetTestCases(c, &input, &output) s.verifyPartitionResult(tk, input, output) } + +func (s *testSuiteWithData) TestRangePartitionBoundariesLtM(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("create database TestRangePartitionBoundariesLtM") + defer tk.MustExec("drop database TestRangePartitionBoundariesLtM") + tk.MustExec("use TestRangePartitionBoundariesLtM") + tk.MustExec("drop table if exists t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1000000), + PARTITION p1 VALUES LESS THAN (2000000), + PARTITION p2 VALUES LESS THAN (3000000))`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} + +func (s *testSuiteWithData) TestRangePartitionBoundariesLtS(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("create database TestRangePartitionBoundariesLtS") + defer tk.MustExec("drop database TestRangePartitionBoundariesLtS") + tk.MustExec("use TestRangePartitionBoundariesLtS") + tk.MustExec("drop table if exists t") + tk.MustExec(`CREATE TABLE t +(a INT, b varchar(255)) +PARTITION BY RANGE (a) ( + PARTITION p0 VALUES LESS THAN (1), + PARTITION p1 VALUES LESS THAN (2), + PARTITION p2 VALUES LESS THAN (3), + PARTITION p3 VALUES LESS THAN (4), + PARTITION p4 VALUES LESS THAN (5), + PARTITION p5 VALUES LESS THAN (6), + PARTITION p6 VALUES LESS THAN (7))`) + + var input []string + var output []testOutput + s.testData.GetTestCases(c, &input, &output) + s.verifyPartitionResult(tk, input, output) +} diff --git a/executor/testdata/executor_suite_in.json b/executor/testdata/executor_suite_in.json index a8db9425a7078..cd8fa234c0117 100644 --- a/executor/testdata/executor_suite_in.json +++ b/executor/testdata/executor_suite_in.json @@ -273,5 +273,340 @@ "SELECT * FROM t WHERE a BETWEEN 2 AND 7", "SELECT * FROM t WHERE a BETWEEN 7 AND 4" ] + }, + { + "name": "TestRangePartitionBoundariesLtM", + "cases": [ + "INSERT INTO t VALUES (999998, '999998 Filler ...'), (999999, '999999 Filler ...'), (1000000, '1000000 Filler ...'), (1000001, '1000001 Filler ...'), (1000002, '1000002 Filler ...')", + "INSERT INTO t VALUES (1999998, '1999998 Filler ...'), (1999999, '1999999 Filler ...'), (2000000, '2000000 Filler ...'), (2000001, '2000001 Filler ...'), (2000002, '2000002 Filler ...')", + "INSERT INTO t VALUES (2999998, '2999998 Filler ...'), (2999999, '2999999 Filler ...')", + "INSERT INTO t VALUES (-2147483648, 'MIN_INT filler...'), (0, '0 Filler...')", + "ANALYZE TABLE t", + "SELECT * FROM t WHERE a < -2147483648", + "SELECT * FROM t WHERE a > -2147483648", + "SELECT * FROM t WHERE a <= -2147483648", + "SELECT * FROM t WHERE a >= -2147483648", + "SELECT * FROM t WHERE a < 0", + "SELECT * FROM t WHERE a > 0", + "SELECT * FROM t WHERE a <= 0", + "SELECT * FROM t WHERE a >= 0", + "SELECT * FROM t WHERE a < 999998", + "SELECT * FROM t WHERE a > 999998", + "SELECT * FROM t WHERE a <= 999998", + "SELECT * FROM t WHERE a >= 999998", + "SELECT * FROM t WHERE a < 999999", + "SELECT * FROM t WHERE a > 999999", + "SELECT * FROM t WHERE a <= 999999", + "SELECT * FROM t WHERE a >= 999999", + "SELECT * FROM t WHERE a < 1000000", + "SELECT * FROM t WHERE a > 1000000", + "SELECT * FROM t WHERE a <= 1000000", + "SELECT * FROM t WHERE a >= 1000000", + "SELECT * FROM t WHERE a < 1000001", + "SELECT * FROM t WHERE a > 1000001", + "SELECT * FROM t WHERE a <= 1000001", + "SELECT * FROM t WHERE a >= 1000001", + "SELECT * FROM t WHERE a < 1000002", + "SELECT * FROM t WHERE a > 1000002", + "SELECT * FROM t WHERE a <= 1000002", + "SELECT * FROM t WHERE a >= 1000002", + "SELECT * FROM t WHERE a < 3000000", + "SELECT * FROM t WHERE a > 3000000", + "SELECT * FROM t WHERE a <= 3000000", + "SELECT * FROM t WHERE a >= 3000000", + "SELECT * FROM t WHERE a < 3000001", + "SELECT * FROM t WHERE a > 3000001", + "SELECT * FROM t WHERE a <= 3000001", + "SELECT * FROM t WHERE a >= 3000001", + "SELECT * FROM t WHERE a < 999997", + "SELECT * FROM t WHERE a > 999997", + "SELECT * FROM t WHERE a <= 999997", + "SELECT * FROM t WHERE a >= 999997", + "SELECT * FROM t WHERE a >= 999997 AND a <= 999999", + "SELECT * FROM t WHERE a > 999997 AND a <= 999999", + "SELECT * FROM t WHERE a > 999997 AND a < 999999", + "SELECT * FROM t WHERE a > 999997 AND a <= 999999", + "SELECT * FROM t WHERE a < 999998", + "SELECT * FROM t WHERE a > 999998", + "SELECT * FROM t WHERE a <= 999998", + "SELECT * FROM t WHERE a >= 999998", + "SELECT * FROM t WHERE a >= 999998 AND a <= 1000000", + "SELECT * FROM t WHERE a > 999998 AND a <= 1000000", + "SELECT * FROM t WHERE a > 999998 AND a < 1000000", + "SELECT * FROM t WHERE a > 999998 AND a <= 1000000", + "SELECT * FROM t WHERE a < 999999", + "SELECT * FROM t WHERE a > 999999", + "SELECT * FROM t WHERE a <= 999999", + "SELECT * FROM t WHERE a >= 999999", + "SELECT * FROM t WHERE a >= 999999 AND a <= 1000001", + "SELECT * FROM t WHERE a > 999999 AND a <= 1000001", + "SELECT * FROM t WHERE a > 999999 AND a < 1000001", + "SELECT * FROM t WHERE a > 999999 AND a <= 1000001", + "SELECT * FROM t WHERE a < 1000000", + "SELECT * FROM t WHERE a > 1000000", + "SELECT * FROM t WHERE a <= 1000000", + "SELECT * FROM t WHERE a >= 1000000", + "SELECT * FROM t WHERE a >= 1000000 AND a <= 1000002", + "SELECT * FROM t WHERE a > 1000000 AND a <= 1000002", + "SELECT * FROM t WHERE a > 1000000 AND a < 1000002", + "SELECT * FROM t WHERE a > 1000000 AND a <= 1000002", + "SELECT * FROM t WHERE a < 1999997", + "SELECT * FROM t WHERE a > 1999997", + "SELECT * FROM t WHERE a <= 1999997", + "SELECT * FROM t WHERE a >= 1999997", + "SELECT * FROM t WHERE a >= 1999997 AND a <= 1999999", + "SELECT * FROM t WHERE a > 1999997 AND a <= 1999999", + "SELECT * FROM t WHERE a > 1999997 AND a < 1999999", + "SELECT * FROM t WHERE a > 1999997 AND a <= 1999999", + "SELECT * FROM t WHERE a < 1999998", + "SELECT * FROM t WHERE a > 1999998", + "SELECT * FROM t WHERE a <= 1999998", + "SELECT * FROM t WHERE a >= 1999998", + "SELECT * FROM t WHERE a >= 1999998 AND a <= 2000000", + "SELECT * FROM t WHERE a > 1999998 AND a <= 2000000", + "SELECT * FROM t WHERE a > 1999998 AND a < 2000000", + "SELECT * FROM t WHERE a > 1999998 AND a <= 2000000", + "SELECT * FROM t WHERE a < 1999999", + "SELECT * FROM t WHERE a > 1999999", + "SELECT * FROM t WHERE a <= 1999999", + "SELECT * FROM t WHERE a >= 1999999", + "SELECT * FROM t WHERE a >= 1999999 AND a <= 2000001", + "SELECT * FROM t WHERE a > 1999999 AND a <= 2000001", + "SELECT * FROM t WHERE a > 1999999 AND a < 2000001", + "SELECT * FROM t WHERE a > 1999999 AND a <= 2000001", + "SELECT * FROM t WHERE a < 2000000", + "SELECT * FROM t WHERE a > 2000000", + "SELECT * FROM t WHERE a <= 2000000", + "SELECT * FROM t WHERE a >= 2000000", + "SELECT * FROM t WHERE a >= 2000000 AND a <= 2000002", + "SELECT * FROM t WHERE a > 2000000 AND a <= 2000002", + "SELECT * FROM t WHERE a > 2000000 AND a < 2000002", + "SELECT * FROM t WHERE a > 2000000 AND a <= 2000002", + "SELECT * FROM t WHERE a < 2999997", + "SELECT * FROM t WHERE a > 2999997", + "SELECT * FROM t WHERE a <= 2999997", + "SELECT * FROM t WHERE a >= 2999997", + "SELECT * FROM t WHERE a >= 2999997 AND a <= 2999999", + "SELECT * FROM t WHERE a > 2999997 AND a <= 2999999", + "SELECT * FROM t WHERE a > 2999997 AND a < 2999999", + "SELECT * FROM t WHERE a > 2999997 AND a <= 2999999", + "SELECT * FROM t WHERE a < 2999998", + "SELECT * FROM t WHERE a > 2999998", + "SELECT * FROM t WHERE a <= 2999998", + "SELECT * FROM t WHERE a >= 2999998", + "SELECT * FROM t WHERE a >= 2999998 AND a <= 3000000", + "SELECT * FROM t WHERE a > 2999998 AND a <= 3000000", + "SELECT * FROM t WHERE a > 2999998 AND a < 3000000", + "SELECT * FROM t WHERE a > 2999998 AND a <= 3000000", + "SELECT * FROM t WHERE a < 2999999", + "SELECT * FROM t WHERE a > 2999999", + "SELECT * FROM t WHERE a <= 2999999", + "SELECT * FROM t WHERE a >= 2999999", + "SELECT * FROM t WHERE a >= 2999999 AND a <= 3000001", + "SELECT * FROM t WHERE a > 2999999 AND a <= 3000001", + "SELECT * FROM t WHERE a > 2999999 AND a < 3000001", + "SELECT * FROM t WHERE a > 2999999 AND a <= 3000001", + "SELECT * FROM t WHERE a < 3000000", + "SELECT * FROM t WHERE a > 3000000", + "SELECT * FROM t WHERE a <= 3000000", + "SELECT * FROM t WHERE a >= 3000000", + "SELECT * FROM t WHERE a >= 3000000 AND a <= 3000002", + "SELECT * FROM t WHERE a > 3000000 AND a <= 3000002", + "SELECT * FROM t WHERE a > 3000000 AND a < 3000002", + "SELECT * FROM t WHERE a > 3000000 AND a <= 3000002" + ] + }, + { + "name": "TestRangePartitionBoundariesLtS", + "cases": [ + "INSERT INTO t VALUES (0, '0 Filler...')", + "INSERT INTO t VALUES (1, '1 Filler...')", + "INSERT INTO t VALUES (2, '2 Filler...')", + "INSERT INTO t VALUES (3, '3 Filler...')", + "INSERT INTO t VALUES (4, '4 Filler...')", + "INSERT INTO t VALUES (5, '5 Filler...')", + "INSERT INTO t VALUES (6, '6 Filler...')", + "ANALYZE TABLE t", + "SELECT * FROM t WHERE a < -1", + "SELECT * FROM t WHERE a > -1", + "SELECT * FROM t WHERE a <= -1", + "SELECT * FROM t WHERE a >= -1", + "SELECT * FROM t WHERE a < 2 OR a > -1", + "SELECT * FROM t WHERE a > 2 AND a < -1", + "SELECT * FROM t WHERE NOT (a < 2 OR a > -1)", + "SELECT * FROM t WHERE NOT (a > 2 AND a < -1)", + "SELECT * FROM t WHERE a < 2 OR a >= -1", + "SELECT * FROM t WHERE a >= 2 AND a < -1", + "SELECT * FROM t WHERE NOT (a < 2 OR a >= -1)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a < -1)", + "SELECT * FROM t WHERE a <= 2 OR a > -1", + "SELECT * FROM t WHERE a > 2 AND a <= -1", + "SELECT * FROM t WHERE NOT (a <= 2 OR a > -1)", + "SELECT * FROM t WHERE NOT (a > 2 AND a <= -1)", + "SELECT * FROM t WHERE a <= 2 OR a >= -1", + "SELECT * FROM t WHERE a >= 2 AND a <= -1", + "SELECT * FROM t WHERE NOT (a <= 2 OR a >= -1)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a <= -1)", + "SELECT * FROM t WHERE a < 0", + "SELECT * FROM t WHERE a > 0", + "SELECT * FROM t WHERE a <= 0", + "SELECT * FROM t WHERE a >= 0", + "SELECT * FROM t WHERE a < 2 OR a > 0", + "SELECT * FROM t WHERE a > 2 AND a < 0", + "SELECT * FROM t WHERE NOT (a < 2 OR a > 0)", + "SELECT * FROM t WHERE NOT (a > 2 AND a < 0)", + "SELECT * FROM t WHERE a < 2 OR a >= 0", + "SELECT * FROM t WHERE a >= 2 AND a < 0", + "SELECT * FROM t WHERE NOT (a < 2 OR a >= 0)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a < 0)", + "SELECT * FROM t WHERE a <= 2 OR a > 0", + "SELECT * FROM t WHERE a > 2 AND a <= 0", + "SELECT * FROM t WHERE NOT (a <= 2 OR a > 0)", + "SELECT * FROM t WHERE NOT (a > 2 AND a <= 0)", + "SELECT * FROM t WHERE a <= 2 OR a >= 0", + "SELECT * FROM t WHERE a >= 2 AND a <= 0", + "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 0)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 0)", + "SELECT * FROM t WHERE a < 1", + "SELECT * FROM t WHERE a > 1", + "SELECT * FROM t WHERE a <= 1", + "SELECT * FROM t WHERE a >= 1", + "SELECT * FROM t WHERE a < 2 OR a > 1", + "SELECT * FROM t WHERE a > 2 AND a < 1", + "SELECT * FROM t WHERE NOT (a < 2 OR a > 1)", + "SELECT * FROM t WHERE NOT (a > 2 AND a < 1)", + "SELECT * FROM t WHERE a < 2 OR a >= 1", + "SELECT * FROM t WHERE a >= 2 AND a < 1", + "SELECT * FROM t WHERE NOT (a < 2 OR a >= 1)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a < 1)", + "SELECT * FROM t WHERE a <= 2 OR a > 1", + "SELECT * FROM t WHERE a > 2 AND a <= 1", + "SELECT * FROM t WHERE NOT (a <= 2 OR a > 1)", + "SELECT * FROM t WHERE NOT (a > 2 AND a <= 1)", + "SELECT * FROM t WHERE a <= 2 OR a >= 1", + "SELECT * FROM t WHERE a >= 2 AND a <= 1", + "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 1)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 1)", + "SELECT * FROM t WHERE a < 2", + "SELECT * FROM t WHERE a > 2", + "SELECT * FROM t WHERE a <= 2", + "SELECT * FROM t WHERE a >= 2", + "SELECT * FROM t WHERE a < 2 OR a > 2", + "SELECT * FROM t WHERE a > 2 AND a < 2", + "SELECT * FROM t WHERE NOT (a < 2 OR a > 2)", + "SELECT * FROM t WHERE NOT (a > 2 AND a < 2)", + "SELECT * FROM t WHERE a < 2 OR a >= 2", + "SELECT * FROM t WHERE a >= 2 AND a < 2", + "SELECT * FROM t WHERE NOT (a < 2 OR a >= 2)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a < 2)", + "SELECT * FROM t WHERE a <= 2 OR a > 2", + "SELECT * FROM t WHERE a > 2 AND a <= 2", + "SELECT * FROM t WHERE NOT (a <= 2 OR a > 2)", + "SELECT * FROM t WHERE NOT (a > 2 AND a <= 2)", + "SELECT * FROM t WHERE a <= 2 OR a >= 2", + "SELECT * FROM t WHERE a >= 2 AND a <= 2", + "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 2)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 2)", + "SELECT * FROM t WHERE a < 3", + "SELECT * FROM t WHERE a > 3", + "SELECT * FROM t WHERE a <= 3", + "SELECT * FROM t WHERE a >= 3", + "SELECT * FROM t WHERE a < 2 OR a > 3", + "SELECT * FROM t WHERE a > 2 AND a < 3", + "SELECT * FROM t WHERE NOT (a < 2 OR a > 3)", + "SELECT * FROM t WHERE NOT (a > 2 AND a < 3)", + "SELECT * FROM t WHERE a < 2 OR a >= 3", + "SELECT * FROM t WHERE a >= 2 AND a < 3", + "SELECT * FROM t WHERE NOT (a < 2 OR a >= 3)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a < 3)", + "SELECT * FROM t WHERE a <= 2 OR a > 3", + "SELECT * FROM t WHERE a > 2 AND a <= 3", + "SELECT * FROM t WHERE NOT (a <= 2 OR a > 3)", + "SELECT * FROM t WHERE NOT (a > 2 AND a <= 3)", + "SELECT * FROM t WHERE a <= 2 OR a >= 3", + "SELECT * FROM t WHERE a >= 2 AND a <= 3", + "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 3)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 3)", + "SELECT * FROM t WHERE a < 4", + "SELECT * FROM t WHERE a > 4", + "SELECT * FROM t WHERE a <= 4", + "SELECT * FROM t WHERE a >= 4", + "SELECT * FROM t WHERE a < 2 OR a > 4", + "SELECT * FROM t WHERE a > 2 AND a < 4", + "SELECT * FROM t WHERE NOT (a < 2 OR a > 4)", + "SELECT * FROM t WHERE NOT (a > 2 AND a < 4)", + "SELECT * FROM t WHERE a < 2 OR a >= 4", + "SELECT * FROM t WHERE a >= 2 AND a < 4", + "SELECT * FROM t WHERE NOT (a < 2 OR a >= 4)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a < 4)", + "SELECT * FROM t WHERE a <= 2 OR a > 4", + "SELECT * FROM t WHERE a > 2 AND a <= 4", + "SELECT * FROM t WHERE NOT (a <= 2 OR a > 4)", + "SELECT * FROM t WHERE NOT (a > 2 AND a <= 4)", + "SELECT * FROM t WHERE a <= 2 OR a >= 4", + "SELECT * FROM t WHERE a >= 2 AND a <= 4", + "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 4)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 4)", + "SELECT * FROM t WHERE a < 5", + "SELECT * FROM t WHERE a > 5", + "SELECT * FROM t WHERE a <= 5", + "SELECT * FROM t WHERE a >= 5", + "SELECT * FROM t WHERE a < 2 OR a > 5", + "SELECT * FROM t WHERE a > 2 AND a < 5", + "SELECT * FROM t WHERE NOT (a < 2 OR a > 5)", + "SELECT * FROM t WHERE NOT (a > 2 AND a < 5)", + "SELECT * FROM t WHERE a < 2 OR a >= 5", + "SELECT * FROM t WHERE a >= 2 AND a < 5", + "SELECT * FROM t WHERE NOT (a < 2 OR a >= 5)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a < 5)", + "SELECT * FROM t WHERE a <= 2 OR a > 5", + "SELECT * FROM t WHERE a > 2 AND a <= 5", + "SELECT * FROM t WHERE NOT (a <= 2 OR a > 5)", + "SELECT * FROM t WHERE NOT (a > 2 AND a <= 5)", + "SELECT * FROM t WHERE a <= 2 OR a >= 5", + "SELECT * FROM t WHERE a >= 2 AND a <= 5", + "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 5)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 5)", + "SELECT * FROM t WHERE a < 6", + "SELECT * FROM t WHERE a > 6", + "SELECT * FROM t WHERE a <= 6", + "SELECT * FROM t WHERE a >= 6", + "SELECT * FROM t WHERE a < 2 OR a > 6", + "SELECT * FROM t WHERE a > 2 AND a < 6", + "SELECT * FROM t WHERE NOT (a < 2 OR a > 6)", + "SELECT * FROM t WHERE NOT (a > 2 AND a < 6)", + "SELECT * FROM t WHERE a < 2 OR a >= 6", + "SELECT * FROM t WHERE a >= 2 AND a < 6", + "SELECT * FROM t WHERE NOT (a < 2 OR a >= 6)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a < 6)", + "SELECT * FROM t WHERE a <= 2 OR a > 6", + "SELECT * FROM t WHERE a > 2 AND a <= 6", + "SELECT * FROM t WHERE NOT (a <= 2 OR a > 6)", + "SELECT * FROM t WHERE NOT (a > 2 AND a <= 6)", + "SELECT * FROM t WHERE a <= 2 OR a >= 6", + "SELECT * FROM t WHERE a >= 2 AND a <= 6", + "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 6)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 6)", + "SELECT * FROM t WHERE a < 7", + "SELECT * FROM t WHERE a > 7", + "SELECT * FROM t WHERE a <= 7", + "SELECT * FROM t WHERE a >= 7", + "SELECT * FROM t WHERE a < 2 OR a > 7", + "SELECT * FROM t WHERE a > 2 AND a < 7", + "SELECT * FROM t WHERE NOT (a < 2 OR a > 7)", + "SELECT * FROM t WHERE NOT (a > 2 AND a < 7)", + "SELECT * FROM t WHERE a < 2 OR a >= 7", + "SELECT * FROM t WHERE a >= 2 AND a < 7", + "SELECT * FROM t WHERE NOT (a < 2 OR a >= 7)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a < 7)", + "SELECT * FROM t WHERE a <= 2 OR a > 7", + "SELECT * FROM t WHERE a > 2 AND a <= 7", + "SELECT * FROM t WHERE NOT (a <= 2 OR a > 7)", + "SELECT * FROM t WHERE NOT (a > 2 AND a <= 7)", + "SELECT * FROM t WHERE a <= 2 OR a >= 7", + "SELECT * FROM t WHERE a >= 2 AND a <= 7", + "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 7)", + "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 7)" + ] } ] diff --git a/executor/testdata/executor_suite_out.json b/executor/testdata/executor_suite_out.json index bd5fbdb486ac0..fd834a5229662 100644 --- a/executor/testdata/executor_suite_out.json +++ b/executor/testdata/executor_suite_out.json @@ -2562,5 +2562,3816 @@ "Res": null } ] + }, + { + "Name": "TestRangePartitionBoundariesLtM", + "Cases": [ + { + "SQL": "INSERT INTO t VALUES (999998, '999998 Filler ...'), (999999, '999999 Filler ...'), (1000000, '1000000 Filler ...'), (1000001, '1000001 Filler ...'), (1000002, '1000002 Filler ...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (1999998, '1999998 Filler ...'), (1999999, '1999999 Filler ...'), (2000000, '2000000 Filler ...'), (2000001, '2000001 Filler ...'), (2000002, '2000002 Filler ...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (2999998, '2999998 Filler ...'), (2999999, '2999999 Filler ...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (-2147483648, 'MIN_INT filler...'), (0, '0 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "ANALYZE TABLE t", + "Plan": null, + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a < -2147483648", + "Plan": [ + "p0" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a > -2147483648", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= -2147483648", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= -2147483648", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 0", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 0", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 0", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 0", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 999998", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999998", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 999998", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "999998 999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 999998", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 999999", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "999998 999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999999", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 999999", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 999999", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 1000000", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1000000", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 1000000", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1000000", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 1000001", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1000001", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 1000001", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1000001", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 1000002", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1000002", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 1000002", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1000002", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 3000000", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 3000000", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a <= 3000000", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 3000000", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a < 3000001", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 3000001", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a <= 3000001", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 3000001", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a < 999997", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999997", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 999997", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 999997", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 999997 AND a <= 999999", + "Plan": [ + "p0" + ], + "Res": [ + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999997 AND a <= 999999", + "Plan": [ + "p0" + ], + "Res": [ + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999997 AND a < 999999", + "Plan": [ + "p0" + ], + "Res": [ + "999998 999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999997 AND a <= 999999", + "Plan": [ + "p0" + ], + "Res": [ + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 999998", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999998", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 999998", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "999998 999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 999998", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 999998 AND a <= 1000000", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999998 AND a <= 1000000", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999998 AND a < 1000000", + "Plan": [ + "p0" + ], + "Res": [ + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999998 AND a <= 1000000", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 999999", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "999998 999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999999", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 999999", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 999999", + "Plan": [ + "all" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 999999 AND a <= 1000001", + "Plan": [ + "p0 p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999999 AND a <= 1000001", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999999 AND a < 1000001", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 999999 AND a <= 1000001", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 1000000", + "Plan": [ + "p0" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1000000", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 1000000", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1000000", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1000000 AND a <= 1000002", + "Plan": [ + "p1" + ], + "Res": [ + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1000000 AND a <= 1000002", + "Plan": [ + "p1" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1000000 AND a < 1000002", + "Plan": [ + "p1" + ], + "Res": [ + "1000001 1000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1000000 AND a <= 1000002", + "Plan": [ + "p1" + ], + "Res": [ + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 1999997", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999997", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 1999997", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1999997", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1999997 AND a <= 1999999", + "Plan": [ + "p1" + ], + "Res": [ + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999997 AND a <= 1999999", + "Plan": [ + "p1" + ], + "Res": [ + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999997 AND a < 1999999", + "Plan": [ + "p1" + ], + "Res": [ + "1999998 1999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999997 AND a <= 1999999", + "Plan": [ + "p1" + ], + "Res": [ + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 1999998", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999998", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 1999998", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1999998", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1999998 AND a <= 2000000", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999998 AND a <= 2000000", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999998 AND a < 2000000", + "Plan": [ + "p1" + ], + "Res": [ + "1999999 1999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999998 AND a <= 2000000", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 1999999", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999999", + "Plan": [ + "p2" + ], + "Res": [ + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 1999999", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1999999", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1999999 AND a <= 2000001", + "Plan": [ + "p1 p2" + ], + "Res": [ + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999999 AND a <= 2000001", + "Plan": [ + "p2" + ], + "Res": [ + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999999 AND a < 2000001", + "Plan": [ + "p2" + ], + "Res": [ + "2000000 2000000 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1999999 AND a <= 2000001", + "Plan": [ + "p2" + ], + "Res": [ + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2000000", + "Plan": [ + "p0 p1" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2000000", + "Plan": [ + "p2" + ], + "Res": [ + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2000000", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2000000", + "Plan": [ + "p2" + ], + "Res": [ + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2000000 AND a <= 2000002", + "Plan": [ + "p2" + ], + "Res": [ + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2000000 AND a <= 2000002", + "Plan": [ + "p2" + ], + "Res": [ + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2000000 AND a < 2000002", + "Plan": [ + "p2" + ], + "Res": [ + "2000001 2000001 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2000000 AND a <= 2000002", + "Plan": [ + "p2" + ], + "Res": [ + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2999997", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999997", + "Plan": [ + "p2" + ], + "Res": [ + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2999997", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2999997", + "Plan": [ + "p2" + ], + "Res": [ + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2999997 AND a <= 2999999", + "Plan": [ + "p2" + ], + "Res": [ + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999997 AND a <= 2999999", + "Plan": [ + "p2" + ], + "Res": [ + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999997 AND a < 2999999", + "Plan": [ + "p2" + ], + "Res": [ + "2999998 2999998 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999997 AND a <= 2999999", + "Plan": [ + "p2" + ], + "Res": [ + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2999998", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999998", + "Plan": [ + "p2" + ], + "Res": [ + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2999998", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2999998", + "Plan": [ + "p2" + ], + "Res": [ + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2999998 AND a <= 3000000", + "Plan": [ + "p2" + ], + "Res": [ + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999998 AND a <= 3000000", + "Plan": [ + "p2" + ], + "Res": [ + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999998 AND a < 3000000", + "Plan": [ + "p2" + ], + "Res": [ + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999998 AND a <= 3000000", + "Plan": [ + "p2" + ], + "Res": [ + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2999999", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999999", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2999999", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2999999", + "Plan": [ + "p2" + ], + "Res": [ + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2999999 AND a <= 3000001", + "Plan": [ + "p2" + ], + "Res": [ + "2999999 2999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999999 AND a <= 3000001", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999999 AND a < 3000001", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a > 2999999 AND a <= 3000001", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a < 3000000", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 3000000", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a <= 3000000", + "Plan": [ + "all" + ], + "Res": [ + "-2147483648 MIN_INT filler...", + "0 0 Filler...", + "1000000 1000000 Filler ...", + "1000001 1000001 Filler ...", + "1000002 1000002 Filler ...", + "1999998 1999998 Filler ...", + "1999999 1999999 Filler ...", + "2000000 2000000 Filler ...", + "2000001 2000001 Filler ...", + "2000002 2000002 Filler ...", + "2999998 2999998 Filler ...", + "2999999 2999999 Filler ...", + "999998 999998 Filler ...", + "999999 999999 Filler ..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 3000000", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a >= 3000000 AND a <= 3000002", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a > 3000000 AND a <= 3000002", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a > 3000000 AND a < 3000002", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a > 3000000 AND a <= 3000002", + "Plan": [ + "dual" + ], + "Res": null + } + ] + }, + { + "Name": "TestRangePartitionBoundariesLtS", + "Cases": [ + { + "SQL": "INSERT INTO t VALUES (0, '0 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (1, '1 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (2, '2 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (3, '3 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (4, '4 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (5, '5 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "INSERT INTO t VALUES (6, '6 Filler...')", + "Plan": null, + "Res": null + }, + { + "SQL": "ANALYZE TABLE t", + "Plan": null, + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a < -1", + "Plan": [ + "p0" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a > -1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= -1", + "Plan": [ + "p0" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a >= -1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a > -1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a < -1", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a > -1)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a < -1)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a >= -1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a < -1", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a >= -1)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a < -1)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a > -1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a <= -1", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a > -1)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a <= -1)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a >= -1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a <= -1", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a >= -1)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a <= -1)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 0", + "Plan": [ + "p0" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a > 0", + "Plan": [ + "p1 p2 p3 p4 p5 p6" + ], + "Res": [ + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 0", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 0", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a > 0", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a < 0", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a > 0)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a < 0)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a >= 0", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a < 0", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a >= 0)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a < 0)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a > 0", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a <= 0", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a > 0)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a <= 0)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a >= 0", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a <= 0", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 0)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 0)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 1", + "Plan": [ + "p0" + ], + "Res": [ + "0 0 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 1", + "Plan": [ + "p2 p3 p4 p5 p6" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 1", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 1", + "Plan": [ + "p1 p2 p3 p4 p5 p6" + ], + "Res": [ + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a > 1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a < 1", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a > 1)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a < 1)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a >= 1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a < 1", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a >= 1)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a < 1)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a > 1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a <= 1", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a > 1)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a <= 1)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a >= 1", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a <= 1", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 1)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 1)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2", + "Plan": [ + "p3 p4 p5 p6" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2", + "Plan": [ + "p0 p1 p2" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2", + "Plan": [ + "p2 p3 p4 p5 p6" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a > 2", + "Plan": [ + "p0 p1 p3 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a < 2", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a > 2)", + "Plan": [ + "p2" + ], + "Res": [ + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a < 2)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a >= 2", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a < 2", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a >= 2)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a < 2)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a > 2", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a <= 2", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a > 2)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a <= 2)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a >= 2", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a <= 2", + "Plan": [ + "p2" + ], + "Res": [ + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 2)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 2)", + "Plan": [ + "p0 p1 p3 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 3", + "Plan": [ + "p0 p1 p2" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 3", + "Plan": [ + "p4 p5 p6" + ], + "Res": [ + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 3", + "Plan": [ + "p0 p1 p2 p3" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 3", + "Plan": [ + "p3 p4 p5 p6" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a > 3", + "Plan": [ + "p0 p1 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a < 3", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a > 3)", + "Plan": [ + "p2 p3" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a < 3)", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a >= 3", + "Plan": [ + "p0 p1 p3 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a < 3", + "Plan": [ + "p2" + ], + "Res": [ + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a >= 3)", + "Plan": [ + "p2" + ], + "Res": [ + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a < 3)", + "Plan": [ + "p0 p1 p3 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a > 3", + "Plan": [ + "p0 p1 p2 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a <= 3", + "Plan": [ + "p3" + ], + "Res": [ + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a > 3)", + "Plan": [ + "p3" + ], + "Res": [ + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a <= 3)", + "Plan": [ + "p0 p1 p2 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a >= 3", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a <= 3", + "Plan": [ + "p2 p3" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 3)", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 3)", + "Plan": [ + "p0 p1 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 4", + "Plan": [ + "p0 p1 p2 p3" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 4", + "Plan": [ + "p5 p6" + ], + "Res": [ + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 4", + "Plan": [ + "p0 p1 p2 p3 p4" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 4", + "Plan": [ + "p4 p5 p6" + ], + "Res": [ + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a > 4", + "Plan": [ + "p0 p1 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a < 4", + "Plan": [ + "p3" + ], + "Res": [ + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a > 4)", + "Plan": [ + "p2 p3 p4" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a < 4)", + "Plan": [ + "p0 p1 p2 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a >= 4", + "Plan": [ + "p0 p1 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a < 4", + "Plan": [ + "p2 p3" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a >= 4)", + "Plan": [ + "p2 p3" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a < 4)", + "Plan": [ + "p0 p1 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a > 4", + "Plan": [ + "p0 p1 p2 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a <= 4", + "Plan": [ + "p3 p4" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a > 4)", + "Plan": [ + "p3 p4" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a <= 4)", + "Plan": [ + "p0 p1 p2 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a >= 4", + "Plan": [ + "p0 p1 p2 p4 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a <= 4", + "Plan": [ + "p2 p3 p4" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 4)", + "Plan": [ + "p3" + ], + "Res": [ + "3 3 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 4)", + "Plan": [ + "p0 p1 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 5", + "Plan": [ + "p0 p1 p2 p3 p4" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 5", + "Plan": [ + "p6" + ], + "Res": [ + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 5", + "Plan": [ + "p0 p1 p2 p3 p4 p5" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 5", + "Plan": [ + "p5 p6" + ], + "Res": [ + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a > 5", + "Plan": [ + "p0 p1 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a < 5", + "Plan": [ + "p3 p4" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a > 5)", + "Plan": [ + "p2 p3 p4 p5" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a < 5)", + "Plan": [ + "p0 p1 p2 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a >= 5", + "Plan": [ + "p0 p1 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a < 5", + "Plan": [ + "p2 p3 p4" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a >= 5)", + "Plan": [ + "p2 p3 p4" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a < 5)", + "Plan": [ + "p0 p1 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a > 5", + "Plan": [ + "p0 p1 p2 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a <= 5", + "Plan": [ + "p3 p4 p5" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a > 5)", + "Plan": [ + "p3 p4 p5" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a <= 5)", + "Plan": [ + "p0 p1 p2 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a >= 5", + "Plan": [ + "p0 p1 p2 p5 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a <= 5", + "Plan": [ + "p2 p3 p4 p5" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 5)", + "Plan": [ + "p3 p4" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 5)", + "Plan": [ + "p0 p1 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 6", + "Plan": [ + "p0 p1 p2 p3 p4 p5" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 6", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a <= 6", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 6", + "Plan": [ + "p6" + ], + "Res": [ + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a > 6", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a < 6", + "Plan": [ + "p3 p4 p5" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a > 6)", + "Plan": [ + "p2 p3 p4 p5 p6" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a < 6)", + "Plan": [ + "p0 p1 p2 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a >= 6", + "Plan": [ + "p0 p1 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a < 6", + "Plan": [ + "p2 p3 p4 p5" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a >= 6)", + "Plan": [ + "p2 p3 p4 p5" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a < 6)", + "Plan": [ + "p0 p1 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a > 6", + "Plan": [ + "p0 p1 p2" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a <= 6", + "Plan": [ + "p3 p4 p5 p6" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a > 6)", + "Plan": [ + "p3 p4 p5 p6" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a <= 6)", + "Plan": [ + "p0 p1 p2" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a >= 6", + "Plan": [ + "p0 p1 p2 p6" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a <= 6", + "Plan": [ + "p2 p3 p4 p5 p6" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 6)", + "Plan": [ + "p3 p4 p5" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 6)", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 7", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 7", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a <= 7", + "Plan": [ + "all" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 7", + "Plan": [ + "dual" + ], + "Res": null + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a > 7", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a < 7", + "Plan": [ + "p3 p4 p5 p6" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a > 7)", + "Plan": [ + "p2 p3 p4 p5 p6" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a < 7)", + "Plan": [ + "p0 p1 p2" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a < 2 OR a >= 7", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a < 7", + "Plan": [ + "p2 p3 p4 p5 p6" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a < 2 OR a >= 7)", + "Plan": [ + "p2 p3 p4 p5 p6" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a < 7)", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a > 7", + "Plan": [ + "p0 p1 p2" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a > 2 AND a <= 7", + "Plan": [ + "p3 p4 p5 p6" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a > 7)", + "Plan": [ + "p3 p4 p5 p6" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a > 2 AND a <= 7)", + "Plan": [ + "p0 p1 p2" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a <= 2 OR a >= 7", + "Plan": [ + "p0 p1 p2" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler...", + "2 2 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE a >= 2 AND a <= 7", + "Plan": [ + "p2 p3 p4 p5 p6" + ], + "Res": [ + "2 2 Filler...", + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a <= 2 OR a >= 7)", + "Plan": [ + "p3 p4 p5 p6" + ], + "Res": [ + "3 3 Filler...", + "4 4 Filler...", + "5 5 Filler...", + "6 6 Filler..." + ] + }, + { + "SQL": "SELECT * FROM t WHERE NOT (a >= 2 AND a <= 7)", + "Plan": [ + "p0 p1" + ], + "Res": [ + "0 0 Filler...", + "1 1 Filler..." + ] + } + ] } ] From db71d7fb831ecf60985745fbbf0749979111315e Mon Sep 17 00:00:00 2001 From: tangenta Date: Thu, 20 May 2021 21:54:21 +0800 Subject: [PATCH 127/343] test: record random seed in TestIssue20658 (#24782) --- executor/aggregate_test.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 94820a028123d..85e8bcc4f236d 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -1293,12 +1293,14 @@ func (s *testSuiteAgg) TestIssue20658(c *C) { tk.MustExec("CREATE TABLE t(a bigint, b bigint);") tk.MustExec("set tidb_init_chunk_size=1;") tk.MustExec("set tidb_max_chunk_size=32;") + randSeed := time.Now().UnixNano() + r := rand.New(rand.NewSource(randSeed)) var insertSQL string for i := 0; i < 1000; i++ { if i == 0 { - insertSQL += fmt.Sprintf("(%d, %d)", rand.Intn(100), rand.Intn(100)) + insertSQL += fmt.Sprintf("(%d, %d)", r.Intn(100), r.Intn(100)) } else { - insertSQL += fmt.Sprintf(",(%d, %d)", rand.Intn(100), rand.Intn(100)) + insertSQL += fmt.Sprintf(",(%d, %d)", r.Intn(100), r.Intn(100)) } } tk.MustExec(fmt.Sprintf("insert into t values %s;", insertSQL)) @@ -1307,7 +1309,7 @@ func (s *testSuiteAgg) TestIssue20658(c *C) { for _, sql := range sqls { var expected [][]interface{} for _, con := range concurrencies { - comment := Commentf("sql: %s; concurrency: %d", sql, con) + comment := Commentf("sql: %s; concurrency: %d, seed: ", sql, con, randSeed) tk.MustExec(fmt.Sprintf("set @@tidb_streamagg_concurrency=%d;", con)) if con == 1 { expected = tk.MustQuery(sql).Sort().Rows() From 789874e201c50d7509620d6b1f48eb6cb41781b6 Mon Sep 17 00:00:00 2001 From: Shirly Date: Thu, 20 May 2021 22:06:21 +0800 Subject: [PATCH 128/343] store/tikv/retry: define Config instead of BackoffType (#24692) --- store/copr/batch_coprocessor.go | 9 +- store/copr/coprocessor.go | 13 +- store/copr/mpp.go | 7 +- store/driver/backoff/backoff.go | 23 +- store/driver/tikv_driver.go | 2 +- store/gcworker/gc_worker.go | 10 +- store/tikv/backoff.go | 38 ++- store/tikv/batch_request_sender.go | 3 +- store/tikv/client_batch.go | 2 +- store/tikv/prewrite.go | 2 +- store/tikv/region_request.go | 2 +- store/tikv/retry/backoff.go | 287 ++++++--------------- store/tikv/retry/backoff_test.go | 2 +- store/tikv/retry/config.go | 159 ++++++++++++ store/tikv/scan.go | 2 +- store/tikv/snapshot.go | 18 +- store/tikv/tests/snapshot_test.go | 2 +- util/stmtsummary/statement_summary_test.go | 14 +- 18 files changed, 313 insertions(+), 282 deletions(-) create mode 100644 store/tikv/retry/config.go diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 8c73f58fbf892..6efbf76775186 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -302,7 +302,7 @@ func buildBatchCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.Key // As mentioned above, nil rpcCtx is always attributed to failed stores. // It's equal to long poll the store but get no response. Here we'd better use // TiFlash error to trigger the TiKV fallback mechanism. - err = bo.Backoff(tikv.BoTiFlashRPC, errors.New("Cannot find region with TiFlash peer")) + err = bo.Backoff(tikv.BoTiFlashRPC(), errors.New("Cannot find region with TiFlash peer")) if err != nil { return nil, errors.Trace(err) } @@ -548,7 +548,7 @@ func (b *batchCopIterator) handleStreamedBatchCopResponse(ctx context.Context, b return nil } - if err1 := bo.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil { + if err1 := bo.Backoff(tikv.BoTiKVRPC(), errors.Errorf("recv stream response error: %v, task store addr: %s", err, task.storeAddr)); err1 != nil { return errors.Trace(err) } @@ -597,9 +597,8 @@ func (b *batchCopIterator) handleBatchCopResponse(bo *Backoffer, response *copro resp.detail.BackoffSleep = make(map[string]time.Duration, len(backoffTimes)) resp.detail.BackoffTimes = make(map[string]int, len(backoffTimes)) for backoff := range backoffTimes { - backoffName := backoff.String() - resp.detail.BackoffTimes[backoffName] = backoffTimes[backoff] - resp.detail.BackoffSleep[backoffName] = time.Duration(bo.GetBackoffSleepMS()[backoff]) * time.Millisecond + resp.detail.BackoffTimes[backoff] = backoffTimes[backoff] + resp.detail.BackoffSleep[backoff] = time.Duration(bo.GetBackoffSleepMS()[backoff]) * time.Millisecond } resp.detail.CalleeAddress = task.storeAddr diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 989a6d835ce0f..c66c3cda9af35 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -833,9 +833,9 @@ func (worker *copIteratorWorker) handleCopStreamResult(bo *Backoffer, rpcCtx *ti err1 := errors.Errorf("recv stream response error: %v, task: %s", err, task) if task.storeType == kv.TiFlash { - err1 = bo.Backoff(tikv.BoTiFlashRPC, err1) + err1 = bo.Backoff(tikv.BoTiFlashRPC(), err1) } else { - err1 = bo.BackoffTiKVRPC(err1) + err1 = bo.Backoff(tikv.BoTiKVRPC(), err1) } if err1 != nil { @@ -869,7 +869,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.R } errStr := fmt.Sprintf("region_id:%v, region_ver:%v, store_type:%s, peer_addr:%s, error:%s", task.region.GetID(), task.region.GetVer(), task.storeType.Name(), task.storeAddr, regionErr.String()) - if err := bo.Backoff(tikv.BoRegionMiss, errors.New(errStr)); err != nil { + if err := bo.Backoff(tikv.BoRegionMiss(), errors.New(errStr)); err != nil { return nil, errors.Trace(err) } // We may meet RegionError at the first packet, but not during visiting the stream. @@ -884,7 +884,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.R return nil, errors.Trace(err1) } if msBeforeExpired > 0 { - if err := bo.BackoffWithMaxSleep(tikv.BoTxnLockFast, int(msBeforeExpired), errors.New(lockErr.String())); err != nil { + if err := bo.BackoffWithMaxSleepTxnLockFast(int(msBeforeExpired), errors.New(lockErr.String())); err != nil { return nil, errors.Trace(err) } } @@ -915,9 +915,8 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.R resp.detail.BackoffSleep = make(map[string]time.Duration, len(backoffTimes)) resp.detail.BackoffTimes = make(map[string]int, len(backoffTimes)) for backoff := range backoffTimes { - backoffName := backoff.String() - resp.detail.BackoffTimes[backoffName] = backoffTimes[backoff] - resp.detail.BackoffSleep[backoffName] = time.Duration(bo.GetBackoffSleepMS()[backoff]) * time.Millisecond + resp.detail.BackoffTimes[backoff] = backoffTimes[backoff] + resp.detail.BackoffSleep[backoff] = time.Duration(bo.GetBackoffSleepMS()[backoff]) * time.Millisecond } if rpcCtx != nil { resp.detail.CalleeAddress = rpcCtx.Addr diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 1941f2b3fbfa4..6d58e4ef732fe 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -340,7 +340,7 @@ func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchReques return } - if err1 := bo.BackoffTiKVRPC(errors.Errorf("recv stream response error: %v", err)); err1 != nil { + if err1 := bo.Backoff(tikv.BoTiKVRPC(), errors.Errorf("recv stream response error: %v", err)); err1 != nil { if errors.Cause(err) == context.Canceled { logutil.BgLogger().Info("stream recv timeout", zap.Error(err)) } else { @@ -383,9 +383,8 @@ func (m *mppIterator) handleMPPStreamResponse(bo *Backoffer, response *mpp.MPPDa resp.detail.BackoffSleep = make(map[string]time.Duration, len(backoffTimes)) resp.detail.BackoffTimes = make(map[string]int, len(backoffTimes)) for backoff := range backoffTimes { - backoffName := backoff.String() - resp.detail.BackoffTimes[backoffName] = backoffTimes[backoff] - resp.detail.BackoffSleep[backoffName] = time.Duration(bo.GetBackoffSleepMS()[backoff]) * time.Millisecond + resp.detail.BackoffTimes[backoff] = backoffTimes[backoff] + resp.detail.BackoffSleep[backoff] = time.Duration(bo.GetBackoffSleepMS()[backoff]) * time.Millisecond } resp.detail.CalleeAddress = req.Meta.GetAddress() diff --git a/store/driver/backoff/backoff.go b/store/driver/backoff/backoff.go index f634366381d06..35979edc638b4 100644 --- a/store/driver/backoff/backoff.go +++ b/store/driver/backoff/backoff.go @@ -43,29 +43,22 @@ func (b *Backoffer) TiKVBackoffer() *tikv.Backoffer { return b.b } -// Backoff sleeps a while base on the backoffType and records the error message. +// Backoff sleeps a while base on the BackoffConfig and records the error message. // It returns a retryable error if total sleep time exceeds maxSleep. -func (b *Backoffer) Backoff(typ tikv.BackoffType, err error) error { - e := b.b.Backoff(typ, err) +func (b *Backoffer) Backoff(cfg *tikv.BackoffConfig, err error) error { + e := b.b.Backoff(cfg, err) return derr.ToTiDBErr(e) } -// BackoffTiKVRPC sleeps a while base on the TiKVRPC and records the error message. -// It returns a retryable error if total sleep time exceeds maxSleep. -func (b *Backoffer) BackoffTiKVRPC(err error) error { - e := b.b.BackoffTiKVRPC(err) - return derr.ToTiDBErr(e) -} - -// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message +// BackoffWithMaxSleepTxnLockFast sleeps a while for the operation TxnLockFast and records the error message // and never sleep more than maxSleepMs for each sleep. -func (b *Backoffer) BackoffWithMaxSleep(typ tikv.BackoffType, maxSleepMs int, err error) error { - e := b.b.BackoffWithMaxSleep(typ, maxSleepMs, err) +func (b *Backoffer) BackoffWithMaxSleepTxnLockFast(maxSleepMs int, err error) error { + e := b.b.BackoffWithMaxSleepTxnLockFast(maxSleepMs, err) return derr.ToTiDBErr(e) } // GetBackoffTimes returns a map contains backoff time count by type. -func (b *Backoffer) GetBackoffTimes() map[tikv.BackoffType]int { +func (b *Backoffer) GetBackoffTimes() map[string]int { return b.b.GetBackoffTimes() } @@ -80,7 +73,7 @@ func (b *Backoffer) GetVars() *tikv.Variables { } // GetBackoffSleepMS returns a map contains backoff sleep time by type. -func (b *Backoffer) GetBackoffSleepMS() map[tikv.BackoffType]int { +func (b *Backoffer) GetBackoffSleepMS() map[string]int { return b.b.GetBackoffSleepMS() } diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index 2d93b7eda4abb..5f5471d8e7251 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -231,7 +231,7 @@ func (s *tikvStore) EtcdAddrs() ([]string, error) { for { members, err := pdClient.GetAllMembers(ctx) if err != nil { - err := bo.Backoff(tikv.BoRegionMiss, err) + err := bo.Backoff(tikv.BoRegionMiss(), err) if err != nil { return nil, err } diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index b5b42df1838b9..72ae8bc34f0fa 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -1089,7 +1089,7 @@ retryScanAndResolve: return stat, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(tikv.BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(tikv.BoRegionMiss(), errors.New(regionErr.String())) if err != nil { return stat, errors.Trace(err) } @@ -1125,7 +1125,7 @@ retryScanAndResolve: return stat, errors.Trace(err1) } if !ok { - err = bo.Backoff(tikv.BoTxnLock, errors.Errorf("remain locks: %d", len(locks))) + err = bo.Backoff(tikv.BoTxnLock(), errors.Errorf("remain locks: %d", len(locks))) if err != nil { return stat, errors.Trace(err) } @@ -1497,7 +1497,7 @@ func (w *GCWorker) resolveLocksAcrossRegions(ctx context.Context, locks []*tikv. return errors.Trace(err) } if !ok { - err = bo.Backoff(tikv.BoTxnLock, errors.Errorf("remain locks: %d", len(locks))) + err = bo.Backoff(tikv.BoTxnLock(), errors.Errorf("remain locks: %d", len(locks))) if err != nil { return errors.Trace(err) } @@ -1525,7 +1525,7 @@ func (w *GCWorker) uploadSafePointToPD(ctx context.Context, safePoint uint64) er if errors.Cause(err) == context.Canceled { return errors.Trace(err) } - err = bo.Backoff(tikv.BoPDRPC, errors.Errorf("failed to upload safe point to PD, err: %v", err)) + err = bo.Backoff(tikv.BoPDRPC(), errors.Errorf("failed to upload safe point to PD, err: %v", err)) if err != nil { return errors.Trace(err) } @@ -1567,7 +1567,7 @@ func (w *GCWorker) doGCForRange(ctx context.Context, startKey []byte, endKey []b // we check regionErr here first, because we know 'regionErr' and 'err' should not return together, to keep it to // make the process correct. if regionErr != nil { - err = bo.Backoff(tikv.BoRegionMiss, errors.New(regionErr.String())) + err = bo.Backoff(tikv.BoRegionMiss(), errors.New(regionErr.String())) if err == nil { continue } diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index c622e21d2ee5d..918acc9addcb8 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -23,17 +23,8 @@ import ( // Backoffer is a utility for retrying queries. type Backoffer = retry.Backoffer -// BackoffType defines the backoff type. -type BackoffType = retry.BackoffType - -// Back off types. -const ( - BoRegionMiss = retry.BoRegionMiss - BoTiFlashRPC = retry.BoTiFlashRPC - BoTxnLockFast = retry.BoTxnLockFast - BoTxnLock = retry.BoTxnLock - BoPDRPC = retry.BoPDRPC -) +// BackoffConfig defines the backoff configuration. +type BackoffConfig = retry.Config // Maximum total sleep time(in ms) for kv/cop commands. const ( @@ -62,6 +53,31 @@ func TxnStartKey() interface{} { return retry.TxnStartKey } +// BoRegionMiss returns the default backoff config for RegionMiss. +func BoRegionMiss() *BackoffConfig { + return retry.BoRegionMiss +} + +// BoTiFlashRPC returns the default backoff config for TiFlashRPC. +func BoTiFlashRPC() *BackoffConfig { + return retry.BoTiFlashRPC +} + +// BoTxnLock returns the default backoff config for TxnLock. +func BoTxnLock() *BackoffConfig { + return retry.BoTxnLock +} + +// BoPDRPC returns the default backoff config for PDRPC. +func BoPDRPC() *BackoffConfig { + return retry.BoPDRPC +} + +// BoTiKVRPC returns the default backoff config for TiKVRPC. +func BoTiKVRPC() *BackoffConfig { + return retry.BoTiKVRPC +} + // NewGcResolveLockMaxBackoffer creates a Backoffer for Gc to resolve lock. func NewGcResolveLockMaxBackoffer(ctx context.Context) *Backoffer { return retry.NewBackofferWithVars(ctx, gcResolveLockMaxBackoff, nil) diff --git a/store/tikv/batch_request_sender.go b/store/tikv/batch_request_sender.go index 9aad070b70306..74a62dcfd781c 100644 --- a/store/tikv/batch_request_sender.go +++ b/store/tikv/batch_request_sender.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -93,6 +94,6 @@ func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx // When a store is not available, the leader of related region should be elected quickly. // TODO: the number of retry time should be limited:since region may be unavailable // when some unrecoverable disaster happened. - err = bo.Backoff(BoTiFlashRPC, errors.Errorf("send request error: %v, ctx: %v, regionInfos: %v", err, ctx, regionInfos)) + err = bo.Backoff(retry.BoTiFlashRPC, errors.Errorf("send request error: %v, ctx: %v, regionInfos: %v", err, ctx, regionInfos)) return errors.Trace(err) } diff --git a/store/tikv/client_batch.go b/store/tikv/client_batch.go index e5ec039fc6911..70f1cf27ccacc 100644 --- a/store/tikv/client_batch.go +++ b/store/tikv/client_batch.go @@ -673,7 +673,7 @@ func (c *batchCommandsClient) recreateStreamingClient(err error, streamClient *b break } - err2 := b.BackoffTiKVRPC(err1) + err2 := b.Backoff(retry.BoTiKVRPC, err1) // As timeout is set to math.MaxUint32, err2 should always be nil. // This line is added to make the 'make errcheck' pass. terror.Log(err2) diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index ffb47e1fb46fa..49ddc1525b748 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -269,7 +269,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff } atomic.AddInt64(&c.getDetail().ResolveLockTime, int64(time.Since(start))) if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(retry.BoTxnLock, int(msBeforeExpired), errors.Errorf("2PC prewrite lockedKeys: %d", len(locks))) + err = bo.BackoffWithCfgAndMaxSleep(retry.BoTxnLock, int(msBeforeExpired), errors.Errorf("2PC prewrite lockedKeys: %d", len(locks))) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index f42f7add092db..36b297c580102 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -575,7 +575,7 @@ func (s *RegionRequestSender) onSendFail(bo *Backoffer, ctx *RPCContext, err err if ctx.Store != nil && ctx.Store.storeType == tikvrpc.TiFlash { err = bo.Backoff(retry.BoTiFlashRPC, errors.Errorf("send tiflash request error: %v, ctx: %v, try next peer later", err, ctx)) } else { - err = bo.BackoffTiKVRPC(errors.Errorf("send tikv request error: %v, ctx: %v, try next peer later", err, ctx)) + err = bo.Backoff(retry.BoTiKVRPC, errors.Errorf("send tikv request error: %v, ctx: %v, try next peer later", err, ctx)) } return errors.Trace(err) } diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go index 9e7a527c69caa..d07b9c4fdccae 100644 --- a/store/tikv/retry/backoff.go +++ b/store/tikv/retry/backoff.go @@ -17,7 +17,6 @@ import ( "context" "fmt" "math" - "math/rand" "strings" "sync/atomic" "time" @@ -30,202 +29,24 @@ import ( "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/util" - "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) -const ( - // NoJitter makes the backoff sequence strict exponential. - NoJitter = 1 + iota - // FullJitter applies random factors to strict exponential. - FullJitter - // EqualJitter is also randomized, but prevents very short sleeps. - EqualJitter - // DecorrJitter increases the maximum jitter based on the last random value. - DecorrJitter -) - -func (t BackoffType) metric() prometheus.Observer { - switch t { - // TODO: distinguish tikv and tiflash in metrics - case boTiKVRPC, BoTiFlashRPC: - return metrics.BackoffHistogramRPC - case BoTxnLock: - return metrics.BackoffHistogramLock - case BoTxnLockFast: - return metrics.BackoffHistogramLockFast - case BoPDRPC: - return metrics.BackoffHistogramPD - case BoRegionMiss: - return metrics.BackoffHistogramRegionMiss - case BoTiKVServerBusy, BoTiFlashServerBusy: - return metrics.BackoffHistogramServerBusy - case BoStaleCmd: - return metrics.BackoffHistogramStaleCmd - } - return metrics.BackoffHistogramEmpty -} - -// NewBackoffFn creates a backoff func which implements exponential backoff with -// optional jitters. -// See http://www.awsarchitectureblog.com/2015/03/backoff.html -func NewBackoffFn(base, cap, jitter int) func(ctx context.Context, maxSleepMs int) int { - if base < 2 { - // Top prevent panic in 'rand.Intn'. - base = 2 - } - attempts := 0 - lastSleep := base - return func(ctx context.Context, maxSleepMs int) int { - var sleep int - switch jitter { - case NoJitter: - sleep = expo(base, cap, attempts) - case FullJitter: - v := expo(base, cap, attempts) - sleep = rand.Intn(v) - case EqualJitter: - v := expo(base, cap, attempts) - sleep = v/2 + rand.Intn(v/2) - case DecorrJitter: - sleep = int(math.Min(float64(cap), float64(base+rand.Intn(lastSleep*3-base)))) - } - logutil.BgLogger().Debug("backoff", - zap.Int("base", base), - zap.Int("sleep", sleep), - zap.Int("attempts", attempts)) - - realSleep := sleep - // when set maxSleepMs >= 0 in `tikv.BackoffWithMaxSleep` will force sleep maxSleepMs milliseconds. - if maxSleepMs >= 0 && realSleep > maxSleepMs { - realSleep = maxSleepMs - } - select { - case <-time.After(time.Duration(realSleep) * time.Millisecond): - attempts++ - lastSleep = sleep - return realSleep - case <-ctx.Done(): - return 0 - } - } -} - -func expo(base, cap, n int) int { - return int(math.Min(float64(cap), float64(base)*math.Pow(2.0, float64(n)))) -} - -// BackoffType defines the backoff type. -type BackoffType int - -// Back off types. -const ( - boTiKVRPC BackoffType = iota - BoTiFlashRPC - BoTxnLock - BoTxnLockFast - BoPDRPC - BoRegionMiss - BoTiKVServerBusy - BoTiFlashServerBusy - BoTxnNotFound - BoStaleCmd - BoMaxTsNotSynced -) - -func (t BackoffType) createFn(vars *kv.Variables) func(context.Context, int) int { - switch t { - case boTiKVRPC, BoTiFlashRPC: - return NewBackoffFn(100, 2000, EqualJitter) - case BoTxnLock: - return NewBackoffFn(200, 3000, EqualJitter) - case BoTxnLockFast: - return NewBackoffFn(vars.BackoffLockFast, 3000, EqualJitter) - case BoPDRPC: - return NewBackoffFn(500, 3000, EqualJitter) - case BoRegionMiss: - // change base time to 2ms, because it may recover soon. - return NewBackoffFn(2, 500, NoJitter) - case BoTxnNotFound: - return NewBackoffFn(2, 500, NoJitter) - case BoTiKVServerBusy, BoTiFlashServerBusy: - return NewBackoffFn(2000, 10000, EqualJitter) - case BoStaleCmd: - return NewBackoffFn(2, 1000, NoJitter) - case BoMaxTsNotSynced: - return NewBackoffFn(2, 500, NoJitter) - } - return nil -} - -func (t BackoffType) String() string { - switch t { - case boTiKVRPC: - return "tikvRPC" - case BoTiFlashRPC: - return "tiflashRPC" - case BoTxnLock: - return "txnLock" - case BoTxnLockFast: - return "txnLockFast" - case BoPDRPC: - return "pdRPC" - case BoRegionMiss: - return "regionMiss" - case BoTiKVServerBusy: - return "tikvServerBusy" - case BoTiFlashServerBusy: - return "tiflashServerBusy" - case BoStaleCmd: - return "staleCommand" - case BoTxnNotFound: - return "txnNotFound" - case BoMaxTsNotSynced: - return "maxTsNotSynced" - } - return "" -} - -// TError returns pingcap/error of the backoff type. -func (t BackoffType) TError() error { - switch t { - case boTiKVRPC: - return tikverr.ErrTiKVServerTimeout - case BoTiFlashRPC: - return tikverr.ErrTiFlashServerTimeout - case BoTxnLock, BoTxnLockFast, BoTxnNotFound: - return tikverr.ErrResolveLockTimeout - case BoPDRPC: - return tikverr.NewErrPDServerTimeout("") - case BoRegionMiss: - return tikverr.ErrRegionUnavailable - case BoTiKVServerBusy: - return tikverr.ErrTiKVServerBusy - case BoTiFlashServerBusy: - return tikverr.ErrTiFlashServerBusy - case BoStaleCmd: - return tikverr.ErrTiKVStaleCommand - case BoMaxTsNotSynced: - return tikverr.ErrTiKVMaxTimestampNotSynced - } - return tikverr.ErrUnknown -} - // Backoffer is a utility for retrying queries. type Backoffer struct { ctx context.Context - fn map[BackoffType]func(context.Context, int) int + fn map[string]backoffFn maxSleep int totalSleep int errors []error - types []fmt.Stringer + configs []fmt.Stringer vars *kv.Variables noop bool - backoffSleepMS map[BackoffType]int - backoffTimes map[BackoffType]int + backoffSleepMS map[string]int + backoffTimes map[string]int } type txnStartCtxKeyType struct{} @@ -265,26 +86,71 @@ func (b *Backoffer) withVars(vars *kv.Variables) *Backoffer { return b } -// Backoff sleeps a while base on the backoffType and records the error message. +// Backoff sleeps a while base on the Config and records the error message. // It returns a retryable error if total sleep time exceeds maxSleep. -func (b *Backoffer) Backoff(typ BackoffType, err error) error { +func (b *Backoffer) Backoff(cfg *Config, err error) error { if span := opentracing.SpanFromContext(b.ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan(fmt.Sprintf("tikv.backoff.%s", typ), opentracing.ChildOf(span.Context())) + span1 := span.Tracer().StartSpan(fmt.Sprintf("tikv.backoff.%s", cfg), opentracing.ChildOf(span.Context())) defer span1.Finish() opentracing.ContextWithSpan(b.ctx, span1) } - return b.BackoffWithMaxSleep(typ, -1, err) + return b.BackoffWithCfgAndMaxSleep(cfg, -1, err) } -// BackoffTiKVRPC sleeps a while base on the TiKVRPC and records the error message. -// It returns a retryable error if total sleep time exceeds maxSleep. -func (b *Backoffer) BackoffTiKVRPC(err error) error { - return b.Backoff(boTiKVRPC, err) +// BackoffWithMaxSleepTxnLockFast sleeps a while base on the MaxSleepTxnLock and records the error message +// and never sleep more than maxSleepMs for each sleep. +func (b *Backoffer) BackoffWithMaxSleepTxnLockFast(maxSleepMs int, err error) error { + cfg := BoTxnLockFast + return b.BackoffWithCfgAndMaxSleep(cfg, maxSleepMs, err) +} + +// BackoffWithMaxSleep is deprecated, please use BackoffWithCfgAndMaxSleep instead. TODO: remove it when br is ready. +func (b *Backoffer) BackoffWithMaxSleep(typ int, maxSleepMs int, err error) error { + // Back off types. + const ( + boTiKVRPC int = iota + boTiFlashRPC + boTxnLock + boTxnLockFast + boPDRPC + boRegionMiss + boTiKVServerBusy + boTiFlashServerBusy + boTxnNotFound + boStaleCmd + boMaxTsNotSynced + ) + switch typ { + case boTiKVRPC: + return b.BackoffWithCfgAndMaxSleep(BoTiKVRPC, maxSleepMs, err) + case boTiFlashRPC: + return b.BackoffWithCfgAndMaxSleep(BoTiFlashRPC, maxSleepMs, err) + case boTxnLock: + return b.BackoffWithCfgAndMaxSleep(BoTxnLock, maxSleepMs, err) + case boTxnLockFast: + return b.BackoffWithCfgAndMaxSleep(BoTxnLockFast, maxSleepMs, err) + case boPDRPC: + return b.BackoffWithCfgAndMaxSleep(BoPDRPC, maxSleepMs, err) + case boRegionMiss: + return b.BackoffWithCfgAndMaxSleep(BoRegionMiss, maxSleepMs, err) + case boTiKVServerBusy: + return b.BackoffWithCfgAndMaxSleep(BoTiKVServerBusy, maxSleepMs, err) + case boTiFlashServerBusy: + return b.BackoffWithCfgAndMaxSleep(BoTiFlashServerBusy, maxSleepMs, err) + case boTxnNotFound: + return b.BackoffWithCfgAndMaxSleep(BoTxnNotFound, maxSleepMs, err) + case boStaleCmd: + return b.BackoffWithCfgAndMaxSleep(BoStaleCmd, maxSleepMs, err) + case boMaxTsNotSynced: + return b.BackoffWithCfgAndMaxSleep(BoMaxTsNotSynced, maxSleepMs, err) + } + cfg := NewConfig("", metrics.BackoffHistogramEmpty, nil, tikverr.ErrUnknown) + return b.BackoffWithCfgAndMaxSleep(cfg, maxSleepMs, err) } -// BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message +// BackoffWithCfgAndMaxSleep sleeps a while base on the Config and records the error message // and never sleep more than maxSleepMs for each sleep. -func (b *Backoffer) BackoffWithMaxSleep(typ BackoffType, maxSleepMs int, err error) error { +func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err error) error { if strings.Contains(err.Error(), tikverr.MismatchClusterID) { logutil.BgLogger().Fatal("critical error", zap.Error(err)) } @@ -295,9 +161,9 @@ func (b *Backoffer) BackoffWithMaxSleep(typ BackoffType, maxSleepMs int, err err } b.errors = append(b.errors, errors.Errorf("%s at %s", err.Error(), time.Now().Format(time.RFC3339Nano))) - b.types = append(b.types, typ) + b.configs = append(b.configs, cfg) if b.noop || (b.maxSleep > 0 && b.totalSleep >= b.maxSleep) { - errMsg := fmt.Sprintf("%s backoffer.maxSleep %dms is exceeded, errors:", typ.String(), b.maxSleep) + errMsg := fmt.Sprintf("%s backoffer.maxSleep %dms is exceeded, errors:", cfg.String(), b.maxSleep) for i, err := range b.errors { // Print only last 3 errors for non-DEBUG log levels. if log.GetLevel() == zapcore.DebugLevel || i >= len(b.errors)-3 { @@ -306,30 +172,29 @@ func (b *Backoffer) BackoffWithMaxSleep(typ BackoffType, maxSleepMs int, err err } logutil.BgLogger().Warn(errMsg) // Use the first backoff type to generate a MySQL error. - return b.types[0].(BackoffType).TError() + return b.configs[0].(*Config).err } // Lazy initialize. if b.fn == nil { - b.fn = make(map[BackoffType]func(context.Context, int) int) + b.fn = make(map[string]backoffFn) } - f, ok := b.fn[typ] + f, ok := b.fn[cfg.name] if !ok { - f = typ.createFn(b.vars) - b.fn[typ] = f + f = cfg.createBackoffFn(b.vars) + b.fn[cfg.name] = f } - realSleep := f(b.ctx, maxSleepMs) - typ.metric().Observe(float64(realSleep) / 1000) + cfg.metric.Observe(float64(realSleep) / 1000) b.totalSleep += realSleep if b.backoffSleepMS == nil { - b.backoffSleepMS = make(map[BackoffType]int) + b.backoffSleepMS = make(map[string]int) } - b.backoffSleepMS[typ] += realSleep + b.backoffSleepMS[cfg.name] += realSleep if b.backoffTimes == nil { - b.backoffTimes = make(map[BackoffType]int) + b.backoffTimes = make(map[string]int) } - b.backoffTimes[typ]++ + b.backoffTimes[cfg.name]++ stmtExec := b.ctx.Value(util.ExecDetailsKey) if stmtExec != nil { @@ -352,7 +217,7 @@ func (b *Backoffer) BackoffWithMaxSleep(typ BackoffType, maxSleepMs int, err err zap.Error(err), zap.Int("totalSleep", b.totalSleep), zap.Int("maxSleep", b.maxSleep), - zap.Stringer("type", typ), + zap.Stringer("type", cfg), zap.Reflect("txnStartTS", startTs)) return nil } @@ -361,7 +226,7 @@ func (b *Backoffer) String() string { if b.totalSleep == 0 { return "" } - return fmt.Sprintf(" backoff(%dms %v)", b.totalSleep, b.types) + return fmt.Sprintf(" backoff(%dms %v)", b.totalSleep, b.configs) } // Clone creates a new Backoffer which keeps current Backoffer's sleep time and errors, and shares @@ -401,7 +266,7 @@ func (b *Backoffer) GetTotalSleep() int { // GetTypes returns type list. func (b *Backoffer) GetTypes() []fmt.Stringer { - return b.types + return b.configs } // GetCtx returns the binded context. @@ -415,12 +280,12 @@ func (b *Backoffer) SetCtx(ctx context.Context) { } // GetBackoffTimes returns a map contains backoff time count by type. -func (b *Backoffer) GetBackoffTimes() map[BackoffType]int { +func (b *Backoffer) GetBackoffTimes() map[string]int { return b.backoffTimes } // GetBackoffSleepMS returns a map contains backoff sleep time by type. -func (b *Backoffer) GetBackoffSleepMS() map[BackoffType]int { +func (b *Backoffer) GetBackoffSleepMS() map[string]int { return b.backoffSleepMS } diff --git a/store/tikv/retry/backoff_test.go b/store/tikv/retry/backoff_test.go index f8dfb9ed120f3..a0a566499b10f 100644 --- a/store/tikv/retry/backoff_test.go +++ b/store/tikv/retry/backoff_test.go @@ -27,7 +27,7 @@ var _ = Suite(&testBackoffSuite{}) func (s *testBackoffSuite) TestBackoffWithMax(c *C) { b := NewBackofferWithVars(context.TODO(), 2000, nil) - err := b.BackoffWithMaxSleep(BoTxnLockFast, 30, errors.New("test")) + err := b.BackoffWithMaxSleepTxnLockFast(30, errors.New("test")) c.Assert(err, IsNil) c.Assert(b.totalSleep, Equals, 30) } diff --git a/store/tikv/retry/config.go b/store/tikv/retry/config.go new file mode 100644 index 0000000000000..bd118cabd8028 --- /dev/null +++ b/store/tikv/retry/config.go @@ -0,0 +1,159 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package retry + +import ( + "context" + "math" + "math/rand" + "strings" + "time" + + tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/zap" +) + +// Config is the configuration of the Backoff function. +type Config struct { + name string + metric prometheus.Observer + fnCfg *BackoffFnCfg + err error +} + +// backoffFn is the backoff function which compute the sleep time and do sleep. +type backoffFn func(ctx context.Context, maxSleepMs int) int + +func (c *Config) createBackoffFn(vars *kv.Variables) backoffFn { + if strings.EqualFold(c.name, txnLockFastName) { + return newBackoffFn(vars.BackoffLockFast, c.fnCfg.cap, c.fnCfg.jitter) + } + return newBackoffFn(c.fnCfg.base, c.fnCfg.cap, c.fnCfg.jitter) +} + +// BackoffFnCfg is the configuration for the backoff func which implements exponential backoff with +// optional jitters. +// See http://www.awsarchitectureblog.com/2015/03/backoff.html +type BackoffFnCfg struct { + base int + cap int + jitter int +} + +// NewBackoffFnCfg creates the config for BackoffFn. +func NewBackoffFnCfg(base, cap, jitter int) *BackoffFnCfg { + return &BackoffFnCfg{ + base, + cap, + jitter, + } +} + +// NewConfig creates a new Config for the Backoff operation. +func NewConfig(name string, metric prometheus.Observer, backoffFnCfg *BackoffFnCfg, err error) *Config { + return &Config{ + name: name, + metric: metric, + fnCfg: backoffFnCfg, + err: err, + } +} + +func (c *Config) String() string { + return c.name +} + +const txnLockFastName = "txnLockFast" + +// Backoff Config samples. +var ( + // TODO: distinguish tikv and tiflash in metrics + BoTiKVRPC = NewConfig("tikvRPC", metrics.BackoffHistogramRPC, NewBackoffFnCfg(100, 2000, EqualJitter), tikverr.ErrTiKVServerTimeout) + BoTiFlashRPC = NewConfig("tiflashRPC", metrics.BackoffHistogramRPC, NewBackoffFnCfg(100, 2000, EqualJitter), tikverr.ErrTiFlashServerTimeout) + BoTxnLock = NewConfig("txnLock", metrics.BackoffHistogramLock, NewBackoffFnCfg(200, 3000, EqualJitter), tikverr.ErrResolveLockTimeout) + BoPDRPC = NewConfig("pdRPC", metrics.BackoffHistogramPD, NewBackoffFnCfg(500, 3000, EqualJitter), tikverr.NewErrPDServerTimeout("")) + // change base time to 2ms, because it may recover soon. + BoRegionMiss = NewConfig("regionMiss", metrics.BackoffHistogramRegionMiss, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrRegionUnavailable) + BoTiKVServerBusy = NewConfig("tikvServerBusy", metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiKVServerBusy) + BoTiFlashServerBusy = NewConfig("tiflashServerBusy", metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiFlashServerBusy) + BoTxnNotFound = NewConfig("txnNotFound", metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrResolveLockTimeout) + BoStaleCmd = NewConfig("staleCommand", metrics.BackoffHistogramStaleCmd, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrTiKVStaleCommand) + BoMaxTsNotSynced = NewConfig("maxTsNotSynced", metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrTiKVMaxTimestampNotSynced) + // TxnLockFast's `base` load from vars.BackoffLockFast when create BackoffFn. + BoTxnLockFast = NewConfig(txnLockFastName, metrics.BackoffHistogramLockFast, NewBackoffFnCfg(2, 3000, EqualJitter), tikverr.ErrResolveLockTimeout) +) + +const ( + // NoJitter makes the backoff sequence strict exponential. + NoJitter = 1 + iota + // FullJitter applies random factors to strict exponential. + FullJitter + // EqualJitter is also randomized, but prevents very short sleeps. + EqualJitter + // DecorrJitter increases the maximum jitter based on the last random value. + DecorrJitter +) + +// newBackoffFn creates a backoff func which implements exponential backoff with +// optional jitters. +// See http://www.awsarchitectureblog.com/2015/03/backoff.html +func newBackoffFn(base, cap, jitter int) backoffFn { + if base < 2 { + // Top prevent panic in 'rand.Intn'. + base = 2 + } + attempts := 0 + lastSleep := base + return func(ctx context.Context, maxSleepMs int) int { + var sleep int + switch jitter { + case NoJitter: + sleep = expo(base, cap, attempts) + case FullJitter: + v := expo(base, cap, attempts) + sleep = rand.Intn(v) + case EqualJitter: + v := expo(base, cap, attempts) + sleep = v/2 + rand.Intn(v/2) + case DecorrJitter: + sleep = int(math.Min(float64(cap), float64(base+rand.Intn(lastSleep*3-base)))) + } + logutil.BgLogger().Debug("backoff", + zap.Int("base", base), + zap.Int("sleep", sleep), + zap.Int("attempts", attempts)) + + realSleep := sleep + // when set maxSleepMs >= 0 in `tikv.BackoffWithMaxSleep` will force sleep maxSleepMs milliseconds. + if maxSleepMs >= 0 && realSleep > maxSleepMs { + realSleep = maxSleepMs + } + select { + case <-time.After(time.Duration(realSleep) * time.Millisecond): + attempts++ + lastSleep = sleep + return realSleep + case <-ctx.Done(): + return 0 + } + } +} + +func expo(base, cap, n int) int { + return int(math.Min(float64(cap), float64(base)*math.Pow(2.0, float64(n)))) +} diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 035291a783aec..94ece80ff067f 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -254,7 +254,7 @@ func (s *Scanner) getData(bo *Backoffer) error { return errors.Trace(err) } if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(retry.BoTxnLockFast, int(msBeforeExpired), errors.Errorf("key is locked during scanning")) + err = bo.BackoffWithMaxSleepTxnLockFast(int(msBeforeExpired), errors.Errorf("key is locked during scanning")) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index ab3862fe4bf0b..180ac59369aca 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -381,7 +381,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec return errors.Trace(err) } if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(retry.BoTxnLockFast, int(msBeforeExpired), errors.Errorf("batchGet lockedKeys: %d", len(lockedKeys))) + err = bo.BackoffWithMaxSleepTxnLockFast(int(msBeforeExpired), errors.Errorf("batchGet lockedKeys: %d", len(lockedKeys))) if err != nil { return errors.Trace(err) } @@ -527,7 +527,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, return nil, errors.Trace(err) } if msBeforeExpired > 0 { - err = bo.BackoffWithMaxSleep(retry.BoTxnLockFast, int(msBeforeExpired), errors.New(keyErr.String())) + err = bo.BackoffWithMaxSleepTxnLockFast(int(msBeforeExpired), errors.New(keyErr.String())) if err != nil { return nil, errors.Trace(err) } @@ -730,8 +730,8 @@ func (s *KVSnapshot) mergeRegionRequestStats(stats map[tikvrpc.CmdType]*RPCRunti // SnapshotRuntimeStats records the runtime stats of snapshot. type SnapshotRuntimeStats struct { rpcStats RegionRequestRuntimeStats - backoffSleepMS map[retry.BackoffType]int - backoffTimes map[retry.BackoffType]int + backoffSleepMS map[string]int + backoffTimes map[string]int scanDetail *util.ScanDetail timeDetail *util.TimeDetail } @@ -745,8 +745,8 @@ func (rs *SnapshotRuntimeStats) Clone() *SnapshotRuntimeStats { } } if len(rs.backoffSleepMS) > 0 { - newRs.backoffSleepMS = make(map[retry.BackoffType]int) - newRs.backoffTimes = make(map[retry.BackoffType]int) + newRs.backoffSleepMS = make(map[string]int) + newRs.backoffTimes = make(map[string]int) for k, v := range rs.backoffSleepMS { newRs.backoffSleepMS[k] += v } @@ -767,10 +767,10 @@ func (rs *SnapshotRuntimeStats) Merge(other *SnapshotRuntimeStats) { } if len(other.backoffSleepMS) > 0 { if rs.backoffSleepMS == nil { - rs.backoffSleepMS = make(map[retry.BackoffType]int) + rs.backoffSleepMS = make(map[string]int) } if rs.backoffTimes == nil { - rs.backoffTimes = make(map[retry.BackoffType]int) + rs.backoffTimes = make(map[string]int) } for k, v := range other.backoffSleepMS { rs.backoffSleepMS[k] += v @@ -791,7 +791,7 @@ func (rs *SnapshotRuntimeStats) String() string { } ms := rs.backoffSleepMS[k] d := time.Duration(ms) * time.Millisecond - buf.WriteString(fmt.Sprintf("%s_backoff:{num:%d, total_time:%s}", k.String(), v, util.FormatDuration(d))) + buf.WriteString(fmt.Sprintf("%s_backoff:{num:%d, total_time:%s}", k, v, util.FormatDuration(d))) } timeDetail := rs.timeDetail.String() if timeDetail != "" { diff --git a/store/tikv/tests/snapshot_test.go b/store/tikv/tests/snapshot_test.go index a126decfc1c7d..ee2a71730b5bf 100644 --- a/store/tikv/tests/snapshot_test.go +++ b/store/tikv/tests/snapshot_test.go @@ -273,7 +273,7 @@ func (s *testSnapshotSuite) TestSnapshotRuntimeStats(c *C) { snapshot.MergeRegionRequestStats(reqStats.Stats) snapshot.MergeRegionRequestStats(reqStats.Stats) bo := tikv.NewBackofferWithVars(context.Background(), 2000, nil) - err := bo.BackoffWithMaxSleep(tikv.BoTxnLockFast, 30, errors.New("test")) + err := bo.BackoffWithMaxSleepTxnLockFast(30, errors.New("test")) c.Assert(err, IsNil) snapshot.RecordBackoffInfo(bo) snapshot.RecordBackoffInfo(bo) diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 5971e83e7980f..66e9af8969edd 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -192,7 +192,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { sync.Mutex BackoffTypes []fmt.Stringer }{ - BackoffTypes: []fmt.Stringer{tikv.BoTxnLock}, + BackoffTypes: []fmt.Stringer{tikv.BoTxnLock()}, }, ResolveLockTime: 10000, WriteKeys: 100000, @@ -268,7 +268,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { expectedSummaryElement.sumTxnRetry += int64(stmtExecInfo2.ExecDetail.CommitDetail.TxnRetry) expectedSummaryElement.maxTxnRetry = stmtExecInfo2.ExecDetail.CommitDetail.TxnRetry expectedSummaryElement.sumBackoffTimes += 1 - expectedSummaryElement.backoffTypes[tikv.BoTxnLock] = 1 + expectedSummaryElement.backoffTypes[tikv.BoTxnLock()] = 1 expectedSummaryElement.sumMem += stmtExecInfo2.MemMax expectedSummaryElement.maxMem = stmtExecInfo2.MemMax expectedSummaryElement.sumDisk += stmtExecInfo2.DiskMax @@ -319,7 +319,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { sync.Mutex BackoffTypes []fmt.Stringer }{ - BackoffTypes: []fmt.Stringer{tikv.BoTxnLock}, + BackoffTypes: []fmt.Stringer{tikv.BoTxnLock()}, }, ResolveLockTime: 1000, WriteKeys: 10000, @@ -374,7 +374,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { expectedSummaryElement.sumPrewriteRegionNum += int64(stmtExecInfo3.ExecDetail.CommitDetail.PrewriteRegionNum) expectedSummaryElement.sumTxnRetry += int64(stmtExecInfo3.ExecDetail.CommitDetail.TxnRetry) expectedSummaryElement.sumBackoffTimes += 1 - expectedSummaryElement.backoffTypes[tikv.BoTxnLock] = 2 + expectedSummaryElement.backoffTypes[tikv.BoTxnLock()] = 2 expectedSummaryElement.sumMem += stmtExecInfo3.MemMax expectedSummaryElement.sumDisk += stmtExecInfo3.DiskMax expectedSummaryElement.sumAffectedRows += stmtExecInfo3.StmtCtx.AffectedRows() @@ -575,7 +575,7 @@ func generateAnyExecInfo() *StmtExecInfo { sync.Mutex BackoffTypes []fmt.Stringer }{ - BackoffTypes: []fmt.Stringer{tikv.BoTxnLock}, + BackoffTypes: []fmt.Stringer{tikv.BoTxnLock()}, }, ResolveLockTime: 2000, WriteKeys: 20000, @@ -960,10 +960,10 @@ func (s *testStmtSummarySuite) TestFormatBackoffTypes(c *C) { backoffMap := make(map[fmt.Stringer]int) c.Assert(formatBackoffTypes(backoffMap), IsNil) - backoffMap[tikv.BoPDRPC] = 1 + backoffMap[tikv.BoPDRPC()] = 1 c.Assert(formatBackoffTypes(backoffMap), Equals, "pdRPC:1") - backoffMap[tikv.BoTxnLock] = 2 + backoffMap[tikv.BoTxnLock()] = 2 c.Assert(formatBackoffTypes(backoffMap), Equals, "txnLock:2,pdRPC:1") } From a2b9cb09291868b3fbe0a747ca7658ca4e764692 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=99=8E?= Date: Thu, 20 May 2021 22:44:22 +0800 Subject: [PATCH 129/343] config: ignore tiflash when show config (#24770) --- executor/memtable_reader.go | 3 +++ executor/memtable_reader_test.go | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/executor/memtable_reader.go b/executor/memtable_reader.go index 76ea478d91e71..f98306b5da51d 100644 --- a/executor/memtable_reader.go +++ b/executor/memtable_reader.go @@ -191,6 +191,9 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String url = fmt.Sprintf("%s://%s%s", util.InternalHTTPSchema(), statusAddr, pdapi.Config) case "tikv", "tidb": url = fmt.Sprintf("%s://%s/config", util.InternalHTTPSchema(), statusAddr) + case "tiflash": + // TODO: support show tiflash config once tiflash supports it + return default: ch <- result{err: errors.Errorf("unknown node type: %s(%s)", typ, address)} return diff --git a/executor/memtable_reader_test.go b/executor/memtable_reader_test.go index 53607dd80ef03..5b55a9d601b03 100644 --- a/executor/memtable_reader_test.go +++ b/executor/memtable_reader_test.go @@ -170,7 +170,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterConfig(c *C) { // mock servers servers := []string{} - for _, typ := range []string{"tidb", "tikv", "pd"} { + for _, typ := range []string{"tidb", "tikv", "tiflash", "pd"} { for _, server := range testServers { servers = append(servers, strings.Join([]string{typ, server.address, server.address}, ",")) } From 4000975ad1785fbc4c0e5175407e8486c0cef763 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Thu, 20 May 2021 08:56:21 -0600 Subject: [PATCH 130/343] privileges: improve dynamic privs registration and tests (#24773) --- privilege/privileges/privileges.go | 9 ++++++-- privilege/privileges/privileges_test.go | 29 ++++++++++++++++++++++++- 2 files changed, 35 insertions(+), 3 deletions(-) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 0e8d88a90c5a1..cdaca2f634263 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -531,7 +531,8 @@ func (p *UserPrivileges) GetAllRoles(user, host string) []*auth.RoleIdentity { } // IsDynamicPrivilege returns true if the DYNAMIC privilege is built-in or has been registered by a plugin -func (p *UserPrivileges) IsDynamicPrivilege(privNameInUpper string) bool { +func (p *UserPrivileges) IsDynamicPrivilege(privName string) bool { + privNameInUpper := strings.ToUpper(privName) for _, priv := range dynamicPrivs { if privNameInUpper == priv { return true @@ -541,7 +542,11 @@ func (p *UserPrivileges) IsDynamicPrivilege(privNameInUpper string) bool { } // RegisterDynamicPrivilege is used by plugins to add new privileges to TiDB -func RegisterDynamicPrivilege(privNameInUpper string) error { +func RegisterDynamicPrivilege(privName string) error { + privNameInUpper := strings.ToUpper(privName) + if len(privNameInUpper) > 32 { + return errors.New("privilege name is longer than 32 characters") + } dynamicPrivLock.Lock() defer dynamicPrivLock.Unlock() for _, priv := range dynamicPrivs { diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 3917d822aa403..1563f8d72cf7b 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/sem" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" @@ -1579,7 +1580,33 @@ func (s *testPrivilegeSuite) TestDynamicPrivsRegistration(c *C) { count := len(privileges.GetDynamicPrivileges()) c.Assert(pm.IsDynamicPrivilege("ACDC_ADMIN"), IsFalse) - privileges.RegisterDynamicPrivilege("ACDC_ADMIN") + c.Assert(privileges.RegisterDynamicPrivilege("ACDC_ADMIN"), IsNil) c.Assert(pm.IsDynamicPrivilege("ACDC_ADMIN"), IsTrue) c.Assert(len(privileges.GetDynamicPrivileges()), Equals, count+1) + + c.Assert(pm.IsDynamicPrivilege("iAmdynamIC"), IsFalse) + c.Assert(privileges.RegisterDynamicPrivilege("IAMdynamic"), IsNil) + c.Assert(pm.IsDynamicPrivilege("IAMdyNAMIC"), IsTrue) + c.Assert(len(privileges.GetDynamicPrivileges()), Equals, count+2) + + c.Assert(privileges.RegisterDynamicPrivilege("THIS_PRIVILEGE_NAME_IS_TOO_LONG_THE_MAX_IS_32_CHARS").Error(), Equals, "privilege name is longer than 32 characters") + c.Assert(pm.IsDynamicPrivilege("THIS_PRIVILEGE_NAME_IS_TOO_LONG_THE_MAX_IS_32_CHARS"), IsFalse) + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("CREATE USER privassigntest") + tk.MustExec("SET tidb_enable_dynamic_privileges=1") + + // Check that all privileges registered are assignable to users, + // including the recently registered ACDC_ADMIN + for _, priv := range privileges.GetDynamicPrivileges() { + sqlGrant, err := sqlexec.EscapeSQL("GRANT %n ON *.* TO privassigntest", priv) + c.Assert(err, IsNil) + tk.MustExec(sqlGrant) + } + // Check that all privileges registered are revokable + for _, priv := range privileges.GetDynamicPrivileges() { + sqlGrant, err := sqlexec.EscapeSQL("REVOKE %n ON *.* FROM privassigntest", priv) + c.Assert(err, IsNil) + tk.MustExec(sqlGrant) + } } From bc7f18229ee1525689bc12d155511595676b71c0 Mon Sep 17 00:00:00 2001 From: TomShawn <41534398+TomShawn@users.noreply.github.com> Date: Thu, 20 May 2021 23:58:21 +0800 Subject: [PATCH 131/343] README: remove the link of TiDB Monthly Update (#24791) --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index f3c07bf0b1ad8..b4fcc35237280 100644 --- a/README.md +++ b/README.md @@ -55,7 +55,6 @@ In addition, you may enjoy following: - [@PingCAP](https://twitter.com/PingCAP) on Twitter - Question tagged [#tidb on StackOverflow](https://stackoverflow.com/questions/tagged/tidb) - The PingCAP Team [English Blog](https://en.pingcap.com/blog) and [Chinese Blog](https://pingcap.com/blog-cn/) -- [TiDB Monthly](https://pingcap.com/weekly/) For support, please contact [PingCAP](http://bit.ly/contact_us_via_github). From 55d26c58323063c0a63bd11ba4498de7a977d886 Mon Sep 17 00:00:00 2001 From: Lei Zhao Date: Fri, 21 May 2021 10:16:22 +0800 Subject: [PATCH 132/343] region_cache: filter peers on tombstone or dropped stores (#24726) --- store/tikv/mockstore/mocktikv/pd.go | 8 ++ store/tikv/region_cache.go | 183 +++++++++++++++------------ store/tikv/region_cache_test.go | 188 ++++++++++++++++++++++++---- 3 files changed, 278 insertions(+), 101 deletions(-) diff --git a/store/tikv/mockstore/mocktikv/pd.go b/store/tikv/mockstore/mocktikv/pd.go index b2db0de8fb343..6531a4ca71cc3 100644 --- a/store/tikv/mockstore/mocktikv/pd.go +++ b/store/tikv/mockstore/mocktikv/pd.go @@ -15,6 +15,7 @@ package mocktikv import ( "context" + "fmt" "math" "sync" "time" @@ -126,6 +127,13 @@ func (c *pdClient) GetStore(ctx context.Context, storeID uint64) (*metapb.Store, default: } store := c.cluster.GetStore(storeID) + // It's same as PD's implementation. + if store == nil { + return nil, fmt.Errorf("invalid store ID %d, not found", storeID) + } + if store.GetState() == metapb.StoreState_Tombstone { + return nil, nil + } return store, nil } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 0d9423a9f5a7e..5dc27503465d4 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -187,7 +187,7 @@ func (r *RegionStore) filterStoreCandidate(aidx AccessIndex, op *storeSelectorOp } // init initializes region after constructed. -func (r *Region) init(c *RegionCache) error { +func (r *Region) init(bo *Backoffer, c *RegionCache) error { // region store pull used store from global store map // to avoid acquire storeMu in later access. rs := &RegionStore{ @@ -197,6 +197,7 @@ func (r *Region) init(c *RegionCache) error { stores: make([]*Store, 0, len(r.meta.Peers)), storeEpochs: make([]uint32, 0, len(r.meta.Peers)), } + availablePeers := r.meta.GetPeers()[:0] for _, p := range r.meta.Peers { c.storeMu.RLock() store, exists := c.storeMu.stores[p.StoreId] @@ -204,10 +205,15 @@ func (r *Region) init(c *RegionCache) error { if !exists { store = c.getStoreByStoreID(p.StoreId) } - _, err := store.initResolve(retry.NewNoopBackoff(context.Background()), c) + addr, err := store.initResolve(bo, c) if err != nil { return err } + // Filter the peer on a tombstone store. + if addr == "" { + continue + } + availablePeers = append(availablePeers, p) switch store.storeType { case tikvrpc.TiKV: rs.accessIndex[TiKVOnly] = append(rs.accessIndex[TiKVOnly], len(rs.stores)) @@ -217,6 +223,13 @@ func (r *Region) init(c *RegionCache) error { rs.stores = append(rs.stores, store) rs.storeEpochs = append(rs.storeEpochs, atomic.LoadUint32(&store.epoch)) } + // TODO(youjiali1995): It's possible the region info in PD is stale for now but it can recover. + // Maybe we need backoff here. + if len(availablePeers) == 0 { + return errors.Errorf("no available peers, region: {%v}", r.meta) + } + r.meta.Peers = availablePeers + atomic.StorePointer(&r.store, unsafe.Pointer(rs)) // mark region has been init accessed. @@ -321,6 +334,18 @@ func NewRegionCache(pdClient pd.Client) *RegionCache { return c } +// clear clears all cached data in the RegionCache. It's only used in tests. +func (c *RegionCache) clear() { + c.mu.Lock() + c.mu.regions = make(map[RegionVerID]*Region) + c.mu.latestVersions = make(map[uint64]RegionVerID) + c.mu.sorted = btree.New(btreeDegree) + c.mu.Unlock() + c.storeMu.Lock() + c.storeMu.stores = make(map[uint64]*Store) + c.storeMu.Unlock() +} + // Close releases region cache's resource. func (c *RegionCache) Close() { close(c.closeCh) @@ -332,32 +357,29 @@ func (c *RegionCache) asyncCheckAndResolveLoop(interval time.Duration) { defer ticker.Stop() var needCheckStores []*Store for { + needCheckStores = needCheckStores[:0] select { case <-c.closeCh: return case <-c.notifyCheckCh: - needCheckStores = needCheckStores[:0] - c.checkAndResolve(needCheckStores) + c.checkAndResolve(needCheckStores, func(s *Store) bool { + return s.getResolveState() == needCheck + }) case <-ticker.C: - // refresh store once a minute to update labels - var stores []*Store - c.storeMu.RLock() - stores = make([]*Store, 0, len(c.storeMu.stores)) - for _, s := range c.storeMu.stores { - stores = append(stores, s) - } - c.storeMu.RUnlock() - for _, store := range stores { - _, err := store.reResolve(c) - terror.Log(err) - } + // refresh store to update labels. + c.checkAndResolve(needCheckStores, func(s *Store) bool { + state := s.getResolveState() + // Only valid stores should be reResolved. In fact, it's impossible + // there's a deleted store in the stores map which guaranteed by reReslve(). + return state != unresolved && state != tombstone && state != deleted + }) } } } // checkAndResolve checks and resolve addr of failed stores. // this method isn't thread-safe and only be used by one goroutine. -func (c *RegionCache) checkAndResolve(needCheckStores []*Store) { +func (c *RegionCache) checkAndResolve(needCheckStores []*Store, needCheck func(*Store) bool) { defer func() { r := recover() if r != nil { @@ -369,8 +391,7 @@ func (c *RegionCache) checkAndResolve(needCheckStores []*Store) { c.storeMu.RLock() for _, store := range c.storeMu.stores { - state := store.getResolveState() - if state == needCheck { + if needCheck(store) { needCheckStores = append(needCheckStores, store) } } @@ -1217,9 +1238,6 @@ func filterUnavailablePeers(region *pd.Region) { new = append(new, p) } } - for i := len(new); i < len(region.Meta.Peers); i++ { - region.Meta.Peers[i] = nil - } region.Meta.Peers = new } @@ -1272,7 +1290,7 @@ func (c *RegionCache) loadRegion(bo *Backoffer, key []byte, isEndKey bool) (*Reg continue } region := &Region{meta: reg.Meta} - err = region.init(c) + err = region.init(bo, c) if err != nil { return nil, err } @@ -1317,7 +1335,7 @@ func (c *RegionCache) loadRegionByID(bo *Backoffer, regionID uint64) (*Region, e return nil, errors.New("receive Region with no available peer") } region := &Region{meta: reg.Meta} - err = region.init(c) + err = region.init(bo, c) if err != nil { return nil, err } @@ -1368,7 +1386,7 @@ func (c *RegionCache) scanRegions(bo *Backoffer, startKey, endKey []byte, limit regions := make([]*Region, 0, len(regionsInfo)) for _, r := range regionsInfo { region := &Region{meta: r.Meta} - err := region.init(c) + err := region.init(bo, c) if err != nil { return nil, err } @@ -1409,6 +1427,8 @@ func (c *RegionCache) getStoreAddr(bo *Backoffer, region *Region, store *Store, case deleted: addr = c.changeToActiveStore(region, store, storeIdx) return + case tombstone: + return "", nil default: panic("unsupported resolve state") } @@ -1456,6 +1476,8 @@ func (c *RegionCache) getProxyStore(region *Region, store *Store, rs *RegionStor return nil, 0, 0 } +// changeToActiveStore replace the deleted store in the region by an up-to-date store in the stores map. +// The order is guaranteed by reResolve() which adds the new store before marking old store deleted. func (c *RegionCache) changeToActiveStore(region *Region, store *Store, storeIdx int) (addr string) { c.storeMu.RLock() store = c.storeMu.stores[store.storeID] @@ -1530,7 +1552,7 @@ func (c *RegionCache) OnRegionEpochNotMatch(bo *Backoffer, ctx *RPCContext, curr } } region := &Region{meta: meta} - err := region.init(c) + err := region.init(bo, c) if err != nil { return err } @@ -1860,19 +1882,31 @@ type Store struct { type resolveState uint64 const ( + // The store is just created and normally is being resolved. + // Store in this state will only be resolved by initResolve(). unresolved resolveState = iota + // The store is resolved and its address is valid. resolved + // Request failed on this store and it will be re-resolved by asyncCheckAndResolveLoop(). needCheck + // The store's address or label is changed and marked deleted. + // There is a new store struct replaced it in the RegionCache and should + // call changeToActiveStore() to get the new struct. deleted + // The store is a tombstone. Should invalidate the region if tries to access it. + tombstone ) -// initResolve resolves addr for store that never resolved. +// initResolve resolves the address of the store that never resolved and returns an +// empty string if it's a tombstone. func (s *Store) initResolve(bo *Backoffer, c *RegionCache) (addr string, err error) { s.resolveMutex.Lock() state := s.getResolveState() defer s.resolveMutex.Unlock() if state != unresolved { - addr = s.addr + if state != tombstone { + addr = s.addr + } return } var store *metapb.Store @@ -1883,35 +1917,33 @@ func (s *Store) initResolve(bo *Backoffer, c *RegionCache) (addr string, err err } else { metrics.RegionCacheCounterWithGetStoreOK.Inc() } - if err != nil { + if bo.GetCtx().Err() != nil && errors.Cause(bo.GetCtx().Err()) == context.Canceled { + return + } + if err != nil && !isStoreNotFoundError(err) { // TODO: more refine PD error status handle. - if errors.Cause(err) == context.Canceled { - return - } err = errors.Errorf("loadStore from PD failed, id: %d, err: %v", s.storeID, err) if err = bo.Backoff(retry.BoPDRPC, err); err != nil { return } continue } + // The store is a tombstone. if store == nil { - return + s.setResolveState(tombstone) + return "", nil } addr = store.GetAddress() + if addr == "" { + return "", errors.Errorf("empty store(%d) address", s.storeID) + } s.addr = addr s.saddr = store.GetStatusAddress() s.storeType = GetStoreTypeByMeta(store) s.labels = store.GetLabels() - retry: - state = s.getResolveState() - if state != unresolved { - addr = s.addr - return - } - if !s.compareAndSwapState(state, resolved) { - goto retry - } - return + // Shouldn't have other one changing its state concurrently, but we still use changeResolveStateTo for safety. + s.changeResolveStateTo(unresolved, resolved) + return s.addr, nil } } @@ -1944,7 +1976,7 @@ func (s *Store) reResolve(c *RegionCache) (bool, error) { logutil.BgLogger().Info("invalidate regions in removed store", zap.Uint64("store", s.storeID), zap.String("add", s.addr)) atomic.AddUint32(&s.epoch, 1) - atomic.StoreUint64(&s.state, uint64(deleted)) + s.setResolveState(tombstone) metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() return false, nil } @@ -1952,33 +1984,14 @@ func (s *Store) reResolve(c *RegionCache) (bool, error) { storeType := GetStoreTypeByMeta(store) addr = store.GetAddress() if s.addr != addr || !s.IsSameLabels(store.GetLabels()) { - state := resolved - newStore := &Store{storeID: s.storeID, addr: addr, saddr: store.GetStatusAddress(), storeType: storeType, labels: store.GetLabels()} - newStore.state = *(*uint64)(&state) + newStore := &Store{storeID: s.storeID, addr: addr, saddr: store.GetStatusAddress(), storeType: storeType, labels: store.GetLabels(), state: uint64(resolved)} c.storeMu.Lock() c.storeMu.stores[newStore.storeID] = newStore c.storeMu.Unlock() - retryMarkDel: - // all region used those - oldState := s.getResolveState() - if oldState == deleted { - return false, nil - } - newState := deleted - if !s.compareAndSwapState(oldState, newState) { - goto retryMarkDel - } + s.setResolveState(deleted) return false, nil } -retryMarkResolved: - oldState := s.getResolveState() - if oldState != needCheck { - return true, nil - } - newState := resolved - if !s.compareAndSwapState(oldState, newState) { - goto retryMarkResolved - } + s.changeResolveStateTo(needCheck, resolved) return true, nil } @@ -1990,23 +2003,35 @@ func (s *Store) getResolveState() resolveState { return resolveState(atomic.LoadUint64(&s.state)) } -func (s *Store) compareAndSwapState(oldState, newState resolveState) bool { - return atomic.CompareAndSwapUint64(&s.state, uint64(oldState), uint64(newState)) +func (s *Store) setResolveState(state resolveState) { + atomic.StoreUint64(&s.state, uint64(state)) +} + +// changeResolveStateTo changes the store resolveState from the old state to the new state. +// Returns true if it changes the state successfully, and false if the store's state +// is changed by another one. +func (s *Store) changeResolveStateTo(from, to resolveState) bool { + for { + state := s.getResolveState() + if state == to { + return true + } + if state != from { + return false + } + if atomic.CompareAndSwapUint64(&s.state, uint64(from), uint64(to)) { + return true + } + } } // markNeedCheck marks resolved store to be async resolve to check store addr change. func (s *Store) markNeedCheck(notifyCheckCh chan struct{}) { -retry: - oldState := s.getResolveState() - if oldState != resolved { - return - } - if !s.compareAndSwapState(oldState, needCheck) { - goto retry - } - select { - case notifyCheckCh <- struct{}{}: - default: + if s.changeResolveStateTo(resolved, needCheck) { + select { + case notifyCheckCh <- struct{}{}: + default: + } } } diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index 7b0e5884abb2a..acd267ceed760 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -181,32 +181,176 @@ func (s *testRegionCacheSuite) TestSimple(c *C) { c.Assert(r, IsNil) } -func (s *testRegionCacheSuite) TestDropStore(c *C) { - bo := NewBackofferWithVars(context.Background(), 100, nil) +// TestResolveStateTransition verifies store's resolve state transition. For example, +// a newly added store is in unresolved state and will be resolved soon if it's an up store, +// or in tombstone state if it's a tombstone. +func (s *testRegionCacheSuite) TestResolveStateTransition(c *C) { + cache := s.cache + bo := retry.NewNoopBackoff(context.Background()) + + // Check resolving normal stores. The resolve state should be resolved. + for _, storeMeta := range s.cluster.GetAllStores() { + store := cache.getStoreByStoreID(storeMeta.GetId()) + c.Assert(store.getResolveState(), Equals, unresolved) + addr, err := store.initResolve(bo, cache) + c.Assert(err, IsNil) + c.Assert(addr, Equals, storeMeta.GetAddress()) + c.Assert(store.getResolveState(), Equals, resolved) + } + + waitResolve := func(s *Store) { + for i := 0; i < 10; i++ { + if s.getResolveState() != needCheck { + break + } + time.Sleep(50 * time.Millisecond) + } + } + + // Mark the store needCheck. The resolve state should be resolved soon. + store := cache.getStoreByStoreID(s.store1) + store.markNeedCheck(cache.notifyCheckCh) + waitResolve(store) + c.Assert(store.getResolveState(), Equals, resolved) + + // Mark the store needCheck and it becomes a tombstone. The resolve state should be tombstone. + s.cluster.MarkTombstone(s.store1) + store.markNeedCheck(cache.notifyCheckCh) + waitResolve(store) + c.Assert(store.getResolveState(), Equals, tombstone) + s.cluster.StartStore(s.store1) + + // Mark the store needCheck and it's deleted from PD. The resolve state should be tombstone. + cache.clear() + store = cache.getStoreByStoreID(s.store1) + store.initResolve(bo, cache) + c.Assert(store.getResolveState(), Equals, resolved) + storeMeta := s.cluster.GetStore(s.store1) s.cluster.RemoveStore(s.store1) - loc, err := s.cache.LocateKey(bo, []byte("a")) - c.Assert(err, IsNil) - ctx, err := s.cache.GetTiKVRPCContext(bo, loc.Region, kv.ReplicaReadLeader, 0) + store.markNeedCheck(cache.notifyCheckCh) + waitResolve(store) + c.Assert(store.getResolveState(), Equals, tombstone) + s.cluster.AddStore(storeMeta.GetId(), storeMeta.GetAddress(), storeMeta.GetLabels()...) + + // Mark the store needCheck and its address and labels are changed. + // The resolve state should be deleted and a new store is added to the cache. + cache.clear() + store = cache.getStoreByStoreID(s.store1) + store.initResolve(bo, cache) + c.Assert(store.getResolveState(), Equals, resolved) + s.cluster.UpdateStoreAddr(s.store1, store.addr+"0", &metapb.StoreLabel{Key: "k", Value: "v"}) + store.markNeedCheck(cache.notifyCheckCh) + waitResolve(store) + c.Assert(store.getResolveState(), Equals, deleted) + newStore := cache.getStoreByStoreID(s.store1) + c.Assert(newStore.getResolveState(), Equals, resolved) + c.Assert(newStore.addr, Equals, store.addr+"0") + c.Assert(newStore.labels, DeepEquals, []*metapb.StoreLabel{{Key: "k", Value: "v"}}) + + // Check initResolve()ing a tombstone store. The resolve state should be tombstone. + cache.clear() + s.cluster.MarkTombstone(s.store1) + store = cache.getStoreByStoreID(s.store1) + for i := 0; i < 2; i++ { + addr, err := store.initResolve(bo, cache) + c.Assert(err, IsNil) + c.Assert(addr, Equals, "") + c.Assert(store.getResolveState(), Equals, tombstone) + } + s.cluster.StartStore(s.store1) + cache.clear() + + // Check initResolve()ing a dropped store. The resolve state should be tombstone. + cache.clear() + storeMeta = s.cluster.GetStore(s.store1) + s.cluster.RemoveStore(s.store1) + store = cache.getStoreByStoreID(s.store1) + for i := 0; i < 2; i++ { + addr, err := store.initResolve(bo, cache) + c.Assert(err, IsNil) + c.Assert(addr, Equals, "") + c.Assert(store.getResolveState(), Equals, tombstone) + } + s.cluster.AddStore(storeMeta.GetId(), storeMeta.GetAddress(), storeMeta.GetLabels()...) +} + +// TestFilterDownPeersOrPeersOnTombstoneOrDroppedStore verifies the RegionCache filter +// region's down peers and peers on tombstone or dropped stores. RegionCache shouldn't +// report errors in such cases if there are available peers. +func (s *testRegionCacheSuite) TestFilterDownPeersOrPeersOnTombstoneOrDroppedStores(c *C) { + key := []byte("a") + bo := NewBackofferWithVars(context.Background(), 100, nil) + + verifyGetRPCCtx := func(meta *metapb.Region) { + loc, err := s.cache.LocateKey(bo, key) + c.Assert(loc, NotNil) + c.Assert(err, IsNil) + ctx, err := s.cache.GetTiKVRPCContext(bo, loc.Region, kv.ReplicaReadLeader, 0) + c.Assert(err, IsNil) + c.Assert(ctx, NotNil) + c.Assert(ctx.Meta, DeepEquals, meta) + ctx, err = s.cache.GetTiKVRPCContext(bo, loc.Region, kv.ReplicaReadFollower, rand.Uint32()) + c.Assert(err, IsNil) + c.Assert(ctx, NotNil) + c.Assert(ctx.Meta, DeepEquals, meta) + } + + // When all peers are normal, the cached region should contain all peers. + reg, err := s.cache.findRegionByKey(bo, key, false) + c.Assert(reg, NotNil) c.Assert(err, IsNil) - c.Assert(ctx, IsNil) - ctx, err = s.cache.GetTiKVRPCContext(bo, loc.Region, kv.ReplicaReadFollower, rand.Uint32()) + regInPD, _ := s.cluster.GetRegion(reg.GetID()) + c.Assert(reg.meta, DeepEquals, regInPD) + c.Assert(len(reg.meta.GetPeers()), Equals, len(reg.getStore().stores)) + verifyGetRPCCtx(reg.meta) + s.checkCache(c, 1) + s.cache.clear() + + // Shouldn't contain the peer on the tombstone store. + s.cluster.MarkTombstone(s.store1) + reg, err = s.cache.findRegionByKey(bo, key, false) + c.Assert(reg, NotNil) c.Assert(err, IsNil) - c.Assert(ctx, IsNil) - s.checkCache(c, 0) -} + c.Assert(len(reg.meta.GetPeers()), Equals, len(regInPD.GetPeers())-1) + c.Assert(len(reg.meta.GetPeers()), Equals, len(reg.getStore().stores)) + for _, peer := range reg.meta.GetPeers() { + c.Assert(peer.GetStoreId(), Not(Equals), s.store1) + } + for _, store := range reg.getStore().stores { + c.Assert(store.storeID, Not(Equals), s.store1) + } + verifyGetRPCCtx(reg.meta) + s.checkCache(c, 1) + s.cache.clear() + s.cluster.StartStore(s.store1) -func (s *testRegionCacheSuite) TestDropStoreRetry(c *C) { + // Shouldn't contain the peer on the dropped store. + store := s.cluster.GetStore(s.store1) s.cluster.RemoveStore(s.store1) - done := make(chan struct{}) - go func() { - time.Sleep(time.Millisecond * 10) - s.cluster.AddStore(s.store1, s.storeAddr(s.store1)) - close(done) - }() - loc, err := s.cache.LocateKey(s.bo, []byte("a")) + reg, err = s.cache.findRegionByKey(bo, key, false) + c.Assert(reg, NotNil) c.Assert(err, IsNil) - c.Assert(loc.Region.id, Equals, s.region1) - <-done + c.Assert(len(reg.meta.GetPeers()), Equals, len(regInPD.GetPeers())-1) + c.Assert(len(reg.meta.GetPeers()), Equals, len(reg.getStore().stores)) + for _, peer := range reg.meta.GetPeers() { + c.Assert(peer.GetStoreId(), Not(Equals), s.store1) + } + for _, store := range reg.getStore().stores { + c.Assert(store.storeID, Not(Equals), s.store1) + } + verifyGetRPCCtx(reg.meta) + s.checkCache(c, 1) + s.cache.clear() + s.cluster.AddStore(store.GetId(), store.GetAddress(), store.GetLabels()...) + + // Report an error when there's no available peers. + s.cluster.MarkTombstone(s.store1) + s.cluster.MarkTombstone(s.store2) + _, err = s.cache.findRegionByKey(bo, key, false) + c.Assert(err, NotNil) + c.Assert(err.Error(), Matches, ".*no available peers.*") + s.cluster.StartStore(s.store1) + s.cluster.StartStore(s.store2) } func (s *testRegionCacheSuite) TestUpdateLeader(c *C) { @@ -312,7 +456,7 @@ func (s *testRegionCacheSuite) TestUpdateLeader3(c *C) { c.Assert(err, IsNil) c.Assert(ctx.Addr, Equals, "store2") s.cache.OnSendFail(retry.NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) - s.cache.checkAndResolve(nil) + s.cache.checkAndResolve(nil, func(*Store) bool { return true }) s.cache.UpdateLeader(loc.Region, s.store2, 0) addr := s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0) c.Assert(addr, Equals, "") @@ -1325,7 +1469,7 @@ func (s *testRegionCacheSuite) TestPeersLenChange(c *C) { } filterUnavailablePeers(cpRegion) region := &Region{meta: cpRegion.Meta} - err = region.init(s.cache) + err = region.init(s.bo, s.cache) c.Assert(err, IsNil) s.cache.insertRegionToCache(region) From 327638c8742ae8e9370dbcdd388be1987c373af6 Mon Sep 17 00:00:00 2001 From: Shirly Date: Fri, 21 May 2021 10:34:21 +0800 Subject: [PATCH 133/343] util/stmtsummary: remove import package tikv (#24776) --- util/stmtsummary/statement_summary_test.go | 29 +++++++++++++--------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 66e9af8969edd..fb4593e26e9b6 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -27,10 +27,10 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/sessionctx/stmtctx" - "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/stringutil" ) var _ = Suite(&testStmtSummarySuite{}) @@ -62,6 +62,10 @@ func TestT(t *testing.T) { TestingT(t) } +const ( + boTxnLockName = stringutil.StringerStr("txnlock") +) + // Test stmtSummaryByDigest.AddStatement. func (s *testStmtSummarySuite) TestAddStatement(c *C) { s.ssMap.Clear() @@ -192,7 +196,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { sync.Mutex BackoffTypes []fmt.Stringer }{ - BackoffTypes: []fmt.Stringer{tikv.BoTxnLock()}, + BackoffTypes: []fmt.Stringer{boTxnLockName}, }, ResolveLockTime: 10000, WriteKeys: 100000, @@ -268,7 +272,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { expectedSummaryElement.sumTxnRetry += int64(stmtExecInfo2.ExecDetail.CommitDetail.TxnRetry) expectedSummaryElement.maxTxnRetry = stmtExecInfo2.ExecDetail.CommitDetail.TxnRetry expectedSummaryElement.sumBackoffTimes += 1 - expectedSummaryElement.backoffTypes[tikv.BoTxnLock()] = 1 + expectedSummaryElement.backoffTypes[boTxnLockName] = 1 expectedSummaryElement.sumMem += stmtExecInfo2.MemMax expectedSummaryElement.maxMem = stmtExecInfo2.MemMax expectedSummaryElement.sumDisk += stmtExecInfo2.DiskMax @@ -319,7 +323,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { sync.Mutex BackoffTypes []fmt.Stringer }{ - BackoffTypes: []fmt.Stringer{tikv.BoTxnLock()}, + BackoffTypes: []fmt.Stringer{boTxnLockName}, }, ResolveLockTime: 1000, WriteKeys: 10000, @@ -374,7 +378,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { expectedSummaryElement.sumPrewriteRegionNum += int64(stmtExecInfo3.ExecDetail.CommitDetail.PrewriteRegionNum) expectedSummaryElement.sumTxnRetry += int64(stmtExecInfo3.ExecDetail.CommitDetail.TxnRetry) expectedSummaryElement.sumBackoffTimes += 1 - expectedSummaryElement.backoffTypes[tikv.BoTxnLock()] = 2 + expectedSummaryElement.backoffTypes[boTxnLockName] = 2 expectedSummaryElement.sumMem += stmtExecInfo3.MemMax expectedSummaryElement.sumDisk += stmtExecInfo3.DiskMax expectedSummaryElement.sumAffectedRows += stmtExecInfo3.StmtCtx.AffectedRows() @@ -575,7 +579,7 @@ func generateAnyExecInfo() *StmtExecInfo { sync.Mutex BackoffTypes []fmt.Stringer }{ - BackoffTypes: []fmt.Stringer{tikv.BoTxnLock()}, + BackoffTypes: []fmt.Stringer{boTxnLockName}, }, ResolveLockTime: 2000, WriteKeys: 20000, @@ -650,7 +654,7 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) { stmtExecInfo1.ExecDetail.CommitDetail.WriteSize, stmtExecInfo1.ExecDetail.CommitDetail.WriteSize, stmtExecInfo1.ExecDetail.CommitDetail.PrewriteRegionNum, stmtExecInfo1.ExecDetail.CommitDetail.PrewriteRegionNum, stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, 0, 0, 1, - "txnLock:1", stmtExecInfo1.MemMax, stmtExecInfo1.MemMax, stmtExecInfo1.DiskMax, stmtExecInfo1.DiskMax, + fmt.Sprintf("%s:1", boTxnLockName), stmtExecInfo1.MemMax, stmtExecInfo1.MemMax, stmtExecInfo1.DiskMax, stmtExecInfo1.DiskMax, 0, 0, 0, 0, 0, stmtExecInfo1.StmtCtx.AffectedRows(), t, t, 0, 0, 0, stmtExecInfo1.OriginalSQL, stmtExecInfo1.PrevSQL, "plan_digest", ""} match(c, datums[0], expectedDatum...) @@ -959,12 +963,13 @@ func (s *testStmtSummarySuite) TestGetMoreThanOnceBindableStmt(c *C) { func (s *testStmtSummarySuite) TestFormatBackoffTypes(c *C) { backoffMap := make(map[fmt.Stringer]int) c.Assert(formatBackoffTypes(backoffMap), IsNil) + bo1 := stringutil.StringerStr("pdrpc") + backoffMap[bo1] = 1 + c.Assert(formatBackoffTypes(backoffMap), Equals, "pdrpc:1") + bo2 := stringutil.StringerStr("txnlock") + backoffMap[bo2] = 2 - backoffMap[tikv.BoPDRPC()] = 1 - c.Assert(formatBackoffTypes(backoffMap), Equals, "pdRPC:1") - - backoffMap[tikv.BoTxnLock()] = 2 - c.Assert(formatBackoffTypes(backoffMap), Equals, "txnLock:2,pdRPC:1") + c.Assert(formatBackoffTypes(backoffMap), Equals, "txnlock:2,pdrpc:1") } // Test refreshing current statement summary periodically. From e167b26e11bde210792c53ccddf99ca01293a775 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 21 May 2021 11:28:21 +0800 Subject: [PATCH 134/343] ddl: grammar check for create unsupported temporary table (#24723) --- ddl/db_integration_test.go | 24 ++++++++++++++++++++++++ ddl/ddl_api.go | 14 +++++++++++++- ddl/error.go | 3 +++ 3 files changed, 40 insertions(+), 1 deletion(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index a39322c2f5b9c..b7a2b2a26e37a 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2801,3 +2801,27 @@ func (s *testIntegrationSuite3) TestIssue21835(c *C) { _, err := tk.Exec("create table t( col decimal(1,2) not null default 0);") c.Assert(err.Error(), Equals, "[types:1427]For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column 'col').") } + +func (s *testIntegrationSuite3) TestCreateTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + + // Grammar error. + tk.MustGetErrCode("create global temporary table t(a double(0, 0))", errno.ErrParse) + tk.MustGetErrCode("create temporary table t(id int) on commit delete rows", errno.ErrParse) + tk.MustGetErrCode("create temporary table t(id int) on commit preserve rows", errno.ErrParse) + tk.MustGetErrCode("create table t(id int) on commit delete rows", errno.ErrParse) + tk.MustGetErrCode("create table t(id int) on commit preserve rows", errno.ErrParse) + + // Not support yet. + tk.MustGetErrCode("create global temporary table t (id int) on commit preserve rows", errno.ErrUnsupportedDDLOperation) + // Engine type can only be 'memory' or empty for now. + tk.MustGetErrCode("create global temporary table t (id int) engine = 'innodb' on commit delete rows", errno.ErrUnsupportedDDLOperation) + // Follow the behaviour of the old version TiDB: parse and ignore the 'temporary' keyword. + tk.MustGetErrCode("create temporary table t(id int)", errno.ErrNotSupportedYet) + + tk.MustExec("set @@tidb_enable_noop_functions = 1") + tk.MustExec("create temporary table t (id int)") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning 1105 local TEMPORARY TABLE is not supported yet, TEMPORARY will be parsed but ignored")) +} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d0289dc19e39f..2cfd3f524a29f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1736,8 +1736,14 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh switch s.TemporaryKeyword { case ast.TemporaryGlobal: tbInfo.TempTableType = model.TempTableGlobal + // "create global temporary table ... on commit preserve rows" + if !s.OnCommitDelete { + return nil, errors.Trace(errUnsupportedOnCommitPreserve) + } case ast.TemporaryLocal: - tbInfo.TempTableType = model.TempTableLocal + // TODO: set "tbInfo.TempTableType = model.TempTableLocal" after local temporary table is supported. + tbInfo.TempTableType = model.TempTableNone + ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("local TEMPORARY TABLE is not supported yet, TEMPORARY will be parsed but ignored")) case ast.TemporaryNone: tbInfo.TempTableType = model.TempTableNone } @@ -2217,6 +2223,12 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err tbInfo.PreSplitRegions = op.UintValue case ast.TableOptionCharset, ast.TableOptionCollate: // We don't handle charset and collate here since they're handled in `getCharsetAndCollateInTableOption`. + case ast.TableOptionEngine: + if tbInfo.TempTableType != model.TempTableNone { + if op.StrValue != "" && !strings.EqualFold(op.StrValue, "memory") { + return errors.Trace(errUnsupportedEngineTemporary) + } + } } } shardingBits := shardingBits(tbInfo) diff --git a/ddl/error.go b/ddl/error.go index 463c9c405a19e..6bde83badb41b 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -277,4 +277,7 @@ var ( // ErrPartitionNoTemporary returns when partition at temporary mode ErrPartitionNoTemporary = dbterror.ClassDDL.NewStd(mysql.ErrPartitionNoTemporary) + + errUnsupportedOnCommitPreserve = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("TiDB doesn't support ON COMMIT PRESERVE ROWS for now", nil)) + errUnsupportedEngineTemporary = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("TiDB doesn't support this kind of engine for temporary table", nil)) ) From 6833280fabbb33df15abb5a0460f037d7e252684 Mon Sep 17 00:00:00 2001 From: Lynn Date: Fri, 21 May 2021 12:58:21 +0800 Subject: [PATCH 135/343] *: update go.etcd.io/bbolt (#24799) --- go.mod | 3 +++ go.sum | 5 ++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index f82a8a187775f..82a394d8406de 100644 --- a/go.mod +++ b/go.mod @@ -89,3 +89,6 @@ require ( ) go 1.13 + +// Fix panic in unit test with go >= 1.14, ref: etcd-io/bbolt#201 https://github.com/etcd-io/bbolt/pull/201 +replace go.etcd.io/bbolt => go.etcd.io/bbolt v1.3.5 diff --git a/go.sum b/go.sum index 14986c3d1f025..cf26e5ebe4089 100644 --- a/go.sum +++ b/go.sum @@ -600,9 +600,8 @@ github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1: github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= -go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= -go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= -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/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 h1:3kC4J3eQF6p1UEfQTkC67eEeb3rTk+shQqdX6tFyq9Q= From 17e8561040c86139d36581803a5f701e1c62a27e Mon Sep 17 00:00:00 2001 From: Lynn Date: Fri, 21 May 2021 14:14:21 +0800 Subject: [PATCH 136/343] ddl: speed up the execution time of `ddl test` and `Test Chunk #7 ddl-other` (#24780) --- ddl/db_integration_test.go | 70 +++++++++++--------------------------- ddl/db_partition_test.go | 24 +++++++------ ddl/db_test.go | 28 ++++++++------- ddl/placement_sql_test.go | 2 +- ddl/serial_test.go | 28 +++++++-------- 5 files changed, 64 insertions(+), 88 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index b7a2b2a26e37a..9fe96a50afa02 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -57,8 +57,6 @@ var _ = Suite(&testIntegrationSuite3{&testIntegrationSuite{}}) var _ = Suite(&testIntegrationSuite4{&testIntegrationSuite{}}) var _ = Suite(&testIntegrationSuite5{&testIntegrationSuite{}}) var _ = Suite(&testIntegrationSuite6{&testIntegrationSuite{}}) -var _ = SerialSuites(&testIntegrationSuite7{&testIntegrationSuite{}}) -var _ = SerialSuites(&testIntegrationSuite8{&testIntegrationSuite{}}) type testIntegrationSuite struct { lease time.Duration @@ -127,8 +125,6 @@ type testIntegrationSuite4 struct{ *testIntegrationSuite } type testIntegrationSuite5 struct{ *testIntegrationSuite } type testIntegrationSuite6 struct{ *testIntegrationSuite } type testIntegrationSuite7 struct{ *testIntegrationSuite } -type testIntegrationSuite8 struct{ *testIntegrationSuite } -type testIntegrationSuite9 struct{ *testIntegrationSuite } func (s *testIntegrationSuite5) TestNoZeroDateMode(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -223,9 +219,10 @@ func (s *testIntegrationSuite2) TestCreateTableWithKeyWord(c *C) { c.Assert(err, IsNil) } -func (s *testIntegrationSuite1) TestUniqueKeyNullValue(c *C) { +func (s *testIntegrationSuite6) TestUniqueKeyNullValue(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test") + tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b varchar(255))") tk.MustExec("insert into t values(1, NULL)") @@ -308,7 +305,7 @@ func (s *testIntegrationSuite2) TestIssue19229(c *C) { tk.MustExec("drop table sett") } -func (s *testIntegrationSuite1) TestIndexLength(c *C) { +func (s *testIntegrationSuite7) TestIndexLength(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table idx_len(a int(0), b timestamp(0), c datetime(0), d time(0), f float(0), g decimal(0))") @@ -384,7 +381,7 @@ func (s *testIntegrationSuite1) TestIssue4432(c *C) { tk.MustExec("drop table tx") } -func (s *testIntegrationSuite1) TestIssue5092(c *C) { +func (s *testIntegrationSuite7) TestIssue5092(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -641,6 +638,9 @@ func (s *testIntegrationSuite3) TestTableDDLWithFloatType(c *C) { } func (s *testIntegrationSuite1) TestTableDDLWithTimeType(c *C) { + if israce.RaceEnabled { + c.Skip("skip race test") + } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1059,7 +1059,7 @@ func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn5(c *C) { tk.MustExec("admin check table t") } -func (s *testIntegrationSuite2) TestCaseInsensitiveCharsetAndCollate(c *C) { +func (s *testIntegrationSuite6) TestCaseInsensitiveCharsetAndCollate(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists test_charset_collate") @@ -1288,41 +1288,6 @@ func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { tk.MustExec("admin check index t idx_b") } -func (s *testIntegrationSuite3) TestMultiRegionGetTableEndHandle(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("drop database if exists test_get_endhandle") - tk.MustExec("create database test_get_endhandle") - tk.MustExec("use test_get_endhandle") - - tk.MustExec("create table t(a bigint PRIMARY KEY nonclustered, b int)") - for i := 0; i < 1000; i++ { - tk.MustExec(fmt.Sprintf("insert into t values(%v, %v)", i, i)) - } - - // Get table ID for split. - dom := domain.GetDomain(tk.Se) - is := dom.InfoSchema() - tbl, err := is.TableByName(model.NewCIStr("test_get_endhandle"), model.NewCIStr("t")) - c.Assert(err, IsNil) - tblID := tbl.Meta().ID - - d := s.dom.DDL() - testCtx := newTestMaxTableRowIDContext(c, d, tbl) - - // Split the table. - tableStart := tablecodec.GenTableRecordPrefix(tblID) - s.cluster.SplitKeys(tableStart, tableStart.PrefixNext(), 100) - - maxHandle, emptyTable := getMaxTableHandle(testCtx, s.store) - c.Assert(emptyTable, IsFalse) - c.Assert(maxHandle, Equals, kv.IntHandle(1000)) - - tk.MustExec("insert into t values(10000, 1000)") - maxHandle, emptyTable = getMaxTableHandle(testCtx, s.store) - c.Assert(emptyTable, IsFalse) - c.Assert(maxHandle, Equals, kv.IntHandle(1001)) -} - type testMaxTableRowIDContext struct { c *C d ddl.DDL @@ -1369,6 +1334,9 @@ func getHistoryDDLJob(store kv.Storage, id int64) (*model.Job, error) { } func (s *testIntegrationSuite6) TestCreateTableTooLarge(c *C) { + if israce.RaceEnabled { + c.Skip("skip race test") + } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1390,7 +1358,7 @@ func (s *testIntegrationSuite6) TestCreateTableTooLarge(c *C) { atomic.StoreUint32(&config.GetGlobalConfig().TableColumnCountLimit, originLimit) } -func (s *testIntegrationSuite8) TestCreateTableTooManyIndexes(c *C) { +func (s *testSerialDBSuite1) TestCreateTableTooManyIndexes(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1519,7 +1487,7 @@ func (s *testIntegrationSuite6) TestAddColumnTooMany(c *C) { tk.MustGetErrCode(alterSQL, errno.ErrTooManyFields) } -func (s *testIntegrationSuite8) TestCreateTooManyIndexes(c *C) { +func (s *testSerialDBSuite1) TestCreateTooManyIndexes(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") count := config.GetGlobalConfig().IndexLimit - 1 @@ -1541,7 +1509,7 @@ func (s *testIntegrationSuite8) TestCreateTooManyIndexes(c *C) { tk.MustGetErrCode(alterSQL, errno.ErrTooManyKeys) } -func (s *testIntegrationSuite8) TestCreateSecondaryIndexInCluster(c *C) { +func (s *testSerialDBSuite1) TestCreateSecondaryIndexInCluster(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -2271,7 +2239,7 @@ func (s *testIntegrationSuite3) TestParserIssue284(c *C) { tk.MustExec("drop table test.t_parser_issue_284_2") } -func (s *testIntegrationSuite7) TestAddExpressionIndex(c *C) { +func (s *testSerialDBSuite1) TestAddExpressionIndex(c *C) { config.UpdateGlobal(func(conf *config.Config) { conf.Experimental.AllowsExpressionIndex = true }) @@ -2337,7 +2305,7 @@ func (s *testIntegrationSuite7) TestAddExpressionIndex(c *C) { tk.MustGetErrMsg("create table t(a int, key ((a+1)));", "[ddl:8200]Unsupported creating expression index without allow-expression-index in config") } -func (s *testIntegrationSuite7) TestCreateExpressionIndexError(c *C) { +func (s *testSerialDBSuite1) TestCreateExpressionIndexError(c *C) { defer config.RestoreFunc()() config.UpdateGlobal(func(conf *config.Config) { conf.Experimental.AllowsExpressionIndex = true @@ -2380,7 +2348,7 @@ func (s *testIntegrationSuite7) TestCreateExpressionIndexError(c *C) { tk.MustGetErrCode("CREATE TABLE t1 (col1 INT, PRIMARY KEY ((ABS(col1))) NONCLUSTERED);", errno.ErrFunctionalIndexPrimaryKey) } -func (s *testIntegrationSuite7) TestAddExpressionIndexOnPartition(c *C) { +func (s *testSerialDBSuite1) TestAddExpressionIndexOnPartition(c *C) { config.UpdateGlobal(func(conf *config.Config) { conf.Experimental.AllowsExpressionIndex = true }) @@ -2634,7 +2602,7 @@ func (s *testSerialDBSuite) TestDropLastVisibleColumnOrColumns(c *C) { tk.MustExec("drop table if exists t_drop_last_column, t_drop_last_columns") } -func (s *testIntegrationSuite7) TestAutoIncrementTableOption(c *C) { +func (s *testSerialDBSuite1) TestAutoIncrementTableOption(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_auto_inc_table_opt;") tk.MustExec("create database test_auto_inc_table_opt;") @@ -2726,7 +2694,7 @@ func (s *testIntegrationSuite3) TestStrictDoubleTypeCheck(c *C) { tk.MustExec(sql) } -func (s *testIntegrationSuite7) TestDuplicateErrorMessage(c *C) { +func (s *testSerialDBSuite) TestDuplicateErrorMessage(c *C) { defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index b52644c151fd4..b261ca3f98519 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -45,6 +45,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" ) @@ -359,7 +360,7 @@ func (s *testIntegrationSuite2) TestCreateTableWithHashPartition(c *C) { tk.MustExec("create table t4 (a int, b int) partition by hash(floor(a-b)) partitions 10") } -func (s *testIntegrationSuite7) TestCreateTableWithRangeColumnPartition(c *C) { +func (s *testSerialDBSuite1) TestCreateTableWithRangeColumnPartition(c *C) { collate.SetNewCollationEnabledForTest(true) defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKit(c, s.store) @@ -593,6 +594,9 @@ create table log_message_1 ( } func (s *testIntegrationSuite1) TestDisableTablePartition(c *C) { + if israce.RaceEnabled { + c.Skip("skip race test") + } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") for _, v := range []string{"'AUTO'", "'OFF'", "0", "'ON'"} { @@ -1603,7 +1607,7 @@ func (s *testIntegrationSuite5) TestMultiPartitionDropAndTruncate(c *C) { result.Check(testkit.Rows(`2010`)) } -func (s *testIntegrationSuite7) TestDropPartitionWithGlobalIndex(c *C) { +func (s *testSerialDBSuite1) TestDropPartitionWithGlobalIndex(c *C) { config.UpdateGlobal(func(conf *config.Config) { conf.EnableGlobalIndex = true }) @@ -1641,7 +1645,7 @@ func (s *testIntegrationSuite7) TestDropPartitionWithGlobalIndex(c *C) { }) } -func (s *testIntegrationSuite7) TestAlterTableExchangePartition(c *C) { +func (s *testSerialDBSuite1) TestAlterTableExchangePartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists e") @@ -2076,7 +2080,7 @@ func (s *testIntegrationSuite4) TestExchangePartitionTableCompatiable(c *C) { c.Assert(err, IsNil) } -func (s *testIntegrationSuite7) TestExchangePartitionExpressIndex(c *C) { +func (s *testSerialDBSuite1) TestExchangePartitionExpressIndex(c *C) { config.UpdateGlobal(func(conf *config.Config) { conf.Experimental.AllowsExpressionIndex = true }) @@ -3205,7 +3209,7 @@ func (s *testIntegrationSuite3) TestUnsupportedPartitionManagementDDLs(c *C) { c.Assert(err, ErrorMatches, ".*alter table partition is unsupported") } -func (s *testIntegrationSuite7) TestCommitWhenSchemaChange(c *C) { +func (s *testSerialDBSuite1) TestCommitWhenSchemaChange(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`create table schema_change (a int, b timestamp) @@ -3270,7 +3274,7 @@ func (s *testIntegrationSuite7) TestCommitWhenSchemaChange(c *C) { tk.MustQuery("select * from nt").Check(testkit.Rows()) } -func (s *testIntegrationSuite7) TestCreatePartitionTableWithWrongType(c *C) { +func (s *testSerialDBSuite1) TestCreatePartitionTableWithWrongType(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -3311,7 +3315,7 @@ func (s *testIntegrationSuite7) TestCreatePartitionTableWithWrongType(c *C) { c.Assert(err, NotNil) } -func (s *testIntegrationSuite7) TestAddPartitionForTableWithWrongType(c *C) { +func (s *testSerialDBSuite1) TestAddPartitionForTableWithWrongType(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop tables if exists t_int, t_char, t_date") @@ -3361,7 +3365,7 @@ func (s *testIntegrationSuite7) TestAddPartitionForTableWithWrongType(c *C) { c.Assert(ddl.ErrWrongTypeColumnValue.Equal(err), IsTrue) } -func (s *testIntegrationSuite7) TestPartitionListWithTimeType(c *C) { +func (s *testSerialDBSuite1) TestPartitionListWithTimeType(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("use test;") tk.MustExec("set @@session.tidb_enable_list_partition = ON") @@ -3370,7 +3374,7 @@ func (s *testIntegrationSuite7) TestPartitionListWithTimeType(c *C) { tk.MustQuery(`select * from t_list1 partition (p0);`).Check(testkit.Rows("2018-02-03")) } -func (s *testIntegrationSuite7) TestPartitionListWithNewCollation(c *C) { +func (s *testSerialDBSuite1) TestPartitionListWithNewCollation(c *C) { collate.SetNewCollationEnabledForTest(true) defer collate.SetNewCollationEnabledForTest(false) tk := testkit.NewTestKitWithInit(c, s.store) @@ -3387,7 +3391,7 @@ func (s *testIntegrationSuite7) TestPartitionListWithNewCollation(c *C) { c.Assert(strings.Contains(str, "partition:p0"), IsTrue) } -func (s *testIntegrationSuite7) TestAddTableWithPartition(c *C) { +func (s *testSerialDBSuite1) TestAddTableWithPartition(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists global_partition_table;") diff --git a/ddl/db_test.go b/ddl/db_test.go index ecef0d0144215..1ae6ac0ff3efd 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -76,8 +76,9 @@ var _ = Suite(&testDBSuite4{&testDBSuite{}}) var _ = Suite(&testDBSuite5{&testDBSuite{}}) var _ = Suite(&testDBSuite6{&testDBSuite{}}) var _ = Suite(&testDBSuite7{&testDBSuite{}}) -var _ = SerialSuites(&testSerialDBSuite{&testDBSuite{}}) var _ = Suite(&testDBSuite8{&testDBSuite{}}) +var _ = SerialSuites(&testSerialDBSuite{&testDBSuite{}}) +var _ = SerialSuites(&testSerialDBSuite1{&testDBSuite{}}) const defaultBatchSize = 1024 const defaultReorgBatchSize = 256 @@ -90,6 +91,7 @@ type testDBSuite struct { s session.Session lease time.Duration autoIDStep int64 + ctx sessionctx.Context } func setUpSuite(s *testDBSuite, c *C) { @@ -114,6 +116,7 @@ func setUpSuite(s *testDBSuite, c *C) { c.Assert(err, IsNil) s.s, err = session.CreateSession4Test(s.store) c.Assert(err, IsNil) + s.ctx = s.s.(sessionctx.Context) _, err = s.s.Execute(context.Background(), "create database test_db") c.Assert(err, IsNil) @@ -145,8 +148,9 @@ type testDBSuite4 struct{ *testDBSuite } type testDBSuite5 struct{ *testDBSuite } type testDBSuite6 struct{ *testDBSuite } type testDBSuite7 struct{ *testDBSuite } -type testSerialDBSuite struct{ *testDBSuite } type testDBSuite8 struct{ *testDBSuite } +type testSerialDBSuite struct{ *testDBSuite } +type testSerialDBSuite1 struct{ *testDBSuite } func testAddIndexWithPK(tk *testkit.TestKit) { tk.MustExec("drop table if exists test_add_index_with_pk") @@ -287,7 +291,7 @@ func backgroundExec(s kv.Storage, sql string, done chan error) { } // TestAddPrimaryKeyRollback1 is used to test scenarios that will roll back when a duplicate primary key is encountered. -func (s *testDBSuite5) TestAddPrimaryKeyRollback1(c *C) { +func (s *testDBSuite8) TestAddPrimaryKeyRollback1(c *C) { hasNullValsInKey := false idxName := "PRIMARY" addIdxSQL := "alter table t1 add primary key c3_index (c3);" @@ -296,7 +300,7 @@ func (s *testDBSuite5) TestAddPrimaryKeyRollback1(c *C) { } // TestAddPrimaryKeyRollback2 is used to test scenarios that will roll back when a null primary key is encountered. -func (s *testDBSuite1) TestAddPrimaryKeyRollback2(c *C) { +func (s *testDBSuite8) TestAddPrimaryKeyRollback2(c *C) { hasNullValsInKey := true idxName := "PRIMARY" addIdxSQL := "alter table t1 add primary key c3_index (c3);" @@ -444,7 +448,7 @@ LOOP: tk.MustExec("drop table t1") } -func (s *testDBSuite5) TestCancelAddPrimaryKey(c *C) { +func (s *testDBSuite8) TestCancelAddPrimaryKey(c *C) { idxName := "primary" addIdxSQL := "alter table t1 add primary key idx_c2 (c2);" testCancelAddIndex(c, s.store, s.dom.DDL(), s.lease, idxName, addIdxSQL, "", s.dom) @@ -460,7 +464,7 @@ func (s *testDBSuite5) TestCancelAddPrimaryKey(c *C) { tk.MustExec("drop table t1") } -func (s *testDBSuite3) TestCancelAddIndex(c *C) { +func (s *testDBSuite7) TestCancelAddIndex(c *C) { idxName := "c3_index " addIdxSQL := "create unique index c3_index on t1 (c3)" testCancelAddIndex(c, s.store, s.dom.DDL(), s.lease, idxName, addIdxSQL, "", s.dom) @@ -1072,7 +1076,7 @@ func (s *testDBSuite6) TestAddMultiColumnsIndexClusterIndex(c *C) { tk.MustExec("admin check table t;") } -func (s *testDBSuite1) TestAddPrimaryKey1(c *C) { +func (s *testDBSuite6) TestAddPrimaryKey1(c *C) { testAddIndex(c, s.store, s.lease, testPlain, "create table test_add_index (c1 bigint, c2 bigint, c3 bigint, unique key(c1))", "primary") } @@ -1105,7 +1109,7 @@ func (s *testDBSuite4) TestAddPrimaryKey4(c *C) { partition p4 values less than maxvalue)`, "primary") } -func (s *testDBSuite1) TestAddIndex1(c *C) { +func (s *testDBSuite6) TestAddIndex1(c *C) { testAddIndex(c, s.store, s.lease, testPlain, "create table test_add_index (c1 bigint, c2 bigint, c3 bigint, primary key(c1))", "") } @@ -1127,7 +1131,7 @@ func (s *testDBSuite3) TestAddIndex3(c *C) { partition by hash (c1) partitions 4;`, "") } -func (s *testDBSuite4) TestAddIndex4(c *C) { +func (s *testDBSuite8) TestAddIndex4(c *C) { testAddIndex(c, s.store, s.lease, testPartition, `create table test_add_index (c1 bigint, c2 bigint, c3 bigint, primary key(c1)) partition by range columns (c1) ( @@ -2035,7 +2039,7 @@ func (s *testDBSuite5) TestCreateIndexType(c *C) { tk.MustExec(sql) } -func (s *testDBSuite1) TestColumn(c *C) { +func (s *testDBSuite6) TestColumn(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use " + s.schemaName) tk.MustExec("create table t2 (c1 int, c2 int, c3 int)") @@ -3026,7 +3030,7 @@ func (s *testDBSuite2) TestTableForeignKey(c *C) { tk.MustExec("drop table if exists t1,t2,t3,t4;") } -func (s *testDBSuite3) TestFKOnGeneratedColumns(c *C) { +func (s *testDBSuite8) TestFKOnGeneratedColumns(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") // test add foreign key to generated column @@ -4902,7 +4906,7 @@ func (s *testDBSuite1) TestModifyColumnTime_TimeToTimestamp(c *C) { testModifyColumnTime(c, s.store, tests) } -func (s *testDBSuite1) TestModifyColumnTime_DateToTime(c *C) { +func (s *testDBSuite7) TestModifyColumnTime_DateToTime(c *C) { tests := []testModifyColumnTimeCase{ // date to time {"date", `"2019-01-02"`, "time", "00:00:00", 0}, diff --git a/ddl/placement_sql_test.go b/ddl/placement_sql_test.go index fb7158681714f..36812454fb662 100644 --- a/ddl/placement_sql_test.go +++ b/ddl/placement_sql_test.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testDBSuite1) TestAlterTableAlterPartition(c *C) { +func (s *testDBSuite6) TestAlterTableAlterPartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") diff --git a/ddl/serial_test.go b/ddl/serial_test.go index a8d245007f39c..b891cce567392 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -55,7 +55,7 @@ import ( var _ = SerialSuites(&testSerialSuite{}) // TODO(tangenta): Move all the parallel tests out of this file. -var _ = Suite(&testIntegrationSuite9{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite7{&testIntegrationSuite{}}) type testSerialSuite struct { CommonHandleSuite @@ -113,7 +113,7 @@ func (s *testSerialSuite) TestChangeMaxIndexLength(c *C) { tk.MustExec("drop table t, t1") } -func (s *testIntegrationSuite9) TestPrimaryKey(c *C) { +func (s *testIntegrationSuite7) TestPrimaryKey(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_primary_key;") tk.MustExec("create database test_primary_key;") @@ -178,7 +178,7 @@ func (s *testIntegrationSuite9) TestPrimaryKey(c *C) { tk.MustExec("drop table t;") } -func (s *testIntegrationSuite9) TestDropAutoIncrementIndex(c *C) { +func (s *testIntegrationSuite7) TestDropAutoIncrementIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -186,7 +186,7 @@ func (s *testIntegrationSuite9) TestDropAutoIncrementIndex(c *C) { tk.MustExec("alter table t1 drop index a") } -func (s *testIntegrationSuite9) TestMultiRegionGetTableEndHandle(c *C) { +func (s *testIntegrationSuite7) TestMultiRegionGetTableEndHandle(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_get_endhandle") tk.MustExec("create database test_get_endhandle") @@ -230,7 +230,7 @@ func (s *testIntegrationSuite9) TestMultiRegionGetTableEndHandle(c *C) { c.Assert(maxHandle, Equals, kv.IntHandle(10000)) } -func (s *testIntegrationSuite9) TestGetTableEndHandle(c *C) { +func (s *testIntegrationSuite7) TestGetTableEndHandle(c *C) { // TestGetTableEndHandle test ddl.GetTableMaxHandle method, which will return the max row id of the table. tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_get_endhandle") @@ -322,7 +322,7 @@ func (s *testIntegrationSuite9) TestGetTableEndHandle(c *C) { c.Assert(emptyTable, IsFalse) } -func (s *testIntegrationSuite9) TestMultiRegionGetTableEndCommonHandle(c *C) { +func (s *testIntegrationSuite7) TestMultiRegionGetTableEndCommonHandle(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_get_endhandle") tk.MustExec("create database test_get_endhandle") @@ -367,7 +367,7 @@ func (s *testIntegrationSuite9) TestMultiRegionGetTableEndCommonHandle(c *C) { c.Assert(maxHandle, HandleEquals, MustNewCommonHandle(c, "a", 1, 1)) } -func (s *testIntegrationSuite9) TestGetTableEndCommonHandle(c *C) { +func (s *testIntegrationSuite7) TestGetTableEndCommonHandle(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_get_endhandle") tk.MustExec("create database test_get_endhandle") @@ -925,7 +925,7 @@ func (s *testSerialSuite) TestTableLocksEnable(c *C) { checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone) } -func (s *testSerialSuite) TestAutoRandom(c *C) { +func (s *testSerialDBSuite) TestAutoRandom(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists auto_random_db") defer tk.MustExec("drop database if exists auto_random_db") @@ -1146,7 +1146,7 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { }) } -func (s *testIntegrationSuite9) TestAutoRandomChangeFromAutoInc(c *C) { +func (s *testIntegrationSuite7) TestAutoRandomChangeFromAutoInc(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("set @@tidb_allow_remove_auto_inc = 1;") @@ -1202,7 +1202,7 @@ func (s *testIntegrationSuite9) TestAutoRandomChangeFromAutoInc(c *C) { tk.MustExec("alter table t modify column a bigint auto_random(4);") } -func (s *testIntegrationSuite9) TestAutoRandomExchangePartition(c *C) { +func (s *testIntegrationSuite7) TestAutoRandomExchangePartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists auto_random_db") defer tk.MustExec("drop database if exists auto_random_db") @@ -1236,7 +1236,7 @@ func (s *testIntegrationSuite9) TestAutoRandomExchangePartition(c *C) { tk.MustQuery("select count(*) from e4").Check(testkit.Rows("4")) } -func (s *testIntegrationSuite9) TestAutoRandomIncBitsIncrementAndOffset(c *C) { +func (s *testIntegrationSuite7) TestAutoRandomIncBitsIncrementAndOffset(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists auto_random_db") defer tk.MustExec("drop database if exists auto_random_db") @@ -1388,7 +1388,7 @@ func (s *testSerialSuite) TestForbidUnsupportedCollations(c *C) { // mustGetUnsupportedCollation("alter table t convert to collate utf8mb4_unicode_ci", "utf8mb4_unicode_ci") } -func (s *testIntegrationSuite9) TestInvisibleIndex(c *C) { +func (s *testIntegrationSuite7) TestInvisibleIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1464,7 +1464,7 @@ func (s *testIntegrationSuite9) TestInvisibleIndex(c *C) { c.Check(len(res.Rows()), Equals, 1) } -func (s *testIntegrationSuite9) TestCreateClusteredIndex(c *C) { +func (s *testIntegrationSuite7) TestCreateClusteredIndex(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.Se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("CREATE TABLE t1 (a int primary key, b int)") @@ -1515,7 +1515,7 @@ func (s *testIntegrationSuite9) TestCreateClusteredIndex(c *C) { c.Assert(tbl.Meta().IsCommonHandle, IsFalse) } -func (s *testSerialSuite) TestCreateTableNoBlock(c *C) { +func (s *testSerialDBSuite) TestCreateTableNoBlock(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/checkOwnerCheckAllVersionsWaitTime", `return(true)`), IsNil) defer func() { From 35f5ff44078777741cae7df7bb106f9e26fac633 Mon Sep 17 00:00:00 2001 From: JmPotato Date: Fri, 21 May 2021 15:54:21 +0800 Subject: [PATCH 137/343] executor: remove the unnecessary use of fmt.Sprintf (#24815) --- executor/partition_table_test.go | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 7398789eadba5..9590c7ed4ceb2 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -469,8 +469,8 @@ func (s *partitionTableSuite) TestDirectReadingwithIndexJoin(c *C) { tk.MustExec("insert ignore into touter values " + strings.Join(vals, ",")) // test indexLookUp + hash - queryPartition := fmt.Sprintf("select /*+ INL_JOIN(touter, thash) */ * from touter join thash use index(idx_b) on touter.b = thash.b") - queryRegular := fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal use index(idx_b) on touter.b = tnormal.b") + queryPartition := "select /*+ INL_JOIN(touter, thash) */ * from touter join thash use index(idx_b) on touter.b = thash.b" + queryRegular := "select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal use index(idx_b) on touter.b = tnormal.b" tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( "IndexJoin 12487.50 root inner join, inner:IndexLookUp, outer key:test_dr_join.touter.b, inner key:test_dr_join.thash.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.thash.b)", "├─TableReader(Build) 9990.00 root data:Selection", @@ -483,8 +483,8 @@ func (s *partitionTableSuite) TestDirectReadingwithIndexJoin(c *C) { tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) // test tableReader + hash - queryPartition = fmt.Sprintf("select /*+ INL_JOIN(touter, thash) */ * from touter join thash on touter.a = thash.a") - queryRegular = fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal on touter.a = tnormal.a") + queryPartition = "select /*+ INL_JOIN(touter, thash) */ * from touter join thash on touter.a = thash.a" + queryRegular = "select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal on touter.a = tnormal.a" tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test_dr_join.touter.a, inner key:test_dr_join.thash.a, equal cond:eq(test_dr_join.touter.a, test_dr_join.thash.a)", "├─TableReader(Build) 9990.00 root data:Selection", @@ -495,8 +495,8 @@ func (s *partitionTableSuite) TestDirectReadingwithIndexJoin(c *C) { tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) // test indexReader + hash - queryPartition = fmt.Sprintf("select /*+ INL_JOIN(touter, thash) */ thash.b from touter join thash use index(idx_b) on touter.b = thash.b;") - queryRegular = fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b;") + queryPartition = "select /*+ INL_JOIN(touter, thash) */ thash.b from touter join thash use index(idx_b) on touter.b = thash.b;" + queryRegular = "select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b;" tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( "IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test_dr_join.touter.b, inner key:test_dr_join.thash.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.thash.b)", "├─TableReader(Build) 9990.00 root data:Selection", @@ -509,8 +509,8 @@ func (s *partitionTableSuite) TestDirectReadingwithIndexJoin(c *C) { // test indexLookUp + range // explain select /*+ INL_JOIN(touter, tinner) */ * from touter join tinner use index(a) on touter.a = tinner.a; - queryPartition = fmt.Sprintf("select /*+ INL_JOIN(touter, trange) */ * from touter join trange use index(idx_b) on touter.b = trange.b;") - queryRegular = fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal use index(idx_b) on touter.b = tnormal.b;") + queryPartition = "select /*+ INL_JOIN(touter, trange) */ * from touter join trange use index(idx_b) on touter.b = trange.b;" + queryRegular = "select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal use index(idx_b) on touter.b = tnormal.b;" tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( "IndexJoin 12487.50 root inner join, inner:IndexLookUp, outer key:test_dr_join.touter.b, inner key:test_dr_join.trange.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.trange.b)", "├─TableReader(Build) 9990.00 root data:Selection", @@ -523,8 +523,8 @@ func (s *partitionTableSuite) TestDirectReadingwithIndexJoin(c *C) { tk.MustQuery(queryPartition).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) // test tableReader + range - queryPartition = fmt.Sprintf("select /*+ INL_JOIN(touter, trange) */ * from touter join trange on touter.a = trange.a;") - queryRegular = fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal on touter.a = tnormal.a;") + queryPartition = "select /*+ INL_JOIN(touter, trange) */ * from touter join trange on touter.a = trange.a;" + queryRegular = "select /*+ INL_JOIN(touter, tnormal) */ * from touter join tnormal on touter.a = tnormal.a;" tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( "IndexJoin 12487.50 root inner join, inner:TableReader, outer key:test_dr_join.touter.a, inner key:test_dr_join.trange.a, equal cond:eq(test_dr_join.touter.a, test_dr_join.trange.a)", "├─TableReader(Build) 9990.00 root data:Selection", @@ -536,8 +536,8 @@ func (s *partitionTableSuite) TestDirectReadingwithIndexJoin(c *C) { // test indexReader + range // explain select /*+ INL_JOIN(touter, tinner) */ tinner.a from touter join tinner on touter.a = tinner.a; - queryPartition = fmt.Sprintf("select /*+ INL_JOIN(touter, trange) */ trange.b from touter join trange use index(idx_b) on touter.b = trange.b;") - queryRegular = fmt.Sprintf("select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b;") + queryPartition = "select /*+ INL_JOIN(touter, trange) */ trange.b from touter join trange use index(idx_b) on touter.b = trange.b;" + queryRegular = "select /*+ INL_JOIN(touter, tnormal) */ tnormal.b from touter join tnormal use index(idx_b) on touter.b = tnormal.b;" tk.MustQuery("explain format = 'brief' " + queryPartition).Check(testkit.Rows( "IndexJoin 12487.50 root inner join, inner:IndexReader, outer key:test_dr_join.touter.b, inner key:test_dr_join.trange.b, equal cond:eq(test_dr_join.touter.b, test_dr_join.trange.b)", "├─TableReader(Build) 9990.00 root data:Selection", From 032d34c7508c64ebb0982f70138b7731da8abd5d Mon Sep 17 00:00:00 2001 From: Zhuomin Liu Date: Fri, 21 May 2021 16:06:21 +0800 Subject: [PATCH 138/343] executor: fix index join panic on prefix index on some cases (#24568) --- executor/index_lookup_join.go | 2 +- executor/index_lookup_join_test.go | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 0e31280b6632a..976ca336d088d 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -537,7 +537,7 @@ func (iw *innerWorker) constructLookupContent(task *lookUpJoinTask) ([]*indexJoi // Store the encoded lookup key in chunk, so we can use it to lookup the matched inners directly. task.encodedLookUpKeys[chkIdx].AppendBytes(0, keyBuf) if iw.hasPrefixCol { - for i, outerOffset := range iw.outerCtx.keyCols { + for i, outerOffset := range iw.keyOff2IdxOff { // If it's a prefix column. Try to fix it. joinKeyColPrefixLen := iw.colLens[outerOffset] if joinKeyColPrefixLen != types.UnspecifiedLength { diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index 9515907b8eba6..f524acc6acca5 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -335,6 +335,18 @@ func (s *testSuite5) TestIssue23722(c *C) { "order by col_15 , col_16 , col_17 , col_18 , col_19;").Check(testkit.Rows("38799.400 20301 KETeFZhkoxnwMAhA Charlie zyhXEppZdqyqNV")) } +func (s *testSuite5) TestIssue24547(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists a") + tk.MustExec("drop table if exists b") + tk.MustExec("CREATE TABLE `a` (\n `v` varchar(100) DEFAULT NULL,\n `k1` varchar(100) NOT NULL,\n `k2` varchar(100) NOT NULL,\n PRIMARY KEY (`k1`(3),`k2`(3)) /*T![clustered_index] CLUSTERED */,\n KEY `kk2` (`k2`(3)),\n UNIQUE KEY `uk1` (`v`)\n)") + tk.MustExec("CREATE TABLE `b` (\n `v` varchar(100) DEFAULT NULL,\n `k1` varchar(100) NOT NULL,\n `k2` varchar(100) NOT NULL,\n PRIMARY KEY (`k1`(3),`k2`(3)) /*T![clustered_index] CLUSTERED */,\n KEY `kk2` (`k2`(3))\n)") + tk.MustExec("insert into a(v, k1, k2) values('1', '1', '1'), ('22', '22', '22'), ('333', '333', '333'), ('3444', '3444', '3444'), ('444', '444', '444')") + tk.MustExec("insert into b(v, k1, k2) values('1', '1', '1'), ('22', '22', '22'), ('333', '333', '333'), ('2333', '2333', '2333'), ('555', '555', '555')") + tk.MustExec("delete a from a inner join b on a.k1 = b.k1 and a.k2 = b.k2 where b.k2 <> '333'") +} + func (s *testSuite5) TestPartitionTableIndexJoinAndIndexReader(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 163eb5f03c6fa664ef1f17b2e2f5f1c692e7071c Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Sat, 22 May 2021 01:23:53 +0800 Subject: [PATCH 139/343] planner: Implement PointGet in TryFastPlan for range/list paritition table (#24588) * implement PointGet in TryFastPlan for range/list partition --- Makefile | 10 +- cmd/explaintest/r/partition_pruning.result | 47 +++---- executor/partition_table_test.go | 74 +++++++++++ executor/testdata/executor_suite_out.json | 8 +- planner/core/point_get_plan.go | 115 +++++++++++++++--- .../core/testdata/partition_pruner_out.json | 4 +- table/tables/partition.go | 5 +- tools/check/go.mod | 2 + tools/check/go.sum | 21 +++- 9 files changed, 223 insertions(+), 63 deletions(-) diff --git a/Makefile b/Makefile index d6681083dc284..d09ed06f2c528 100644 --- a/Makefile +++ b/Makefile @@ -239,11 +239,13 @@ tools/bin/unconvert: tools/check/go.mod cd tools/check; \ $(GO) build -o ../bin/unconvert github.com/mdempsky/unconvert -tools/bin/failpoint-ctl: go.mod - $(GO) build -o $@ github.com/pingcap/failpoint/failpoint-ctl +tools/bin/failpoint-ctl: tools/check/go.mod + cd tools/check; \ + $(GO) build -o ../bin/failpoint-ctl github.com/pingcap/failpoint/failpoint-ctl -tools/bin/errdoc-gen: go.mod - $(GO) build -o $@ github.com/pingcap/errors/errdoc-gen +tools/bin/errdoc-gen: tools/check/go.mod + cd tools/check; \ + $(GO) build -o ../bin/errdoc-gen github.com/pingcap/errors/errdoc-gen tools/bin/golangci-lint: curl -sfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh| sh -s -- -b ./tools/bin v1.29.0 diff --git a/cmd/explaintest/r/partition_pruning.result b/cmd/explaintest/r/partition_pruning.result index b3c3a8b3169b9..2e7d5dfcf0989 100644 --- a/cmd/explaintest/r/partition_pruning.result +++ b/cmd/explaintest/r/partition_pruning.result @@ -206,50 +206,43 @@ a 1 explain format = 'brief' SELECT * FROM t1 WHERE a = 1; id estRows task access object operator info -TableReader 1.00 root partition:p1 data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[1,1], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:p1 handle:1 SELECT * FROM t1 WHERE a = 2 order by a; a 2 explain format = 'brief' SELECT * FROM t1 WHERE a = 2; id estRows task access object operator info -TableReader 1.00 root partition:p2 data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[2,2], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:p2 handle:2 SELECT * FROM t1 WHERE a = 3 order by a; a 3 explain format = 'brief' SELECT * FROM t1 WHERE a = 3; id estRows task access object operator info -TableReader 1.00 root partition:p3 data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[3,3], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:p3 handle:3 SELECT * FROM t1 WHERE a = 4 order by a; a 4 explain format = 'brief' SELECT * FROM t1 WHERE a = 4; id estRows task access object operator info -TableReader 1.00 root partition:p4 data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[4,4], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:p4 handle:4 SELECT * FROM t1 WHERE a = 5 order by a; a 5 explain format = 'brief' SELECT * FROM t1 WHERE a = 5; id estRows task access object operator info -TableReader 1.00 root partition:p5 data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[5,5], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:p5 handle:5 SELECT * FROM t1 WHERE a = 6 order by a; a 6 explain format = 'brief' SELECT * FROM t1 WHERE a = 6; id estRows task access object operator info -TableReader 1.00 root partition:max data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[6,6], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:max handle:6 SELECT * FROM t1 WHERE a = 7 order by a; a 7 explain format = 'brief' SELECT * FROM t1 WHERE a = 7; id estRows task access object operator info -TableReader 1.00 root partition:max data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[7,7], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:max handle:7 SELECT * FROM t1 WHERE a >= 1 order by a; a 1 @@ -544,43 +537,37 @@ a 1 explain format = 'brief' SELECT * FROM t1 WHERE a = 1; id estRows task access object operator info -TableReader 1.00 root partition:p1 data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[1,1], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:p1 handle:1 SELECT * FROM t1 WHERE a = 2; a 2 explain format = 'brief' SELECT * FROM t1 WHERE a = 2; id estRows task access object operator info -TableReader 1.00 root partition:p2 data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[2,2], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:p2 handle:2 SELECT * FROM t1 WHERE a = 3; a 3 explain format = 'brief' SELECT * FROM t1 WHERE a = 3; id estRows task access object operator info -TableReader 1.00 root partition:p3 data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[3,3], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:p3 handle:3 SELECT * FROM t1 WHERE a = 4; a 4 explain format = 'brief' SELECT * FROM t1 WHERE a = 4; id estRows task access object operator info -TableReader 1.00 root partition:p4 data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[4,4], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:p4 handle:4 SELECT * FROM t1 WHERE a = 5; a 5 explain format = 'brief' SELECT * FROM t1 WHERE a = 5; id estRows task access object operator info -TableReader 1.00 root partition:max data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[5,5], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:max handle:5 SELECT * FROM t1 WHERE a = 6; a 6 explain format = 'brief' SELECT * FROM t1 WHERE a = 6; id estRows task access object operator info -TableReader 1.00 root partition:max data:TableRangeScan -└─TableRangeScan 1.00 cop[tikv] table:t1 range:[6,6], keep order:false, stats:pseudo +Point_Get 1.00 root table:t1, partition:max handle:6 SELECT * FROM t1 WHERE a >= 1 order by a; a 1 @@ -1795,9 +1782,7 @@ TableReader 3323.33 root partition:all data:Selection └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo explain format = 'brief' select * from t7 where a = 90; id estRows task access object operator info -TableReader 10.00 root partition:dual data:Selection -└─Selection 10.00 cop[tikv] eq(test.t7.a, 90) - └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo +TableDual 0.00 root rows:0 explain format = 'brief' select * from t7 where a > 90; id estRows task access object operator info TableReader 3333.33 root partition:dual data:Selection @@ -1919,9 +1904,7 @@ TableReader 3323.33 root partition:all data:Selection └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo explain format = 'brief' select * from t7 where a = 90; id estRows task access object operator info -TableReader 10.00 root partition:dual data:Selection -└─Selection 10.00 cop[tikv] eq(test.t7.a, 90) - └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo +TableDual 0.00 root rows:0 explain format = 'brief' select * from t7 where a > 90; id estRows task access object operator info TableReader 3333.33 root partition:dual data:Selection diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 9590c7ed4ceb2..73520eea1792a 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -111,6 +111,80 @@ func (s *partitionTableSuite) TestPartitionUnionScanIndexJoin(c *C) { tk.MustExec("commit") } +func (s *partitionTableSuite) TestPointGetwithRangeAndListPartitionTable(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_pointget_list_hash") + tk.MustExec("use test_pointget_list_hash") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + + // list partition table + tk.MustExec(`create table tlist(a int, b int, unique index idx_a(a), index idx_b(b)) partition by list(a)( + partition p0 values in (NULL, 1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8), + partition p2 values in (9, 10, 11, 12));`) + + // range partition table + tk.MustExec(`create table trange1(a int, unique key(a)) partition by range(a) ( + partition p0 values less than (30), + partition p1 values less than (60), + partition p2 values less than (90), + partition p3 values less than (120));`) + + // range partition table + unsigned int + tk.MustExec(`create table trange2(a int unsigned, unique key(a)) partition by range(a) ( + partition p0 values less than (30), + partition p1 values less than (60), + partition p2 values less than (90), + partition p3 values less than (120));`) + + // insert data into list partition table + tk.MustExec("insert into tlist values(1,1), (2,2), (3, 3), (4, 4), (5,5), (6, 6), (7,7), (8, 8), (9, 9), (10, 10), (11, 11), (12, 12), (NULL, NULL);") + + vals := make([]string, 0, 100) + // insert data into range partition table and hash partition table + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v)", i+1)) + } + tk.MustExec("insert into trange1 values " + strings.Join(vals, ",")) + tk.MustExec("insert into trange2 values " + strings.Join(vals, ",")) + + // test PointGet + for i := 0; i < 100; i++ { + // explain select a from t where a = {x}; // x >= 1 and x <= 100 Check if PointGet is used + // select a from t where a={x}; // the result is {x} + x := rand.Intn(100) + 1 + queryRange1 := fmt.Sprintf("select a from trange1 where a=%v", x) + c.Assert(tk.HasPlan(queryRange1, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryRange1).Check(testkit.Rows(fmt.Sprintf("%v", x))) + + queryRange2 := fmt.Sprintf("select a from trange1 where a=%v", x) + c.Assert(tk.HasPlan(queryRange2, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryRange2).Check(testkit.Rows(fmt.Sprintf("%v", x))) + + y := rand.Intn(12) + 1 + queryList := fmt.Sprintf("select a from tlist where a=%v", y) + c.Assert(tk.HasPlan(queryList, "Point_Get"), IsTrue) // check if PointGet is used + tk.MustQuery(queryList).Check(testkit.Rows(fmt.Sprintf("%v", y))) + } + + // test table dual + queryRange1 := fmt.Sprintf("select a from trange1 where a=200") + c.Assert(tk.HasPlan(queryRange1, "TableDual"), IsTrue) // check if TableDual is used + tk.MustQuery(queryRange1).Check(testkit.Rows()) + + queryRange2 := fmt.Sprintf("select a from trange2 where a=200") + c.Assert(tk.HasPlan(queryRange2, "TableDual"), IsTrue) // check if TableDual is used + tk.MustQuery(queryRange2).Check(testkit.Rows()) + + queryList := fmt.Sprintf("select a from tlist where a=200") + c.Assert(tk.HasPlan(queryList, "TableDual"), IsTrue) // check if TableDual is used + tk.MustQuery(queryList).Check(testkit.Rows()) +} + func (s *partitionTableSuite) TestPartitionReaderUnderApply(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("use test") diff --git a/executor/testdata/executor_suite_out.json b/executor/testdata/executor_suite_out.json index fd834a5229662..eab098751f2ba 100644 --- a/executor/testdata/executor_suite_out.json +++ b/executor/testdata/executor_suite_out.json @@ -755,9 +755,7 @@ }, { "SQL": "SELECT * FROM t WHERE a = 3000000", - "Plan": [ - "dual" - ], + "Plan": null, "Res": null }, { @@ -769,9 +767,7 @@ }, { "SQL": "SELECT * FROM t WHERE a = 3000001", - "Plan": [ - "dual" - ], + "Plan": null, "Res": null }, { diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index fbc0bf9333a29..28e324b4459c2 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -17,6 +17,7 @@ import ( "bytes" "fmt" math2 "math" + "sort" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" @@ -816,9 +817,7 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt, check bool return nil } pi := tbl.GetPartitionInfo() - if pi != nil && pi.Type != model.PartitionTypeHash { - return nil - } + for _, col := range tbl.Columns { // Do not handle generated columns. if col.IsGenerated() { @@ -847,7 +846,12 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt, check bool var partitionInfo *model.PartitionDefinition var pos int if pi != nil { - partitionInfo, pos = getPartitionInfo(ctx, tbl, pairs) + partitionInfo, pos, isTableDual = getPartitionInfo(ctx, tbl, pairs) + if isTableDual { + p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names) + p.IsTableDual = true + return p + } if partitionInfo == nil { return nil } @@ -1469,20 +1473,77 @@ func buildHandleCols(ctx sessionctx.Context, tbl *model.TableInfo, schema *expre return &IntHandleCols{col: handleCol} } -func getPartitionInfo(ctx sessionctx.Context, tbl *model.TableInfo, pairs []nameValuePair) (*model.PartitionDefinition, int) { - partitionColName := getHashPartitionColumnName(ctx, tbl) - if partitionColName == nil { - return nil, 0 +func getPartitionInfo(ctx sessionctx.Context, tbl *model.TableInfo, pairs []nameValuePair) (*model.PartitionDefinition, int, bool) { + partitionExpr := getPartitionExpr(ctx, tbl) + + pi := tbl.GetPartitionInfo() + if pi == nil { + return nil, 0, false } - pi := tbl.Partition - for i, pair := range pairs { - if partitionColName.Name.L == pair.colName { - val := pair.value.GetInt64() - pos := math.Abs(val % int64(pi.Num)) - return &pi.Definitions[pos], i + + switch pi.Type { + case model.PartitionTypeHash: + expr := partitionExpr.OrigExpr + col, ok := expr.(*ast.ColumnNameExpr) + if !ok { + return nil, 0, false + } + + partitionColName := col.Name + if partitionColName == nil { + return nil, 0, false + } + + for i, pair := range pairs { + if partitionColName.Name.L == pair.colName { + val := pair.value.GetInt64() + pos := math.Abs(val % int64(pi.Num)) + return &pi.Definitions[pos], i, false + } + } + case model.PartitionTypeRange: + // left range columns partition for future development + if len(pi.Columns) == 0 { + if col, ok := partitionExpr.Expr.(*expression.Column); ok { + colInfo := findColNameByColID(tbl.Columns, col) + for i, pair := range pairs { + if colInfo.Name.L == pair.colName { + val := pair.value.GetInt64() // val cannot be Null, we've check this in func getNameValuePairs + unsigned := mysql.HasUnsignedFlag(col.GetType().Flag) + ranges := partitionExpr.ForRangePruning + length := len(ranges.LessThan) + pos := sort.Search(length, func(i int) bool { + return ranges.Compare(i, val, unsigned) > 0 + }) + if pos >= 0 && pos < length { + return &pi.Definitions[pos], i, false + } + return nil, 0, true + } + } + } + } + case model.PartitionTypeList: + // left list columns partition for future development + if partitionExpr.ForListPruning.ColPrunes == nil { + locateExpr := partitionExpr.ForListPruning.LocateExpr + if locateExpr, ok := locateExpr.(*expression.Column); ok { + colInfo := findColNameByColID(tbl.Columns, locateExpr) + for i, pair := range pairs { + if colInfo.Name.L == pair.colName { + val := pair.value.GetInt64() // val cannot be Null, we've check this in func getNameValuePairs + isNull := false + pos := partitionExpr.ForListPruning.LocatePartition(val, isNull) + if pos >= 0 { + return &pi.Definitions[pos], i, false + } + return nil, 0, true + } + } + } } } - return nil, 0 + return nil, 0, false } func findPartitionIdx(idxInfo *model.IndexInfo, pos int, pairs []nameValuePair) int { @@ -1507,6 +1568,21 @@ func getPartitionColumnPos(idx *model.IndexInfo, partitionColName *ast.ColumnNam panic("unique index must include all partition columns") } +func getPartitionExpr(ctx sessionctx.Context, tbl *model.TableInfo) *tables.PartitionExpr { + is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + table, ok := is.TableByID(tbl.ID) + if !ok { + return nil + } + // PartitionExpr don't need columns and names for hash partition. + partitionExpr, err := table.(partitionTable).PartitionExpr() + if err != nil { + return nil + } + + return partitionExpr +} + func getHashPartitionColumnName(ctx sessionctx.Context, tbl *model.TableInfo) *ast.ColumnName { pi := tbl.GetPartitionInfo() if pi == nil { @@ -1532,3 +1608,12 @@ func getHashPartitionColumnName(ctx sessionctx.Context, tbl *model.TableInfo) *a } return col.Name } + +func findColNameByColID(cols []*model.ColumnInfo, col *expression.Column) *model.ColumnInfo { + for _, c := range cols { + if c.ID == col.ID { + return c + } + } + return nil +} diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index 962894b5c35c9..c1085f5fb30fe 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -598,9 +598,7 @@ "SQL": "select * from t1 where a = 100", "Result": null, "Plan": [ - "TableReader 10.00 root partition:dual data:Selection", - "└─Selection 10.00 cop[tikv] eq(test_partition.t1.a, 100)", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + "TableDual 0.00 root rows:0" ] }, { diff --git a/table/tables/partition.go b/table/tables/partition.go index 7f27785931e28..d83373e9601cb 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -998,7 +998,7 @@ func (t *partitionedTable) locateRangePartition(ctx sessionctx.Context, pi *mode if isNull { return true } - return ranges.compare(i, ret, unsigned) > 0 + return ranges.Compare(i, ret, unsigned) > 0 }) if isNull { pos = 0 @@ -1243,7 +1243,8 @@ func compareUnsigned(v1, v2 int64) int { return -1 } -func (lt *ForRangePruning) compare(ith int, v int64, unsigned bool) int { +// Compare is to be used in the binary search to locate partition +func (lt *ForRangePruning) Compare(ith int, v int64, unsigned bool) int { if ith == len(lt.LessThan)-1 { if lt.MaxValue { return 1 diff --git a/tools/check/go.mod b/tools/check/go.mod index 171d04af71830..8754267765791 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -11,6 +11,8 @@ require ( github.com/mgechev/revive v0.0.0-20181210140514-b4cc152955fb github.com/nicksnyder/go-i18n v1.10.0 // indirect github.com/pelletier/go-toml v1.2.0 // indirect + github.com/pingcap/errors v0.11.5-0.20210513014640-40f9a1999b3b // indirect + github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce // indirect github.com/securego/gosec v0.0.0-20181211171558-12400f9a1ca7 gopkg.in/alecthomas/gometalinter.v2 v2.0.12 // indirect gopkg.in/alecthomas/gometalinter.v3 v3.0.0 // indirect diff --git a/tools/check/go.sum b/tools/check/go.sum index b4f8676c60fe9..7558a1677d777 100644 --- a/tools/check/go.sum +++ b/tools/check/go.sum @@ -5,6 +5,7 @@ github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf h1:qet1QNfXsQxTZq github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03 h1:0wUHjDfbCAROEAZ96zAJGwcNMkPIheFaIjtQyv3QqfM= github.com/chzchzchz/goword v0.0.0-20170907005317-a9744cb52b03/go.mod h1:uFE9hX+zXEwvyUThZ4gDb9vkAwc5DoHUnRSEpH0VrOs= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/dnephin/govet v0.0.0-20171012192244-4a96d43e39d3/go.mod h1:pPTX0MEEoAnfbrAGFj4nSVNhl6YbugRj6eardUZdtGo= github.com/fatih/color v1.7.0 h1:DkWD4oS2D8LGGgTQ6IvwJJXSL5Vp2ffcQg58nFV38Ys= @@ -44,17 +45,33 @@ github.com/onsi/ginkgo v0.0.0-20170829012221-11459a886d9c/go.mod h1:lLunBs/Ym6LB github.com/onsi/gomega v0.0.0-20170829124025-dcabb60a477c/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA= github.com/pelletier/go-toml v1.2.0 h1:T5zMGML61Wp+FlcbWjRDT7yAxhJNAiPPLOFECq181zc= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= +github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= +github.com/pingcap/errors v0.11.4 h1:lFuQV/oaUMGcD2tqt+01ROSmJs75VG1ToEOkZIZ4nE4= +github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= +github.com/pingcap/errors v0.11.5-0.20210513014640-40f9a1999b3b h1:j9MP8ma4e75tckq11+n4EhB2xq0xwYNoxL8w9JTZRhs= +github.com/pingcap/errors v0.11.5-0.20210513014640-40f9a1999b3b/go.mod h1:X2r9ueLEUZgtx2cIogM0v4Zj5uvvzhuuiu7Pn8HzMPg= +github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce h1:Y1kCxlCtlPTMtVcOkjUcuQKh+YrluSo7+7YMCQSzy30= +github.com/pingcap/failpoint v0.0.0-20200702092429-9f69995143ce/go.mod h1:w4PEZ5y16LeofeeGwdgZB4ddv9bLyDuIX+ljstgKZyk= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/ryanuber/go-glob v0.0.0-20170128012129-256dc444b735 h1:7YvPJVmEeFHR1Tj9sZEYsmarJEQfMVYpd/Vyy/A8dqE= github.com/ryanuber/go-glob v0.0.0-20170128012129-256dc444b735/go.mod h1:807d1WSdnB0XRJzKNil9Om6lcp/3a0v4qIHxIXzX/Yc= github.com/securego/gosec v0.0.0-20181211171558-12400f9a1ca7 h1:Ca7U7/rZ+caxjW2na7wbmgmaPsoSCIlpc6sm0aWtFg0= github.com/securego/gosec v0.0.0-20181211171558-12400f9a1ca7/go.mod h1:m3KbCTwh9vLhm6AKBjE+ALesKilKcQHezI1uVOti0Ks= +github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= +github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= +go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/mod v0.1.1-0.20191105210325-c90efee705ee/go.mod h1:QqPTAvyqsEbceGzBzNggFXnrqF1CaUcvgkdR5Ot7KZg= golang.org/x/net v0.0.0-20170915142106-8351a756f30f/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -72,8 +89,10 @@ golang.org/x/tools v0.0.0-20180911133044-677d2ff680c1/go.mod h1:n7NCudcB/nEzxVGm golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563 h1:NIou6eNFigscvKJmsbyez16S2cIS6idossORlFtSt2E= golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/tools v0.0.0-20190925020647-22afafe3322a h1:3GxqzBPBt1O2dIiPnzldQ5d25CAMWJFBZTpqxLPfjs8= golang.org/x/tools v0.0.0-20190925020647-22afafe3322a/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20191029041327-9cc4af7d6b2c/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= golang.org/x/tools v0.0.0-20200225230052-807dcd883420 h1:4RJNOV+2rLxMEfr6QIpC7GEv9MjD6ApGXTCLrNF9+eA= golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -90,4 +109,4 @@ gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8 gopkg.in/yaml.v2 v2.0.0-20170812160011-eb3733d160e7/go.mod h1:JAlM8MvJe8wmxCU4Bli9HhUf9+ttbYbLASfIpnQbh74= gopkg.in/yaml.v2 v2.2.2 h1:ZCJp+EgiOT7lHqUV2J862kp8Qj64Jo6az82+3Td9dZw= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= -honnef.co/go/tools v0.0.0-20180920025451-e3ad64cb4ed3/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= \ No newline at end of file +honnef.co/go/tools v0.0.0-20180920025451-e3ad64cb4ed3/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= From 49f614e05f400c094a63a62de494f1bb640ce249 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Sat, 22 May 2021 02:58:21 +0800 Subject: [PATCH 140/343] executor: add a test case for 'tidb_constraint_check_in_place' on temporary table (#24784) --- executor/write_test.go | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/executor/write_test.go b/executor/write_test.go index cf7a51985a450..e9111154cfc87 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -2860,6 +2860,37 @@ func (s *testSuite7) TestDeferConstraintCheckForInsert(c *C) { tk.MustExec("insert into t values (1, 3)") _, err = tk.Exec("commit") c.Assert(err, NotNil) + + // Cover the temporary table. + for val := range []int{0, 1} { + tk.MustExec("set tidb_constraint_check_in_place = ?", val) + + tk.MustExec("drop table t") + tk.MustExec("create global temporary table t (a int primary key, b int) on commit delete rows") + tk.MustExec("begin") + tk.MustExec("insert into t values (1, 1)") + _, err = tk.Exec(`insert into t values (1, 3)`) + c.Assert(err, NotNil) + tk.MustExec("insert into t values (2, 2)") + _, err = tk.Exec("update t set a = a + 1 where a = 1") + c.Assert(err, NotNil) + _, err = tk.Exec("insert into t values (1, 3) on duplicated key update a = a + 1") + c.Assert(err, NotNil) + tk.MustExec("commit") + + tk.MustExec("drop table t") + tk.MustExec("create global temporary table t (a int, b int unique) on commit delete rows") + tk.MustExec("begin") + tk.MustExec("insert into t values (1, 1)") + _, err = tk.Exec(`insert into t values (3, 1)`) + c.Assert(err, NotNil) + tk.MustExec("insert into t values (2, 2)") + _, err = tk.Exec("update t set b = b + 1 where a = 1") + c.Assert(err, NotNil) + _, err = tk.Exec("insert into t values (3, 1) on duplicated key update b = b + 1") + c.Assert(err, NotNil) + tk.MustExec("commit") + } } func (s *testSuite7) TestPessimisticDeleteYourWrites(c *C) { From 19e100c27a37601d674df625034ebddb5196a299 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Sun, 23 May 2021 09:31:32 -0600 Subject: [PATCH 141/343] expression: add integration test for JDBC's SHOW VARIABLES (#24834) --- expression/integration_test.go | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/expression/integration_test.go b/expression/integration_test.go index c6baa9bcea350..22e401bab559f 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9576,3 +9576,20 @@ func (s *testIntegrationSuite) TestControlFunctionWithEnumOrSet(c *C) { tk.MustQuery("select elt(1,s) = 4 from s").Check(testkit.Rows("1")) tk.MustQuery("select s from s where elt(1,s)").Check(testkit.Rows("a")) } + +func (s *testIntegrationSuite) TestComplexShowVariables(c *C) { + // This is an example SHOW VARIABLES from mysql-connector-java-5.1.34 + // It returns 19 rows in MySQL 5.7 (the language sysvar no longer exists in 5.6+) + // and 16 rows in MySQL 8.0 (the aliases for tx_isolation is removed, along with query cache) + // In the event that we hide noop sysvars in future, we must keep these variables. + tk := testkit.NewTestKit(c, s.store) + c.Assert(tk.MustQuery(`SHOW VARIABLES WHERE Variable_name ='language' OR Variable_name = 'net_write_timeout' OR Variable_name = 'interactive_timeout' +OR Variable_name = 'wait_timeout' OR Variable_name = 'character_set_client' OR Variable_name = 'character_set_connection' +OR Variable_name = 'character_set' OR Variable_name = 'character_set_server' OR Variable_name = 'tx_isolation' +OR Variable_name = 'transaction_isolation' OR Variable_name = 'character_set_results' OR Variable_name = 'timezone' +OR Variable_name = 'time_zone' OR Variable_name = 'system_time_zone' +OR Variable_name = 'lower_case_table_names' OR Variable_name = 'max_allowed_packet' OR Variable_name = 'net_buffer_length' +OR Variable_name = 'sql_mode' OR Variable_name = 'query_cache_type' OR Variable_name = 'query_cache_size' +OR Variable_name = 'license' OR Variable_name = 'init_connect'`).Rows(), HasLen, 19) + +} From 0191281b92aa8ac436dbe9e838b7a228d3e912da Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Sun, 23 May 2021 21:55:33 -0600 Subject: [PATCH 142/343] session: improve collation_server handling (#24835) --- session/session.go | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/session/session.go b/session/session.go index 2874330a02fd3..151f0be8704c6 100644 --- a/session/session.go +++ b/session/session.go @@ -2552,7 +2552,6 @@ var builtinGlobalVariable = []string{ variable.NetBufferLength, variable.QueryCacheType, variable.QueryCacheSize, - variable.CharacterSetServer, variable.AutoIncrementIncrement, variable.AutoIncrementOffset, variable.CollationServer, @@ -2691,9 +2690,7 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { continue // skip variables that are not loaded. } } - // `collation_server` is related to `character_set_server`, set `character_set_server` will also set `collation_server`. - // We have to make sure we set the `collation_server` with right value. - if _, ok := vars.GetSystemVar(varName); !ok || varName == variable.CollationServer { + if _, ok := vars.GetSystemVar(varName); !ok { err = vars.SetSystemVarWithRelaxedValidation(varName, varVal) if err != nil { return err From 375e438be4148ac4d701f346375f30e72b0bfac1 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Sun, 23 May 2021 22:29:33 -0600 Subject: [PATCH 143/343] session, variable: invert builtinGlobalVariable from allow to deny list (#24836) --- session/session.go | 129 +---------------------- sessionctx/variable/sysvar.go | 159 +++++++++++++++++------------ sessionctx/variable/sysvar_test.go | 14 +++ 3 files changed, 109 insertions(+), 193 deletions(-) diff --git a/session/session.go b/session/session.go index 151f0be8704c6..fcecd49a6353c 100644 --- a/session/session.go +++ b/session/session.go @@ -2535,130 +2535,6 @@ func finishBootstrap(store kv.Storage) { const quoteCommaQuote = "', '" -var builtinGlobalVariable = []string{ - variable.AutoCommit, - variable.SQLModeVar, - variable.MaxAllowedPacket, - variable.TimeZone, - variable.BlockEncryptionMode, - variable.WaitTimeout, - variable.InteractiveTimeout, - variable.MaxPreparedStmtCount, - variable.InitConnect, - variable.TxnIsolation, - variable.TxReadOnly, - variable.TransactionIsolation, - variable.TransactionReadOnly, - variable.NetBufferLength, - variable.QueryCacheType, - variable.QueryCacheSize, - variable.AutoIncrementIncrement, - variable.AutoIncrementOffset, - variable.CollationServer, - variable.NetWriteTimeout, - variable.MaxExecutionTime, - variable.InnodbLockWaitTimeout, - variable.WindowingUseHighPrecision, - variable.SQLSelectLimit, - variable.DefaultWeekFormat, - - /* TiDB specific global variables: */ - variable.TiDBSkipASCIICheck, - variable.TiDBSkipUTF8Check, - variable.TiDBIndexJoinBatchSize, - variable.TiDBIndexLookupSize, - variable.TiDBIndexLookupConcurrency, - variable.TiDBIndexLookupJoinConcurrency, - variable.TiDBIndexSerialScanConcurrency, - variable.TiDBHashJoinConcurrency, - variable.TiDBProjectionConcurrency, - variable.TiDBHashAggPartialConcurrency, - variable.TiDBHashAggFinalConcurrency, - variable.TiDBWindowConcurrency, - variable.TiDBMergeJoinConcurrency, - variable.TiDBStreamAggConcurrency, - variable.TiDBExecutorConcurrency, - variable.TiDBBackoffLockFast, - variable.TiDBBackOffWeight, - variable.TiDBConstraintCheckInPlace, - variable.TiDBDDLReorgWorkerCount, - variable.TiDBDDLReorgBatchSize, - variable.TiDBDDLErrorCountLimit, - variable.TiDBOptInSubqToJoinAndAgg, - variable.TiDBOptPreferRangeScan, - variable.TiDBOptCorrelationThreshold, - variable.TiDBOptCorrelationExpFactor, - variable.TiDBOptCPUFactor, - variable.TiDBOptCopCPUFactor, - variable.TiDBOptNetworkFactor, - variable.TiDBOptScanFactor, - variable.TiDBOptDescScanFactor, - variable.TiDBOptMemoryFactor, - variable.TiDBOptDiskFactor, - variable.TiDBOptConcurrencyFactor, - variable.TiDBDistSQLScanConcurrency, - variable.TiDBInitChunkSize, - variable.TiDBMaxChunkSize, - variable.TiDBEnableCascadesPlanner, - variable.TiDBRetryLimit, - variable.TiDBDisableTxnAutoRetry, - variable.TiDBEnableWindowFunction, - variable.TiDBEnableStrictDoubleTypeCheck, - variable.TiDBEnableTablePartition, - variable.TiDBEnableVectorizedExpression, - variable.TiDBEnableFastAnalyze, - variable.TiDBExpensiveQueryTimeThreshold, - variable.TiDBEnableNoopFuncs, - variable.TiDBEnableIndexMerge, - variable.TiDBTxnMode, - variable.TiDBAllowBatchCop, - variable.TiDBAllowMPPExecution, - variable.TiDBOptBCJ, - variable.TiDBBCJThresholdSize, - variable.TiDBBCJThresholdCount, - variable.TiDBRowFormatVersion, - variable.TiDBEnableStmtSummary, - variable.TiDBStmtSummaryInternalQuery, - variable.TiDBStmtSummaryRefreshInterval, - variable.TiDBStmtSummaryHistorySize, - variable.TiDBStmtSummaryMaxStmtCount, - variable.TiDBStmtSummaryMaxSQLLength, - variable.TiDBMaxDeltaSchemaCount, - variable.TiDBCapturePlanBaseline, - variable.TiDBUsePlanBaselines, - variable.TiDBEvolvePlanBaselines, - variable.TiDBEnableExtendedStats, - variable.TiDBIsolationReadEngines, - variable.TiDBStoreLimit, - variable.TiDBAllowAutoRandExplicitInsert, - variable.TiDBEnableClusteredIndex, - variable.TiDBPartitionPruneMode, - variable.TiDBRedactLog, - variable.TiDBEnableTelemetry, - variable.TiDBShardAllocateStep, - variable.TiDBEnableChangeColumnType, - variable.TiDBEnableChangeMultiSchema, - variable.TiDBEnablePointGetCache, - variable.TiDBEnableAlterPlacement, - variable.TiDBEnableAmendPessimisticTxn, - variable.TiDBMemQuotaApplyCache, - variable.TiDBEnableParallelApply, - variable.TiDBMemoryUsageAlarmRatio, - variable.TiDBEnableRateLimitAction, - variable.TiDBEnableAsyncCommit, - variable.TiDBEnable1PC, - variable.TiDBGuaranteeLinearizability, - variable.TiDBAnalyzeVersion, - variable.TiDBEnableIndexMergeJoin, - variable.TiDBTrackAggregateMemoryUsage, - variable.TiDBMultiStatementMode, - variable.TiDBEnableExchangePartition, - variable.TiDBAllowFallbackToTiKV, - variable.TiDBEnableDynamicPrivileges, - variable.CTEMaxRecursionDepth, - variable.TiDBDMLBatchSize, -} - // loadCommonGlobalVariablesIfNeeded loads and applies commonly used global variables for the session. func (s *session) loadCommonGlobalVariablesIfNeeded() error { vars := s.sessionVars @@ -2678,7 +2554,10 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { if err != nil { return err } - for _, varName := range builtinGlobalVariable { + for varName, sv := range variable.GetSysVars() { + if sv.SkipInit() { + continue + } // The item should be in the sessionCache, but due to a strange current behavior there are some Global-only // vars that are in builtinGlobalVariable. For compatibility we need to fall back to the Global cache on these items. // TODO: don't load these globals into the session! diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 1df9e3c0f582c..06321e831a91a 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -119,6 +119,10 @@ type SysVar struct { // Aliases is a list of sysvars that should also be updated when this sysvar is updated. // Updating aliases calls the SET function of the aliases, but does not update their aliases (preventing SET recursion) Aliases []string + // skipInit defines if the sysvar should be loaded into the session on init. + // This is only important to set for sysvars that include session scope, + // since global scoped sysvars are not-applicable. + skipInit bool } // SetSessionFromHook calls the SetSession func if it exists. @@ -471,6 +475,24 @@ func (sv *SysVar) GetNativeValType(val string) (types.Datum, byte, uint) { return types.NewStringDatum(val), mysql.TypeVarString, 0 } +// SkipInit returns true if when a new session is created we should "skip" copying +// an initial value to it (and call the SetSession func if it exists) +func (sv *SysVar) SkipInit() bool { + if sv.skipInit { + return true + } + // These a special "Global-only" sysvars that for backward compatibility + // are currently cached in the session. Please don't add to this list. + switch sv.Name { + case TiDBEnableChangeMultiSchema, TiDBDDLReorgBatchSize, TiDBEnableAlterPlacement, + TiDBMaxDeltaSchemaCount, InitConnect, MaxPreparedStmtCount, + TiDBDDLReorgWorkerCount, TiDBDDLErrorCountLimit, TiDBRowFormatVersion, + TiDBEnableTelemetry, TiDBEnablePointGetCache: + return false + } + return !sv.HasSessionScope() +} + var sysVars map[string]*SysVar var sysVarsLock sync.RWMutex @@ -537,6 +559,7 @@ func init() { RegisterSysVar(v) } for _, v := range noopSysVars { + v.skipInit = true // by definition a noop can skipInit RegisterSysVar(v) } } @@ -584,7 +607,7 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: SQLLogBin, Value: On, Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: SQLLogBin, Value: On, Type: TypeBool, skipInit: true}, {Scope: ScopeGlobal | ScopeSession, Name: TimeZone, Value: "SYSTEM", IsHintUpdatable: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if strings.EqualFold(normalizedValue, "SYSTEM") { return "SYSTEM", nil @@ -600,7 +623,7 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeNone, Name: SystemTimeZone, Value: "CST"}, - {Scope: ScopeGlobal | ScopeSession, Name: ForeignKeyChecks, Value: Off, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: ForeignKeyChecks, Value: Off, Type: TypeBool, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if TiDBOptOn(normalizedValue) { // TiDB does not yet support foreign keys. // Return the original value in the warning, so that users are not confused. @@ -612,11 +635,11 @@ var defaultSysVars = []*SysVar{ return normalizedValue, ErrWrongValueForVar.GenWithStackByArgs(ForeignKeyChecks, originalValue) }}, {Scope: ScopeNone, Name: Hostname, Value: DefHostname}, - {Scope: ScopeSession, Name: Timestamp, Value: ""}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetFilesystem, Value: "binary", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeSession, Name: Timestamp, Value: "", skipInit: true}, + {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetFilesystem, Value: "binary", skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCharacterSet(normalizedValue, CharacterSetFilesystem) }}, - {Scope: ScopeGlobal | ScopeSession, Name: CollationDatabase, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: CollationDatabase, Value: mysql.DefaultCollationName, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCollation(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { if coll, err := collate.GetCollationByName(val); err == nil { @@ -634,13 +657,13 @@ var defaultSysVars = []*SysVar{ s.AutoIncrementOffset = tidbOptPositiveInt32(val, DefAutoIncrementOffset) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetClient, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetClient, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCharacterSet(normalizedValue, CharacterSetClient) }}, {Scope: ScopeNone, Name: Port, Value: "4000", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint16}, {Scope: ScopeNone, Name: LowerCaseTableNames, Value: "2"}, {Scope: ScopeNone, Name: LogBin, Value: Off, Type: TypeBool}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetResults, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetResults, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { if normalizedValue == "" { return normalizedValue, nil } @@ -656,7 +679,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TransactionIsolation, Value: "REPEATABLE-READ", Type: TypeEnum, Aliases: []string{TxnIsolation}, PossibleValues: []string{"READ-UNCOMMITTED", "READ-COMMITTED", "REPEATABLE-READ", "SERIALIZABLE"}, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }}, - {Scope: ScopeGlobal | ScopeSession, Name: CollationConnection, Value: mysql.DefaultCollationName, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: CollationConnection, Value: mysql.DefaultCollationName, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCollation(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { if coll, err := collate.GetCollationByName(val); err == nil { @@ -673,7 +696,7 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: CharsetDatabase, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: CharsetDatabase, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCharacterSet(normalizedValue, CharsetDatabase) }, SetSession: func(s *SessionVars, val string) error { if _, coll, err := charset.GetCharsetInfo(val); err == nil { @@ -690,7 +713,7 @@ var defaultSysVars = []*SysVar{ s.LockWaitTimeout = lockWaitSec * 1000 return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: GroupConcatMaxLen, Value: "1024", AutoConvertOutOfRange: true, IsHintUpdatable: true, Type: TypeUnsigned, MinValue: 4, MaxValue: math.MaxUint64, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: GroupConcatMaxLen, Value: "1024", AutoConvertOutOfRange: true, IsHintUpdatable: true, skipInit: true, Type: TypeUnsigned, MinValue: 4, MaxValue: math.MaxUint64, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { // https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_group_concat_max_len // Minimum Value 4 // Maximum Value (64-bit platforms) 18446744073709551615 @@ -706,7 +729,7 @@ var defaultSysVars = []*SysVar{ return normalizedValue, nil }}, {Scope: ScopeNone, Name: Socket, Value: "/tmp/myssock"}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetConnection, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetConnection, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCharacterSet(normalizedValue, CharacterSetConnection) }, SetSession: func(s *SessionVars, val string) error { if _, coll, err := charset.GetCharsetInfo(val); err == nil { @@ -714,7 +737,7 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetServer, Value: mysql.DefaultCharset, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: CharacterSetServer, Value: mysql.DefaultCharset, skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkCharacterSet(normalizedValue, CharacterSetServer) }, SetSession: func(s *SessionVars, val string) error { if _, coll, err := charset.GetCharsetInfo(val); err == nil { @@ -723,15 +746,15 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: MaxAllowedPacket, Value: "67108864", Type: TypeUnsigned, MinValue: 1024, MaxValue: MaxOfMaxAllowedPacket, AutoConvertOutOfRange: true}, - {Scope: ScopeSession, Name: WarningCount, Value: "0", ReadOnly: true}, - {Scope: ScopeSession, Name: ErrorCount, Value: "0", ReadOnly: true}, + {Scope: ScopeSession, Name: WarningCount, Value: "0", ReadOnly: true, skipInit: true}, + {Scope: ScopeSession, Name: ErrorCount, Value: "0", ReadOnly: true, skipInit: true}, {Scope: ScopeGlobal | ScopeSession, Name: WindowingUseHighPrecision, Value: On, Type: TypeBool, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error { s.WindowingUseHighPrecision = TiDBOptOn(val) return nil }}, {Scope: ScopeNone, Name: "license", Value: "Apache License 2.0"}, {Scope: ScopeGlobal | ScopeSession, Name: BlockEncryptionMode, Value: "aes-128-ecb"}, - {Scope: ScopeSession, Name: "last_insert_id", Value: ""}, + {Scope: ScopeSession, Name: "last_insert_id", Value: "", skipInit: true}, {Scope: ScopeNone, Name: "have_ssl", Value: "DISABLED"}, {Scope: ScopeNone, Name: "have_openssl", Value: "DISABLED"}, {Scope: ScopeNone, Name: "ssl_ca", Value: ""}, @@ -740,7 +763,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: InitConnect, Value: ""}, /* TiDB specific variables */ - {Scope: ScopeSession, Name: TiDBTxnScope, Value: func() string { + {Scope: ScopeSession, Name: TiDBTxnScope, skipInit: true, Value: func() string { if isGlobal, _ := config.GetTxnScopeFromConfig(); isGlobal { return kv.GlobalTxnScope } @@ -768,14 +791,14 @@ var defaultSysVars = []*SysVar{ s.BroadcastJoinThresholdSize = tidbOptInt64(val, DefBroadcastJoinThresholdSize) return nil }}, - {Scope: ScopeSession, Name: TiDBSnapshot, Value: "", SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBSnapshot, Value: "", skipInit: true, SetSession: func(s *SessionVars, val string) error { err := setSnapshotTS(s, val) if err != nil { return err } return nil }}, - {Scope: ScopeSession, Name: TiDBOptAggPushDown, Value: BoolToOnOff(DefOptAggPushDown), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBOptAggPushDown, Value: BoolToOnOff(DefOptAggPushDown), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.AllowAggPushDown = TiDBOptOn(val) return nil }}, @@ -788,19 +811,19 @@ var defaultSysVars = []*SysVar{ s.AllowBCJ = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBOptDistinctAggPushDown, Value: BoolToOnOff(config.GetGlobalConfig().Performance.DistinctAggPushDown), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBOptDistinctAggPushDown, Value: BoolToOnOff(config.GetGlobalConfig().Performance.DistinctAggPushDown), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.AllowDistinctAggPushDown = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBOptWriteRowID, Value: BoolToOnOff(DefOptWriteRowID), SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBOptWriteRowID, Value: BoolToOnOff(DefOptWriteRowID), skipInit: true, SetSession: func(s *SessionVars, val string) error { s.AllowWriteRowID = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildStatsConcurrency, Value: strconv.Itoa(DefBuildStatsConcurrency)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildStatsConcurrency, skipInit: true, Value: strconv.Itoa(DefBuildStatsConcurrency)}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeRatio, Value: strconv.FormatFloat(DefAutoAnalyzeRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeStartTime, Value: DefAutoAnalyzeStartTime, Type: TypeTime}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeEndTime, Value: DefAutoAnalyzeEndTime, Type: TypeTime}, - {Scope: ScopeSession, Name: TiDBChecksumTableConcurrency, Value: strconv.Itoa(DefChecksumTableConcurrency)}, + {Scope: ScopeSession, Name: TiDBChecksumTableConcurrency, skipInit: true, Value: strconv.Itoa(DefChecksumTableConcurrency)}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBExecutorConcurrency, Value: strconv.Itoa(DefExecutorConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { s.ExecutorConcurrency = tidbOptPositiveInt32(val, DefExecutorConcurrency) return nil @@ -829,7 +852,7 @@ var defaultSysVars = []*SysVar{ s.CPUFactor = tidbOptFloat64(val, DefOptCPUFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptTiFlashConcurrencyFactor, Value: strconv.FormatFloat(DefOptTiFlashConcurrencyFactor, 'f', -1, 64), skipInit: true, Type: TypeFloat, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { s.CopTiFlashConcurrencyFactor = tidbOptFloat64(val, DefOptTiFlashConcurrencyFactor) return nil }}, @@ -849,7 +872,7 @@ var defaultSysVars = []*SysVar{ s.DescScanFactor = tidbOptFloat64(val, DefOptDescScanFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptSeekFactor, Value: strconv.FormatFloat(DefOptSeekFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptSeekFactor, Value: strconv.FormatFloat(DefOptSeekFactor, 'f', -1, 64), skipInit: true, Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { s.SeekFactor = tidbOptFloat64(val, DefOptSeekFactor) return nil }}, @@ -899,15 +922,15 @@ var defaultSysVars = []*SysVar{ s.SkipASCIICheck = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBBatchInsert, Value: BoolToOnOff(DefBatchInsert), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBBatchInsert, Value: BoolToOnOff(DefBatchInsert), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.BatchInsert = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBBatchDelete, Value: BoolToOnOff(DefBatchDelete), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBBatchDelete, Value: BoolToOnOff(DefBatchDelete), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.BatchDelete = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBBatchCommit, Value: BoolToOnOff(DefBatchCommit), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBBatchCommit, Value: BoolToOnOff(DefBatchCommit), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.BatchCommit = TiDBOptOn(val) return nil }}, @@ -915,9 +938,9 @@ var defaultSysVars = []*SysVar{ s.DMLBatchSize = int(tidbOptInt64(val, DefDMLBatchSize)) return nil }}, - {Scope: ScopeSession, Name: TiDBCurrentTS, Value: strconv.Itoa(DefCurretTS), ReadOnly: true}, - {Scope: ScopeSession, Name: TiDBLastTxnInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true}, - {Scope: ScopeSession, Name: TiDBLastQueryInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true}, + {Scope: ScopeSession, Name: TiDBCurrentTS, Value: strconv.Itoa(DefCurretTS), skipInit: true, ReadOnly: true}, + {Scope: ScopeSession, Name: TiDBLastTxnInfo, Value: strconv.Itoa(DefCurretTS), skipInit: true, ReadOnly: true}, + {Scope: ScopeSession, Name: TiDBLastQueryInfo, Value: strconv.Itoa(DefCurretTS), skipInit: true, ReadOnly: true}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxChunkSize, Value: strconv.Itoa(DefMaxChunkSize), Type: TypeUnsigned, MinValue: maxChunkSizeLowerBound, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { s.MaxChunkSize = tidbOptPositiveInt32(val, DefMaxChunkSize) return nil @@ -943,61 +966,61 @@ var defaultSysVars = []*SysVar{ s.SetEnableIndexMerge(TiDBOptOn(val)) return nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaQuery, Value: strconv.FormatInt(config.GetGlobalConfig().MemQuotaQuery, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TIDBMemQuotaQuery, Value: strconv.FormatInt(config.GetGlobalConfig().MemQuotaQuery, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaQuery = tidbOptInt64(val, config.GetGlobalConfig().MemQuotaQuery) return nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaHashJoin, Value: strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TIDBMemQuotaHashJoin, Value: strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaHashJoin = tidbOptInt64(val, DefTiDBMemQuotaHashJoin) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TIDBMemQuotaHashJoin, TIDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaMergeJoin, Value: strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TIDBMemQuotaMergeJoin, Value: strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaMergeJoin = tidbOptInt64(val, DefTiDBMemQuotaMergeJoin) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TIDBMemQuotaMergeJoin, TIDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaSort, Value: strconv.FormatInt(DefTiDBMemQuotaSort, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TIDBMemQuotaSort, Value: strconv.FormatInt(DefTiDBMemQuotaSort, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaSort = tidbOptInt64(val, DefTiDBMemQuotaSort) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TIDBMemQuotaSort, TIDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaTopn, Value: strconv.FormatInt(DefTiDBMemQuotaTopn, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TIDBMemQuotaTopn, Value: strconv.FormatInt(DefTiDBMemQuotaTopn, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaTopn = tidbOptInt64(val, DefTiDBMemQuotaTopn) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TIDBMemQuotaTopn, TIDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupReader, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupReader, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupReader, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupReader, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaIndexLookupReader = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupReader) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TIDBMemQuotaIndexLookupReader, TIDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupJoin, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupJoin, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaIndexLookupJoin = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupJoin) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TIDBMemQuotaIndexLookupJoin, TIDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TiDBEnableStreaming, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBEnableStreaming, Value: Off, Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.EnableStreaming = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBEnableChunkRPC, Value: On, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBEnableChunkRPC, Value: On, Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.EnableChunkRPC = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TxnIsolationOneShot, Value: "", Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeSession, Name: TxnIsolationOneShot, Value: "", skipInit: true, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { return checkIsolationLevel(vars, normalizedValue, originalValue, scope) }, SetSession: func(s *SessionVars, val string) error { s.txnIsolationLevelOneShot.state = oneShotSet @@ -1008,7 +1031,7 @@ var defaultSysVars = []*SysVar{ s.EnableTablePartition = val return nil }}, - {Scope: ScopeSession, Name: TiDBEnableListTablePartition, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBEnableListTablePartition, Value: Off, Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.EnableListTablePartition = TiDBOptOn(val) return nil }}, @@ -1104,7 +1127,7 @@ var defaultSysVars = []*SysVar{ SetDDLReorgRowFormat(tidbOptInt64(val, DefTiDBRowFormatV2)) return nil }}, - {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), skipInit: true, Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { s.OptimizerSelectivityLevel = tidbOptPositiveInt32(val, DefTiDBOptimizerSelectivityLevel) return nil }}, @@ -1124,7 +1147,7 @@ var defaultSysVars = []*SysVar{ s.EnableFastAnalyze = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipIsolationLevelCheck, Value: BoolToOnOff(DefTiDBSkipIsolationLevelCheck), Type: TypeBool}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBSkipIsolationLevelCheck, skipInit: true, Value: BoolToOnOff(DefTiDBSkipIsolationLevelCheck), Type: TypeBool}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableRateLimitAction, Value: BoolToOnOff(DefTiDBEnableRateLimitAction), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnabledRateLimitAction = TiDBOptOn(val) return nil @@ -1167,19 +1190,19 @@ var defaultSysVars = []*SysVar{ s.EnableDynamicPrivileges = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBGeneralLog, Value: BoolToOnOff(DefTiDBGeneralLog), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBGeneralLog, Value: BoolToOnOff(DefTiDBGeneralLog), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { ProcessGeneralLog.Store(TiDBOptOn(val)) return nil }}, - {Scope: ScopeSession, Name: TiDBPProfSQLCPU, Value: strconv.Itoa(DefTiDBPProfSQLCPU), Type: TypeInt, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBPProfSQLCPU, Value: strconv.Itoa(DefTiDBPProfSQLCPU), Type: TypeInt, skipInit: true, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { EnablePProfSQLCPU.Store(uint32(tidbOptPositiveInt32(val, DefTiDBPProfSQLCPU)) > 0) return nil }}, - {Scope: ScopeSession, Name: TiDBDDLSlowOprThreshold, Value: strconv.Itoa(DefTiDBDDLSlowOprThreshold), SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBDDLSlowOprThreshold, Value: strconv.Itoa(DefTiDBDDLSlowOprThreshold), skipInit: true, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint32(&DDLSlowOprThreshold, uint32(tidbOptPositiveInt32(val, DefTiDBDDLSlowOprThreshold))) return nil }}, - {Scope: ScopeSession, Name: TiDBConfig, Value: "", ReadOnly: true}, + {Scope: ScopeSession, Name: TiDBConfig, Value: "", ReadOnly: true, skipInit: true}, {Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: uint64(maxDDLReorgWorkerCount), SetSession: func(s *SessionVars, val string) error { SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount))) return nil @@ -1192,7 +1215,7 @@ var defaultSysVars = []*SysVar{ SetDDLErrorCountLimit(tidbOptInt64(val, DefTiDBDDLErrorCountLimit)) return nil }}, - {Scope: ScopeSession, Name: TiDBDDLReorgPriority, Value: "PRIORITY_LOW", SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBDDLReorgPriority, Value: "PRIORITY_LOW", skipInit: true, SetSession: func(s *SessionVars, val string) error { s.setDDLReorgPriority(val) return nil }}, @@ -1217,32 +1240,32 @@ var defaultSysVars = []*SysVar{ s.EnableAlterPlacement = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBForcePriority, Value: mysql.Priority2Str[DefTiDBForcePriority], SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBForcePriority, skipInit: true, Value: mysql.Priority2Str[DefTiDBForcePriority], SetSession: func(s *SessionVars, val string) error { atomic.StoreInt32(&ForcePriority, int32(mysql.Str2Priority(val))) return nil }}, - {Scope: ScopeSession, Name: TiDBEnableRadixJoin, Value: BoolToOnOff(DefTiDBUseRadixJoin), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBEnableRadixJoin, Value: BoolToOnOff(DefTiDBUseRadixJoin), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.EnableRadixJoin = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptJoinReorderThreshold, Value: strconv.Itoa(DefTiDBOptJoinReorderThreshold), Type: TypeUnsigned, MinValue: 0, MaxValue: 63, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptJoinReorderThreshold, Value: strconv.Itoa(DefTiDBOptJoinReorderThreshold), skipInit: true, Type: TypeUnsigned, MinValue: 0, MaxValue: 63, SetSession: func(s *SessionVars, val string) error { s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) return nil }}, - {Scope: ScopeSession, Name: TiDBSlowQueryFile, Value: "", SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBSlowQueryFile, Value: "", skipInit: true, SetSession: func(s *SessionVars, val string) error { s.SlowQueryFile = val return nil }}, {Scope: ScopeGlobal, Name: TiDBScatterRegion, Value: BoolToOnOff(DefTiDBScatterRegion), Type: TypeBool}, - {Scope: ScopeSession, Name: TiDBWaitSplitRegionFinish, Value: BoolToOnOff(DefTiDBWaitSplitRegionFinish), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBWaitSplitRegionFinish, Value: BoolToOnOff(DefTiDBWaitSplitRegionFinish), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.WaitSplitRegionFinish = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBWaitSplitRegionTimeout, Value: strconv.Itoa(DefWaitSplitRegionTimeout), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBWaitSplitRegionTimeout, Value: strconv.Itoa(DefWaitSplitRegionTimeout), skipInit: true, Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.WaitSplitRegionTimeout = uint64(tidbOptPositiveInt32(val, DefWaitSplitRegionTimeout)) return nil }}, - {Scope: ScopeSession, Name: TiDBLowResolutionTSO, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBLowResolutionTSO, Value: Off, Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.LowResolutionTSO = TiDBOptOn(val) return nil }}, @@ -1280,7 +1303,7 @@ var defaultSysVars = []*SysVar{ s.EnableNoopFuncs = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBReplicaRead, Value: "leader", Type: TypeEnum, PossibleValues: []string{"leader", "follower", "leader-and-follower"}, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBReplicaRead, Value: "leader", Type: TypeEnum, PossibleValues: []string{"leader", "follower", "leader-and-follower"}, skipInit: true, SetSession: func(s *SessionVars, val string) error { if strings.EqualFold(val, "follower") { s.SetReplicaRead(kv.ReplicaReadFollower) } else if strings.EqualFold(val, "leader-and-follower") { @@ -1290,7 +1313,7 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeSession, Name: TiDBAllowRemoveAutoInc, Value: BoolToOnOff(DefTiDBAllowRemoveAutoInc), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBAllowRemoveAutoInc, Value: BoolToOnOff(DefTiDBAllowRemoveAutoInc), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.AllowRemoveAutoInc = TiDBOptOn(val) return nil }}, @@ -1354,27 +1377,27 @@ var defaultSysVars = []*SysVar{ tikvstore.StoreLimit.Store(tidbOptInt64(val, DefTiDBStoreLimit)) return nil }}, - {Scope: ScopeSession, Name: TiDBMetricSchemaStep, Value: strconv.Itoa(DefTiDBMetricSchemaStep), Type: TypeUnsigned, MinValue: 10, MaxValue: 60 * 60 * 60, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBMetricSchemaStep, Value: strconv.Itoa(DefTiDBMetricSchemaStep), Type: TypeUnsigned, skipInit: true, MinValue: 10, MaxValue: 60 * 60 * 60, SetSession: func(s *SessionVars, val string) error { s.MetricSchemaStep = tidbOptInt64(val, DefTiDBMetricSchemaStep) return nil }}, - {Scope: ScopeSession, Name: TiDBMetricSchemaRangeDuration, Value: strconv.Itoa(DefTiDBMetricSchemaRangeDuration), Type: TypeUnsigned, MinValue: 10, MaxValue: 60 * 60 * 60, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBMetricSchemaRangeDuration, Value: strconv.Itoa(DefTiDBMetricSchemaRangeDuration), skipInit: true, Type: TypeUnsigned, MinValue: 10, MaxValue: 60 * 60 * 60, SetSession: func(s *SessionVars, val string) error { s.MetricSchemaRangeDuration = tidbOptInt64(val, DefTiDBMetricSchemaRangeDuration) return nil }}, - {Scope: ScopeSession, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint64(&config.GetGlobalConfig().Log.SlowThreshold, uint64(tidbOptInt64(val, logutil.DefaultSlowThreshold))) return nil }}, - {Scope: ScopeSession, Name: TiDBRecordPlanInSlowLog, Value: int32ToBoolStr(logutil.DefaultRecordPlanInSlowLog), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBRecordPlanInSlowLog, Value: int32ToBoolStr(logutil.DefaultRecordPlanInSlowLog), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog, uint32(tidbOptInt64(val, logutil.DefaultRecordPlanInSlowLog))) return nil }}, - {Scope: ScopeSession, Name: TiDBEnableSlowLog, Value: BoolToOnOff(logutil.DefaultTiDBEnableSlowLog), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBEnableSlowLog, Value: BoolToOnOff(logutil.DefaultTiDBEnableSlowLog), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { config.GetGlobalConfig().Log.EnableSlowLog = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBQueryLogMaxLen, Value: strconv.Itoa(logutil.DefaultQueryLogMaxLen), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBQueryLogMaxLen, Value: strconv.Itoa(logutil.DefaultQueryLogMaxLen), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, skipInit: true, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen, uint64(tidbOptInt64(val, logutil.DefaultQueryLogMaxLen))) return nil }}, @@ -1382,19 +1405,19 @@ var defaultSysVars = []*SysVar{ s.CTEMaxRecursionDepth = tidbOptInt(val, DefCTEMaxRecursionDepth) return nil }}, - {Scope: ScopeSession, Name: TiDBCheckMb4ValueInUTF8, Value: BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBCheckMb4ValueInUTF8, Value: BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBFoundInPlanCache, Value: BoolToOnOff(DefTiDBFoundInPlanCache), Type: TypeBool, ReadOnly: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBFoundInPlanCache, Value: BoolToOnOff(DefTiDBFoundInPlanCache), Type: TypeBool, ReadOnly: true, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.FoundInPlanCache = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBFoundInBinding, Value: BoolToOnOff(DefTiDBFoundInBinding), Type: TypeBool, ReadOnly: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBFoundInBinding, Value: BoolToOnOff(DefTiDBFoundInBinding), Type: TypeBool, ReadOnly: true, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.FoundInBinding = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBEnableCollectExecutionInfo, Value: BoolToOnOff(DefTiDBEnableCollectExecutionInfo), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBEnableCollectExecutionInfo, Value: BoolToOnOff(DefTiDBEnableCollectExecutionInfo), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { oldConfig := config.GetGlobalConfig() newValue := TiDBOptOn(val) if oldConfig.EnableCollectExecutionInfo != newValue { @@ -1427,7 +1450,7 @@ var defaultSysVars = []*SysVar{ s.PartitionPruneMode.Store(strings.ToLower(strings.TrimSpace(val))) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBSlowLogMasking, Value: BoolToOnOff(DefTiDBRedactLog), Aliases: []string{TiDBRedactLog}, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBSlowLogMasking, Value: BoolToOnOff(DefTiDBRedactLog), Aliases: []string{TiDBRedactLog}, skipInit: true, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TiDBSlowLogMasking, TiDBRedactLog) return normalizedValue, nil }}, diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index d236dc142f1dd..eb8d276ba7375 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -451,6 +451,20 @@ func (*testSysVarSuite) TestGetScopeNoneSystemVar(c *C) { c.Assert(val, Equals, "") } +func (*testSysVarSuite) TestSkipInit(c *C) { + sv := SysVar{Scope: ScopeGlobal, Name: "skipinit1", Value: On, Type: TypeBool} + c.Assert(sv.SkipInit(), IsTrue) + + sv = SysVar{Scope: ScopeGlobal | ScopeSession, Name: "skipinit1", Value: On, Type: TypeBool} + c.Assert(sv.SkipInit(), IsFalse) + + sv = SysVar{Scope: ScopeSession, Name: "skipinit1", Value: On, Type: TypeBool} + c.Assert(sv.SkipInit(), IsFalse) + + sv = SysVar{Scope: ScopeSession, Name: "skipinit1", Value: On, Type: TypeBool, skipInit: true} + c.Assert(sv.SkipInit(), IsTrue) +} + func (*testSysVarSuite) TestInstanceScopedVars(c *C) { // This tests instance scoped variables through GetSessionOrGlobalSystemVar(). // Eventually these should be changed to use getters so that the switch From a126b06a1dcbc3eacd519278b3007b04660f5606 Mon Sep 17 00:00:00 2001 From: Howie Date: Mon, 24 May 2021 13:31:33 +0800 Subject: [PATCH 144/343] ddl: the clustered index primary key conflict prompt information is not an index value (#24831) --- executor/ddl_test.go | 11 +++++++++++ store/driver/txn/error.go | 3 +++ 2 files changed, 14 insertions(+) diff --git a/executor/ddl_test.go b/executor/ddl_test.go index c55908066de62..7d50b66e5a65c 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -1526,3 +1526,14 @@ func (s *testRecoverTable) TestRenameMultiTables(c *C) { tk.MustExec("drop database rename2") tk.MustExec("drop database rename3") } + +// See https://github.com/pingcap/tidb/issues/24582 +func (s *testSuite6) TestDuplicatedEntryErr(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(a int, b varchar(20), primary key(a,b(3)) clustered);") + tk.MustExec("insert into t1 values(1,'aaaaa');") + _, err := tk.Exec("insert into t1 values(1,'aaaaa');") + c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '1-aaa' for key 'PRIMARY'") +} diff --git a/store/driver/txn/error.go b/store/driver/txn/error.go index 39931357567be..cde513e9f1e7d 100644 --- a/store/driver/txn/error.go +++ b/store/driver/txn/error.go @@ -94,6 +94,9 @@ func extractKeyExistsErrFromHandle(key kv.Key, value []byte, tblInfo *model.Tabl if err != nil { return genKeyExistsError(name, key.String(), err) } + if col.Length > 0 { + str = str[:col.Length] + } valueStr = append(valueStr, str) } return genKeyExistsError(name, strings.Join(valueStr, "-"), nil) From 2e7ace474ce77917df00f81a0570ed3d7994579b Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 24 May 2021 14:05:33 +0800 Subject: [PATCH 145/343] ci acceleration: reduce execution time for TestDirectReadingWithAgg (#24818) --- executor/partition_table_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 73520eea1792a..dd52a733de9c6 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -1560,7 +1560,7 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { tk.MustExec("insert into tregular2 values " + strings.Join(vals, ",")) // test range partition - for i := 0; i < 2000; i++ { + for i := 0; i < 200; i++ { // select /*+ stream_agg() */ a from t where a > ? group by a; // select /*+ hash_agg() */ a from t where a > ? group by a; // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; @@ -1592,7 +1592,7 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { } // test hash partition - for i := 0; i < 2000; i++ { + for i := 0; i < 200; i++ { // select /*+ stream_agg() */ a from t where a > ? group by a; // select /*+ hash_agg() */ a from t where a > ? group by a; // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; @@ -1624,7 +1624,7 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { } // test list partition - for i := 0; i < 2000; i++ { + for i := 0; i < 200; i++ { // select /*+ stream_agg() */ a from t where a > ? group by a; // select /*+ hash_agg() */ a from t where a > ? group by a; // select /*+ stream_agg() */ a from t where a in(?, ?, ?) group by a; From 5acff639cb8d98cf98ada2ab51b58e5449312a2f Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Mon, 24 May 2021 14:21:33 +0800 Subject: [PATCH 146/343] docs: add the temporary table design proposal (#24168) --- docs/design/2021-04-20-temporary-table.md | 403 ++++++++++++++++++++++ 1 file changed, 403 insertions(+) create mode 100644 docs/design/2021-04-20-temporary-table.md diff --git a/docs/design/2021-04-20-temporary-table.md b/docs/design/2021-04-20-temporary-table.md new file mode 100644 index 0000000000000..dae11f7fcce97 --- /dev/null +++ b/docs/design/2021-04-20-temporary-table.md @@ -0,0 +1,403 @@ +# Temporary Table Design + +- Authors: [Kangli Mao](http://github.com/tiancaiamao), [Ming Zhang](http://github.com/djshow832) +- Discussion PR: https://github.com/pingcap/tidb/pull/24168 +- Tracking Issue: https://github.com/pingcap/tidb/issues/24169 + +# Introduction + +A temporary table is a special table whose rows data are only temporarily available. + +TiDB will implement both local temporary tables and global temporary tables. The local temporary table is basically compatible with MySQL temporary tables. The global temporary table is a subset of the SQL standard, which supports only one table commit action. + +For a local temporary table, it is visible only within the current session, and the table is dropped automatically when the session is closed. + +For a global temporary table, the table schema is visible to all the sessions, but the changes into the table are only available within the current transaction, when the transaction commits, all changes to the global temporary table are discarded. + +## Syntax + +The syntax of creating and dropping a global temporary table: + +```sql +CREATE GLOBAL TEMPORARY TABLE tbl_name (create_definition) +[ENGINE=engine_name] +ON COMMIT DELETE ROWS; + +DROP TABLE tbl_name; +``` + +The syntax of creating and dropping a session(local) temporary table: + +```sql +CREATE TEMPORARY TABLE tbl_name (create_definition) +[ENGINE=engine_name]; + +DROP [TEMPORARY] TABLE tbl_name; +``` + +In the following sectionsn session temporary tables and local temporary tables are used interchangeably. + +## Visibility of table definition + +There are 2 kinds of table definition visibility. + +- Global: global temporary tables are visible to all sessions. These tables only need to be created once and the metadata is persistent. +- Local: local temporary tables are visible to the session that creates them. These tables must be created in the session before being used and the metadata is only kept in memory. + +## Visibility of data + +There are 2 kinds of data visibility: + +- Session: when the table is a session temporary table, the data will be kept after a transaction commits. Subsequent transactions will see the data committed by the previous transactions. +- Transaction: when the table is defined as `ON COMMIT DELETE ROWS`, this data will be cleared automatically after a transaction commits. Subsequent transactions won’t see the data committed by the previous transactions. + +## Storage engines + +TiDB uses TiKV and TiFlash to store the data of normal tables. This is also true for temporary tables by default because MySQL does so. However, temporary tables sometimes are used to boost performance, so it's also reasonable to support in-memory temporary tables. There are 2 kinds of storage engines available: + +- Memory or TempTable: TiDB keeps data in memory, and if the memory consumed by a temporary table exceeds a threshold, the data will be spilled to the local disk on TiDB. This is the default storage engine, which is different from MySQL. +- InnoDB: TiDB stores data on TiKV and TiFlash with more than one replica, just like normal tables. According to MySQL, this is the default storage engine, even for temporary tables. + +# Motivation or Background + +Temporary tables are useful in applications where a result set is to be buffered (temporarily persisted), perhaps because it is constructed by running multiple DML operations. + +The purposes of temporary tables include: + +- Usability. Temporary tables are typically for a temporary use. Applications don't need to frequently truncate the table. +- Performance. In-memory temporary tables are stored in memory, which boosts performance. +- Materialize middle data for queries. Some queries apply internal temporary tables for materializing middle data, such as CTE. + +# Detailed Design + +## Metadata + +The table ID of global temporary tables must be globally unique, while the local temporary tables don't. However, logical or physical plans involve table IDs, which means the temporary table IDs must be different from the normal table IDs. To achieve this goal, it's straightforward to also allocate local temporary tables globally. + +For a global temporary table, its table name should not be duplicated with a normal table. For a local temporary table, when its name conflicts with an existing table, it will take a higher priority. We can keep the temporary tables in a local schema, and overload the original one. The databases where temporary tables belong depend on the identifier in `CREATE TABLE` statements, just like normal tables. + +Since the metadata of global temporary tables are persistent on TiKV, it's straightforward to execute DDL in the same procedure as normal tables. However, the metadata of local temporary tables are only kept in the memory of the current TiDB instance, so we can bypass the complex online DDL procedure. We need only to generate the metadata locally and then merge it into the information schema. Thus, users cannot see the DDL jobs of local temporary tables through `ADMIN SHOW DDL JOBS`. + +Local temporary tables don’t support altering table operations because few users will do that. TiDB should report errors when users try to do that. + +As all DDL statements do, any DDL on a global temporary table will cause an implicit commit. However, creating and dropping a local temporary table doesn’t cause an implicit commit, according to [the MySQL documentation](https://dev.mysql.com/doc/refman/8.0/en/implicit-commit.html). + +Each temporary table belongs to its own database. Local temporary tables have a very loose relationship with databases. Dropping a database does not automatically drop any local temporary tables created within that database. The local temporary tables still stay in a virtual database with the same name. + +Truncating global temporary tables also conforms to the online DDL procedure, which affects other sessions. However, it's different for local temporary tables because the metadata is kept in memory. Truncating local temporary tables just drops the current metadata and creates a new one in memory. + +DDL operations, including those using `INSTANT` and `COPY` algorithms, are simple to apply on global temporary tables. For example, adding an index on a global temporary table is easy, because the table must be empty before adding the index. This benefits from implicit commit before adding the index. Local temporary tables, on the other hand, do support adding indexes, as other altering table operations. + +Some options in `CREATE TABLE` statements are not suitable for temporary tables. These options include: `AUTO_RANDOM`, `SHARD_ROW_ID_BITS`, `PRE_SPLIT_REGIONS`, `PARTITION BY`, `FOREIGN KEY`. Similarly, some related DDL operations are not supported, such as `SPLIT TABLE`, `SHOW TABLE REGIONS`, `ALTER PLACEMENT POLICY`. Table partition option is useless to a temporary table in the real use cases, so it's also not supported. Errors should be reported when users declare such options in the statements. + +Since some options are not suitable for temporary tables, when a user creates a temporary table from the `CREATE TABLE LIKE` statement and the source table has these options, an error should be reported. + +Other options will be kept. For example, clustered indexes and secondary indexes are kept because they can improve performance. + +- Altering table types is not allowed, since few users will do that: +- Altering a temporary table to a normal table or conversely. +- Altering a global temporary table to a local temporary table or conversely. +- Altering the storage engine of temporary tables. + +The result of `SHOW TABLES` contains global temporary tables, but not local temporary tables. The motivation is that local temporary tables are only for temporary usage, so the user should know what he is doing. However, `SHOW CREATE TABLE` works for all temporary tables. + +Similarly, system tables `TABLES` and `COLUMNS` in `information_schema` do not contain local temporary tables, but they contain global temporary tables. For global temporary tables, the value of field `TABLE_TYPE` in `information_schema.TABLES` is `GLOBAL TEMPORARY TABLE`. + +`ADMIN CHECK TABLE` and `ADMIN CHECKSUM TABLE` are used to check data integrity of the table. Data of temporary tables might also be corrupted due to unexpected problems, but it's impossible to check them because they are invisible to other sessions. So TiDB doesn’t support these commands. + +Creating views on a global temporary table makes sense, and the view will also be persistent on TiKV. However, it's unreasonable to persist a view that is based on a local temporary table, because the table will be discarded when the session ends. + +As the metadata of global temporary tables are persistent on TiKV, the binlog of DDL should also be exported. However, this is unnecessary for local temporary tables. + +## Optimizer + +Statistics of temporary tables are very different from those of normal tables. The data of each temporary table is invisible to other sessions, so it's unnecessary to persist statistics on TiKV. On the other hand, even if the sizes of temporary tables are relatively small, it's also necessary to consider statistics since it may improve the query performance significantly. + +Updating statistics is a little different from normal tables. We can’t rely on `AUTO ANALYZE` anymore, because the lifecycle of one session is relatively short, it's unlikely to wait for `AUTO ANALYZE`. What’s more, `AUTO ANALYZE` runs in background sessions, which means they can’t visit the temporary data. + +it's also unreasonable to force users to run `ANALYZE TABLE` periodically in applications. Intuitively, there are 2 ways to maintain statistics: +Update statistics once updating the table. When users run `ANALYZE TABLE`, TiDB updates statistics in the current session, instead of in background sessions. +Instead of maintaining statistics, collecting needed statistics before each query is another option. The collection can be boosted by sampling. `ANALYZE TABLE` needs to do nothing in this way. + +Both ways are also easy to implement, and it's not concrete that we really need statistics for temporary tables for now. So we only maintain row count, and just skip maintaining NDV, selectivity and others until there’s a need. + +Statistics of the same global temporary table are different in different sessions using, so every session keeps a copy of statistics for each global temporary table. However, the statistics of normal tables are stored in a global cache, which can be visited concurrently. So there needs to be a way to store the statistics of global temporary tables separately. + +Obviously, the cost of reading temporary tables is much lower than reading normal tables, since TiDB doesn’t need to request TiKV or consume any network resources. So the factors of such operations should be lower or 0. + +SQL binding is used to bind statements with specific execution plans. Global SQL binding affects all sessions and session SQL binding affects the current session. Since local temporary tables are invisible to other sessions, global SQL binding is meaningless. TiDB should report errors when users try to use global SQL binding on local temporary tables. Creating global or session SQL binding on global temporary tables, and creating session SQL binding on local temporary tables are both allowed. + +Baseline is used to capture the plans of statements that appear more than once. The appearance of statements is counted by SQL digests. Even if all sessions share the same global temporary table definitions, the data and statistics is different from one session to another. Thus baseline and SPM is useless for temporary tables. TiDB will ignore this feature for temporary tables. + +Prepared plan cache is used to cache plans for prepared statements to avoid duplicate optimization. Each session has a cache and the scope of each cache is the current session. Even if the cached plan stays in the cache after the temporary table is dropped, the plan won’t take effect and will be removed by the LRU list finally. So we just leave it as it was. + +## Storage + +Before going further to the executor, we need to determine the storage form of temporary tables. + +Basically, there are a few alternatives to store the data of in-memory temporary tables, and the most promising ones are: + +- Unistore. Unistore is a module that simulates TiKV on a standalone TiDB instance. +- UnionScan. UnionScan is a module that unit's membuffer and TiKV data. Membuffer buffers the dirty data a transaction writ's. Query operators read UnionScan and UnionScan will read both buffered data and persistent data. Thus, if the persistent part is always empty, then the UnionScan itself is a temporary table. + +| | Unistore | UnionScan | +| ------------- | :-------------: | -----: | +| execution | Y | Y | +| indexes | Y | Y | +| spilling to disk | Y | N | +| MVCC | Y | N | + + +TiDB uses UnionScan to store the data of temporary tables for the following reasons: + +- The performance should be better. It can cut down the marshal/unmarshal cost and a lot of the coprocessor code path, which is inevitable in the Unistore as a standalone storage engine. +- The implementation is easier. As long as we don’t take the spilling to disk feature into consideration for now, the global temporary table is almost handly. And we do not bother by the tons of background goroutines of the Unistore engine when dealing with resource releasing. How to keep the atomicity is another headache if we choose the Unistore, imaging that a transaction would write to both temporary tables and normal tables at the same time. +- A lower risk of introducing bugs. Although we implement the coprocessor in the Unistore, we just use it for the testing purpose, and we also have some experimental features first implemented in the Unistore, so its behavior may slightly differ from the real TiKV, and that difference would introduce bugs. + +Nothing needs to be changed for the KV encoding, the temporary table uses the same strategy with the normal table. + +When the memory consumed by a temporary table exceeds a threshold, the data should be spilled to the local disk to avoid OOM. The threshold is defined by the system variable `temptable_max_ram`, which is 1G by default. Membuffer does not support disk storage for now, so we need to implement it. + +A possible implementation is to use the AOF (append-only file) persistent storage, the membuffer is implemented as a red-black tree and its node is allocated from a customized allocator. That allocator manages the memory in an arena manner. A continuous block of memory becomes an arena block and an arena consists of several arena blocks. We can dump those blocks into the disk and maintain some LRU records for them. A precondition of the AOF implementation is that once a block is dumped, it is never modified. Currently, the value of a node is append-only, but the key is modified in-place, so some changes are necessary. We can keep all the red-black tree nodes in memory, while keeping part of the key-value data in the disk. + +Another option is changing the red-black tree to a B+ tree, this option is more disk friendly but the change takes more effort. + +When a temporary table needs to be cleared, its disk data should also be cleared. That can be done asynchronously by a background goroutine. The goroutine needs to know whether a file is in use. So TiDB needs to maintain a map which contains in-use files, each session updates the map periodically if it'still needs the file. If some files are not touched for a long time, we can treat the session as crashed and collect the disk files. + +For on-disk temporary tables, it's straightforward to store them on TiKV like normal tables. Since each local temporary table has a unique table ID, the data from different sessions are separate. However, multiple sessions using the same global temporary table will share the same table ID, which means the data from multiple sessions is continuous and stored together. Sessions will affect each other in this case. + +Clearing the data of on-disk temporary tables is a little different from in-memory temporary tables. When a TiDB instance is down, the storage space needs to be collected by other TiDB instances. Thus the maintenance of in-use temporary tables should be on the TiKV side. + +## Executor + +For normal tables, auto-increment IDs and row IDs are allocated in batches from TiKV, which significantly improves performance. However, as temporary tables are usually inserted, it may cause write hotspot on TiKV. So the best way is to allocate IDs locally on TiDB. The ID of a global temporary table is allocated separately among sessions and rebases to 0 every time a transaction ends. That means, each session needs a copy of allocators, rather than sharing the same allocators. + +Besides, `last_insert_id` is also affected by inserting into temporary tables. + +Since the data of in-memory temporary tables are not needed to be cached anymore, coprocessor cache and point-get cache are ignored. But they still work for on-disk temporary tables. + +Follower read indicates TiDB to read the follower replicas to release the load of leaders. For in-memory temporary tables, this hint is ignored. But it'still works for on-disk temporary tables. + +Users can also choose the storage engine to read by setting `tidb_isolation_read_engines`. For in-memory temporary tables, this setting will also be ignored. But it'still works for on-disk temporary tables. + +Since in-memory temporary tables are not persistent on TiKV or TiFlash, writing binlog for DML is also unnecessary. This also stays true for on-disk temporary tables, because data is invisible to other sessions. + +it's straightforward to support MPP on on-disk temporary tables, because the data is also synchronized to TiFlash. Most operators on in-memory temporary tables can still be processed in TiDB, such as Aggregation and TopN. These operators will not cost much memory because the sizes of in-memory temporary tables are relatively small. + +However, joining normal tables with in-memory temporary tables might be a problem, because the sizes of normal tables might be huge and thus merge sort join will cause OOM, while hash join and index lookup join will be too slow. Supporting broadcast join and shuffled hash join on in-memory temporary tables is very difficult. Fortunately, MPP typically happens in OLAP applications, where writing and scanning duration is relatively short compared to computing duration. So users can choose to define on-disk temporary tables in this case. + +## Transaction + +Because it's rare to read historical data of a temporary table, temporary tables don’t support features that rely on MVCC, like flashback table, recover table, stale read, and historical read. Errors will be reported when users execute such statements on temporary tables. + +If a transaction only writ's to temporary tables without normal tables, it does not really need a TSO for committing, because MVCC is unsupported and the data is invisible to other sessions. But for the sake of simplicity, we still fetch commit's just like normal tables. + +When a transaction commits, the data in in-memory temporary tables should not be committed on TiKV. When a transaction rollbacks, operations on normal tables and temporary tables should be rollbacked together. The data in on-disk temporary tables will be committed. However, this can be omitted by global on-disk temporary tables, because the data will be cleared anyway. TiDB can iterate KV ranges to filter out the data. + +Since there won’t be concurrent modifications on a same temporary table, there won’t be lock conflicts. So `FOR UPDATE` and `LOCK IN SHARE MODE` clauses will be ignored. + +Transactions might retry write operations when commit fails. DML on normal tables might rely on the data on temporary tables, so DML on temporary tables should also be retried. For example: + +``` +INSERT INTO normal_table SELECT * FROM temp_table +UPDATE normal_table, temp_table SET … WHERE normal_table.id=temp_table.id +``` + +If DML on temporary tables is not retried, such statements won’t write any data. + +Specially, as mentioned above, creating and dropping local temporary tables might also be in a transaction, but they needn’t be retried. + +TiDB comes with an optimization when the variable `tidb_constraint_check_in_place` is disabled: checking for duplicate values in UNIQUE indexes is deferred until the transaction commits. For those cases where temporary tables skip 2PC, this optimization should be disabled. + +Local transactions are special transactions that fetch TSO from the local PD. They can not access the data that is bound to the current available zone. Although temporary tables are not bound to any zones, they are invisible to other sessions, which means local transactions can still guarantee linearizability even when they access temporary tables. + +Schema change on a global temporary table may happen during a transaction which writ's to the temporary table. Unlike normal tables, the transaction won’t overwrite other transactions, so it's fine to commit. Schema change on a local temporary table will never happen during a transaction which writ's to the temporary table. + +## Privileges + +Creating a local temporary table checks the `CREATE TEMPORARY TABLES` privilege. No access rights are checked when dropping a local temporary table, according to [the MySQL documentation](https://dev.mysql.com/doc/refman/8.0/en/drop-table.html). + +All DDL on global temporary tables check the corresponding privileges like normal tables do. + +Writing to a global temporary table checks the privileges like the normal table. But there is no privilege check for a local temporary table. + +Privileges can not be granted to local temporary tables, because the tables are invisible to other users. Granting privileges to global temporary tables is possible. + +Ecosystem Tools +As mentioned above, DDL binlog of global temporary tables needs to be recorded, but not for local temporary tables. DML binlog is always skipped for temporary tables. DDL of global temporary tables should be supported by all data migration tools whose upstream is TiDB, such as Dumpling, TiDB-binlog, and TiCDC. + +Since `information_schema.tables` lists global temporary tables, these tables will be processed by tools like Dumpling. Fortunately, querying global temporary tables in a new session just returns empty results, so nothing needs to be handled. + +When backup tools read TiKV data, the data of temporary tables should never be read. However, on-disk temporary tables are stored on TiKV and TiFlash, so they need to be ignored by those tools, such as BR and TiCDC. Since these tools can see the metadata, they should also be capable of skipping tables that are not normal tables. + +Telemetry is used to report the usage information of various features in TiDB. Global and local temporary tables will be reported by telemetry separately, because the scenarios of them are different. + +Internal temporary tables +In MySQL, temporary tables will be used internally as infrastructures of other features, such as CTE, derived tables, and UNION statements. These temporary tables are called [internal temporary tables](https://dev.mysql.com/doc/refman/8.0/en/internal-temporary-tables.html). + +In the future, temporary tables will probably be used internally for some new features, such as CTE. So TiDB should be capable of creating a local temporary table when it's given a list of columns. Internal temporary tables might be in-memory or on-disk, depending on the optimizer. Physical plans that don’t apply MPP should use in-memory temporary tables, otherwise they will use on-disk temporary tables + +Transactions that apply internal temporary tables might be read-only, but there are some steps which write to TiKV: + +- Assigning table ID for temporary tables +- Writing to on-disk temporary tables +- Writing to on-disk temporary tables is inevitable, so the best way is NOT to report errors in this case. + +When executing a CTE in MPP mode, TiFlash has to write to the temporary table by itself, because the middle data is generated by TiFlash. + +## Modification to the write path + +The data of the temporary table is all in the membuffer of UnionScan. Writing to the temporary table is writing to the transaction cache. + +The data writing path in TiDB is typically through the `table.Table` interface, rows data are converted into key-values, and then written to TiKV via 2PC. For normal transactions, the data is cached in the membuffer until the transaction commits. The temporary table should never be written to the real TiKV. + +A transaction can write to both the temporary table and the normal table. Should the temporary table share the membuffer with the normal table, or use a separate one? +it's better to share the membuffer so the commit/rollback operation can be atomic. The risk is that in case of a bug, the temporary table data may be written to the real table. + +For global temporary tables, since the transaction commits automatically clears the temporary table, we can filter out and discard the transaction cache data from the temporary table when committing, and the implementation is relatively simple. + +For local temporary tables, the temporary table data is cleared at the end of the session, so the data should survive the transaction’s commit. We can copy the key-value data belonging to the temporary table to another place and use it in the later read operation. + +## Modification to the read path + +In TiDB, reading is implemented by the coprocessor. Read operations should see their own writ's. TiDB uses a UnionScan operator on top of the coprocessor's executor. This operator takes the data read by the coprocessor as a snapshot, then merges it with the membuffer, and passes the result to its parent operator. + +For transactional temporary tables, there is no need to do any additional changes, the current code works well. + +For the local temporary table, the key-value data of the temporary table in the current transaction should be taken out. We keep a temporary table membuffer in the session variable and if it is not null, the UnionScan operator needs to combine it with the original data. So now the data hierarchy looks like this: + + TiKV snapshot => Old membuffer => current transaction’s membuffer + +## Upgrade and Downgrade Compatibility + +When users downgrade TiDB to an older version after they have created a global temporary table, the table should not be seen by the older version. Otherwise, they might write to the table and then upgrade TiDB, which will be a problem. + +# Test Design + +A brief description of how the implementation will be tested. Both the integration test and the unit test should be considered. + +## Functional Tests + +it's used to ensure the basic feature function works as expected. Both the integration test and the unit test should be considered. + +- DDL + - Create table / create table like + - Drop table + - Truncate table + - Other DDL (for global) +- DML + - Insert / replace / update / delete / load data + - All kinds of query operators + - Show create table / show tables +- Transaction + - Atomicity (for local) + - Isolation (RC / SI, linearizability) + - Data visibility between transactions + - Optimistic / Pessimistic transactions +- Information_schema + - Tables + - Columns +- Privileges + +## Scenario Tests + +it's used to ensure this feature works as expected in some common scenarios. + +Before we implement the spilling to disk feature, we have to know the memory usage for some scenarios. For example, 100M for each temporary table, and 500-600 concurrent connections, how much memory does TiDB use. + +## Compatibility Tests + +| Feature | Compatibility | Temporary Table type | Reason | +| ------------- | :-------------: | :-----: | :---: | +| placement rules | Report error: not support | | Meaningless | +| partition | Report error: not support | | Meaningless | +| show table regions / split table / pre_split_regions | Report error: not support | | Meaningless | +| stale read / historical read | Report error: not support | | Meaningless | +| auto_random / `shard_row_id_bits` | Report error: not support | | No need to release writing hotspots | +| flashback / recover table | Report error: not support | | Meaningless | +| global SQL binding | Report error: not support | local | Bindings are meaningless after session ends & Tables are different amo ng sessions | +| view | Report error: not support | local | Views are meaningless after session ends & Tables are different among sessions | +| copr cache | Ignore this setting | in-memory | No need to cache | +| point get cache | Ignore this setting | in-memory | No need to cache | +| follower read | Ignore this setting | in-memory | Data is neither on TiKV nor on TiFlash | +| read engine | Ignore this setting | in-memory | Data is neither on TiKV nor on TiFlash | +| GC | Ignore this setting | | No need to GC data | +| select for update / lock in share mode | Ignore this setting | | No lock is needed | +| broadcast join / shuffle hash join | Ignore this setting | | Data is not on TiFlash | +| baseline / SPM | Ignore this setting | | | +| `tidb_constraint_check_in_place` | Ignore this setting | in-memory & local | Data is not committed on TiKV | +| auto analyze | Ignore this setting | | Background sessions can’t access private data | +| global session-specific | Need to backfill in memory | | | +| global txn / local txn | Need to deal with | | No limitation for read / write | +| analyze | Need to deal with | on-disk | Update in-memory stats in the current session instead of in system sessions | +| telemetry | Need to deal with | | Report the usage of temporary tables | +| view | Need to test | global | Drop views on dropping temporary tables | +| `auto_increment` / `last_insert_id` | Need to deal with | | AutoID is allocated locally | +| add index | Need to deal with | | The table is always empty before adding index | +| all hints | Need to test | | | +| plan cache | Need to test | | plan cache is session-scope | +| show fields / index / keys | Need to test | | | +| SQL binding | Need to test | | | +| clustered index | Need to test | | | +| async commit / 1PC | Need to test | | | +| checksum / check table | Report error: not support | | Meaningless | +| collation / charset | Need to test | | | +| batch insert | Need to test | | | +| feedback | Need to test | | | +| `statements_summary` / `slow_query` | Need to test | | SQL normalization | +| big transaction | Need to test | | | +| memory tracker | Need to test | | | +| explain / explain analyze | Need to test | | | + +Compatibility with other external components, like TiDB, PD, TiKV, TiFlash, BR, TiCDC, Dumpling, TiUP, K8s, etc. + +Upgrade compatibility + +Downgrade compatibility + +## Benchmark Tests + +The following two parts need to be measured: + +- measure the performance of this feature under different parameters +- measure the performance influence on the online workload + +sysbench for the temporary table, comparing its performance with the normal table. It means comparing the performance of an in-memory system with a distributed system. + +# Investigation & Alternatives + +How do other systems solve this issue? What other designs have been considered and what is the rationale for not choosing them? + +[MySQL documentation](https://dev.mysql.com/doc/refman/8.0/en/internal-temporary-tables.html) about the temporary table + +[CockroachDB](https://github.com/cockroachdb/cockroach/issues/5807) just uses the normal table as the temporary table. All the temporary tables are stored in a special schema, and it is scanned and cleaned periodically. If a session is finished, the temporary tables of that session are also cleaned. + +[Oracle](https://docs.oracle.com/cd/B28359_01/server.111/b28310/tables003.htm#ADMIN11633) uses global temporary tables in the old versions, and later on they also have the private temporary tables. The private temporary table in Oracle looks like MySQL, those tables are visible to the session rather than global. For global temporary tables, Oracle does not need to handle the schema change, because `alter table` is not allowed when some transactions are using the table. + +# Development Plan + +Stage 1(Support transactional, in-memory temporary table) + +- The architecture design for global/local, session/transaction, in-memory/disk spilling (L) +- DDL(2XL) + - Basic DDL e.g. create/drop table/add index etc + - Show tables/show create table for global temporary table +- DML (2XL) + + +- Backup and recover the temporary table meta information (L) +- Sync the temporary table DDL through CDC or Binlog to the downstream +- Privilege (L) +- Compatibility with other features (2XL) +- Performance test (L) + +Stage 2 (Support for session-specific temporary table) + +- create/drop session temporary table (3XL) +- Duplicated table name handling +- Compatibility with MySQL 8.0 temporary table (in-memory) +- Compatibility with other features (2XL) + +Stage 3 + +- Support spilling to disk for all kind of temporary tables (4XL) From 559a0067fa3bd7023ad3a43fcee70e42537dfebf Mon Sep 17 00:00:00 2001 From: disksing Date: Mon, 24 May 2021 15:59:33 +0800 Subject: [PATCH 147/343] store/tikv: extract methods for LockCtx (#24736) --- executor/batch_point_get.go | 9 +++------ executor/point_get.go | 14 ++++---------- store/tikv/kv/kv.go | 28 ++++++++++++++++++++++++++++ store/tikv/tests/2pc_test.go | 3 +-- 4 files changed, 36 insertions(+), 18 deletions(-) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 726603a0ff88f..1e1fcd581cf61 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" driver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -417,8 +416,7 @@ func LockKeys(ctx context.Context, seCtx sessionctx.Context, lockWaitTime int64, txnCtx := seCtx.GetSessionVars().TxnCtx lctx := newLockCtx(seCtx.GetSessionVars(), lockWaitTime) if txnCtx.IsPessimistic { - lctx.ReturnValues = true - lctx.Values = make(map[string]tikvstore.ReturnedValue, len(keys)) + lctx.InitReturnValues(len(keys)) } err := doLockKeys(ctx, seCtx, lctx, keys...) if err != nil { @@ -428,9 +426,8 @@ func LockKeys(ctx context.Context, seCtx sessionctx.Context, lockWaitTime int64, // When doLockKeys returns without error, no other goroutines access the map, // it's safe to read it without mutex. for _, key := range keys { - rv := lctx.Values[string(key)] - if !rv.AlreadyLocked { - txnCtx.SetPessimisticLockCache(key, rv.Value) + if v, ok := lctx.GetValueNotLocked(key); ok { + txnCtx.SetPessimisticLockCache(key, v) } } } diff --git a/executor/point_get.go b/executor/point_get.go index c5ff4b98fa2ba..76132623e621f 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -31,7 +31,6 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -321,19 +320,14 @@ func (e *PointGetExecutor) lockKeyIfNeeded(ctx context.Context, key []byte) erro if e.lock { seVars := e.ctx.GetSessionVars() lockCtx := newLockCtx(seVars, e.lockWaitTime) - lockCtx.ReturnValues = true - lockCtx.Values = map[string]tikvstore.ReturnedValue{} + lockCtx.InitReturnValues(1) err := doLockKeys(ctx, e.ctx, lockCtx, key) if err != nil { return err } - lockCtx.ValuesLock.Lock() - defer lockCtx.ValuesLock.Unlock() - for key, val := range lockCtx.Values { - if !val.AlreadyLocked { - seVars.TxnCtx.SetPessimisticLockCache(kv.Key(key), val.Value) - } - } + lockCtx.IterateValuesNotLocked(func(k, v []byte) { + seVars.TxnCtx.SetPessimisticLockCache(kv.Key(k), v) + }) if len(e.handleVal) > 0 { seVars.TxnCtx.SetPessimisticLockCache(e.idxKey, e.handleVal) } diff --git a/store/tikv/kv/kv.go b/store/tikv/kv/kv.go index 980b95842a361..0d900d6facddb 100644 --- a/store/tikv/kv/kv.go +++ b/store/tikv/kv/kv.go @@ -31,3 +31,31 @@ type LockCtx struct { ResourceGroupTag []byte OnDeadlock func(*tikverr.ErrDeadlock) } + +// InitReturnValues creates the map to store returned value. +func (ctx *LockCtx) InitReturnValues(valueLen int) { + ctx.ReturnValues = true + ctx.Values = make(map[string]ReturnedValue, valueLen) +} + +// GetValueNotLocked returns a value if the key is not already locked. +// (nil, false) means already locked. +func (ctx *LockCtx) GetValueNotLocked(key []byte) ([]byte, bool) { + rv := ctx.Values[string(key)] + if !rv.AlreadyLocked { + return rv.Value, true + } + return nil, false +} + +// IterateValuesNotLocked applies f to all key-values that are not already +// locked. +func (ctx *LockCtx) IterateValuesNotLocked(f func([]byte, []byte)) { + ctx.ValuesLock.Lock() + defer ctx.ValuesLock.Unlock() + for key, val := range ctx.Values { + if !val.AlreadyLocked { + f([]byte(key), val.Value) + } + } +} diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 12aa9466a3837..8fea337bfd61b 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -712,8 +712,7 @@ func (s *testCommitterSuite) TestPessimisticLockReturnValues(c *C) { txn = s.begin(c) txn.SetPessimistic(true) lockCtx := &kv.LockCtx{ForUpdateTS: txn.StartTS(), WaitStartTime: time.Now()} - lockCtx.ReturnValues = true - lockCtx.Values = map[string]kv.ReturnedValue{} + lockCtx.InitReturnValues(2) c.Assert(txn.LockKeys(context.Background(), lockCtx, key, key2), IsNil) c.Assert(lockCtx.Values, HasLen, 2) c.Assert(lockCtx.Values[string(key)].Value, BytesEquals, key) From 9d8935e39a3f8b0cb9ca1c5dbf871585a367626f Mon Sep 17 00:00:00 2001 From: rebelice Date: Mon, 24 May 2021 16:27:33 +0800 Subject: [PATCH 148/343] executor: add partition test with expression (#24744) --- executor/partition_table_test.go | 225 +++++++++++++++++++++++++++++++ 1 file changed, 225 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index dd52a733de9c6..b5aeebd9551ca 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -1013,6 +1013,231 @@ type testData4Expression struct { partitions []string } +func (s *partitionTableSuite) TestDateColWithUnequalExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_datetime_unequal_expression") + tk.MustExec("create database db_datetime_unequal_expression") + tk.MustExec("use db_datetime_unequal_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec(`create table tp(a datetime, b int) partition by range columns (a) (partition p0 values less than("2012-12-10 00:00:00"), partition p1 values less than("2022-12-30 00:00:00"), partition p2 values less than("2025-12-12 00:00:00"))`) + tk.MustExec(`create table t(a datetime, b int) partition by range columns (a) (partition p0 values less than("2012-12-10 00:00:00"), partition p1 values less than("2022-12-30 00:00:00"), partition p2 values less than("2025-12-12 00:00:00"))`) + tk.MustExec(`insert into tp values("2015-09-09 00:00:00", 1), ("2020-08-08 19:00:01", 2), ("2024-01-01 01:01:01", 3)`) + tk.MustExec(`insert into t values("2015-09-09 00:00:00", 1), ("2020-08-08 19:00:01", 2), ("2024-01-01 01:01:01", 3)`) + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a != '2024-01-01 01:01:01'", + partitions: []string{"all"}, + }, + { + sql: "select * from %s where a != '2024-01-01 01:01:01' and a > '2015-09-09 00:00:00'", + partitions: []string{"p1,p2"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestToDaysColWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_to_days_expression") + tk.MustExec("create database db_to_days_expression") + tk.MustExec("use db_to_days_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a date, b int) partition by range(to_days(a)) (partition p0 values less than (737822), partition p1 values less than (738019), partition p2 values less than (738154))") + tk.MustExec("create table t(a date, b int)") + tk.MustExec("insert into tp values('2020-01-01', 1), ('2020-03-02', 2), ('2020-05-05', 3), ('2020-11-11', 4)") + tk.MustExec("insert into t values('2020-01-01', 1), ('2020-03-02', 2), ('2020-05-05', 3), ('2020-11-11', 4)") + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a < '2020-08-16'", + partitions: []string{"p0,p1"}, + }, + { + sql: "select * from %s where a between '2020-05-01' and '2020-10-01'", + partitions: []string{"p1,p2"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestWeekdayWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_weekday_expression") + tk.MustExec("create database db_weekday_expression") + tk.MustExec("use db_weekday_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a datetime, b int) partition by range(weekday(a)) (partition p0 values less than(3), partition p1 values less than(5), partition p2 values less than(8))") + tk.MustExec("create table t(a datetime, b int)") + tk.MustExec(`insert into tp values("2020-08-17 00:00:00", 1), ("2020-08-18 00:00:00", 2), ("2020-08-19 00:00:00", 4), ("2020-08-20 00:00:00", 5), ("2020-08-21 00:00:00", 6), ("2020-08-22 00:00:00", 0)`) + tk.MustExec(`insert into t values("2020-08-17 00:00:00", 1), ("2020-08-18 00:00:00", 2), ("2020-08-19 00:00:00", 4), ("2020-08-20 00:00:00", 5), ("2020-08-21 00:00:00", 6), ("2020-08-22 00:00:00", 0)`) + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a = '2020-08-17 00:00:00'", + partitions: []string{"p0"}, + }, + { + sql: "select * from %s where a= '2020-08-20 00:00:00' and a < '2020-08-22 00:00:00'", + partitions: []string{"p1"}, + }, + { + sql: " select * from %s where a < '2020-08-19 00:00:00'", + partitions: []string{"all"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestFloorUnixTimestampAndIntColWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_floor_unix_timestamp_int_expression") + tk.MustExec("create database db_floor_unix_timestamp_int_expression") + tk.MustExec("use db_floor_unix_timestamp_int_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a timestamp, b int) partition by range(floor(unix_timestamp(a))) (partition p0 values less than(1580670000), partition p1 values less than(1597622400), partition p2 values less than(1629158400))") + tk.MustExec("create table t(a timestamp, b int)") + tk.MustExec("insert into tp values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3)") + tk.MustExec("insert into t values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3)") + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a > '2020-09-11 00:00:00'", + partitions: []string{"p2"}, + }, + { + sql: "select * from %s where a < '2020-07-07 01:00:00'", + partitions: []string{"p0,p1"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestUnixTimestampAndIntColWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_unix_timestamp_int_expression") + tk.MustExec("create database db_unix_timestamp_int_expression") + tk.MustExec("use db_unix_timestamp_int_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a timestamp, b int) partition by range(unix_timestamp(a)) (partition p0 values less than(1580670000), partition p1 values less than(1597622400), partition p2 values less than(1629158400))") + tk.MustExec("create table t(a timestamp, b int)") + tk.MustExec("insert into tp values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3)") + tk.MustExec("insert into t values('2020-01-01 19:00:00', 1),('2020-08-15 00:00:00', -1), ('2020-08-18 05:00:01', 2), ('2020-10-01 14:13:15', 3)") + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a > '2020-09-11 00:00:00'", + partitions: []string{"p2"}, + }, + { + sql: "select * from %s where a < '2020-07-07 01:00:00'", + partitions: []string{"p0,p1"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestDatetimeColAndIntColWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_datetime_int_expression") + tk.MustExec("create database db_datetime_int_expression") + tk.MustExec("use db_datetime_int_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a datetime, b int) partition by range columns(a) (partition p0 values less than('2020-02-02 00:00:00'), partition p1 values less than('2020-09-01 00:00:00'), partition p2 values less than('2020-12-20 00:00:00'))") + tk.MustExec("create table t(a datetime, b int)") + tk.MustExec("insert into tp values('2020-01-01 12:00:00', 1), ('2020-08-22 10:00:00', 2), ('2020-09-09 11:00:00', 3), ('2020-10-01 00:00:00', 4)") + tk.MustExec("insert into t values('2020-01-01 12:00:00', 1), ('2020-08-22 10:00:00', 2), ('2020-09-09 11:00:00', 3), ('2020-10-01 00:00:00', 4)") + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a < '2020-09-01 00:00:00'", + partitions: []string{"p0,p1"}, + }, + { + sql: "select * from %s where a > '2020-07-07 01:00:00'", + partitions: []string{"p1,p2"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + +func (s *partitionTableSuite) TestVarcharColAndIntColWithExpression(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop database if exists db_varchar_int_expression") + tk.MustExec("create database db_varchar_int_expression") + tk.MustExec("use db_varchar_int_expression") + tk.MustExec("set tidb_partition_prune_mode='dynamic'") + tk.MustExec("create table tp(a varchar(255), b int) partition by range columns(a) (partition p0 values less than('ddd'), partition p1 values less than('ggggg'), partition p2 values less than('mmmmmm'))") + tk.MustExec("create table t(a varchar(255), b int)") + tk.MustExec("insert into tp values('aaa', 1), ('bbbb', 2), ('ccc', 3), ('dfg', 4), ('kkkk', 5), ('10', 6)") + tk.MustExec("insert into t values('aaa', 1), ('bbbb', 2), ('ccc', 3), ('dfg', 4), ('kkkk', 5), ('10', 6)") + tk.MustExec("analyze table tp") + tk.MustExec("analyze table t") + + tests := []testData4Expression{ + { + sql: "select * from %s where a < '10'", + partitions: []string{"p0"}, + }, + { + sql: "select * from %s where a > 0", + partitions: []string{"all"}, + }, + { + sql: "select * from %s where a < 0", + partitions: []string{"all"}, + }, + } + + for _, t := range tests { + tpSQL := fmt.Sprintf(t.sql, "tp") + tSQL := fmt.Sprintf(t.sql, "t") + tk.MustPartition(tpSQL, t.partitions[0]).Sort().Check(tk.MustQuery(tSQL).Sort().Rows()) + } +} + func (s *partitionTableSuite) TestDynamicPruneModeWithExpression(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop database if exists db_equal_expression") From 23ce65724a011279b2eca3282ba6ca752e3913cb Mon Sep 17 00:00:00 2001 From: Howie Date: Mon, 24 May 2021 17:25:33 +0800 Subject: [PATCH 149/343] planner: rewrite `LIKE` as range for expression index (#24250) --- .../explain_generate_column_substitute.result | 25 +++++++++++++++++++ .../t/explain_generate_column_substitute.test | 8 ++++++ .../core/rule_generate_column_substitute.go | 6 +++++ 3 files changed, 39 insertions(+) diff --git a/cmd/explaintest/r/explain_generate_column_substitute.result b/cmd/explaintest/r/explain_generate_column_substitute.result index 821988170dd87..2442479202c1a 100644 --- a/cmd/explaintest/r/explain_generate_column_substitute.result +++ b/cmd/explaintest/r/explain_generate_column_substitute.result @@ -389,3 +389,28 @@ explain format = 'brief' select c0 from t0; id estRows task access object operator info TableReader 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t0 keep order:false, stats:pseudo + -- TableRead +drop table if exists tbl1; +create table tbl1 (id int unsigned not null auto_increment primary key, s int, index((md5(s)))); +insert into tbl1 (id) select null; + insert into tbl1 (id) select null from tbl1; + insert into tbl1 (id) select null from tbl1; + insert into tbl1 (id) select null from tbl1; + insert into tbl1 (id) select null from tbl1; + insert into tbl1 (id) select null from tbl1; + insert into tbl1 (id) select null from tbl1; + insert into tbl1 (id) select null from tbl1; + insert into tbl1 (id) select null from tbl1; + insert into tbl1 (id) select null from tbl1; + insert into tbl1 (id) select null from tbl1; + insert into tbl1 (id) select null from tbl1; +update tbl1 set s=id%32; +explain format = 'brief' select count(*) from tbl1 where md5(s) like '02e74f10e0327ad868d138f2b4fdd6f%'; +id estRows task access object operator info +StreamAgg 1.00 root funcs:count(Column#6)->Column#4 +└─IndexReader 1.00 root index:StreamAgg + └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#6 + └─IndexRangeScan 250.00 cop[tikv] table:tbl1, index:expression_index(md5(`s`)) range:["02e74f10e0327ad868d138f2b4fdd6f","02e74f10e0327ad868d138f2b4fdd6g"), keep order:false, stats:pseudo +select count(*) from tbl1 use index() where md5(s) like '02e74f10e0327ad868d138f2b4fdd6f%'; +count(*) +64 diff --git a/cmd/explaintest/t/explain_generate_column_substitute.test b/cmd/explaintest/t/explain_generate_column_substitute.test index 78096628c293d..395ed2311fda6 100644 --- a/cmd/explaintest/t/explain_generate_column_substitute.test +++ b/cmd/explaintest/t/explain_generate_column_substitute.test @@ -174,3 +174,11 @@ explain format = 'brief' select c0 from t0; -- TableRead drop table if exists t0; create table t0(c0 double, c1 float as (c0) unique); explain format = 'brief' select c0 from t0; -- TableRead + +drop table if exists tbl1; +create table tbl1 (id int unsigned not null auto_increment primary key, s int, index((md5(s)))); +insert into tbl1 (id) select null; insert into tbl1 (id) select null from tbl1; insert into tbl1 (id) select null from tbl1; insert into tbl1 (id) select null from tbl1; insert into tbl1 (id) select null from tbl1; insert into tbl1 (id) select null from tbl1; insert into tbl1 (id) select null from tbl1; insert into tbl1 (id) select null from tbl1; insert into tbl1 (id) select null from tbl1; insert into tbl1 (id) select null from tbl1; insert into tbl1 (id) select null from tbl1; insert into tbl1 (id) select null from tbl1; +update tbl1 set s=id%32; +explain format = 'brief' select count(*) from tbl1 where md5(s) like '02e74f10e0327ad868d138f2b4fdd6f%'; +select count(*) from tbl1 use index() where md5(s) like '02e74f10e0327ad868d138f2b4fdd6f%'; + diff --git a/planner/core/rule_generate_column_substitute.go b/planner/core/rule_generate_column_substitute.go index bc4a31e0b88e8..670753d0833ec 100644 --- a/planner/core/rule_generate_column_substitute.go +++ b/planner/core/rule_generate_column_substitute.go @@ -119,6 +119,12 @@ func (gc *gcSubstituter) substitute(ctx context.Context, lp LogicalPlan, exprToC tryToSubstituteExpr(expr, lp.SCtx(), candidateExpr, tp, x.Schema(), column) } } + case ast.Like: + expr := &sf.GetArgs()[0] + tp = sf.GetArgs()[1].GetType().EvalType() + for candidateExpr, column := range exprToColumn { + tryToSubstituteExpr(expr, lp.SCtx(), candidateExpr, tp, x.Schema(), column) + } } } case *LogicalProjection: From 26cf50ebe5c8e442de73546c9fecabc444d4230f Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 24 May 2021 17:49:34 +0800 Subject: [PATCH 150/343] executor: implement cteutil.Storage (#24193) --- util/cteutil/storage.go | 271 +++++++++++++++++++++++++++++++++++ util/cteutil/storage_test.go | 262 +++++++++++++++++++++++++++++++++ 2 files changed, 533 insertions(+) create mode 100644 util/cteutil/storage.go create mode 100644 util/cteutil/storage_test.go diff --git a/util/cteutil/storage.go b/util/cteutil/storage.go new file mode 100644 index 0000000000000..3397e81fa7c05 --- /dev/null +++ b/util/cteutil/storage.go @@ -0,0 +1,271 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cteutil + +import ( + "sync" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/disk" + "github.com/pingcap/tidb/util/memory" +) + +var _ Storage = &StorageRC{} + +// Storage is a temporary storage to store the intermidate data of CTE. +// +// Common usage as follows: +// +// storage.Lock() +// if !storage.Done() { +// fill all data into storage +// } +// storage.UnLock() +// read data from storage +type Storage interface { + // If is first called, will open underlying storage. Otherwise will add ref count by one. + OpenAndRef() error + + // Minus ref count by one, if ref count is zero, close underlying storage. + DerefAndClose() (err error) + + // SwapData swaps data of two storage. + // Other metainfo is not touched, such ref count/done flag etc. + SwapData(other Storage) error + + // Reopen reset storage and related info. + // So the status of Storage is like a new created one. + Reopen() error + + // Add chunk into underlying storage. + Add(chk *chunk.Chunk) error + + // Get Chunk by index. + GetChunk(chkIdx int) (*chunk.Chunk, error) + + // Get row by RowPtr. + GetRow(ptr chunk.RowPtr) (chunk.Row, error) + + // NumChunks return chunk number of the underlying storage. + NumChunks() int + + // Storage is not thread-safe. + // By using Lock(), users can achieve the purpose of ensuring thread safety. + Lock() + Unlock() + + // Usually, Storage is filled first, then user can read it. + // User can check whether Storage is filled first, if not, they can fill it. + Done() bool + SetDone() + + // Readers use iter information to determine + // whether they need to read data from the beginning. + 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() memory.ActionOnExceed +} + +// StorageRC implements Storage interface using RowContainer. +type StorageRC struct { + mu sync.Mutex + refCnt int + tp []*types.FieldType + chkSize int + + begCh chan struct{} + done bool + iter int + + rc *chunk.RowContainer +} + +// NewStorageRC create a new StorageRC. +func NewStorageRC(tp []*types.FieldType, chkSize int) *StorageRC { + return &StorageRC{tp: tp, chkSize: chkSize} +} + +// OpenAndRef impls Storage OpenAndRef interface. +func (s *StorageRC) OpenAndRef() (err error) { + if !s.valid() { + s.rc = chunk.NewRowContainer(s.tp, s.chkSize) + s.refCnt = 1 + s.begCh = make(chan struct{}) + s.iter = 0 + } else { + s.refCnt += 1 + } + return nil +} + +// DerefAndClose impls Storage DerefAndClose interface. +func (s *StorageRC) DerefAndClose() (err error) { + if !s.valid() { + return errors.New("Storage not opend yet") + } + s.refCnt -= 1 + if s.refCnt < 0 { + return errors.New("Storage ref count is less than zero") + } else if s.refCnt == 0 { + // TODO: unreg memtracker + if err = s.rc.Close(); err != nil { + return err + } + if err = s.resetAll(); err != nil { + return err + } + } + return nil +} + +// SwapData impls Storage Swap interface. +func (s *StorageRC) SwapData(other Storage) (err error) { + otherRC, ok := other.(*StorageRC) + if !ok { + return errors.New("cannot swap if underlying storages are different") + } + s.tp, otherRC.tp = otherRC.tp, s.tp + s.chkSize, otherRC.chkSize = otherRC.chkSize, s.chkSize + + s.rc, otherRC.rc = otherRC.rc, s.rc + return nil +} + +// Reopen impls Storage Reopen interface. +func (s *StorageRC) Reopen() (err error) { + if err = s.rc.Reset(); err != nil { + return err + } + s.iter = 0 + s.begCh = make(chan struct{}) + s.done = false + // Create a new RowContainer. + // Because some meta infos in old RowContainer are not resetted. + // Such as memTracker/actionSpill etc. So we just use a new one. + s.rc = chunk.NewRowContainer(s.tp, s.chkSize) + return nil +} + +// Add impls Storage Add interface. +func (s *StorageRC) Add(chk *chunk.Chunk) (err error) { + if !s.valid() { + return errors.New("Storage is not valid") + } + if chk.NumRows() == 0 { + return nil + } + return s.rc.Add(chk) +} + +// GetChunk impls Storage GetChunk interface. +func (s *StorageRC) GetChunk(chkIdx int) (*chunk.Chunk, error) { + if !s.valid() { + return nil, errors.New("Storage is not valid") + } + return s.rc.GetChunk(chkIdx) +} + +// GetRow impls Storage GetRow interface. +func (s *StorageRC) GetRow(ptr chunk.RowPtr) (chunk.Row, error) { + if !s.valid() { + return chunk.Row{}, errors.New("Storage is not valid") + } + return s.rc.GetRow(ptr) +} + +// NumChunks impls Storage NumChunks interface. +func (s *StorageRC) NumChunks() int { + return s.rc.NumChunks() +} + +// Lock impls Storage Lock interface. +func (s *StorageRC) Lock() { + s.mu.Lock() +} + +// Unlock impls Storage Unlock interface. +func (s *StorageRC) Unlock() { + s.mu.Unlock() +} + +// Done impls Storage Done interface. +func (s *StorageRC) Done() bool { + return s.done +} + +// SetDone impls Storage SetDone interface. +func (s *StorageRC) SetDone() { + s.done = true +} + +// SetIter impls Storage SetIter interface. +func (s *StorageRC) SetIter(iter int) { + s.iter = iter +} + +// GetIter impls Storage GetIter interface. +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() +} + +// GetDiskTracker impls Storage GetDiskTracker interface. +func (s *StorageRC) GetDiskTracker() *memory.Tracker { + return s.rc.GetDiskTracker() +} + +// ActionSpill impls Storage ActionSpill interface. +func (s *StorageRC) ActionSpill() memory.ActionOnExceed { + return s.rc.ActionSpill() +} + +// ActionSpillForTest is for test. +func (s *StorageRC) ActionSpillForTest() *chunk.SpillDiskAction { + return s.rc.ActionSpillForTest() +} + +func (s *StorageRC) resetAll() error { + s.refCnt = -1 + s.begCh = nil + s.done = false + s.iter = 0 + if err := s.rc.Reset(); err != nil { + return err + } + s.rc = nil + return nil +} + +func (s *StorageRC) valid() bool { + return s.refCnt > 0 && s.rc != nil +} diff --git a/util/cteutil/storage_test.go b/util/cteutil/storage_test.go new file mode 100644 index 0000000000000..89376fc8580b8 --- /dev/null +++ b/util/cteutil/storage_test.go @@ -0,0 +1,262 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package cteutil + +import ( + "reflect" + "strconv" + "testing" + + "github.com/pingcap/check" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" +) + +func TestT(t *testing.T) { + check.TestingT(t) +} + +var _ = check.Suite(&StorageRCTestSuite{}) + +type StorageRCTestSuite struct{} + +func (test *StorageRCTestSuite) TestStorageBasic(c *check.C) { + fields := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} + chkSize := 1 + storage := NewStorageRC(fields, chkSize) + c.Assert(storage, check.NotNil) + + // Close before open. + err := storage.DerefAndClose() + c.Assert(err, check.NotNil) + + err = storage.OpenAndRef() + c.Assert(err, check.IsNil) + + err = storage.DerefAndClose() + c.Assert(err, check.IsNil) + + err = storage.DerefAndClose() + c.Assert(err, check.NotNil) + + // Open twice. + err = storage.OpenAndRef() + c.Assert(err, check.IsNil) + err = storage.OpenAndRef() + c.Assert(err, check.IsNil) + err = storage.DerefAndClose() + c.Assert(err, check.IsNil) + err = storage.DerefAndClose() + c.Assert(err, check.IsNil) + err = storage.DerefAndClose() + c.Assert(err, check.NotNil) +} + +func (test *StorageRCTestSuite) TestOpenAndClose(c *check.C) { + fields := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} + chkSize := 1 + storage := NewStorageRC(fields, chkSize) + + for i := 0; i < 10; i++ { + err := storage.OpenAndRef() + c.Assert(err, check.IsNil) + } + + for i := 0; i < 9; i++ { + err := storage.DerefAndClose() + c.Assert(err, check.IsNil) + } + err := storage.DerefAndClose() + c.Assert(err, check.IsNil) + + err = storage.DerefAndClose() + c.Assert(err, check.NotNil) +} + +func (test *StorageRCTestSuite) TestAddAndGetChunk(c *check.C) { + fields := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} + chkSize := 10 + + storage := NewStorageRC(fields, chkSize) + + inChk := chunk.NewChunkWithCapacity(fields, chkSize) + for i := 0; i < chkSize; i++ { + inChk.AppendInt64(0, int64(i)) + } + + err := storage.Add(inChk) + c.Assert(err, check.NotNil) + + err = storage.OpenAndRef() + c.Assert(err, check.IsNil) + + err = storage.Add(inChk) + c.Assert(err, check.IsNil) + + outChk, err1 := storage.GetChunk(0) + c.Assert(err1, check.IsNil) + + in64s := inChk.Column(0).Int64s() + out64s := outChk.Column(0).Int64s() + + c.Assert(reflect.DeepEqual(in64s, out64s), check.IsTrue) +} + +func (test *StorageRCTestSuite) TestSpillToDisk(c *check.C) { + fields := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} + chkSize := 10 + storage := NewStorageRC(fields, chkSize) + var tmp interface{} = storage + + inChk := chunk.NewChunkWithCapacity(fields, chkSize) + for i := 0; i < chkSize; i++ { + inChk.AppendInt64(0, int64(i)) + } + + err := storage.OpenAndRef() + c.Assert(err, check.IsNil) + + memTracker := storage.GetMemTracker() + memTracker.SetBytesLimit(inChk.MemoryUsage() + 1) + action := tmp.(*StorageRC).ActionSpillForTest() + memTracker.FallbackOldAndSetNewAction(action) + diskTracker := storage.GetDiskTracker() + + // All data is in memory. + err = storage.Add(inChk) + c.Assert(err, check.IsNil) + outChk, err1 := storage.GetChunk(0) + c.Assert(err1, check.IsNil) + in64s := inChk.Column(0).Int64s() + out64s := outChk.Column(0).Int64s() + c.Assert(reflect.DeepEqual(in64s, out64s), check.IsTrue) + + c.Assert(memTracker.BytesConsumed(), check.Greater, int64(0)) + c.Assert(memTracker.MaxConsumed(), check.Greater, int64(0)) + c.Assert(diskTracker.BytesConsumed(), check.Equals, int64(0)) + c.Assert(diskTracker.MaxConsumed(), check.Equals, int64(0)) + + // Add again, and will trigger spill to disk. + err = storage.Add(inChk) + c.Assert(err, check.IsNil) + action.WaitForTest() + c.Assert(memTracker.BytesConsumed(), check.Equals, int64(0)) + c.Assert(memTracker.MaxConsumed(), check.Greater, int64(0)) + c.Assert(diskTracker.BytesConsumed(), check.Greater, int64(0)) + c.Assert(diskTracker.MaxConsumed(), check.Greater, int64(0)) + + outChk, err = storage.GetChunk(0) + c.Assert(err, check.IsNil) + out64s = outChk.Column(0).Int64s() + c.Assert(reflect.DeepEqual(in64s, out64s), check.IsTrue) + + outChk, err = storage.GetChunk(1) + c.Assert(err, check.IsNil) + out64s = outChk.Column(0).Int64s() + c.Assert(reflect.DeepEqual(in64s, out64s), check.IsTrue) +} + +func (test *StorageRCTestSuite) TestReopen(c *check.C) { + fields := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} + chkSize := 10 + storage := NewStorageRC(fields, chkSize) + err := storage.OpenAndRef() + c.Assert(err, check.IsNil) + + inChk := chunk.NewChunkWithCapacity(fields, chkSize) + for i := 0; i < chkSize; i++ { + inChk.AppendInt64(0, int64(i)) + } + err = storage.Add(inChk) + c.Assert(err, check.IsNil) + c.Assert(storage.NumChunks(), check.Equals, 1) + + err = storage.Reopen() + c.Assert(err, check.IsNil) + c.Assert(storage.NumChunks(), check.Equals, 0) + + err = storage.Add(inChk) + c.Assert(err, check.IsNil) + c.Assert(storage.NumChunks(), check.Equals, 1) + + outChk, err := storage.GetChunk(0) + c.Assert(err, check.IsNil) + in64s := inChk.Column(0).Int64s() + out64s := outChk.Column(0).Int64s() + c.Assert(reflect.DeepEqual(in64s, out64s), check.IsTrue) + + // Reopen multiple times. + for i := 0; i < 100; i++ { + err = storage.Reopen() + c.Assert(err, check.IsNil) + } + err = storage.Add(inChk) + c.Assert(err, check.IsNil) + c.Assert(storage.NumChunks(), check.Equals, 1) + + outChk, err = storage.GetChunk(0) + c.Assert(err, check.IsNil) + in64s = inChk.Column(0).Int64s() + out64s = outChk.Column(0).Int64s() + c.Assert(reflect.DeepEqual(in64s, out64s), check.IsTrue) +} + +func (test *StorageRCTestSuite) TestSwapData(c *check.C) { + tp1 := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} + chkSize := 10 + storage1 := NewStorageRC(tp1, chkSize) + err := storage1.OpenAndRef() + c.Assert(err, check.IsNil) + inChk1 := chunk.NewChunkWithCapacity(tp1, chkSize) + for i := 0; i < chkSize; i++ { + inChk1.AppendInt64(0, int64(i)) + } + in1 := inChk1.Column(0).Int64s() + err = storage1.Add(inChk1) + c.Assert(err, check.IsNil) + + tp2 := []*types.FieldType{types.NewFieldType(mysql.TypeVarString)} + storage2 := NewStorageRC(tp2, chkSize) + err = storage2.OpenAndRef() + c.Assert(err, check.IsNil) + + inChk2 := chunk.NewChunkWithCapacity(tp2, chkSize) + for i := 0; i < chkSize; i++ { + inChk2.AppendString(0, strconv.FormatInt(int64(i), 10)) + } + var in2 []string + for i := 0; i < inChk2.NumRows(); i++ { + in2 = append(in2, inChk2.Column(0).GetString(i)) + } + err = storage2.Add(inChk2) + c.Assert(err, check.IsNil) + + err = storage1.SwapData(storage2) + c.Assert(err, check.IsNil) + + outChk1, err := storage1.GetChunk(0) + c.Assert(err, check.IsNil) + outChk2, err := storage2.GetChunk(0) + c.Assert(err, check.IsNil) + + var out1 []string + for i := 0; i < outChk1.NumRows(); i++ { + out1 = append(out1, outChk1.Column(0).GetString(i)) + } + out2 := outChk2.Column(0).Int64s() + + c.Assert(reflect.DeepEqual(in1, out2), check.IsTrue) + c.Assert(reflect.DeepEqual(in2, out1), check.IsTrue) +} From b7c93e8952d3dfb9f3ecc1f748741f3931d42bda Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Mon, 24 May 2021 18:05:34 +0800 Subject: [PATCH 151/343] variable: remove radix join variable (#24566) --- sessionctx/variable/session.go | 10 ---------- sessionctx/variable/sysvar.go | 4 ---- sessionctx/variable/tidb_vars.go | 5 ----- sessionctx/variable/varsutil_test.go | 1 - 4 files changed, 20 deletions(-) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index d38881a8ddcfa..0af4769c54930 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -28,7 +28,6 @@ import ( "sync/atomic" "time" - "github.com/klauspost/cpuid" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" @@ -632,13 +631,6 @@ type SessionVars struct { writeStmtBufs WriteStmtBufs - // L2CacheSize indicates the size of CPU L2 cache, using byte as unit. - L2CacheSize int - - // EnableRadixJoin indicates whether to use radix hash join to execute - // HashJoin. - EnableRadixJoin bool - // ConstraintCheckInPlace indicates whether to check the constraint when the SQL executing. ConstraintCheckInPlace bool @@ -1019,9 +1011,7 @@ func NewSessionVars() *SessionVars { MemoryFactor: DefOptMemoryFactor, DiskFactor: DefOptDiskFactor, ConcurrencyFactor: DefOptConcurrencyFactor, - EnableRadixJoin: false, EnableVectorizedExpression: DefEnableVectorizedExpression, - L2CacheSize: cpuid.CPU.Cache.L2, CommandValue: uint32(mysql.ComSleep), TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 06321e831a91a..fadfda444e1ac 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1244,10 +1244,6 @@ var defaultSysVars = []*SysVar{ atomic.StoreInt32(&ForcePriority, int32(mysql.Str2Priority(val))) return nil }}, - {Scope: ScopeSession, Name: TiDBEnableRadixJoin, Value: BoolToOnOff(DefTiDBUseRadixJoin), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { - s.EnableRadixJoin = TiDBOptOn(val) - return nil - }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptJoinReorderThreshold, Value: strconv.Itoa(DefTiDBOptJoinReorderThreshold), skipInit: true, Type: TypeUnsigned, MinValue: 0, MaxValue: 63, SetSession: func(s *SessionVars, val string) error { s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 7fdccecb5a97e..3fb99204ee2ea 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -393,10 +393,6 @@ const ( // It can be "NO_PRIORITY", "LOW_PRIORITY", "HIGH_PRIORITY", "DELAYED" TiDBForcePriority = "tidb_force_priority" - // tidb_enable_radix_join indicates to use radix hash join algorithm to execute - // HashJoin. - TiDBEnableRadixJoin = "tidb_enable_radix_join" - // tidb_constraint_check_in_place indicates to check the constraint when the SQL executing. // It could hurt the performance of bulking insert when it is ON. TiDBConstraintCheckInPlace = "tidb_constraint_check_in_place" @@ -632,7 +628,6 @@ const ( DefTiDBMergeJoinConcurrency = 1 // disable optimization by default DefTiDBStreamAggConcurrency = 1 DefTiDBForcePriority = mysql.NoPriority - DefTiDBUseRadixJoin = false DefEnableWindowFunction = true DefEnableStrictDoubleTypeCheck = true DefEnableVectorizedExpression = true diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 1d8e629b7df4a..5843e4b91c77d 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -95,7 +95,6 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.MemQuotaIndexLookupReader, Equals, int64(DefTiDBMemQuotaIndexLookupReader)) c.Assert(vars.MemQuotaIndexLookupJoin, Equals, int64(DefTiDBMemQuotaIndexLookupJoin)) c.Assert(vars.MemQuotaApplyCache, Equals, int64(DefTiDBMemQuotaApplyCache)) - c.Assert(vars.EnableRadixJoin, Equals, DefTiDBUseRadixJoin) c.Assert(vars.AllowWriteRowID, Equals, DefOptWriteRowID) c.Assert(vars.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) c.Assert(vars.EnableFastAnalyze, Equals, DefTiDBUseFastAnalyze) From bfdad7c75210a30c796d82ccd882f381c2c20478 Mon Sep 17 00:00:00 2001 From: xhe Date: Mon, 24 May 2021 18:27:34 +0800 Subject: [PATCH 152/343] *: milisecond test and infoschema fix (#24820) --- distsql/request_builder.go | 10 ++--- distsql/request_builder_test.go | 4 +- executor/adapter.go | 2 +- executor/admin.go | 3 ++ executor/analyze.go | 4 +- executor/analyze_test.go | 12 +++--- executor/builder.go | 2 +- executor/compiler.go | 2 +- executor/coprocessor.go | 2 +- executor/distsql.go | 11 +----- executor/executor_test.go | 4 +- executor/grant.go | 2 +- executor/index_merge_reader.go | 3 +- executor/infoschema_reader.go | 16 ++++---- executor/load_stats.go | 2 +- executor/metrics_reader_test.go | 2 +- executor/partition_table_test.go | 2 +- executor/point_get.go | 2 +- executor/prepared.go | 2 +- executor/simple.go | 3 +- executor/stale_txn_test.go | 34 +++++++---------- executor/table_reader.go | 6 +-- expression/builtin_info.go | 6 +-- infoschema/tables.go | 2 +- planner/core/cacheable_checker_test.go | 2 +- planner/core/explain.go | 8 ++-- planner/core/expression_rewriter.go | 2 +- planner/core/integration_test.go | 2 +- planner/core/point_get_plan.go | 8 ++-- planner/core/prepare_test.go | 2 +- session/session.go | 52 +++++++++++++++++--------- sessionctx/binloginfo/binloginfo.go | 2 +- sessionctx/context.go | 10 +++++ sessionctx/variable/session.go | 28 +------------- statistics/handle/ddl.go | 2 +- statistics/handle/update.go | 2 +- util/mock/context.go | 14 +++++++ 37 files changed, 135 insertions(+), 137 deletions(-) diff --git a/distsql/request_builder.go b/distsql/request_builder.go index aced1a71aaa7b..42a098cd05440 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -229,10 +229,6 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req builder.Request.TaskID = sv.StmtCtx.TaskID builder.Request.Priority = builder.getKVPriority(sv) builder.Request.ReplicaRead = sv.GetReplicaRead() - // in tests, it may be null - if is, ok := sv.GetInfoSchema().(infoschema.InfoSchema); ok { - builder.Request.SchemaVar = is.SchemaMetaVersion() - } builder.txnScope = sv.TxnCtx.TxnScope builder.IsStaleness = sv.TxnCtx.IsStaleness if builder.IsStaleness && builder.txnScope != kv.GlobalTxnScope { @@ -268,11 +264,13 @@ func (builder *RequestBuilder) SetTiDBServerID(serverID uint64) *RequestBuilder // SetFromInfoSchema sets the following fields from infoSchema: // "bundles" -func (builder *RequestBuilder) SetFromInfoSchema(is infoschema.InfoSchema) *RequestBuilder { - if is == nil { +func (builder *RequestBuilder) SetFromInfoSchema(pis interface{}) *RequestBuilder { + is, ok := pis.(infoschema.InfoSchema) + if !ok { return builder } builder.is = is + builder.Request.SchemaVar = is.SchemaMetaVersion() return builder } diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index ed921c391d6a3..b9837f77c1ff1 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -19,7 +19,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -612,7 +611,6 @@ func (s *testSuite) TestRequestBuilder7(c *C) { func (s *testSuite) TestRequestBuilder8(c *C) { sv := variable.NewSessionVars() - sv.SnapshotInfoschema = infoschema.MockInfoSchemaWithSchemaVer(nil, 10000) actual, err := (&RequestBuilder{}). SetFromSessionVars(sv). Build() @@ -625,7 +623,7 @@ func (s *testSuite) TestRequestBuilder8(c *C) { IsolationLevel: 0, Priority: 0, MemTracker: (*memory.Tracker)(nil), - SchemaVar: 10000, + SchemaVar: 0, } c.Assert(actual, DeepEquals, expect) } diff --git a/executor/adapter.go b/executor/adapter.go index 784696996cb94..15a823dd0f724 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -268,7 +268,7 @@ func (a *ExecStmt) IsReadOnly(vars *variable.SessionVars) bool { // RebuildPlan rebuilds current execute statement plan. // It returns the current information schema version that 'a' is using. func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { - is := a.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := a.Ctx.GetInfoSchema().(infoschema.InfoSchema) a.InfoSchema = is if err := plannercore.Preprocess(a.Ctx, a.StmtNode, is, plannercore.InTxnRetry); err != nil { return 0, err diff --git a/executor/admin.go b/executor/admin.go index be46e39a1d2a3..df29c6a62aa8b 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -126,6 +126,7 @@ func (e *CheckIndexRangeExec) Open(ctx context.Context) error { SetStartTS(txn.StartTS()). SetKeepOrder(true). SetFromSessionVars(e.ctx.GetSessionVars()). + SetFromInfoSchema(e.ctx.GetInfoSchema()). Build() if err != nil { return err @@ -272,6 +273,7 @@ func (e *RecoverIndexExec) buildTableScan(ctx context.Context, txn kv.Transactio SetStartTS(txn.StartTS()). SetKeepOrder(true). SetFromSessionVars(e.ctx.GetSessionVars()). + SetFromInfoSchema(e.ctx.GetInfoSchema()). Build() if err != nil { return nil, err @@ -735,6 +737,7 @@ func (e *CleanupIndexExec) buildIndexScan(ctx context.Context, txn kv.Transactio SetStartTS(txn.StartTS()). SetKeepOrder(true). SetFromSessionVars(e.ctx.GetSessionVars()). + SetFromInfoSchema(e.ctx.GetInfoSchema()). Build() if err != nil { return nil, err diff --git a/executor/analyze.go b/executor/analyze.go index 633f6e4dcc69e..ed7463b2ac46f 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -186,7 +186,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } if needGlobalStats { for globalStatsID, info := range globalStatsMap { - globalStats, err := statsHandle.MergePartitionStats2GlobalStatsByTableID(e.ctx, e.opts, e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), globalStatsID.tableID, info.isIndex, info.idxID) + globalStats, err := statsHandle.MergePartitionStats2GlobalStatsByTableID(e.ctx, e.opts, e.ctx.GetInfoSchema().(infoschema.InfoSchema), globalStatsID.tableID, info.isIndex, info.idxID) if err != nil { if types.ErrPartitionStatsMissing.Equal(err) { // When we find some partition-level stats are missing, we need to report warning. @@ -205,7 +205,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } } } - return statsHandle.Update(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)) + return statsHandle.Update(e.ctx.GetInfoSchema().(infoschema.InfoSchema)) } func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) { diff --git a/executor/analyze_test.go b/executor/analyze_test.go index b527875c1498b..f5b822b51f45f 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -69,7 +69,7 @@ PARTITION BY RANGE ( a ) ( } tk.MustExec("analyze table t") - is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pi := table.Meta().GetPartitionInfo() @@ -96,7 +96,7 @@ PARTITION BY RANGE ( a ) ( tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "hello")`, i, i)) } tk.MustExec("alter table t analyze partition p0") - is = tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is = tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pi = table.Meta().GetPartitionInfo() @@ -176,7 +176,7 @@ func (s *testSuite1) TestAnalyzeParameters(c *C) { tk.MustExec("set @@tidb_enable_fast_analyze = 1") tk.MustExec("analyze table t with 30 samples") - is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -227,7 +227,7 @@ func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) { tk.MustExec(fmt.Sprintf("insert into t values ('%s')", value)) tk.MustExec("analyze table t") - is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -259,7 +259,7 @@ func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("analyze table t with 10 cmsketch width") - is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() @@ -435,7 +435,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { } tk.MustExec("analyze table t with 5 buckets, 6 samples") - is := tk.Se.(sessionctx.Context).GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() diff --git a/executor/builder.go b/executor/builder.go index c8a128e505562..bf1ad7e46c9f0 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3462,7 +3462,7 @@ func (builder *dataReaderBuilder) buildTableReaderBase(ctx context.Context, e *T SetKeepOrder(e.keepOrder). SetStreaming(e.streaming). SetFromSessionVars(e.ctx.GetSessionVars()). - SetFromInfoSchema(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)). + SetFromInfoSchema(e.ctx.GetInfoSchema()). Build() if err != nil { return nil, err diff --git a/executor/compiler.go b/executor/compiler.go index bb00bfe14602d..5a658878ca747 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -53,7 +53,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm ctx = opentracing.ContextWithSpan(ctx, span1) } - infoSchema := c.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + infoSchema := c.Ctx.GetInfoSchema().(infoschema.InfoSchema) if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil { return nil, err } diff --git a/executor/coprocessor.go b/executor/coprocessor.go index 490b981add461..f46ba8774c2be 100644 --- a/executor/coprocessor.go +++ b/executor/coprocessor.go @@ -159,7 +159,7 @@ func (h *CoprocessorDAGHandler) buildDAGExecutor(req *coprocessor.Request) (Exec return nil, errors.Trace(err) } h.dagReq = dagReq - is := h.sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := h.sctx.GetInfoSchema().(infoschema.InfoSchema) // Build physical plan. bp := core.NewPBPlanBuilder(h.sctx, is, req.Ranges) plan, err := bp.Build(dagReq.Executors) diff --git a/executor/distsql.go b/executor/distsql.go index 5e83a63d8f9b1..23fa8bd4ce58e 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" @@ -278,11 +277,8 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) SetKeepOrder(e.keepOrder). SetStreaming(e.streaming). SetFromSessionVars(e.ctx.GetSessionVars()). + SetFromInfoSchema(e.ctx.GetInfoSchema()). SetMemTracker(e.memTracker) - // for tests, infoschema may be null - if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { - builder.SetFromInfoSchema(is) - } kvReq, err := builder.Build() if err != nil { e.feedback.Invalidate() @@ -530,11 +526,8 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, workCh chan< SetKeepOrder(e.keepOrder). SetStreaming(e.indexStreaming). SetFromSessionVars(e.ctx.GetSessionVars()). + SetFromInfoSchema(e.ctx.GetInfoSchema()). SetMemTracker(tracker) - // for tests, infoschema may be null - if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { - builder.SetFromInfoSchema(is) - } for partTblIdx, kvRange := range kvRanges { // check if executor is closed diff --git a/executor/executor_test.go b/executor/executor_test.go index a796a27326a70..6ed0309d75a42 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2346,7 +2346,7 @@ func (s *testSuiteP2) TestIsPointGet(c *C) { "select * from help_topic where help_topic_id=1": true, "select * from help_topic where help_category_id=1": false, } - infoSchema := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + infoSchema := ctx.GetInfoSchema().(infoschema.InfoSchema) for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") @@ -2378,7 +2378,7 @@ func (s *testSuiteP2) TestClusteredIndexIsPointGet(c *C) { "select * from t where a='x' and c='x'": true, "select * from t where a='x' and c='x' and b=1": false, } - infoSchema := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + infoSchema := ctx.GetInfoSchema().(infoschema.InfoSchema) for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") c.Check(err, IsNil) diff --git a/executor/grant.go b/executor/grant.go index 86620cc0124b1..54b5b66b70612 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -74,7 +74,7 @@ func (e *GrantExec) Next(ctx context.Context, req *chunk.Chunk) error { // Make sure the table exist. if e.Level.Level == ast.GrantLevelTable { dbNameStr := model.NewCIStr(dbName) - schema := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + schema := e.ctx.GetInfoSchema().(infoschema.InfoSchema) tbl, err := schema.TableByName(dbNameStr, model.NewCIStr(e.Level.TableName)) if err != nil { return err diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 84b4b810b059d..343ee4cb06105 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/distsql" - "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" @@ -245,7 +244,7 @@ func (e *IndexMergeReaderExecutor) startPartialIndexWorker(ctx context.Context, SetStreaming(e.partialStreamings[workID]). SetFromSessionVars(e.ctx.GetSessionVars()). SetMemTracker(e.memTracker). - SetFromInfoSchema(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)) + SetFromInfoSchema(e.ctx.GetInfoSchema()) worker := &partialIndexWorker{ stats: e.stats, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 491eb3a3fe26f..cfc29c5ccc513 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -80,7 +80,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex // Cache the ret full rows in schemataRetriever if !e.initialized { - is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := sctx.GetInfoSchema().(infoschema.InfoSchema) dbs := is.AllSchemas() sort.Sort(infoschema.SchemasSorter(dbs)) var err error @@ -304,7 +304,7 @@ func (c *statsCache) get(ctx sessionctx.Context) (map[int64]uint64, map[tableHis } func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *model.TableInfo) (int64, error) { - is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := ctx.GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(schema.Name, tblInfo.Name) if err != nil { return 0, err @@ -592,7 +592,7 @@ func (e *hugeMemTableRetriever) setDataForColumns(ctx context.Context, sctx sess } func (e *hugeMemTableRetriever) dataForColumnsInTable(ctx context.Context, sctx sessionctx.Context, schema *model.DBInfo, tbl *model.TableInfo) { - if err := tryFillViewColumnType(ctx, sctx, sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), schema.Name, tbl); err != nil { + if err := tryFillViewColumnType(ctx, sctx, sctx.GetInfoSchema().(infoschema.InfoSchema), schema.Name, tbl); err != nil { sctx.GetSessionVars().StmtCtx.AppendWarning(err) return } @@ -1339,7 +1339,7 @@ func (e *memtableRetriever) setDataForTiKVRegionStatus(ctx sessionctx.Context) e if err != nil { return err } - allSchemas := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema).AllSchemas() + allSchemas := ctx.GetInfoSchema().(infoschema.InfoSchema).AllSchemas() tableInfos := tikvHelper.GetRegionsTableInfo(regionsInfo, allSchemas) for _, region := range regionsInfo.Regions { tableList := tableInfos[region.ID] @@ -1451,7 +1451,7 @@ func (e *memtableRetriever) setDataForTiDBHotRegions(ctx sessionctx.Context) err if !ok { return errors.New("Information about hot region can be gotten only when the storage is TiKV") } - allSchemas := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema).AllSchemas() + allSchemas := ctx.GetInfoSchema().(infoschema.InfoSchema).AllSchemas() tikvHelper := &helper.Helper{ Store: tikvStore, RegionCache: tikvStore.GetRegionCache(), @@ -1600,7 +1600,7 @@ type initialTable struct { } func (e *tableStorageStatsRetriever) initialize(sctx sessionctx.Context) error { - is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := sctx.GetInfoSchema().(infoschema.InfoSchema) var databases []string schemas := e.extractor.TableSchema tables := e.extractor.TableName @@ -1891,7 +1891,7 @@ func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context, func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) error { checker := privilege.GetPrivilegeManager(ctx) - is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := ctx.GetInfoSchema().(infoschema.InfoSchema) var rows [][]types.Datum for _, bundle := range is.RuleBundles() { id, err := placement.ObjectIDFromGroupID(bundle.ID) @@ -2099,7 +2099,7 @@ func (e *hugeMemTableRetriever) retrieve(ctx context.Context, sctx sessionctx.Co } if !e.initialized { - is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := sctx.GetInfoSchema().(infoschema.InfoSchema) dbs := is.AllSchemas() sort.Sort(infoschema.SchemasSorter(dbs)) e.dbs = dbs diff --git a/executor/load_stats.go b/executor/load_stats.go index 984f649e1291f..473a14150e711 100644 --- a/executor/load_stats.go +++ b/executor/load_stats.go @@ -86,5 +86,5 @@ func (e *LoadStatsInfo) Update(data []byte) error { if h == nil { return errors.New("Load Stats: handle is nil") } - return h.LoadStatsFromJSON(e.Ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), jsonTbl) + return h.LoadStatsFromJSON(e.Ctx.GetInfoSchema().(infoschema.InfoSchema), jsonTbl) } diff --git a/executor/metrics_reader_test.go b/executor/metrics_reader_test.go index 662c3b917ce9c..51bdc560ed6d0 100644 --- a/executor/metrics_reader_test.go +++ b/executor/metrics_reader_test.go @@ -62,7 +62,7 @@ func (s *testSuite7) TestStmtLabel(c *C) { for _, tt := range tests { stmtNode, err := parser.New().ParseOneStmt(tt.sql, "", "") c.Check(err, IsNil) - is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) err = plannercore.Preprocess(tk.Se.(sessionctx.Context), stmtNode, is) c.Assert(err, IsNil) _, _, err = planner.Optimize(context.TODO(), tk.Se, stmtNode, is) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index b5aeebd9551ca..b9823a32a647e 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -284,7 +284,7 @@ func (s *partitionTableSuite) TestPartitionInfoDisable(c *C) { PARTITION p202010 VALUES LESS THAN ("2020-11-01"), PARTITION p202011 VALUES LESS THAN ("2020-12-01") )`) - is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t_info_null")) c.Assert(err, IsNil) diff --git a/executor/point_get.go b/executor/point_get.go index 76132623e621f..bc476794888fe 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -391,7 +391,7 @@ func (e *PointGetExecutor) verifyTxnScope() error { var tblID int64 var tblName string var partName string - is := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := e.ctx.GetInfoSchema().(infoschema.InfoSchema) if e.partInfo != nil { tblID = e.partInfo.ID tblInfo, _, partInfo := is.FindTableByPartitionID(tblID) diff --git a/executor/prepared.go b/executor/prepared.go index 448ee3b7fdc66..50d555cff3d34 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -320,7 +320,7 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, return nil, false, false, err } execStmt.BinaryArgs = args - is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := sctx.GetInfoSchema().(infoschema.InfoSchema) execPlan, names, err := planner.Optimize(ctx, sctx, execStmt, is) if err != nil { return nil, false, false, err diff --git a/executor/simple.go b/executor/simple.go index 193949eda3fdd..9a0b4b919b1e3 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -1456,6 +1456,7 @@ func killRemoteConn(ctx context.Context, sctx sessionctx.Context, connID *util.G kvReq, err := builder. SetDAGRequest(dagReq). SetFromSessionVars(sctx.GetSessionVars()). + SetFromInfoSchema(sctx.GetInfoSchema()). SetStoreType(kv.TiDB). SetTiDBServerID(connID.ServerID). Build() @@ -1544,7 +1545,7 @@ func (e *SimpleExec) executeDropStats(s *ast.DropStatsStmt) (err error) { if err := h.DeleteTableStatsFromKV(statsIDs); err != nil { return err } - return h.Update(e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema)) + return h.Update(e.ctx.GetInfoSchema().(infoschema.InfoSchema)) } func (e *SimpleExec) autoNewTxn() bool { diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index dc5ddd0785208..a9336fbfac882 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -300,27 +300,21 @@ func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key);") - schemaVer1 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() - time.Sleep(time.Second) + // test exact + schemaVer1 := tk.Se.GetInfoSchema().SchemaMetaVersion() + time1 := time.Now() tk.MustExec("drop table if exists t") - schemaVer2 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() - // confirm schema changed - c.Assert(schemaVer1, Less, schemaVer2) - - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:01'`) - schemaVer3 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() - // got an old infoSchema - c.Assert(schemaVer3, Equals, schemaVer1) + c.Assert(schemaVer1, Less, tk.Se.GetInfoSchema().SchemaMetaVersion()) + tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) + c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) + tk.MustExec("commit") - schemaVer4 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() - time.Sleep(time.Second) + // test as of + schemaVer2 := tk.Se.GetInfoSchema().SchemaMetaVersion() + time2 := time.Now() tk.MustExec("create table t (id int primary key);") - schemaVer5 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() - // confirm schema changed - c.Assert(schemaVer4, Less, schemaVer5) - - tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW() - INTERVAL 1 SECOND`) - schemaVer6 := tk.Se.GetSessionVars().GetInfoSchema().SchemaMetaVersion() - // got an old infoSchema - c.Assert(schemaVer6, Equals, schemaVer4) + c.Assert(schemaVer2, Less, tk.Se.GetInfoSchema().SchemaMetaVersion()) + tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time2.Format("2006-1-2 15:04:05.000"))) + c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + tk.MustExec("commit") } diff --git a/executor/table_reader.go b/executor/table_reader.go index 1a76598fb2250..29d9e4e6908c5 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" @@ -228,13 +227,10 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra SetKeepOrder(e.keepOrder). SetStreaming(e.streaming). SetFromSessionVars(e.ctx.GetSessionVars()). + SetFromInfoSchema(e.ctx.GetInfoSchema()). SetMemTracker(e.memTracker). SetStoreType(e.storeType). SetAllowBatchCop(e.batchCop) - // infoschema maybe null for tests - if is, ok := e.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { - reqBuilder.SetFromInfoSchema(is) - } kvReq, err := reqBuilder.Build() if err != nil { return nil, err diff --git a/expression/builtin_info.go b/expression/builtin_info.go index fda57a884f1d8..7b439d6dce14d 100644 --- a/expression/builtin_info.go +++ b/expression/builtin_info.go @@ -847,7 +847,7 @@ func (b *builtinNextValSig) evalInt(row chunk.Row) (int64, bool, error) { db = b.ctx.GetSessionVars().CurrentDB } // Check the tableName valid. - sequence, err := b.ctx.GetSessionVars().GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) + sequence, err := b.ctx.GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) if err != nil { return 0, false, err } @@ -903,7 +903,7 @@ func (b *builtinLastValSig) evalInt(row chunk.Row) (int64, bool, error) { db = b.ctx.GetSessionVars().CurrentDB } // Check the tableName valid. - sequence, err := b.ctx.GetSessionVars().GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) + sequence, err := b.ctx.GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) if err != nil { return 0, false, err } @@ -953,7 +953,7 @@ func (b *builtinSetValSig) evalInt(row chunk.Row) (int64, bool, error) { db = b.ctx.GetSessionVars().CurrentDB } // Check the tableName valid. - sequence, err := b.ctx.GetSessionVars().GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) + sequence, err := b.ctx.GetInfoSchema().(util.SequenceSchema).SequenceByName(model.NewCIStr(db), model.NewCIStr(seq)) if err != nil { return 0, false, err } diff --git a/infoschema/tables.go b/infoschema/tables.go index 40451046fe8ec..fac85f799ed66 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1775,7 +1775,7 @@ func (s SchemasSorter) Less(i, j int) bool { } func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) (fullRows [][]types.Datum, err error) { - is := ctx.GetSessionVars().GetInfoSchema().(InfoSchema) + is := ctx.GetInfoSchema().(InfoSchema) dbs := is.AllSchemas() sort.Sort(SchemasSorter(dbs)) switch it.meta.Name.O { diff --git a/planner/core/cacheable_checker_test.go b/planner/core/cacheable_checker_test.go index fb9d05d528ec0..3becdce3d2cc1 100644 --- a/planner/core/cacheable_checker_test.go +++ b/planner/core/cacheable_checker_test.go @@ -42,7 +42,7 @@ func (s *testCacheableSuite) TestCacheable(c *C) { tk.MustExec("create table t2(a int, b int) partition by hash(a) partitions 11") tk.MustExec("create table t3(a int, b int)") tbl := &ast.TableName{Schema: model.NewCIStr("test"), Name: model.NewCIStr("t3")} - is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) // test non-SelectStmt/-InsertStmt/-DeleteStmt/-UpdateStmt/-SetOprStmt var stmt ast.Node = &ast.ShowStmt{} c.Assert(core.Cacheable(stmt, is), IsFalse) diff --git a/planner/core/explain.go b/planner/core/explain.go index 913b4a88b5dac..57334d172a0d6 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -306,7 +306,7 @@ func (p *PhysicalTableReader) accessObject(sctx sessionctx.Context) string { return "" } - is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := sctx.GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { return "partition table not found" + strconv.FormatInt(ts.Table.ID, 10) @@ -366,7 +366,7 @@ func (p *PhysicalIndexReader) accessObject(sctx sessionctx.Context) string { } var buffer bytes.Buffer - is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := sctx.GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { fmt.Fprintf(&buffer, "partition table not found: %d", ts.Table.ID) @@ -394,7 +394,7 @@ func (p *PhysicalIndexLookUpReader) accessObject(sctx sessionctx.Context) string } var buffer bytes.Buffer - is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := sctx.GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { fmt.Fprintf(&buffer, "partition table not found: %d", ts.Table.ID) @@ -417,7 +417,7 @@ func (p *PhysicalIndexMergeReader) accessObject(sctx sessionctx.Context) string return "" } - is := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := sctx.GetInfoSchema().(infoschema.InfoSchema) tmp, ok := is.TableByID(ts.Table.ID) if !ok { return "partition table not found" + strconv.FormatInt(ts.Table.ID, 10) diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index eb154d0201ecb..423aef3376a77 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -65,7 +65,7 @@ func evalAstExpr(sctx sessionctx.Context, expr ast.ExprNode) (types.Datum, error func rewriteAstExpr(sctx sessionctx.Context, expr ast.ExprNode, schema *expression.Schema, names types.NameSlice) (expression.Expression, error) { var is infoschema.InfoSchema // in tests, it may be null - if s, ok := sctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema); ok { + if s, ok := sctx.GetInfoSchema().(infoschema.InfoSchema); ok { is = s } b, savedBlockNames := NewPlanBuilder(sctx, is, &hint.BlockHintProcessor{}) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 62e5b032c86df..8c5c105922536 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -1219,7 +1219,7 @@ func (s *testIntegrationSuite) TestPartitionPruningForEQ(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a datetime, b int) partition by range(weekday(a)) (partition p0 values less than(10), partition p1 values less than (100))") - is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) pt := tbl.(table.PartitionedTable) diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 28e324b4459c2..ec8c84e7b207e 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -1017,7 +1017,7 @@ func checkFastPlanPrivilege(ctx sessionctx.Context, dbName, tableName string, ch }) } - infoSchema := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + infoSchema := ctx.GetInfoSchema().(infoschema.InfoSchema) return CheckTableLock(ctx, infoSchema, visitInfos) } @@ -1323,7 +1323,7 @@ func buildPointUpdatePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName VirtualAssignmentsOffset: len(orderedList), }.Init(ctx) updatePlan.names = pointPlan.OutputNames() - is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := ctx.GetInfoSchema().(infoschema.InfoSchema) t, _ := is.TableByID(tbl.ID) updatePlan.tblID2Table = map[int64]table.Table{ tbl.ID: t, @@ -1569,7 +1569,7 @@ func getPartitionColumnPos(idx *model.IndexInfo, partitionColName *ast.ColumnNam } func getPartitionExpr(ctx sessionctx.Context, tbl *model.TableInfo) *tables.PartitionExpr { - is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := ctx.GetInfoSchema().(infoschema.InfoSchema) table, ok := is.TableByID(tbl.ID) if !ok { return nil @@ -1591,7 +1591,7 @@ func getHashPartitionColumnName(ctx sessionctx.Context, tbl *model.TableInfo) *a if pi.Type != model.PartitionTypeHash { return nil } - is := ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := ctx.GetInfoSchema().(infoschema.InfoSchema) table, ok := is.TableByID(tbl.ID) if !ok { return nil diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index d6bfe69f82b39..7bb87c8736b74 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -201,7 +201,7 @@ func (s *testPlanSerialSuite) TestPrepareCacheDeferredFunction(c *C) { for i := 0; i < 2; i++ { stmt, err := s.ParseOneStmt(sql1, "", "") c.Check(err, IsNil) - is := tk.Se.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) builder, _ := core.NewPlanBuilder(tk.Se, is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Check(err, IsNil) diff --git a/session/session.go b/session/session.go index fcecd49a6353c..9ef43bef3d7ed 100644 --- a/session/session.go +++ b/session/session.go @@ -408,7 +408,7 @@ func (s *session) StoreIndexUsage(tblID int64, idxID int64, rowsSelected int64) // FieldList returns fields list of a table. func (s *session) FieldList(tableName string) ([]*ast.ResultField, error) { - is := s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := s.GetInfoSchema().(infoschema.InfoSchema) dbName := model.NewCIStr(s.GetSessionVars().CurrentDB) tName := model.NewCIStr(tableName) pm := privilege.GetPrivilegeManager(s) @@ -515,7 +515,7 @@ func (s *session) doCommit(ctx context.Context) error { physicalTableIDs = append(physicalTableIDs, id) } // Set this option for 2 phase commit to validate schema lease. - s.txn.SetOption(kv.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.sessionVars.GetInfoSchema().SchemaMetaVersion(), physicalTableIDs)) + s.txn.SetOption(kv.SchemaChecker, domain.NewSchemaChecker(domain.GetDomain(s), s.GetInfoSchema().SchemaMetaVersion(), physicalTableIDs)) s.txn.SetOption(kv.InfoSchema, s.sessionVars.TxnCtx.InfoSchema) s.txn.SetOption(kv.CommitHook, func(info string, _ error) { s.sessionVars.LastTxnInfo = info }) if s.GetSessionVars().EnableAmendPessimisticTxn { @@ -1523,12 +1523,12 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex s.currentPlan = stmt.Plan // Execute the physical plan. - logStmt(stmt, s.sessionVars) + logStmt(stmt, s) recordSet, err := runStmt(ctx, s, stmt) if err != nil { if !kv.ErrKeyExists.Equal(err) { logutil.Logger(ctx).Warn("run statement failed", - zap.Int64("schemaVersion", s.sessionVars.GetInfoSchema().SchemaMetaVersion()), + zap.Int64("schemaVersion", s.GetInfoSchema().SchemaMetaVersion()), zap.Error(err), zap.String("session", s.String())) } @@ -1703,7 +1703,7 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields // So we have to call PrepareTxnCtx here. s.PrepareTxnCtx(ctx) s.PrepareTSFuture(ctx) - prepareExec := executor.NewPrepareExec(s, s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema), sql) + prepareExec := executor.NewPrepareExec(s, s.GetInfoSchema().(infoschema.InfoSchema), sql) err = prepareExec.Next(ctx, nil) if err != nil { return @@ -1731,7 +1731,7 @@ func (s *session) preparedStmtExec(ctx context.Context, } } sessionExecuteCompileDurationGeneral.Observe(time.Since(s.sessionVars.StartTime).Seconds()) - logQuery(st.OriginText(), s.sessionVars) + logQuery(st.OriginText(), s) return runStmt(ctx, s, st) } @@ -1744,7 +1744,7 @@ func (s *session) cachedPlanExec(ctx context.Context, if prepareStmt.ForUpdateRead { is = domain.GetDomain(s).InfoSchema() } else { - is = s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is = s.GetInfoSchema().(infoschema.InfoSchema) } execAst := &ast.ExecuteStmt{ExecID: stmtID} if err := executor.ResetContextOfStmt(s, execAst); err != nil { @@ -1774,7 +1774,7 @@ func (s *session) cachedPlanExec(ctx context.Context, stmtCtx.OriginalSQL = stmt.Text stmtCtx.InitSQLDigest(prepareStmt.NormalizedSQL, prepareStmt.SQLDigest) stmtCtx.SetPlanDigest(prepareStmt.NormalizedPlan, prepareStmt.PlanDigest) - logQuery(stmt.GetTextToLog(), s.sessionVars) + logQuery(stmt.GetTextToLog(), s) if !s.isInternal() && config.GetGlobalConfig().EnableTelemetry { telemetry.CurrentExecuteCount.Inc() @@ -1824,7 +1824,7 @@ func (s *session) IsCachedExecOk(ctx context.Context, preparedStmt *plannercore. return false, nil } // check schema version - is := s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := s.GetInfoSchema().(infoschema.InfoSchema) if prepared.SchemaVersion != is.SchemaMetaVersion() { prepared.CachedPlan = nil return false, nil @@ -1992,9 +1992,8 @@ func (s *session) checkBeforeNewTxn(ctx context.Context) error { if err != nil { return err } - vars := s.GetSessionVars() logutil.Logger(ctx).Info("Try to create a new txn inside a transaction auto commit", - zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), + zap.Int64("schemaVersion", s.GetInfoSchema().SchemaMetaVersion()), zap.Uint64("txnStartTS", txnStartTS), zap.String("txnScope", txnScope)) } @@ -2732,14 +2731,15 @@ func (s *session) ShowProcess() *util.ProcessInfo { // logStmt logs some crucial SQL including: CREATE USER/GRANT PRIVILEGE/CHANGE PASSWORD/DDL etc and normal SQL // if variable.ProcessGeneralLog is set. -func logStmt(execStmt *executor.ExecStmt, vars *variable.SessionVars) { +func logStmt(execStmt *executor.ExecStmt, s *session) { + vars := s.GetSessionVars() switch stmt := execStmt.StmtNode.(type) { case *ast.CreateUserStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.SetPwdStmt, *ast.GrantStmt, *ast.RevokeStmt, *ast.AlterTableStmt, *ast.CreateDatabaseStmt, *ast.CreateIndexStmt, *ast.CreateTableStmt, *ast.DropDatabaseStmt, *ast.DropIndexStmt, *ast.DropTableStmt, *ast.RenameTableStmt, *ast.TruncateTableStmt, *ast.RenameUserStmt: user := vars.User - schemaVersion := vars.GetInfoSchema().SchemaMetaVersion() + schemaVersion := s.GetInfoSchema().SchemaMetaVersion() if ss, ok := execStmt.StmtNode.(ast.SensitiveStmtNode); ok { logutil.BgLogger().Info("CRUCIAL OPERATION", zap.Uint64("conn", vars.ConnectionID), @@ -2755,11 +2755,12 @@ func logStmt(execStmt *executor.ExecStmt, vars *variable.SessionVars) { zap.Stringer("user", user)) } default: - logQuery(execStmt.GetTextToLog(), vars) + logQuery(execStmt.GetTextToLog(), s) } } -func logQuery(query string, vars *variable.SessionVars) { +func logQuery(query string, s *session) { + vars := s.GetSessionVars() if variable.ProcessGeneralLog.Load() && !vars.InRestrictedSQL { query = executor.QueryReplacer.Replace(query) if !vars.EnableRedactLog { @@ -2768,7 +2769,7 @@ func logQuery(query string, vars *variable.SessionVars) { logutil.BgLogger().Info("GENERAL_LOG", zap.Uint64("conn", vars.ConnectionID), zap.Stringer("user", vars.User), - zap.Int64("schemaVersion", vars.GetInfoSchema().SchemaMetaVersion()), + zap.Int64("schemaVersion", s.GetInfoSchema().SchemaMetaVersion()), zap.Uint64("txnStartTS", vars.TxnCtx.StartTS), zap.Uint64("forUpdateTS", vars.TxnCtx.GetForUpdateTS()), zap.Bool("isReadConsistency", vars.IsIsolation(ast.ReadCommitted)), @@ -2806,7 +2807,7 @@ func (s *session) checkPlacementPolicyBeforeCommit() error { txnScope = kv.GlobalTxnScope } if txnScope != kv.GlobalTxnScope { - is := s.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := s.GetInfoSchema().(infoschema.InfoSchema) deltaMap := s.GetSessionVars().TxnCtx.TableDeltaMap for physicalTableID := range deltaMap { var tableName string @@ -2875,3 +2876,20 @@ func (s *session) SetPort(port string) { func (s *session) GetTxnWriteThroughputSLI() *sli.TxnWriteThroughputSLI { return &s.txn.writeSLI } + +// GetInfoSchema returns snapshotInfoSchema if snapshot schema is set. +// Transaction infoschema is returned if inside an explicit txn. +// Otherwise the latest infoschema is returned. +func (s *session) GetInfoSchema() sessionctx.InfoschemaMetaVersion { + vars := s.GetSessionVars() + if snap, ok := vars.SnapshotInfoschema.(infoschema.InfoSchema); ok { + logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", vars.ConnectionID), zap.Int64("schemaVersion", snap.SchemaMetaVersion())) + return snap + } + if vars.TxnCtx != nil && vars.InTxn() { + if is, ok := vars.TxnCtx.InfoSchema.(infoschema.InfoSchema); ok { + return is + } + } + return domain.GetDomain(s).InfoSchema() +} diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 5645b7496ebcc..a2d1047e93c65 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -100,7 +100,7 @@ func GetPrewriteValue(ctx sessionctx.Context, createIfNotExists bool) *binlog.Pr vars := ctx.GetSessionVars() v, ok := vars.TxnCtx.Binlog.(*binlog.PrewriteValue) if !ok && createIfNotExists { - schemaVer := ctx.GetSessionVars().GetInfoSchema().SchemaMetaVersion() + schemaVer := ctx.GetInfoSchema().SchemaMetaVersion() v = &binlog.PrewriteValue{SchemaVersion: schemaVer} vars.TxnCtx.Binlog = v } diff --git a/sessionctx/context.go b/sessionctx/context.go index 59a917f86a9bc..68767aef30e7c 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -28,6 +28,14 @@ import ( "github.com/pingcap/tipb/go-binlog" ) +// InfoschemaMetaVersion is a workaround. Due to circular dependency, +// can not return the complete interface. But SchemaMetaVersion is widely used for logging. +// So we give a convenience for that. +// FIXME: remove this interface +type InfoschemaMetaVersion interface { + SchemaMetaVersion() int64 +} + // Context is an interface for transaction and executive args environment. type Context interface { // NewTxn creates a new transaction for further execution. @@ -56,6 +64,8 @@ type Context interface { // ClearValue clears the value associated with this context for key. ClearValue(key fmt.Stringer) + GetInfoSchema() InfoschemaMetaVersion + GetSessionVars() *variable.SessionVars GetSessionManager() util.SessionManager diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 0af4769c54930..f2536e4edcb16 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -46,14 +46,12 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/execdetails" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tidb/util/tableutil" "github.com/pingcap/tidb/util/timeutil" "github.com/twmb/murmur3" atomic2 "go.uber.org/atomic" - "go.uber.org/zap" ) // PreparedStmtCount is exported for test. @@ -876,30 +874,6 @@ func (s *SessionVars) BuildParserConfig() parser.ParserConfig { } } -// FIXME: remove this interface -// infoschemaMetaVersion is a workaround. Due to circular dependency, -// can not return the complete interface. But SchemaMetaVersion is widely used for logging. -// So we give a convenience for that -type infoschemaMetaVersion interface { - SchemaMetaVersion() int64 -} - -// GetInfoSchema returns snapshotInfoSchema if snapshot schema is set. -// Otherwise, transaction infoschema is returned. -// Nil if there is no available infoschema. -func (s *SessionVars) GetInfoSchema() infoschemaMetaVersion { - if snap, ok := s.SnapshotInfoschema.(infoschemaMetaVersion); ok { - logutil.BgLogger().Info("use snapshot schema", zap.Uint64("conn", s.ConnectionID), zap.Int64("schemaVersion", snap.SchemaMetaVersion())) - return snap - } - if s.TxnCtx != nil { - if is, ok := s.TxnCtx.InfoSchema.(infoschemaMetaVersion); ok { - return is - } - } - return nil -} - // PartitionPruneMode presents the prune mode used. type PartitionPruneMode string @@ -1259,7 +1233,7 @@ func (s *SessionVars) GetStatusFlag(flag uint16) bool { func (s *SessionVars) SetInTxn(val bool) { s.SetStatusFlag(mysql.ServerStatusInTrans, val) if val { - s.TxnCtx.IsExplicit = true + s.TxnCtx.IsExplicit = val } } diff --git a/statistics/handle/ddl.go b/statistics/handle/ddl.go index 2ddc460ff690b..6f7c37556a36c 100644 --- a/statistics/handle/ddl.go +++ b/statistics/handle/ddl.go @@ -74,7 +74,7 @@ var analyzeOptionDefault = map[ast.AnalyzeOptionType]uint64{ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error { // We need to merge the partition-level stats to global-stats when we drop table partition in dynamic mode. tableID := tblInfo.ID - is := h.mu.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := h.mu.ctx.GetInfoSchema().(infoschema.InfoSchema) globalStats, err := h.TableStatsFromStorage(tblInfo, tableID, true, 0) if err != nil { return err diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 21438adc5b29d..03df857ec4a5d 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -480,7 +480,7 @@ func (h *Handle) dumpTableStatCountToKV(id int64, delta variable.TableDelta) (up affectedRows := h.mu.ctx.GetSessionVars().StmtCtx.AffectedRows() // if it's a partitioned table and its global-stats exists, update its count and modify_count as well. - is := h.mu.ctx.GetSessionVars().GetInfoSchema().(infoschema.InfoSchema) + is := h.mu.ctx.GetInfoSchema().(infoschema.InfoSchema) if is == nil { return false, errors.New("cannot get the information schema") } diff --git a/util/mock/context.go b/util/mock/context.go index d23124e555ea2..4e9a9761f637a 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -144,6 +144,20 @@ func (c *Context) GetMPPClient() kv.MPPClient { return c.Store.GetMPPClient() } +// GetInfoSchema implements sessionctx.Context GetInfoSchema interface. +func (c *Context) GetInfoSchema() sessionctx.InfoschemaMetaVersion { + vars := c.GetSessionVars() + if snap, ok := vars.SnapshotInfoschema.(sessionctx.InfoschemaMetaVersion); ok { + return snap + } + if vars.TxnCtx != nil && vars.InTxn() { + if is, ok := vars.TxnCtx.InfoSchema.(sessionctx.InfoschemaMetaVersion); ok { + return is + } + } + return nil +} + // GetGlobalSysVar implements GlobalVarAccessor GetGlobalSysVar interface. func (c *Context) GetGlobalSysVar(ctx sessionctx.Context, name string) (string, error) { v := variable.GetSysVar(name) From 9865d51f799fea5cd0faa8fea5b77903ab9c64de Mon Sep 17 00:00:00 2001 From: baishen Date: Mon, 24 May 2021 07:29:34 -0500 Subject: [PATCH 153/343] planner: fix the ORDER BY column not match the GROUP BY clause (#24708) --- executor/aggregate_test.go | 15 +++++++++++++++ planner/core/logical_plan_builder.go | 2 +- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 85e8bcc4f236d..b93d8dd619d88 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -824,6 +824,21 @@ func (s *testSuiteAgg) TestIssue16279(c *C) { tk.MustQuery("select count(a) , date_format(a, '%Y-%m-%d') as xx from s group by xx") } +func (s *testSuiteAgg) TestIssue24676(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("set sql_mode = 'ONLY_FULL_GROUP_BY'") + tk.MustExec("drop table if exists t1") + tk.MustExec(`create table t1( + id int(11) NOT NULL PRIMARY KEY, + c1 int(11) NOT NULL DEFAULT '0' + )`) + tk.MustQuery("SELECT c1 FROM t1 GROUP BY c1 ORDER BY c1 ASC;") + tk.MustQuery("SELECT ((floor(((`c1` - 0.0) / 50000)) * 50000) + 0.0) AS `c1` FROM `t1` GROUP BY ((floor(((`c1` - 0.0) / 50000)) * 50000) + 0.0) ORDER BY ((floor(((`c1` - 0.0) / 50000)) * 50000) + 0.0) ASC;") + err := tk.ExecToErr("SELECT ((floor(((`c1` - 10) / 300)) * 50000) + 0.0) AS `c1` FROM `t1` GROUP BY ((floor(((`c1` - 0.0) / 50000)) * 50000) + 0.0) ORDER BY ((floor(((`c1` - 0.0) / 50000)) * 50000) + 0.0) ASC;") + c.Assert(terror.ErrorEqual(err, plannercore.ErrFieldNotInGroupBy), IsTrue, Commentf("err %v", err)) +} + func (s *testSuiteAgg) TestAggPushDownPartitionTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index fc6fb53dcff44..10f7ee31323f1 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -2825,7 +2825,7 @@ func (b *PlanBuilder) checkOnlyFullGroupByWithGroupClause(p LogicalPlan, sel *as continue } } - checkExprInGroupByOrIsSingleValue(p, item.Expr, offset, ErrExprInOrderBy, gbyOrSingleValueColNames, gbyExprs, notInGbyOrSingleValueColNames) + checkExprInGroupByOrIsSingleValue(p, getInnerFromParenthesesAndUnaryPlus(item.Expr), offset, ErrExprInOrderBy, gbyOrSingleValueColNames, gbyExprs, notInGbyOrSingleValueColNames) } } if len(notInGbyOrSingleValueColNames) == 0 { From d5e0ed0aaed72d2f2dfe24e9deec31cb6cb5fdf0 Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Mon, 24 May 2021 22:13:33 +0800 Subject: [PATCH 154/343] ddl: ban the setting replica for tables in system database (MySQL) (#24745) --- ddl/db_test.go | 18 ++++++++++++++++++ ddl/ddl_api.go | 4 ++++ ddl/error.go | 1 + ddl/table.go | 6 ++++++ util/misc.go | 7 ++++++- 5 files changed, 35 insertions(+), 1 deletion(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 1ae6ac0ff3efd..80a821a40e387 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -5299,6 +5299,24 @@ func (s *testSerialDBSuite) TestSetTableFlashReplica(c *C) { c.Assert(err.Error(), Equals, "the tiflash replica count: 2 should be less than the total tiflash server count: 0") } +func (s *testSerialDBSuite) TestSetTableFlashReplicaForSystemTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + sysTables := make([]string, 0, 24) + memOrSysDB := []string{"MySQL", "INFORMATION_SCHEMA", "PERFORMANCE_SCHEMA", "METRICS_SCHEMA"} + for _, db := range memOrSysDB { + tk.MustExec("use " + db) + rows := tk.MustQuery("show tables").Rows() + for i := 0; i < len(rows); i++ { + sysTables = append(sysTables, rows[i][0].(string)) + } + for _, one := range sysTables { + _, err := tk.Exec(fmt.Sprintf("alter table `%s` set tiflash replica 1", one)) + c.Assert(err.Error(), Equals, "[ddl:8200]ALTER table replica for tables in system database is currently unsupported") + } + sysTables = sysTables[:0] + } +} + func (s *testSerialDBSuite) TestAlterShardRowIDBits(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) defer func() { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2cfd3f524a29f..eaf73bccade73 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4372,6 +4372,10 @@ func (d *ddl) AlterTableSetTiFlashReplica(ctx sessionctx.Context, ident ast.Iden if err != nil { return errors.Trace(err) } + // Ban setting replica count for tables in system database. + if util.IsMemOrSysDB(schema.Name.L) { + return errors.Trace(errUnsupportedAlterReplicaForSysTable) + } tbReplicaInfo := tb.Meta().TiFlashReplica if tbReplicaInfo != nil && tbReplicaInfo.Count == replicaInfo.Count && diff --git a/ddl/error.go b/ddl/error.go index 6bde83badb41b..fb046f31c9f38 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -48,6 +48,7 @@ var ( errUnsupportedAlterTableWithValidation = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("ALTER TABLE WITH VALIDATION is currently unsupported", nil)) errUnsupportedAlterTableWithoutValidation = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("ALTER TABLE WITHOUT VALIDATION is currently unsupported", nil)) errUnsupportedAlterTableOption = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("This type of ALTER TABLE is currently unsupported", nil)) + errUnsupportedAlterReplicaForSysTable = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("ALTER table replica for tables in system database is currently unsupported", nil)) errBlobKeyWithoutLength = dbterror.ClassDDL.NewStd(mysql.ErrBlobKeyWithoutLength) errKeyPart0 = dbterror.ClassDDL.NewStd(mysql.ErrKeyPart0) errIncorrectPrefixKey = dbterror.ClassDDL.NewStd(mysql.ErrWrongSubKey) diff --git a/ddl/table.go b/ddl/table.go index 424dd040a0de9..6c113fc855b23 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" + tidb_util "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/gcutil" ) @@ -875,6 +876,11 @@ func (w *worker) onSetTableFlashReplica(t *meta.Meta, job *model.Job) (ver int64 return ver, errors.Trace(err) } + // Ban setting replica count for tables in system database. + if tidb_util.IsMemOrSysDB(job.SchemaName) { + return ver, errors.Trace(errUnsupportedAlterReplicaForSysTable) + } + err = w.checkTiFlashReplicaCount(replicaInfo.Count) if err != nil { job.State = model.JobStateCancelled diff --git a/util/misc.go b/util/misc.go index 13192f5811ccd..683fc3857d930 100644 --- a/util/misc.go +++ b/util/misc.go @@ -175,7 +175,7 @@ var ( // IsMemOrSysDB uses to check whether dbLowerName is memory database or system database. func IsMemOrSysDB(dbLowerName string) bool { - return IsMemDB(dbLowerName) || dbLowerName == mysql.SystemDB + return IsMemDB(dbLowerName) || IsSysDB(dbLowerName) } // IsMemDB checks whether dbLowerName is memory database. @@ -189,6 +189,11 @@ func IsMemDB(dbLowerName string) bool { return false } +// IsSysDB checks whether dbLowerName is system database. +func IsSysDB(dbLowerName string) bool { + return dbLowerName == mysql.SystemDB +} + // IsSystemView is similar to IsMemOrSyDB, but does not include the mysql schema func IsSystemView(dbLowerName string) bool { switch dbLowerName { From d1b3e47111a34a2d3e9ca9aec94292f822c2f5a4 Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 25 May 2021 10:37:33 +0800 Subject: [PATCH 155/343] server, store: speed up server test (#24833) --- server/conn_test.go | 14 +++++++++++++- store/copr/batch_coprocessor.go | 9 ++++++++- 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/server/conn_test.go b/server/conn_test.go index 0ed451a4fde8e..c2347e95e4bdd 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -764,11 +764,22 @@ func (ts *ConnTestSuite) TestTiFlashFallback(c *C) { tb := testGetTableByName(c, tk.Se, "test", "t") err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) c.Assert(err, IsNil) + + dml := "insert into t values" for i := 0; i < 50; i++ { - tk.MustExec(fmt.Sprintf("insert into t values(%v, 0)", i)) + dml += fmt.Sprintf("(%v, 0)", i) + if i != 49 { + dml += "," + } } + tk.MustExec(dml) tk.MustQuery("select count(*) from t").Check(testkit.Rows("50")) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/copr/ReduceCopNextMaxBackoff", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/copr/ReduceCopNextMaxBackoff"), IsNil) + }() + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/BatchCopRpcErrtiflash0", "return(\"tiflash0\")"), IsNil) // test COM_STMT_EXECUTE ctx := context.Background() @@ -777,6 +788,7 @@ func (ts *ConnTestSuite) TestTiFlashFallback(c *C) { c.Assert(cc.handleStmtPrepare(ctx, "select sum(a) from t"), IsNil) c.Assert(cc.handleStmtExecute(ctx, []byte{0x1, 0x0, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0}), IsNil) tk.MustQuery("show warnings").Check(testkit.Rows("Error 9012 TiFlash server timeout")) + // test COM_STMT_FETCH (cursor mode) c.Assert(cc.handleStmtExecute(ctx, []byte{0x1, 0x0, 0x0, 0x0, 0x1, 0x1, 0x0, 0x0, 0x0}), IsNil) c.Assert(cc.handleStmtFetch(ctx, []byte{0x1, 0x0, 0x0, 0x0, 0x1, 0x0, 0x0, 0x0}), NotNil) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 6efbf76775186..8e45b546f10a9 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -24,6 +24,7 @@ import ( "unsafe" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" @@ -389,7 +390,13 @@ func (b *batchCopIterator) run(ctx context.Context) { // We run workers for every batch cop. for _, task := range b.tasks { b.wg.Add(1) - bo := backoff.NewBackofferWithVars(ctx, copNextMaxBackoff, b.vars) + boMaxSleep := copNextMaxBackoff + failpoint.Inject("ReduceCopNextMaxBackoff", func(value failpoint.Value) { + if value.(bool) { + boMaxSleep = 2 + } + }) + bo := backoff.NewBackofferWithVars(ctx, boMaxSleep, b.vars) go b.handleTask(ctx, bo, task) } b.wg.Wait() From 39bd1812132efefc22f433d66a3ce45cac00f890 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BA=84=E5=A4=A9=E7=BF=BC?= Date: Tue, 25 May 2021 12:07:34 +0800 Subject: [PATCH 156/343] *: Upgrade to go 1.16 && remove deprecated io/ioutil (#24527) --- Dockerfile | 2 +- cmd/benchfilesort/main.go | 3 +-- cmd/benchkv/main.go | 4 ++-- cmd/explaintest/main.go | 11 +++++------ cmd/importcheck/importcheck.go | 3 +-- cmd/importer/stats.go | 4 ++-- config/config_util.go | 3 +-- config/config_util_test.go | 5 ++--- domain/infosync/info.go | 3 +-- executor/benchmark_test.go | 3 +-- executor/executor_test.go | 9 ++++----- executor/infoschema_reader.go | 4 ++-- executor/memtable_reader_test.go | 5 ++--- executor/select_into_test.go | 3 +-- executor/set_config.go | 4 ++-- executor/set_test.go | 6 +++--- executor/slow_query.go | 5 ++--- expression/generator/compare_vec.go | 6 +++--- expression/generator/control_vec.go | 6 +++--- expression/generator/other_vec.go | 6 +++--- expression/generator/string_vec.go | 6 +++--- expression/generator/time_vec.go | 6 +++--- expression/util.go | 2 +- go.mod | 4 +--- go.sum | 4 ---- infoschema/tables.go | 4 ++-- planner/core/cacheable_checker.go | 2 +- planner/core/cacheable_checker_test.go | 2 +- planner/core/cbo_test.go | 4 ++-- server/http_handler_test.go | 15 +++++++-------- server/http_status.go | 4 ++-- server/server_test.go | 5 ++--- server/statistics_handler_test.go | 8 ++++---- server/tidb_test.go | 5 ++--- .../unistore/cophandler/cop_handler_test.go | 5 ++--- store/mockstore/unistore/mock.go | 3 +-- store/mockstore/unistore/tikv/mvcc_test.go | 5 ++--- store/tikv/config/security.go | 4 ++-- tests/globalkilltest/go.mod | 2 +- tidb-server/main.go | 4 ++-- tools/check/go.mod | 2 +- util/chunk/chunk_test.go | 3 +-- util/chunk/disk.go | 3 +-- util/chunk/disk_test.go | 3 +-- util/disk/tempDir.go | 3 +-- util/disk/tempDir_test.go | 3 +-- util/expensivequery/memory_usage_alarm.go | 3 +-- util/filesort/filesort_test.go | 15 +++++++-------- util/format/format_test.go | 4 ++-- util/memory/meminfo.go | 6 +++--- util/misc.go | 3 +-- util/profile/profile.go | 3 +-- util/rowDecoder/decoder_test.go | 2 +- util/testutil/testutil.go | 4 ++-- 54 files changed, 107 insertions(+), 139 deletions(-) diff --git a/Dockerfile b/Dockerfile index 46d7d05db347e..c6fca6bc22b75 100644 --- a/Dockerfile +++ b/Dockerfile @@ -12,7 +12,7 @@ # limitations under the License. # Builder image -FROM golang:1.13-alpine as builder +FROM golang:1.16-alpine as builder RUN apk add --no-cache \ wget \ diff --git a/cmd/benchfilesort/main.go b/cmd/benchfilesort/main.go index 1a580f5ca14d6..943968a0ea046 100644 --- a/cmd/benchfilesort/main.go +++ b/cmd/benchfilesort/main.go @@ -17,7 +17,6 @@ import ( "encoding/binary" "flag" "fmt" - "io/ioutil" "math/rand" "os" "path/filepath" @@ -332,7 +331,7 @@ func driveRunCmd() { for i := 0; i < keySize; i++ { byDesc[i] = false } - dir, err = ioutil.TempDir(tmpDir, "benchfilesort_test") + dir, err = os.MkdirTemp(tmpDir, "benchfilesort_test") terror.MustNil(err) fs, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(nWorkers).SetDesc(byDesc).SetDir(dir).Build() terror.MustNil(err) diff --git a/cmd/benchkv/main.go b/cmd/benchkv/main.go index 23bfd44a7476a..e8c3cd1b941a6 100644 --- a/cmd/benchkv/main.go +++ b/cmd/benchkv/main.go @@ -17,7 +17,7 @@ import ( "context" "flag" "fmt" - "io/ioutil" + "io" "net/http" _ "net/http/pprof" "sync" @@ -129,7 +129,7 @@ func main() { terror.MustNil(err) defer terror.Call(resp.Body.Close) - text, err1 := ioutil.ReadAll(resp.Body) + text, err1 := io.ReadAll(resp.Body) terror.Log(errors.Trace(err1)) fmt.Println(string(text)) diff --git a/cmd/explaintest/main.go b/cmd/explaintest/main.go index a85c8ce82dd3c..ae4fa18d5a461 100644 --- a/cmd/explaintest/main.go +++ b/cmd/explaintest/main.go @@ -19,7 +19,6 @@ import ( "flag" "fmt" "io" - "io/ioutil" "net/http" "os" "os/exec" @@ -163,7 +162,7 @@ LOOP: } func (t *tester) loadQueries() ([]query, error) { - data, err := ioutil.ReadFile(t.testFileName()) + data, err := os.ReadFile(t.testFileName()) if err != nil { return nil, err } @@ -427,12 +426,12 @@ func (t *tester) create(tableName string, qText string) error { return err } - js, err := ioutil.ReadAll(resp.Body) + js, err := io.ReadAll(resp.Body) if err != nil { return err } - return ioutil.WriteFile(t.statsFileName(tableName), js, 0644) + return os.WriteFile(t.statsFileName(tableName), js, 0644) } func (t *tester) commit() error { @@ -531,7 +530,7 @@ func (t *tester) flushResult() error { if !record { return nil } - return ioutil.WriteFile(t.resultFileName(), t.buf.Bytes(), 0644) + return os.WriteFile(t.resultFileName(), t.buf.Bytes(), 0644) } func (t *tester) statsFileName(tableName string) string { @@ -550,7 +549,7 @@ func (t *tester) resultFileName() string { func loadAllTests() ([]string, error) { // tests must be in t folder - files, err := ioutil.ReadDir("./t") + files, err := os.ReadDir("./t") if err != nil { return nil, err } diff --git a/cmd/importcheck/importcheck.go b/cmd/importcheck/importcheck.go index e668705230e35..ee4042e052760 100644 --- a/cmd/importcheck/importcheck.go +++ b/cmd/importcheck/importcheck.go @@ -20,7 +20,6 @@ import ( "go/ast" "go/parser" "go/token" - "io/ioutil" "os" "path/filepath" "strings" @@ -60,7 +59,7 @@ func run() error { } func checkFile(path string) error { - src, err := ioutil.ReadFile(path) + src, err := os.ReadFile(path) if err != nil { return err } diff --git a/cmd/importer/stats.go b/cmd/importer/stats.go index 2f258462b93dc..08c52b61a775a 100644 --- a/cmd/importer/stats.go +++ b/cmd/importer/stats.go @@ -15,8 +15,8 @@ package main import ( "encoding/json" - "io/ioutil" "math/rand" + "os" "time" "github.com/pingcap/errors" @@ -29,7 +29,7 @@ import ( ) func loadStats(tblInfo *model.TableInfo, path string) (*stats.Table, error) { - data, err := ioutil.ReadFile(path) + data, err := os.ReadFile(path) if err != nil { return nil, errors.Trace(err) } diff --git a/config/config_util.go b/config/config_util.go index 0bf3374e001d7..35df9421097a0 100644 --- a/config/config_util.go +++ b/config/config_util.go @@ -17,7 +17,6 @@ import ( "bytes" "encoding/json" "fmt" - "io/ioutil" "os" "path/filepath" "reflect" @@ -110,7 +109,7 @@ func atomicWriteConfig(c *Config, confPath string) (err error) { return err } tmpConfPath := filepath.Join(os.TempDir(), fmt.Sprintf("tmp_conf_%v.toml", time.Now().Format("20060102150405"))) - if err := ioutil.WriteFile(tmpConfPath, []byte(content), 0666); err != nil { + if err := os.WriteFile(tmpConfPath, []byte(content), 0666); err != nil { return errors.Trace(err) } return errors.Trace(os.Rename(tmpConfPath, confPath)) diff --git a/config/config_util_test.go b/config/config_util_test.go index 7972fcf706000..e83c1e2046072 100644 --- a/config/config_util_test.go +++ b/config/config_util_test.go @@ -16,7 +16,6 @@ package config import ( "encoding/json" "fmt" - "io/ioutil" "os" "path/filepath" "reflect" @@ -100,7 +99,7 @@ func (s *testConfigSuite) TestAtomicWriteConfig(c *C) { conf.Performance.PseudoEstimateRatio = 3.45 c.Assert(atomicWriteConfig(conf, confPath), IsNil) - content, err := ioutil.ReadFile(confPath) + content, err := os.ReadFile(confPath) c.Assert(err, IsNil) dconf, err := decodeConfig(string(content)) c.Assert(err, IsNil) @@ -113,7 +112,7 @@ func (s *testConfigSuite) TestAtomicWriteConfig(c *C) { conf.Performance.PseudoEstimateRatio = 54.3 c.Assert(atomicWriteConfig(conf, confPath), IsNil) - content, err = ioutil.ReadFile(confPath) + content, err = os.ReadFile(confPath) c.Assert(err, IsNil) dconf, err = decodeConfig(string(content)) c.Assert(err, IsNil) diff --git a/domain/infosync/info.go b/domain/infosync/info.go index a94a15fb7e212..8596dcc6ce477 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -19,7 +19,6 @@ import ( "encoding/json" "fmt" "io" - "io/ioutil" "net/http" "os" "path" @@ -334,7 +333,7 @@ func doRequest(ctx context.Context, addrs []string, route, method string, body i } }) if err == nil { - bodyBytes, err := ioutil.ReadAll(res.Body) + bodyBytes, err := io.ReadAll(res.Body) if err != nil { return nil, err } diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 48fd34d4bb2a0..adf6cc6c854ed 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -17,7 +17,6 @@ import ( "context" "encoding/base64" "fmt" - "io/ioutil" "math/rand" "os" "sort" @@ -1991,7 +1990,7 @@ func BenchmarkReadLastLinesOfHugeLine(b *testing.B) { hugeLine[i] = 'a' + byte(i%26) } fileName := "tidb.log" - err := ioutil.WriteFile(fileName, hugeLine, 0644) + err := os.WriteFile(fileName, hugeLine, 0644) if err != nil { b.Fatal(err) } diff --git a/executor/executor_test.go b/executor/executor_test.go index 6ed0309d75a42..15594a73fe235 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -17,7 +17,6 @@ import ( "context" "flag" "fmt" - "io/ioutil" "math" "net" "os" @@ -6998,7 +6997,7 @@ func (s *testSlowQuery) TestSlowQuerySensitiveQuery(c *C) { originCfg := config.GetGlobalConfig() newCfg := *originCfg - f, err := ioutil.TempFile("", "tidb-slow-*.log") + f, err := os.CreateTemp("", "tidb-slow-*.log") c.Assert(err, IsNil) f.Close() newCfg.Log.SlowQueryFile = f.Name() @@ -7032,7 +7031,7 @@ func (s *testSlowQuery) TestSlowQueryPrepared(c *C) { originCfg := config.GetGlobalConfig() newCfg := *originCfg - f, err := ioutil.TempFile("", "tidb-slow-*.log") + f, err := os.CreateTemp("", "tidb-slow-*.log") c.Assert(err, IsNil) f.Close() newCfg.Log.SlowQueryFile = f.Name() @@ -7068,7 +7067,7 @@ func (s *testSlowQuery) TestSlowQueryPrepared(c *C) { func (s *testSlowQuery) TestLogSlowLogIndex(c *C) { tk := testkit.NewTestKit(c, s.store) - f, err := ioutil.TempFile("", "tidb-slow-*.log") + f, err := os.CreateTemp("", "tidb-slow-*.log") c.Assert(err, IsNil) f.Close() @@ -7094,7 +7093,7 @@ func (s *testSlowQuery) TestLogSlowLogIndex(c *C) { func (s *testSlowQuery) TestSlowQuery(c *C) { tk := testkit.NewTestKit(c, s.store) - f, err := ioutil.TempFile("", "tidb-slow-*.log") + f, err := os.CreateTemp("", "tidb-slow-*.log") c.Assert(err, IsNil) _, err = f.WriteString(` # Time: 2020-10-13T20:08:13.970563+08:00 diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index cfc29c5ccc513..ee5b4af84b574 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -18,7 +18,7 @@ import ( "context" "encoding/json" "fmt" - "io/ioutil" + "io" "net/http" "sort" "strconv" @@ -2266,7 +2266,7 @@ func (e *TiFlashSystemTableRetriever) dataForTiFlashSystemTables(ctx sessionctx. if err != nil { return nil, errors.Trace(err) } - body, err := ioutil.ReadAll(resp.Body) + body, err := io.ReadAll(resp.Body) terror.Log(resp.Body.Close()) if err != nil { return nil, errors.Trace(err) diff --git a/executor/memtable_reader_test.go b/executor/memtable_reader_test.go index 5b55a9d601b03..94a8b49ca26bb 100644 --- a/executor/memtable_reader_test.go +++ b/executor/memtable_reader_test.go @@ -16,7 +16,6 @@ package executor_test import ( "context" "fmt" - "io/ioutil" "log" "net" "net/http/httptest" @@ -431,7 +430,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterConfig(c *C) { } func (s *testClusterTableBase) writeTmpFile(c *C, dir, filename string, lines []string) { - err := ioutil.WriteFile(filepath.Join(dir, filename), []byte(strings.Join(lines, "\n")), os.ModePerm) + err := os.WriteFile(filepath.Join(dir, filename), []byte(strings.Join(lines, "\n")), os.ModePerm) c.Assert(err, IsNil, Commentf("write tmp file %s failed", filename)) } @@ -449,7 +448,7 @@ func (s *testClusterTableBase) setupClusterGRPCServer(c *C) map[string]*testServ // create gRPC servers for _, typ := range []string{"tidb", "tikv", "pd"} { - tmpDir, err := ioutil.TempDir("", typ) + tmpDir, err := os.MkdirTemp("", typ) c.Assert(err, IsNil) server := grpc.NewServer() diff --git a/executor/select_into_test.go b/executor/select_into_test.go index c499db9bdd322..e9b268703ff26 100644 --- a/executor/select_into_test.go +++ b/executor/select_into_test.go @@ -15,7 +15,6 @@ package executor_test import ( "fmt" - "io/ioutil" "os" "path/filepath" "strings" @@ -29,7 +28,7 @@ import ( ) func cmpAndRm(expected, outfile string, c *C) { - content, err := ioutil.ReadFile(outfile) + content, err := os.ReadFile(outfile) c.Assert(err, IsNil) c.Assert(string(content), Equals, expected) c.Assert(os.Remove(outfile), IsNil) diff --git a/executor/set_config.go b/executor/set_config.go index 05064b614d7a6..e31bab643cbb8 100644 --- a/executor/set_config.go +++ b/executor/set_config.go @@ -17,7 +17,7 @@ import ( "bytes" "context" "fmt" - "io/ioutil" + "io" "net" "net/http" "strings" @@ -143,7 +143,7 @@ func (s *SetConfigExec) doRequest(url string) (retErr error) { if resp.StatusCode == http.StatusOK { return nil } else if resp.StatusCode >= 400 && resp.StatusCode < 600 { - message, err := ioutil.ReadAll(resp.Body) + message, err := io.ReadAll(resp.Body) if err != nil { return err } diff --git a/executor/set_test.go b/executor/set_test.go index 4f24ca83519de..eda1696497cea 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -18,7 +18,7 @@ import ( "context" "errors" "fmt" - "io/ioutil" + "io" "net/http" "strconv" @@ -1320,7 +1320,7 @@ func (s *testSuite5) TestSetClusterConfig(c *C) { httpCnt := 0 tk.Se.SetValue(executor.TestSetConfigHTTPHandlerKey, func(*http.Request) (*http.Response, error) { httpCnt++ - return &http.Response{StatusCode: http.StatusOK, Body: ioutil.NopCloser(nil)}, nil + return &http.Response{StatusCode: http.StatusOK, Body: io.NopCloser(nil)}, nil }) tk.MustExec("set config tikv log.level='info'") c.Assert(httpCnt, Equals, 2) @@ -1338,7 +1338,7 @@ func (s *testSuite5) TestSetClusterConfig(c *C) { "Warning 1105 something wrong", "Warning 1105 something wrong")) tk.Se.SetValue(executor.TestSetConfigHTTPHandlerKey, func(*http.Request) (*http.Response, error) { - return &http.Response{StatusCode: http.StatusBadRequest, Body: ioutil.NopCloser(bytes.NewBufferString("WRONG"))}, nil + return &http.Response{StatusCode: http.StatusBadRequest, Body: io.NopCloser(bytes.NewBufferString("WRONG"))}, nil }) tk.MustExec("set config tikv log.level='info'") tk.MustQuery("show warnings").Check(testkit.Rows( diff --git a/executor/slow_query.go b/executor/slow_query.go index 9e32cb175e4b8..ad64fa5355893 100755 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -18,7 +18,6 @@ import ( "context" "fmt" "io" - "io/ioutil" "os" "path/filepath" "sort" @@ -998,11 +997,11 @@ func (e *slowQueryRetriever) getAllFiles(ctx context.Context, sctx sessionctx.Co } return nil } - files, err := ioutil.ReadDir(logDir) + files, err := os.ReadDir(logDir) if err != nil { return nil, err } - walkFn := func(path string, info os.FileInfo) error { + walkFn := func(path string, info os.DirEntry) error { if info.IsDir() { return nil } diff --git a/expression/generator/compare_vec.go b/expression/generator/compare_vec.go index 5864e61c02c3c..4908b63af477b 100644 --- a/expression/generator/compare_vec.go +++ b/expression/generator/compare_vec.go @@ -19,8 +19,8 @@ import ( "bytes" "flag" "go/format" - "io/ioutil" "log" + "os" "path/filepath" "text/template" @@ -414,7 +414,7 @@ func generateDotGo(fileName string, compares []CompareContext, types []TypeConte log.Println("[Warn]", fileName+": gofmt failed", err) data = w.Bytes() // write original data for debugging } - return ioutil.WriteFile(fileName, data, 0644) + return os.WriteFile(fileName, data, 0644) } func generateTestDotGo(fileName string, compares []CompareContext, types []TypeContext) error { @@ -459,7 +459,7 @@ func generateTestDotGo(fileName string, compares []CompareContext, types []TypeC log.Println("[Warn]", fileName+": gofmt failed", err) data = w.Bytes() // write original data for debugging } - return ioutil.WriteFile(fileName, data, 0644) + return os.WriteFile(fileName, data, 0644) } // generateOneFile generate one xxx.go file and the associated xxx_test.go file. diff --git a/expression/generator/control_vec.go b/expression/generator/control_vec.go index 0b77d127d6f9a..99e77fd4052b8 100644 --- a/expression/generator/control_vec.go +++ b/expression/generator/control_vec.go @@ -18,8 +18,8 @@ package main import ( "bytes" "go/format" - "io/ioutil" "log" + "os" "path/filepath" "text/template" @@ -636,7 +636,7 @@ func generateDotGo(fileName string) error { log.Println("[Warn]", fileName+": gofmt failed", err) data = w.Bytes() // write original data for debugging } - return ioutil.WriteFile(fileName, data, 0644) + return os.WriteFile(fileName, data, 0644) } func generateTestDotGo(fileName string) error { @@ -650,7 +650,7 @@ func generateTestDotGo(fileName string) error { log.Println("[Warn]", fileName+": gofmt failed", err) data = w.Bytes() // write original data for debugging } - return ioutil.WriteFile(fileName, data, 0644) + return os.WriteFile(fileName, data, 0644) } // generateOneFile generate one xxx.go file and the associated xxx_test.go file. diff --git a/expression/generator/other_vec.go b/expression/generator/other_vec.go index 8016ba7619766..9c57d179f123d 100644 --- a/expression/generator/other_vec.go +++ b/expression/generator/other_vec.go @@ -18,8 +18,8 @@ package main import ( "bytes" "go/format" - "io/ioutil" "log" + "os" "path/filepath" "text/template" @@ -459,7 +459,7 @@ func generateDotGo(fileName string) error { log.Println("[Warn]", fileName+": gofmt failed", err) data = w.Bytes() // write original data for debugging } - return ioutil.WriteFile(fileName, data, 0644) + return os.WriteFile(fileName, data, 0644) } func generateTestDotGo(fileName string) error { @@ -473,7 +473,7 @@ func generateTestDotGo(fileName string) error { log.Println("[Warn]", fileName+": gofmt failed", err) data = w.Bytes() // write original data for debugging } - return ioutil.WriteFile(fileName, data, 0644) + return os.WriteFile(fileName, data, 0644) } // generateOneFile generate one xxx.go file and the associated xxx_test.go file. diff --git a/expression/generator/string_vec.go b/expression/generator/string_vec.go index 8eaeaee32a387..72785b98b7df4 100644 --- a/expression/generator/string_vec.go +++ b/expression/generator/string_vec.go @@ -19,8 +19,8 @@ import ( "bytes" "flag" "go/format" - "io/ioutil" "log" + "os" "path/filepath" "text/template" @@ -168,7 +168,7 @@ func generateDotGo(fileName string, types []TypeContext) (err error) { log.Println("[Warn]", fileName+": gofmt failed", err) data = w.Bytes() // write original data for debugging } - return ioutil.WriteFile(fileName, data, 0644) + return os.WriteFile(fileName, data, 0644) } func generateTestDotGo(fileName string, types []TypeContext) error { @@ -186,7 +186,7 @@ func generateTestDotGo(fileName string, types []TypeContext) error { log.Println("[Warn]", fileName+": gofmt failed", err) data = w.Bytes() // write original data for debugging } - return ioutil.WriteFile(fileName, data, 0644) + return os.WriteFile(fileName, data, 0644) } // generateOneFile generate one xxx.go file and the associated xxx_test.go file. diff --git a/expression/generator/time_vec.go b/expression/generator/time_vec.go index b6c13dc1af760..72b35ec17f8b0 100644 --- a/expression/generator/time_vec.go +++ b/expression/generator/time_vec.go @@ -18,8 +18,8 @@ package main import ( "bytes" "go/format" - "io/ioutil" "log" + "os" "path/filepath" "text/template" @@ -944,7 +944,7 @@ func generateDotGo(fileName string) error { log.Println("[Warn]", fileName+": gofmt failed", err) data = w.Bytes() // write original data for debugging } - return ioutil.WriteFile(fileName, data, 0644) + return os.WriteFile(fileName, data, 0644) } func generateTestDotGo(fileName string) error { @@ -958,7 +958,7 @@ func generateTestDotGo(fileName string) error { log.Println("[Warn]", fileName+": gofmt failed", err) data = w.Bytes() // write original data for debugging } - return ioutil.WriteFile(fileName, data, 0644) + return os.WriteFile(fileName, data, 0644) } // generateOneFile generate one xxx.go file and the associated xxx_test.go file. diff --git a/expression/util.go b/expression/util.go index 9819dbb447ac1..3a9ecfe27b53a 100644 --- a/expression/util.go +++ b/expression/util.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/types/parser_driver" + driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" diff --git a/go.mod b/go.mod index 82a394d8406de..32d76211a667b 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,6 @@ require ( github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d github.com/carlmjohnson/flagext v0.21.0 // indirect github.com/cheggaaa/pb/v3 v3.0.4 // indirect - github.com/codahale/hdrhistogram v0.9.0 // indirect github.com/coocood/freecache v1.1.1 github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 github.com/cznic/sortutil v0.0.0-20181122101858-f5f958428db8 @@ -30,7 +29,6 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.1.0 github.com/iancoleman/strcase v0.0.0-20191112232945-16388991a334 github.com/joho/sqltocsv v0.0.0-20210208114054-cb2c3a95fb99 // indirect - github.com/klauspost/cpuid v1.2.1 github.com/kr/text v0.2.0 // indirect github.com/mattn/go-runewidth v0.0.10 // indirect github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 @@ -88,7 +86,7 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -go 1.13 +go 1.16 // Fix panic in unit test with go >= 1.14, ref: etcd-io/bbolt#201 https://github.com/etcd-io/bbolt/pull/201 replace go.etcd.io/bbolt => go.etcd.io/bbolt v1.3.5 diff --git a/go.sum b/go.sum index cf26e5ebe4089..c488db7c0347a 100644 --- a/go.sum +++ b/go.sum @@ -86,8 +86,6 @@ github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMn github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa h1:OaNxuTZr7kxeODyLWsRMC+OD03aFUH+mW6r2d+MWa5Y= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= -github.com/codahale/hdrhistogram v0.9.0 h1:9GjrtRI+mLEFPtTfR/AZhcxp+Ii8NZYWq5104FbZQY0= -github.com/codahale/hdrhistogram v0.9.0/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64 h1:W1SHiII3e0jVwvaQFglwu3kS9NLxOeTpvik7MbKCyuQ= github.com/coocood/bbloom v0.0.0-20190830030839-58deb6228d64/go.mod h1:F86k/6c7aDUdwSUevnLpHS/3Q9hzYCE99jGk2xsHnt0= @@ -151,7 +149,6 @@ github.com/fatih/color v1.10.0 h1:s36xzo75JdqLaaWoiEHk767eHiwo0598uUxyfiPkDsg= github.com/fatih/color v1.10.0/go.mod h1:ELkj/draVOlAH/xkhN6mQ50Qd0MPOk5AAr3maGEBuJM= github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= -github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsouza/fake-gcs-server v1.17.0 h1:OeH75kBZcZa3ZE+zz/mFdJ2btt9FgqfjI7gIh9+5fvk= github.com/fsouza/fake-gcs-server v1.17.0/go.mod h1:D1rTE4YCyHFNa99oyJJ5HyclvN/0uQR+pM/VdlL83bw= @@ -594,7 +591,6 @@ github.com/xitongsys/parquet-go v1.5.1/go.mod h1:xUxwM8ELydxh4edHGegYq1pA8NnMKDx github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457 h1:tBbuFCtyJNKT+BFAv6qjvTFpVdy97IYNaBwGUXifIUs= github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457/go.mod h1:pheqtXeHQFzxJk45lRQ0UIGIivKnLXvialZSFWs81A8= github.com/xitongsys/parquet-go-source v0.0.0-20190524061010-2b72cbee77d5/go.mod h1:xxCx7Wpym/3QCo6JhujJX51dzSXrwmb0oH6FQb39SEA= -github.com/xitongsys/parquet-go-source v0.0.0-20200817004010-026bad9b25d0 h1:a742S4V5A15F93smuVxA60LQWsrCnN8bKeWDBARU1/k= github.com/xitongsys/parquet-go-source v0.0.0-20200817004010-026bad9b25d0/go.mod h1:HYhIKsdns7xz80OgkbgJYrtQY7FjHWHKH6cvN7+czGE= github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= diff --git a/infoschema/tables.go b/infoschema/tables.go index fac85f799ed66..04988cb07c2b7 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -17,7 +17,7 @@ import ( "context" "encoding/json" "fmt" - "io/ioutil" + "io" "net" "net/http" "sort" @@ -1557,7 +1557,7 @@ func GetPDServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { if err != nil { return nil, errors.Trace(err) } - pdVersion, err := ioutil.ReadAll(resp.Body) + pdVersion, err := io.ReadAll(resp.Body) terror.Log(resp.Body.Close()) if err != nil { return nil, errors.Trace(err) diff --git a/planner/core/cacheable_checker.go b/planner/core/cacheable_checker.go index 9e2593f380d3b..b78e02fa6a1fb 100644 --- a/planner/core/cacheable_checker.go +++ b/planner/core/cacheable_checker.go @@ -17,7 +17,7 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/types/parser_driver" + driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" ) diff --git a/planner/core/cacheable_checker_test.go b/planner/core/cacheable_checker_test.go index 3becdce3d2cc1..7d6db5dd611d5 100644 --- a/planner/core/cacheable_checker_test.go +++ b/planner/core/cacheable_checker_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/types/parser_driver" + driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/testkit" ) diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 822e52a0a0254..a46a48c7191ff 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -17,7 +17,7 @@ import ( "context" "encoding/json" "fmt" - "io/ioutil" + "os" "path/filepath" "strings" "testing" @@ -60,7 +60,7 @@ func (s *testAnalyzeSuite) TearDownSuite(c *C) { func (s *testAnalyzeSuite) loadTableStats(fileName string, dom *domain.Domain) error { statsPath := filepath.Join("testdata", fileName) - bytes, err := ioutil.ReadFile(statsPath) + bytes, err := os.ReadFile(statsPath) if err != nil { return err } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 7cdd7137138bd..40ec398f8a650 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -23,7 +23,6 @@ import ( "encoding/json" "fmt" "io" - "io/ioutil" "net" "net/http" "net/http/httputil" @@ -740,7 +739,7 @@ func (ts *HTTPHandlerTestSuite) TestTiFlashReplica(c *C) { resp, err = ts.postStatus("/tiflash/replica", "application/json", bytes.NewBuffer([]byte(`{"id":84,"region_count":3,"flash_region_count":3}`))) c.Assert(err, IsNil) c.Assert(resp, NotNil) - body, err := ioutil.ReadAll(resp.Body) + body, err := io.ReadAll(resp.Body) c.Assert(err, IsNil) c.Assert(string(body), Equals, "[schema:1146]Table which ID = 84 does not exist.") @@ -750,7 +749,7 @@ func (ts *HTTPHandlerTestSuite) TestTiFlashReplica(c *C) { resp, err = ts.postStatus("/tiflash/replica", "application/json", bytes.NewBuffer([]byte(req))) c.Assert(err, IsNil) c.Assert(resp, NotNil) - body, err = ioutil.ReadAll(resp.Body) + body, err = io.ReadAll(resp.Body) c.Assert(err, IsNil) c.Assert(string(body), Equals, "") @@ -1237,7 +1236,7 @@ func (ts *HTTPHandlerTestSuite) TestPprof(c *C) { for retry := 0; retry < retryTime; retry++ { resp, err := ts.fetchStatus("/debug/pprof/heap") if err == nil { - _, err = ioutil.ReadAll(resp.Body) + _, err = io.ReadAll(resp.Body) c.Assert(err, IsNil) err = resp.Body.Close() c.Assert(err, IsNil) @@ -1398,7 +1397,7 @@ func (ts *HTTPHandlerTestSuite) TestZipInfoForSQL(c *C) { resp, err = ts.formStatus("/debug/sub-optimal-plan", urlValues) c.Assert(err, IsNil) c.Assert(resp.StatusCode, Equals, http.StatusInternalServerError) - b, err = ioutil.ReadAll(resp.Body) + b, err = io.ReadAll(resp.Body) c.Assert(err, IsNil) c.Assert(string(b), Equals, "use database non_exists_db failed, err: [schema:1049]Unknown database 'non_exists_db'\n") c.Assert(resp.Body.Close(), IsNil) @@ -1420,7 +1419,7 @@ func (ts *HTTPHandlerTestSuite) TestFailpointHandler(c *C) { resp, err = ts.fetchStatus("/fail/") c.Assert(err, IsNil) c.Assert(resp.StatusCode, Equals, http.StatusOK) - b, err := ioutil.ReadAll(resp.Body) + b, err := io.ReadAll(resp.Body) c.Assert(err, IsNil) c.Assert(strings.Contains(string(b), "github.com/pingcap/tidb/server/enableTestAPI=return"), IsTrue) c.Assert(resp.Body.Close(), IsNil) @@ -1488,7 +1487,7 @@ func (ts *HTTPHandlerTestSuite) TestDDLHookHandler(c *C) { resp, err = ts.postStatus("/test/ddl/hook", "application/x-www-form-urlencoded", bytes.NewBuffer([]byte(`ddl_hook=ctc_hook`))) c.Assert(err, IsNil) c.Assert(resp, NotNil) - body, err := ioutil.ReadAll(resp.Body) + body, err := io.ReadAll(resp.Body) c.Assert(err, IsNil) c.Assert(string(body), Equals, "\"success!\"") c.Assert(resp.StatusCode, Equals, http.StatusOK) @@ -1496,7 +1495,7 @@ func (ts *HTTPHandlerTestSuite) TestDDLHookHandler(c *C) { resp, err = ts.postStatus("/test/ddl/hook", "application/x-www-form-urlencoded", bytes.NewBuffer([]byte(`ddl_hook=default_hook`))) c.Assert(err, IsNil) c.Assert(resp, NotNil) - body, err = ioutil.ReadAll(resp.Body) + body, err = io.ReadAll(resp.Body) c.Assert(err, IsNil) c.Assert(string(body), Equals, "\"success!\"") c.Assert(resp.StatusCode, Equals, http.StatusOK) diff --git a/server/http_status.go b/server/http_status.go index b385ea0c45890..b0fc7615caf49 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -21,7 +21,7 @@ import ( "crypto/x509" "encoding/json" "fmt" - "io/ioutil" + "io" "net" "net/http" "net/http/pprof" @@ -193,7 +193,7 @@ func (s *Server) startHTTPServer() { _, err := w.Write([]byte(strconv.Itoa(util.GetGOGC()))) terror.Log(err) case http.MethodPost: - body, err := ioutil.ReadAll(r.Body) + body, err := io.ReadAll(r.Body) if err != nil { terror.Log(err) return diff --git a/server/server_test.go b/server/server_test.go index a342dab77d79a..4caa4f69563f7 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -19,7 +19,6 @@ import ( "encoding/json" "fmt" "io" - "io/ioutil" "math/rand" "net/http" "net/url" @@ -1913,7 +1912,7 @@ func (cli *testServerClient) runTestSumAvg(c *C) { func (cli *testServerClient) getMetrics(t *C) []byte { resp, err := cli.fetchStatus("/metrics") t.Assert(err, IsNil) - content, err := ioutil.ReadAll(resp.Body) + content, err := io.ReadAll(resp.Body) t.Assert(err, IsNil) err = resp.Body.Close() t.Assert(err, IsNil) @@ -1955,7 +1954,7 @@ func (cli *testServerClient) waitUntilServerOnline() { // fetch http status resp, err := cli.fetchStatus("/status") if err == nil { - _, err = ioutil.ReadAll(resp.Body) + _, err = io.ReadAll(resp.Body) if err != nil { panic(err) } diff --git a/server/statistics_handler_test.go b/server/statistics_handler_test.go index 2388392db57c8..4eb1e35f89752 100644 --- a/server/statistics_handler_test.go +++ b/server/statistics_handler_test.go @@ -16,7 +16,7 @@ package server import ( "database/sql" "fmt" - "io/ioutil" + "io" "os" "time" @@ -106,7 +106,7 @@ func (ds *testDumpStatsSuite) TestDumpStatsAPI(c *C) { c.Assert(os.Remove(path), IsNil) }() - js, err := ioutil.ReadAll(resp.Body) + js, err := io.ReadAll(resp.Body) c.Assert(err, IsNil) _, err = fp.Write(js) c.Assert(err, IsNil) @@ -123,7 +123,7 @@ func (ds *testDumpStatsSuite) TestDumpStatsAPI(c *C) { resp1, err := ds.fetchStatus("/stats/dump/tidb/test") c.Assert(err, IsNil) defer resp1.Body.Close() - js, err = ioutil.ReadAll(resp1.Body) + js, err = io.ReadAll(resp1.Body) c.Assert(err, IsNil) c.Assert(string(js), Equals, "null") @@ -139,7 +139,7 @@ func (ds *testDumpStatsSuite) TestDumpStatsAPI(c *C) { resp1, err = ds.fetchStatus("/stats/dump/tidb/test/" + snapshot) c.Assert(err, IsNil) - js, err = ioutil.ReadAll(resp1.Body) + js, err = io.ReadAll(resp1.Body) c.Assert(err, IsNil) _, err = fp1.Write(js) c.Assert(err, IsNil) diff --git a/server/tidb_test.go b/server/tidb_test.go index b9d46b3d2bf00..426e45cd0c57b 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -22,7 +22,6 @@ import ( "crypto/x509" "crypto/x509/pkix" "encoding/pem" - "io/ioutil" "math/big" "net/http" "os" @@ -342,7 +341,7 @@ func (ts *tidbTestSuite) TestStatusAPIWithTLSCNCheck(c *C) { func newTLSHttpClient(c *C, caFile, certFile, keyFile string) *http.Client { cert, err := tls.LoadX509KeyPair(certFile, keyFile) c.Assert(err, IsNil) - caCert, err := ioutil.ReadFile(caFile) + caCert, err := os.ReadFile(caFile) c.Assert(err, IsNil) caCertPool := x509.NewCertPool() caCertPool.AppendCertsFromPEM(caCert) @@ -500,7 +499,7 @@ func generateCert(sn int, commonName string, parentCert *x509.Certificate, paren // See https://godoc.org/github.com/go-sql-driver/mysql#RegisterTLSConfig for details. func registerTLSConfig(configName string, caCertPath string, clientCertPath string, clientKeyPath string, serverName string, verifyServer bool) error { rootCertPool := x509.NewCertPool() - data, err := ioutil.ReadFile(caCertPath) + data, err := os.ReadFile(caCertPath) if err != nil { return err } diff --git a/store/mockstore/unistore/cophandler/cop_handler_test.go b/store/mockstore/unistore/cophandler/cop_handler_test.go index 5cbd24f08a581..c013d04c404a3 100644 --- a/store/mockstore/unistore/cophandler/cop_handler_test.go +++ b/store/mockstore/unistore/cophandler/cop_handler_test.go @@ -16,7 +16,6 @@ package cophandler import ( "errors" "fmt" - "io/ioutil" "math" "os" "path/filepath" @@ -443,11 +442,11 @@ func (ts *testStore) commit(keys [][]byte, startTS, commitTS uint64) error { } func newTestStore(dbPrefix string, logPrefix string) (*testStore, error) { - dbPath, err := ioutil.TempDir("", dbPrefix) + dbPath, err := os.MkdirTemp("", dbPrefix) if err != nil { return nil, err } - LogPath, err := ioutil.TempDir("", logPrefix) + LogPath, err := os.MkdirTemp("", logPrefix) if err != nil { return nil, err } diff --git a/store/mockstore/unistore/mock.go b/store/mockstore/unistore/mock.go index c37235488477f..818016436f922 100644 --- a/store/mockstore/unistore/mock.go +++ b/store/mockstore/unistore/mock.go @@ -14,7 +14,6 @@ package unistore import ( - "io/ioutil" "os" "github.com/pingcap/errors" @@ -28,7 +27,7 @@ func New(path string) (*RPCClient, pd.Client, *Cluster, error) { persistent := true if path == "" { var err error - if path, err = ioutil.TempDir("", "tidb-unistore-temp"); err != nil { + if path, err = os.MkdirTemp("", "tidb-unistore-temp"); err != nil { return nil, nil, nil, err } persistent = false diff --git a/store/mockstore/unistore/tikv/mvcc_test.go b/store/mockstore/unistore/tikv/mvcc_test.go index 8819fe21ab398..c2564a5b4ebdf 100644 --- a/store/mockstore/unistore/tikv/mvcc_test.go +++ b/store/mockstore/unistore/tikv/mvcc_test.go @@ -16,7 +16,6 @@ package tikv import ( "bytes" "fmt" - "io/ioutil" "math" "os" "path/filepath" @@ -89,11 +88,11 @@ func CreateTestDB(dbPath, LogPath string) (*badger.DB, error) { } func NewTestStore(dbPrefix string, logPrefix string, c *C) (*TestStore, error) { - dbPath, err := ioutil.TempDir("", dbPrefix) + dbPath, err := os.MkdirTemp("", dbPrefix) if err != nil { return nil, err } - LogPath, err := ioutil.TempDir("", logPrefix) + LogPath, err := os.MkdirTemp("", logPrefix) if err != nil { return nil, err } diff --git a/store/tikv/config/security.go b/store/tikv/config/security.go index 4cb414c050675..513dc40791509 100644 --- a/store/tikv/config/security.go +++ b/store/tikv/config/security.go @@ -16,7 +16,7 @@ package config import ( "crypto/tls" "crypto/x509" - "io/ioutil" + "os" "github.com/pingcap/errors" ) @@ -45,7 +45,7 @@ func (s *Security) ToTLSConfig() (tlsConfig *tls.Config, err error) { certPool := x509.NewCertPool() // Create a certificate pool from the certificate authority var ca []byte - ca, err = ioutil.ReadFile(s.ClusterSSLCA) + ca, err = os.ReadFile(s.ClusterSSLCA) if err != nil { err = errors.Errorf("could not read ca certificate: %s", err) return diff --git a/tests/globalkilltest/go.mod b/tests/globalkilltest/go.mod index f5919d7a2df4b..0fccf3572c3fe 100644 --- a/tests/globalkilltest/go.mod +++ b/tests/globalkilltest/go.mod @@ -1,6 +1,6 @@ module github.com/pingcap/tests/globalkilltest -go 1.13 +go 1.16 require ( github.com/go-sql-driver/mysql v1.5.0 diff --git a/tidb-server/main.go b/tidb-server/main.go index 05f4ecc59c25a..ed1da6abfb7fb 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -17,7 +17,7 @@ import ( "context" "flag" "fmt" - "io/ioutil" + "io" "os" "runtime" "strconv" @@ -594,7 +594,7 @@ func setupLog() { if len(os.Getenv("GRPC_DEBUG")) > 0 { grpclog.SetLoggerV2(grpclog.NewLoggerV2WithVerbosity(os.Stderr, os.Stderr, os.Stderr, 999)) } else { - grpclog.SetLoggerV2(grpclog.NewLoggerV2(ioutil.Discard, ioutil.Discard, os.Stderr)) + grpclog.SetLoggerV2(grpclog.NewLoggerV2(io.Discard, io.Discard, os.Stderr)) } // trigger internal http(s) client init. util.InternalHTTPClient() diff --git a/tools/check/go.mod b/tools/check/go.mod index 8754267765791..f3827700f1589 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -22,4 +22,4 @@ require ( honnef.co/go/tools v0.0.0-20180920025451-e3ad64cb4ed3 ) -go 1.13 +go 1.16 diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index 22a6ac5b473dc..8c9eec60f99aa 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -16,7 +16,6 @@ package chunk import ( "bytes" "fmt" - "io/ioutil" "math" "os" "strconv" @@ -36,7 +35,7 @@ import ( ) func TestT(t *testing.T) { - path, _ := ioutil.TempDir("", "oom-use-tmp-storage") + path, _ := os.MkdirTemp("", "oom-use-tmp-storage") config.UpdateGlobal(func(conf *config.Config) { conf.TempStoragePath = path }) diff --git a/util/chunk/disk.go b/util/chunk/disk.go index ef269213e9d0d..47e6b25c0b492 100644 --- a/util/chunk/disk.go +++ b/util/chunk/disk.go @@ -16,7 +16,6 @@ package chunk import ( "errors" "io" - "io/ioutil" "os" "strconv" "sync" @@ -70,7 +69,7 @@ func (l *ListInDisk) initDiskFile() (err error) { if err != nil { return } - l.disk, err = ioutil.TempFile(config.GetGlobalConfig().TempStoragePath, defaultChunkListInDiskPath+strconv.Itoa(l.diskTracker.Label())) + l.disk, err = os.CreateTemp(config.GetGlobalConfig().TempStoragePath, defaultChunkListInDiskPath+strconv.Itoa(l.diskTracker.Label())) if err != nil { return errors2.Trace(err) } diff --git a/util/chunk/disk_test.go b/util/chunk/disk_test.go index 36750aa898244..a3cd929a4886f 100644 --- a/util/chunk/disk_test.go +++ b/util/chunk/disk_test.go @@ -17,7 +17,6 @@ import ( "bytes" "fmt" "io" - "io/ioutil" "math/rand" "os" "path/filepath" @@ -146,7 +145,7 @@ type listInDiskWriteDisk struct { func newListInDiskWriteDisk(fieldTypes []*types.FieldType) (*listInDiskWriteDisk, error) { l := listInDiskWriteDisk{*NewListInDisk(fieldTypes)} - disk, err := ioutil.TempFile(config.GetGlobalConfig().TempStoragePath, strconv.Itoa(l.diskTracker.Label())) + disk, err := os.CreateTemp(config.GetGlobalConfig().TempStoragePath, strconv.Itoa(l.diskTracker.Label())) if err != nil { return nil, err } diff --git a/util/disk/tempDir.go b/util/disk/tempDir.go index 2d42d21d82d39..7c0a1a9d45239 100644 --- a/util/disk/tempDir.go +++ b/util/disk/tempDir.go @@ -14,7 +14,6 @@ package disk import ( - "io/ioutil" "os" "path/filepath" @@ -81,7 +80,7 @@ func InitializeTempDir() error { return err } - subDirs, err := ioutil.ReadDir(tempDir) + subDirs, err := os.ReadDir(tempDir) if err != nil { return err } diff --git a/util/disk/tempDir_test.go b/util/disk/tempDir_test.go index 019f122f22da8..436f53a4c4a7f 100644 --- a/util/disk/tempDir_test.go +++ b/util/disk/tempDir_test.go @@ -14,7 +14,6 @@ package disk import ( - "io/ioutil" "os" "sync" "testing" @@ -24,7 +23,7 @@ import ( ) func TestT(t *testing.T) { - path, _ := ioutil.TempDir("", "tmp-storage-disk-pkg") + path, _ := os.MkdirTemp("", "tmp-storage-disk-pkg") config.UpdateGlobal(func(conf *config.Config) { conf.TempStoragePath = path }) diff --git a/util/expensivequery/memory_usage_alarm.go b/util/expensivequery/memory_usage_alarm.go index 03d53c12342c9..dbe13ccb973bc 100644 --- a/util/expensivequery/memory_usage_alarm.go +++ b/util/expensivequery/memory_usage_alarm.go @@ -15,7 +15,6 @@ package expensivequery import ( "fmt" - "io/ioutil" "os" "path/filepath" "runtime" @@ -65,7 +64,7 @@ func (record *memoryUsageAlarm) initMemoryUsageAlarmRecord() { } record.lastProfileFileName = make([][]string, 2) // Read last records - files, err := ioutil.ReadDir(record.tmpDir) + files, err := os.ReadDir(record.tmpDir) if err != nil { record.err = err return diff --git a/util/filesort/filesort_test.go b/util/filesort/filesort_test.go index fbbb48204c747..d518f28e9ac1d 100644 --- a/util/filesort/filesort_test.go +++ b/util/filesort/filesort_test.go @@ -14,7 +14,6 @@ package filesort import ( - "io/ioutil" "math/rand" "os" "testing" @@ -127,7 +126,7 @@ func (s *testFileSortSuite) TestInMemory(c *C) { ret bool ) - tmpDir, err = ioutil.TempDir("", "util_filesort_test") + tmpDir, err = os.MkdirTemp("", "util_filesort_test") c.Assert(err, IsNil) fsBuilder := new(Builder) @@ -177,7 +176,7 @@ func (s *testFileSortSuite) TestMultipleFiles(c *C) { ret bool ) - tmpDir, err = ioutil.TempDir("", "util_filesort_test") + tmpDir, err = os.MkdirTemp("", "util_filesort_test") c.Assert(err, IsNil) fsBuilder := new(Builder) @@ -245,7 +244,7 @@ func (s *testFileSortSuite) TestMultipleWorkers(c *C) { ret bool ) - tmpDir, err = ioutil.TempDir("", "util_filesort_test") + tmpDir, err = os.MkdirTemp("", "util_filesort_test") c.Assert(err, IsNil) fsBuilder := new(Builder) @@ -294,13 +293,13 @@ func (s *testFileSortSuite) TestClose(c *C) { // Prepare two FileSorter instances for tests fsBuilder := new(Builder) - tmpDir0, err = ioutil.TempDir("", "util_filesort_test") + tmpDir0, err = os.MkdirTemp("", "util_filesort_test") c.Assert(err, IsNil) fs0, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(1).SetDesc(byDesc).SetDir(tmpDir0).Build() c.Assert(err, IsNil) defer fs0.Close() - tmpDir1, err = ioutil.TempDir("", "util_filesort_test") + tmpDir1, err = os.MkdirTemp("", "util_filesort_test") c.Assert(err, IsNil) fs1, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(1).SetDesc(byDesc).SetDir(tmpDir1).Build() c.Assert(err, IsNil) @@ -373,13 +372,13 @@ func (s *testFileSortSuite) TestMismatchedUsage(c *C) { // Prepare two FileSorter instances for tests fsBuilder := new(Builder) - tmpDir, err = ioutil.TempDir("", "util_filesort_test") + tmpDir, err = os.MkdirTemp("", "util_filesort_test") c.Assert(err, IsNil) fs0, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(1).SetDesc(byDesc).SetDir(tmpDir).Build() c.Assert(err, IsNil) defer fs0.Close() - tmpDir, err = ioutil.TempDir("", "util_filesort_test") + tmpDir, err = os.MkdirTemp("", "util_filesort_test") c.Assert(err, IsNil) fs1, err = fsBuilder.SetSC(sc).SetSchema(keySize, valSize).SetBuf(bufSize).SetWorkers(1).SetDesc(byDesc).SetDir(tmpDir).Build() c.Assert(err, IsNil) diff --git a/util/format/format_test.go b/util/format/format_test.go index 6dd9b4f8d730f..5ce2159f48207 100644 --- a/util/format/format_test.go +++ b/util/format/format_test.go @@ -15,7 +15,7 @@ package format import ( "bytes" - "io/ioutil" + "io" "testing" . "github.com/pingcap/check" @@ -35,7 +35,7 @@ type testFormatSuite struct { func checkFormat(c *C, f Formatter, buf *bytes.Buffer, str, expect string) { _, err := f.Format(str, 3) c.Assert(err, IsNil) - b, err := ioutil.ReadAll(buf) + b, err := io.ReadAll(buf) c.Assert(err, IsNil) c.Assert(string(b), Equals, expect) } diff --git a/util/memory/meminfo.go b/util/memory/meminfo.go index 6289a46c8b286..3ba93e826ad5e 100644 --- a/util/memory/meminfo.go +++ b/util/memory/meminfo.go @@ -14,7 +14,7 @@ package memory import ( - "io/ioutil" + "os" "strconv" "strings" "sync" @@ -138,7 +138,7 @@ func init() { } func inContainer() bool { - v, err := ioutil.ReadFile(selfCGroupPath) + v, err := os.ReadFile(selfCGroupPath) if err != nil { return false } @@ -171,7 +171,7 @@ func parseUint(s string, base, bitSize int) (uint64, error) { // refer to https://github.com/containerd/cgroups/blob/318312a373405e5e91134d8063d04d59768a1bff/utils.go#L243 func readUint(path string) (uint64, error) { - v, err := ioutil.ReadFile(path) + v, err := os.ReadFile(path) if err != nil { return 0, err } diff --git a/util/misc.go b/util/misc.go index 683fc3857d930..817c5715af818 100644 --- a/util/misc.go +++ b/util/misc.go @@ -19,7 +19,6 @@ import ( "crypto/x509" "crypto/x509/pkix" "fmt" - "io/ioutil" "net" "net/http" "os" @@ -456,7 +455,7 @@ func LoadTLSCertificates(ca, key, cert string) (tlsConfig *tls.Config, err error var certPool *x509.CertPool if len(ca) > 0 { var caCert []byte - caCert, err = ioutil.ReadFile(ca) + caCert, err = os.ReadFile(ca) if err != nil { logutil.BgLogger().Warn("read file failed", zap.Error(err)) err = errors.Trace(err) diff --git a/util/profile/profile.go b/util/profile/profile.go index 3e598a4328441..ff817032617fe 100644 --- a/util/profile/profile.go +++ b/util/profile/profile.go @@ -16,7 +16,6 @@ package profile import ( "bytes" "io" - "io/ioutil" "runtime/pprof" "strconv" "strings" @@ -103,7 +102,7 @@ func (c *Collector) ProfileGraph(name string) ([][]types.Datum, error) { // ParseGoroutines returns the groutine list for given string representation func (c *Collector) ParseGoroutines(reader io.Reader) ([][]types.Datum, error) { - content, err := ioutil.ReadAll(reader) + content, err := io.ReadAll(reader) if err != nil { return nil, err } diff --git a/util/rowDecoder/decoder_test.go b/util/rowDecoder/decoder_test.go index 04d71dda0d19a..4439ecd90bdd7 100644 --- a/util/rowDecoder/decoder_test.go +++ b/util/rowDecoder/decoder_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/rowDecoder" + decoder "github.com/pingcap/tidb/util/rowDecoder" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" diff --git a/util/testutil/testutil.go b/util/testutil/testutil.go index d1ecd061a5624..5bb70ac21cf87 100644 --- a/util/testutil/testutil.go +++ b/util/testutil/testutil.go @@ -20,7 +20,7 @@ import ( "encoding/json" "flag" "fmt" - "io/ioutil" + "io" "os" "path/filepath" "reflect" @@ -287,7 +287,7 @@ func loadTestSuiteCases(filePath string) (res []testCases, err error) { err = err1 } }() - byteValue, err := ioutil.ReadAll(jsonFile) + byteValue, err := io.ReadAll(jsonFile) if err != nil { return res, err } From d5f01e1477fffc91158a02f08a6b24baf053d85f Mon Sep 17 00:00:00 2001 From: disksing Date: Tue, 25 May 2021 12:35:33 +0800 Subject: [PATCH 157/343] kv: extract subset of tikv.KeyFlags (#24739) --- kv/keyflags.go | 55 +++++++++++++++++++++++++++ kv/kv.go | 11 ++---- session/tidb_test.go | 3 +- session/txn.go | 5 +-- store/driver/txn/unionstore_driver.go | 43 +++++++++++++++++---- table/tables/index.go | 5 +-- table/tables/tables.go | 3 +- 7 files changed, 101 insertions(+), 24 deletions(-) create mode 100644 kv/keyflags.go diff --git a/kv/keyflags.go b/kv/keyflags.go new file mode 100644 index 0000000000000..8e721061409d1 --- /dev/null +++ b/kv/keyflags.go @@ -0,0 +1,55 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package kv + +// KeyFlags are metadata associated with key +type KeyFlags uint8 + +const ( + flagPresumeKNE KeyFlags = 1 << iota + flagNeedLocked +) + +// HasPresumeKeyNotExists returns whether the associated key use lazy check. +func (f KeyFlags) HasPresumeKeyNotExists() bool { + return f&flagPresumeKNE != 0 +} + +// HasNeedLocked returns whether the key needed to be locked +func (f KeyFlags) HasNeedLocked() bool { + return f&flagNeedLocked != 0 +} + +// FlagsOp describes KeyFlags modify operation. +type FlagsOp uint16 + +const ( + // SetPresumeKeyNotExists marks the existence of the associated key is checked lazily. + SetPresumeKeyNotExists FlagsOp = iota + // SetNeedLocked marks the associated key need to be acquired lock. + SetNeedLocked +) + +// ApplyFlagsOps applys flagspos to origin. +func ApplyFlagsOps(origin KeyFlags, ops ...FlagsOp) KeyFlags { + for _, op := range ops { + switch op { + case SetPresumeKeyNotExists: + origin |= flagPresumeKNE + case SetNeedLocked: + origin |= flagNeedLocked + } + } + return origin +} diff --git a/kv/kv.go b/kv/kv.go index 471dfe09a110b..0889106ba9fbf 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -44,9 +44,6 @@ var ( TxnTotalSizeLimit uint64 = config.DefTxnTotalSizeLimit ) -// FlagsOp describes KeyFlags modify operation. TODO:remove it when br is ready -type FlagsOp = tikvstore.FlagsOp - // Getter is the interface for the Get method. type Getter interface { // Get gets the value for key k from kv store. @@ -108,11 +105,11 @@ type MemBuffer interface { RUnlock() // GetFlags returns the latest flags associated with key. - GetFlags(Key) (tikvstore.KeyFlags, error) + GetFlags(Key) (KeyFlags, error) // SetWithFlags put key-value into the last active staging buffer with the given KeyFlags. - SetWithFlags(Key, []byte, ...tikvstore.FlagsOp) error + SetWithFlags(Key, []byte, ...FlagsOp) error // DeleteWithFlags delete key with the given KeyFlags - DeleteWithFlags(Key, ...tikvstore.FlagsOp) error + DeleteWithFlags(Key, ...FlagsOp) error // Staging create a new staging buffer inside the MemBuffer. // Subsequent writes will be temporarily stored in this new staging buffer. @@ -124,7 +121,7 @@ type MemBuffer interface { // If the changes are not published by `Release`, they will be discarded. Cleanup(StagingHandle) // InspectStage used to inspect the value updates in the given stage. - InspectStage(StagingHandle, func(Key, tikvstore.KeyFlags, []byte)) + InspectStage(StagingHandle, func(Key, KeyFlags, []byte)) // SnapshotGetter returns a Getter for a snapshot of MemBuffer. SnapshotGetter() Getter diff --git a/session/tidb_test.go b/session/tidb_test.go index 615f388ff1e2e..e351fa533a158 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/store/mockstore" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" @@ -211,7 +210,7 @@ func (s *testMainSuite) TestKeysNeedLock(c *C) { for _, tt := range tests { c.Assert(keyNeedToLock(tt.key, tt.val, 0), Equals, tt.need) } - flag := tikvstore.KeyFlags(1) + flag := kv.KeyFlags(1) c.Assert(flag.HasPresumeKeyNotExists(), IsTrue) c.Assert(keyNeedToLock(indexKey, deleteVal, flag), IsTrue) } diff --git a/session/txn.go b/session/txn.go index 294725f8efaa0..12b2bd7069db2 100644 --- a/session/txn.go +++ b/session/txn.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/logutil" @@ -342,7 +341,7 @@ func (txn *LazyTxn) KeysNeedToLock() ([]kv.Key, error) { } keys := make([]kv.Key, 0, txn.countHint()) buf := txn.Transaction.GetMemBuffer() - buf.InspectStage(txn.stagingHandle, func(k kv.Key, flags tikvstore.KeyFlags, v []byte) { + buf.InspectStage(txn.stagingHandle, func(k kv.Key, flags kv.KeyFlags, v []byte) { if !keyNeedToLock(k, v, flags) { return } @@ -351,7 +350,7 @@ func (txn *LazyTxn) KeysNeedToLock() ([]kv.Key, error) { return keys, nil } -func keyNeedToLock(k, v []byte, flags tikvstore.KeyFlags) bool { +func keyNeedToLock(k, v []byte, flags kv.KeyFlags) bool { isTableKey := bytes.HasPrefix(k, tablecodec.TablePrefix()) if !isTableKey { // meta key always need to lock. diff --git a/store/driver/txn/unionstore_driver.go b/store/driver/txn/unionstore_driver.go index b54bffa139700..3023140cd2c92 100644 --- a/store/driver/txn/unionstore_driver.go +++ b/store/driver/txn/unionstore_driver.go @@ -38,8 +38,8 @@ func (m *memBuffer) Delete(k kv.Key) error { return m.MemDB.Delete(k) } -func (m *memBuffer) DeleteWithFlags(k kv.Key, ops ...tikvstore.FlagsOp) error { - err := m.MemDB.DeleteWithFlags(k, ops...) +func (m *memBuffer) DeleteWithFlags(k kv.Key, ops ...kv.FlagsOp) error { + err := m.MemDB.DeleteWithFlags(k, getTiKVFlagsOps(ops)...) return derr.ToTiDBErr(err) } @@ -48,9 +48,9 @@ func (m *memBuffer) Get(_ context.Context, key kv.Key) ([]byte, error) { return data, derr.ToTiDBErr(err) } -func (m *memBuffer) GetFlags(key kv.Key) (tikvstore.KeyFlags, error) { +func (m *memBuffer) GetFlags(key kv.Key) (kv.KeyFlags, error) { data, err := m.MemDB.GetFlags(key) - return data, derr.ToTiDBErr(err) + return getTiDBKeyFlags(data), derr.ToTiDBErr(err) } func (m *memBuffer) Staging() kv.StagingHandle { @@ -65,9 +65,9 @@ func (m *memBuffer) Release(h kv.StagingHandle) { m.MemDB.Release(int(h)) } -func (m *memBuffer) InspectStage(handle kv.StagingHandle, f func(kv.Key, tikvstore.KeyFlags, []byte)) { +func (m *memBuffer) InspectStage(handle kv.StagingHandle, f func(kv.Key, kv.KeyFlags, []byte)) { tf := func(key []byte, flag tikvstore.KeyFlags, value []byte) { - f(kv.Key(key), flag, value) + f(kv.Key(key), getTiDBKeyFlags(flag), value) } m.MemDB.InspectStage(int(handle), tf) } @@ -78,7 +78,7 @@ func (m *memBuffer) Set(key kv.Key, value []byte) error { } func (m *memBuffer) SetWithFlags(key kv.Key, value []byte, ops ...kv.FlagsOp) error { - err := m.MemDB.SetWithFlags(key, value, ops...) + err := m.MemDB.SetWithFlags(key, value, getTiKVFlagsOps(ops)...) return derr.ToTiDBErr(err) } @@ -139,3 +139,32 @@ func newKVIterator(it unionstore.Iterator) kv.Iterator { func (it *tikvIterator) Key() kv.Key { return kv.Key(it.Iterator.Key()) } + +func getTiDBKeyFlags(flag tikvstore.KeyFlags) kv.KeyFlags { + var v kv.KeyFlags + if flag.HasPresumeKeyNotExists() { + v = kv.ApplyFlagsOps(v, kv.SetPresumeKeyNotExists) + } + if flag.HasNeedLocked() { + v = kv.ApplyFlagsOps(v, kv.SetNeedLocked) + } + return v +} + +func getTiKVFlagsOp(op kv.FlagsOp) tikvstore.FlagsOp { + switch op { + case kv.SetPresumeKeyNotExists: + return tikvstore.SetPresumeKeyNotExists + case kv.SetNeedLocked: + return tikvstore.SetNeedLocked + } + return 0 +} + +func getTiKVFlagsOps(ops []kv.FlagsOp) []tikvstore.FlagsOp { + v := make([]tikvstore.FlagsOp, len(ops)) + for i := range ops { + v[i] = getTiKVFlagsOp(ops[i]) + } + return v +} diff --git a/table/tables/index.go b/table/tables/index.go index ab6296390fbfd..aef03d0590aaa 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -210,7 +209,7 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue } if err != nil || len(value) == 0 { if sctx.GetSessionVars().LazyCheckKeyNotExists() && err != nil { - err = txn.GetMemBuffer().SetWithFlags(key, idxVal, tikvstore.SetPresumeKeyNotExists) + err = txn.GetMemBuffer().SetWithFlags(key, idxVal, kv.SetPresumeKeyNotExists) } else { err = txn.GetMemBuffer().Set(key, idxVal) } @@ -231,7 +230,7 @@ func (c *index) Delete(sc *stmtctx.StatementContext, txn kv.Transaction, indexed return err } if distinct { - err = txn.GetMemBuffer().DeleteWithFlags(key, tikvstore.SetNeedLocked) + err = txn.GetMemBuffer().DeleteWithFlags(key, kv.SetNeedLocked) } else { err = txn.GetMemBuffer().Delete(key) } diff --git a/table/tables/tables.go b/table/tables/tables.go index 37e6ab1103a89..4fb118c36c24d 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -770,7 +769,7 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts . } if setPresume { - err = memBuffer.SetWithFlags(key, value, tikvstore.SetPresumeKeyNotExists) + err = memBuffer.SetWithFlags(key, value, kv.SetPresumeKeyNotExists) } else { err = memBuffer.Set(key, value) } From 64bf4b1904b954efea7af8bc6ec33b5a8ff7c2eb Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Tue, 25 May 2021 12:53:33 +0800 Subject: [PATCH 158/343] ddl: wait 1pc only once for ddl reorg job (#24859) --- ddl/reorg.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/ddl/reorg.go b/ddl/reorg.go index 37bfe82d4ce3d..6b5048eaa1643 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -156,11 +156,6 @@ func (rc *reorgCtx) clean() { } func (w *worker) runReorgJob(t *meta.Meta, reorgInfo *reorgInfo, tblInfo *model.TableInfo, lease time.Duration, f func() error) error { - // lease = 0 means it's in an integration test. In this case we don't delay so the test won't run too slowly. - if lease > 0 { - delayForAsyncCommit() - } - job := reorgInfo.Job // This is for tests compatible, because most of the early tests try to build the reorg job manually // without reorg meta info, which will cause nil pointer in here. @@ -172,6 +167,12 @@ func (w *worker) runReorgJob(t *meta.Meta, reorgInfo *reorgInfo, tblInfo *model. } } if w.reorgCtx.doneCh == nil { + // Since reorg job will be interrupted for polling the cancel action outside. we don't need to wait for 2.5s + // for the later entrances. + // lease = 0 means it's in an integration test. In this case we don't delay so the test won't run too slowly. + if lease > 0 { + delayForAsyncCommit() + } // start a reorganization job w.wg.Add(1) w.reorgCtx.doneCh = make(chan error, 1) From 02049548a9691a44c16981c1c893d9ae6e038c43 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 24 May 2021 23:07:34 -0600 Subject: [PATCH 159/343] variable: Add getters for global/session scope (#24647) --- sessionctx/variable/noop.go | 2 - sessionctx/variable/sysvar.go | 119 ++++++++++++++++++++-- sessionctx/variable/sysvar_test.go | 17 ---- sessionctx/variable/varsutil.go | 153 +++++++---------------------- 4 files changed, 146 insertions(+), 145 deletions(-) diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 1ad37b512d807..053d028641016 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -205,8 +205,6 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: "sort_buffer_size", Value: "262144", IsHintUpdatable: true}, {Scope: ScopeGlobal, Name: "innodb_flush_neighbors", Value: "1"}, {Scope: ScopeNone, Name: "innodb_use_sys_malloc", Value: "1"}, - {Scope: ScopeSession, Name: PluginLoad, Value: ""}, - {Scope: ScopeSession, Name: PluginDir, Value: "/data/deploy/plugin"}, {Scope: ScopeNone, Name: "performance_schema_max_socket_classes", Value: "10"}, {Scope: ScopeNone, Name: "performance_schema_max_stage_classes", Value: "150"}, {Scope: ScopeGlobal, Name: "innodb_purge_batch_size", Value: "300"}, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index fadfda444e1ac..ee633e0d1afdf 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -14,6 +14,7 @@ package variable import ( + "encoding/json" "fmt" "math" "strconv" @@ -119,12 +120,51 @@ type SysVar struct { // Aliases is a list of sysvars that should also be updated when this sysvar is updated. // Updating aliases calls the SET function of the aliases, but does not update their aliases (preventing SET recursion) Aliases []string + // GetSession is a getter function for session scope. + // It can be used by instance-scoped variables to overwrite the previously expected value. + GetSession func(*SessionVars) (string, error) + // GetGlobal is a getter function for global scope. + GetGlobal func(*SessionVars) (string, error) // skipInit defines if the sysvar should be loaded into the session on init. // This is only important to set for sysvars that include session scope, // since global scoped sysvars are not-applicable. skipInit bool } +// GetGlobalFromHook calls the GetSession func if it exists. +func (sv *SysVar) GetGlobalFromHook(s *SessionVars) (string, error) { + // Call the Getter if there is one defined. + if sv.GetGlobal != nil { + return sv.GetGlobal(s) + } + if sv.HasNoneScope() { + return sv.Value, nil + } + return s.GlobalVarsAccessor.GetGlobalSysVar(sv.Name) +} + +// GetSessionFromHook calls the GetSession func if it exists. +func (sv *SysVar) GetSessionFromHook(s *SessionVars) (string, error) { + if sv.HasNoneScope() { + return sv.Value, nil + } + // Call the Getter if there is one defined. + if sv.GetSession != nil { + return sv.GetSession(s) + } + var ( + ok bool + val string + ) + if val, ok = s.stmtVars[sv.Name]; ok { + return val, nil + } + if val, ok = s.systems[sv.Name]; !ok { + return val, errors.New("sysvar has not yet loaded") + } + return val, nil +} + // SetSessionFromHook calls the SetSession func if it exists. func (sv *SysVar) SetSessionFromHook(s *SessionVars, val string) error { if sv.SetSession != nil { @@ -746,8 +786,12 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: MaxAllowedPacket, Value: "67108864", Type: TypeUnsigned, MinValue: 1024, MaxValue: MaxOfMaxAllowedPacket, AutoConvertOutOfRange: true}, - {Scope: ScopeSession, Name: WarningCount, Value: "0", ReadOnly: true, skipInit: true}, - {Scope: ScopeSession, Name: ErrorCount, Value: "0", ReadOnly: true, skipInit: true}, + {Scope: ScopeSession, Name: WarningCount, Value: "0", ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + return strconv.Itoa(s.SysWarningCount), nil + }}, + {Scope: ScopeSession, Name: ErrorCount, Value: "0", ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + return strconv.Itoa(int(s.SysErrorCount)), nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: WindowingUseHighPrecision, Value: On, Type: TypeBool, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error { s.WindowingUseHighPrecision = TiDBOptOn(val) return nil @@ -778,6 +822,8 @@ var defaultSysVars = []*SysVar{ return ErrWrongValueForVar.GenWithStack("@@txn_scope value should be global or local") } return nil + }, GetSession: func(s *SessionVars) (string, error) { + return s.TxnScope.GetVarValue(), nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "ENFORCE"}, SetSession: func(s *SessionVars, val string) error { s.allowMPPExecution = val @@ -938,9 +984,19 @@ var defaultSysVars = []*SysVar{ s.DMLBatchSize = int(tidbOptInt64(val, DefDMLBatchSize)) return nil }}, - {Scope: ScopeSession, Name: TiDBCurrentTS, Value: strconv.Itoa(DefCurretTS), skipInit: true, ReadOnly: true}, - {Scope: ScopeSession, Name: TiDBLastTxnInfo, Value: strconv.Itoa(DefCurretTS), skipInit: true, ReadOnly: true}, - {Scope: ScopeSession, Name: TiDBLastQueryInfo, Value: strconv.Itoa(DefCurretTS), skipInit: true, ReadOnly: true}, + {Scope: ScopeSession, Name: TiDBCurrentTS, Value: strconv.Itoa(DefCurretTS), ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + return fmt.Sprintf("%d", s.TxnCtx.StartTS), nil + }}, + {Scope: ScopeSession, Name: TiDBLastTxnInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + return s.LastTxnInfo, nil + }}, + {Scope: ScopeSession, Name: TiDBLastQueryInfo, Value: strconv.Itoa(DefCurretTS), ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + info, err := json.Marshal(s.LastQueryInfo) + if err != nil { + return "", err + } + return string(info), nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxChunkSize, Value: strconv.Itoa(DefMaxChunkSize), Type: TypeUnsigned, MinValue: maxChunkSizeLowerBound, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { s.MaxChunkSize = tidbOptPositiveInt32(val, DefMaxChunkSize) return nil @@ -1193,16 +1249,33 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TiDBGeneralLog, Value: BoolToOnOff(DefTiDBGeneralLog), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { ProcessGeneralLog.Store(TiDBOptOn(val)) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return BoolToOnOff(ProcessGeneralLog.Load()), nil }}, {Scope: ScopeSession, Name: TiDBPProfSQLCPU, Value: strconv.Itoa(DefTiDBPProfSQLCPU), Type: TypeInt, skipInit: true, MinValue: 0, MaxValue: 1, SetSession: func(s *SessionVars, val string) error { EnablePProfSQLCPU.Store(uint32(tidbOptPositiveInt32(val, DefTiDBPProfSQLCPU)) > 0) return nil + }, GetSession: func(s *SessionVars) (string, error) { + val := "0" + if EnablePProfSQLCPU.Load() { + val = "1" + } + return val, nil }}, {Scope: ScopeSession, Name: TiDBDDLSlowOprThreshold, Value: strconv.Itoa(DefTiDBDDLSlowOprThreshold), skipInit: true, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint32(&DDLSlowOprThreshold, uint32(tidbOptPositiveInt32(val, DefTiDBDDLSlowOprThreshold))) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return strconv.FormatUint(uint64(atomic.LoadUint32(&DDLSlowOprThreshold)), 10), nil + }}, + {Scope: ScopeSession, Name: TiDBConfig, Value: "", ReadOnly: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { + conf := config.GetGlobalConfig() + j, err := json.MarshalIndent(conf, "", "\t") + if err != nil { + return "", err + } + return config.HideConfig(string(j)), nil }}, - {Scope: ScopeSession, Name: TiDBConfig, Value: "", ReadOnly: true, skipInit: true}, {Scope: ScopeGlobal, Name: TiDBDDLReorgWorkerCount, Value: strconv.Itoa(DefTiDBDDLReorgWorkerCount), Type: TypeUnsigned, MinValue: 1, MaxValue: uint64(maxDDLReorgWorkerCount), SetSession: func(s *SessionVars, val string) error { SetDDLReorgWorkerCounter(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgWorkerCount))) return nil @@ -1243,6 +1316,8 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TiDBForcePriority, skipInit: true, Value: mysql.Priority2Str[DefTiDBForcePriority], SetSession: func(s *SessionVars, val string) error { atomic.StoreInt32(&ForcePriority, int32(mysql.Str2Priority(val))) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))], nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptJoinReorderThreshold, Value: strconv.Itoa(DefTiDBOptJoinReorderThreshold), skipInit: true, Type: TypeUnsigned, MinValue: 0, MaxValue: 63, SetSession: func(s *SessionVars, val string) error { s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) @@ -1268,10 +1343,14 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TiDBExpensiveQueryTimeThreshold, Value: strconv.Itoa(DefTiDBExpensiveQueryTimeThreshold), Type: TypeUnsigned, MinValue: int64(MinExpensiveQueryTimeThreshold), MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint64(&ExpensiveQueryTimeThreshold, uint64(tidbOptPositiveInt32(val, DefTiDBExpensiveQueryTimeThreshold))) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveQueryTimeThreshold)), nil }}, {Scope: ScopeSession, Name: TiDBMemoryUsageAlarmRatio, Value: strconv.FormatFloat(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0.0, MaxValue: 1.0, SetSession: func(s *SessionVars, val string) error { MemoryUsageAlarmRatio.Store(tidbOptFloat64(val, 0.8)) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return fmt.Sprintf("%g", MemoryUsageAlarmRatio.Load()), nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNoopFuncs, Value: BoolToOnOff(DefTiDBEnableNoopFuncs), Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { @@ -1319,7 +1398,9 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryHistorySize, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.HistorySize), Type: TypeInt, MinValue: 0, MaxValue: uint64(math.MaxUint8), AllowEmpty: true}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxStmtCount, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxStmtCount), 10), Type: TypeInt, MinValue: 1, MaxValue: uint64(math.MaxInt16), AllowEmpty: true}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxSQLLength, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxSQLLength), 10), Type: TypeInt, MinValue: 0, MaxValue: uint64(math.MaxInt32), AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBCapturePlanBaseline, Value: Off, Type: TypeBool, AllowEmptyAll: true}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBCapturePlanBaseline, Value: Off, Type: TypeBool, AllowEmptyAll: true, GetSession: func(s *SessionVars) (string, error) { + return CapturePlanBaseline.GetVal(), nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBUsePlanBaselines, Value: BoolToOnOff(DefTiDBUsePlanBaselines), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.UsePlanBaselines = TiDBOptOn(val) return nil @@ -1384,18 +1465,26 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TiDBSlowLogThreshold, Value: strconv.Itoa(logutil.DefaultSlowThreshold), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint64(&config.GetGlobalConfig().Log.SlowThreshold, uint64(tidbOptInt64(val, logutil.DefaultSlowThreshold))) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10), nil }}, {Scope: ScopeSession, Name: TiDBRecordPlanInSlowLog, Value: int32ToBoolStr(logutil.DefaultRecordPlanInSlowLog), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog, uint32(tidbOptInt64(val, logutil.DefaultRecordPlanInSlowLog))) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return strconv.FormatUint(uint64(atomic.LoadUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog)), 10), nil }}, {Scope: ScopeSession, Name: TiDBEnableSlowLog, Value: BoolToOnOff(logutil.DefaultTiDBEnableSlowLog), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { config.GetGlobalConfig().Log.EnableSlowLog = TiDBOptOn(val) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return BoolToOnOff(config.GetGlobalConfig().Log.EnableSlowLog), nil }}, {Scope: ScopeSession, Name: TiDBQueryLogMaxLen, Value: strconv.Itoa(logutil.DefaultQueryLogMaxLen), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, skipInit: true, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen, uint64(tidbOptInt64(val, logutil.DefaultQueryLogMaxLen))) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen), 10), nil }}, {Scope: ScopeGlobal | ScopeSession, Name: CTEMaxRecursionDepth, Value: strconv.Itoa(DefCTEMaxRecursionDepth), Type: TypeInt, MinValue: 0, MaxValue: 4294967295, AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { s.CTEMaxRecursionDepth = tidbOptInt(val, DefCTEMaxRecursionDepth) @@ -1404,14 +1493,20 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeSession, Name: TiDBCheckMb4ValueInUTF8, Value: BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8), nil }}, {Scope: ScopeSession, Name: TiDBFoundInPlanCache, Value: BoolToOnOff(DefTiDBFoundInPlanCache), Type: TypeBool, ReadOnly: true, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.FoundInPlanCache = TiDBOptOn(val) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return BoolToOnOff(s.PrevFoundInPlanCache), nil }}, {Scope: ScopeSession, Name: TiDBFoundInBinding, Value: BoolToOnOff(DefTiDBFoundInBinding), Type: TypeBool, ReadOnly: true, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.FoundInBinding = TiDBOptOn(val) return nil + }, GetSession: func(s *SessionVars) (string, error) { + return BoolToOnOff(s.PrevFoundInBinding), nil }}, {Scope: ScopeSession, Name: TiDBEnableCollectExecutionInfo, Value: BoolToOnOff(DefTiDBEnableCollectExecutionInfo), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { oldConfig := config.GetGlobalConfig() @@ -1422,6 +1517,8 @@ var defaultSysVars = []*SysVar{ config.StoreGlobalConfig(&newConfig) } return nil + }, GetSession: func(s *SessionVars) (string, error) { + return BoolToOnOff(config.GetGlobalConfig().EnableCollectExecutionInfo), nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowAutoRandExplicitInsert, Value: BoolToOnOff(DefTiDBAllowAutoRandExplicitInsert), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.AllowAutoRandExplicitInsert = TiDBOptOn(val) @@ -1449,6 +1546,8 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal | ScopeSession, Name: TiDBSlowLogMasking, Value: BoolToOnOff(DefTiDBRedactLog), Aliases: []string{TiDBRedactLog}, skipInit: true, Type: TypeBool, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TiDBSlowLogMasking, TiDBRedactLog) return normalizedValue, nil + }, GetSession: func(s *SessionVars) (string, error) { + return s.systems[TiDBRedactLog], nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBRedactLog, Value: BoolToOnOff(DefTiDBRedactLog), Aliases: []string{TiDBSlowLogMasking}, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableRedactLog = TiDBOptOn(val) @@ -1513,6 +1612,12 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeNone, Name: TiDBEnableEnhancedSecurity, Value: Off, Type: TypeBool}, + {Scope: ScopeSession, Name: PluginLoad, Value: "", GetSession: func(s *SessionVars) (string, error) { + return config.GetGlobalConfig().Plugin.Dir, nil + }}, + {Scope: ScopeSession, Name: PluginDir, Value: "/data/deploy/plugin", GetSession: func(s *SessionVars) (string, error) { + return config.GetGlobalConfig().Plugin.Load, nil + }}, /* tikv gc metrics */ {Scope: ScopeGlobal, Name: TiDBGCEnable, Value: On, Type: TypeBool}, diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index eb8d276ba7375..eb5090bbb5a42 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -434,23 +434,6 @@ func (*testSysVarSuite) TestReadOnlyNoop(c *C) { } } -func (*testSysVarSuite) TestGetScopeNoneSystemVar(c *C) { - val, ok, err := GetScopeNoneSystemVar(Port) - c.Assert(err, IsNil) - c.Assert(ok, IsTrue) - c.Assert(val, Equals, "4000") - - val, ok, err = GetScopeNoneSystemVar("nonsensevar") - c.Assert(err.Error(), Equals, "[variable:1193]Unknown system variable 'nonsensevar'") - c.Assert(ok, IsFalse) - c.Assert(val, Equals, "") - - val, ok, err = GetScopeNoneSystemVar(CharacterSetClient) - c.Assert(err, IsNil) - c.Assert(ok, IsFalse) - c.Assert(val, Equals, "") -} - func (*testSysVarSuite) TestSkipInit(c *C) { sv := SysVar{Scope: ScopeGlobal, Name: "skipinit1", Value: On, Type: TypeBool} c.Assert(sv.SkipInit(), IsTrue) diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 809420ca8ee6c..e47b5d46b9aa7 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -14,8 +14,6 @@ package variable import ( - "encoding/json" - "fmt" "strconv" "strings" "sync" @@ -25,7 +23,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/timeutil" @@ -161,128 +158,46 @@ func checkIsolationLevel(vars *SessionVars, normalizedValue string, originalValu return normalizedValue, nil } -// GetSessionOrGlobalSystemVar gets a system variable of session or global scope. -// It also respects TIDB's special "instance" scope in GetSessionOnlySysVars. -func GetSessionOrGlobalSystemVar(s *SessionVars, key string) (string, error) { - key = strings.ToLower(key) - gVal, ok, err := GetSessionOnlySysVars(s, key) - if err != nil || ok { - return gVal, err - } - gVal, err = s.GlobalVarsAccessor.GetGlobalSysVar(key) - if err != nil { - return "", err - } - // This cache results in incorrect behavior since changes to global - // variables will not be picked up. It should be removed once - // https://github.com/pingcap/tidb/issues/24368 is closed. - s.systems[key] = gVal - return gVal, nil -} - -// GetSessionOnlySysVars get the default value defined in code for session only variable. -// The return bool value indicates whether it's a session only variable. -func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { - sysVar := GetSysVar(key) - if sysVar == nil { - return "", false, ErrUnknownSystemVar.GenWithStackByArgs(key) - } - // For virtual system variables: - switch sysVar.Name { - case TiDBCurrentTS: - return fmt.Sprintf("%d", s.TxnCtx.StartTS), true, nil - case TiDBLastTxnInfo: - return s.LastTxnInfo, true, nil - case TiDBLastQueryInfo: - info, err := json.Marshal(s.LastQueryInfo) - if err != nil { - return "", true, err - } - return string(info), true, nil - case TiDBGeneralLog: - return BoolToOnOff(ProcessGeneralLog.Load()), true, nil - case TiDBPProfSQLCPU: - val := "0" - if EnablePProfSQLCPU.Load() { - val = "1" +// GetSessionOrGlobalSystemVar gets a system variable. +// If it is a session only variable, use the default value defined in code. +// Returns error if there is no such variable. +func GetSessionOrGlobalSystemVar(s *SessionVars, name string) (string, error) { + sv := GetSysVar(name) + if sv == nil { + return "", ErrUnknownSystemVar.GenWithStackByArgs(name) + } + if sv.HasNoneScope() { + s.systems[sv.Name] = sv.Value + return sv.Value, nil + } + if sv.HasSessionScope() { + // Populate the value to s.systems if it is not there already. + // in future should be already loaded on session init + if sv.GetSession != nil { + // shortcut to the getter, we won't use the value + return sv.GetSessionFromHook(s) } - return val, true, nil - case TiDBExpensiveQueryTimeThreshold: - return fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveQueryTimeThreshold)), true, nil - case TiDBMemoryUsageAlarmRatio: - return fmt.Sprintf("%g", MemoryUsageAlarmRatio.Load()), true, nil - case TiDBConfig: - conf := config.GetGlobalConfig() - j, err := json.MarshalIndent(conf, "", "\t") - if err != nil { - return "", false, err + if _, ok := s.systems[sv.Name]; !ok { + if sv.HasGlobalScope() { + if val, err := s.GlobalVarsAccessor.GetGlobalSysVar(sv.Name); err == nil { + s.systems[sv.Name] = val + } + } else { + s.systems[sv.Name] = sv.Value // no global scope, use default + } } - return config.HideConfig(string(j)), true, nil - case TiDBForcePriority: - return mysql.Priority2Str[mysql.PriorityEnum(atomic.LoadInt32(&ForcePriority))], true, nil - case TiDBDDLSlowOprThreshold: - return strconv.FormatUint(uint64(atomic.LoadUint32(&DDLSlowOprThreshold)), 10), true, nil - case PluginDir: - return config.GetGlobalConfig().Plugin.Dir, true, nil - case PluginLoad: - return config.GetGlobalConfig().Plugin.Load, true, nil - case TiDBSlowLogThreshold: - return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.SlowThreshold), 10), true, nil - case TiDBRecordPlanInSlowLog: - return strconv.FormatUint(uint64(atomic.LoadUint32(&config.GetGlobalConfig().Log.RecordPlanInSlowLog)), 10), true, nil - case TiDBEnableSlowLog: - return BoolToOnOff(config.GetGlobalConfig().Log.EnableSlowLog), true, nil - case TiDBQueryLogMaxLen: - return strconv.FormatUint(atomic.LoadUint64(&config.GetGlobalConfig().Log.QueryLogMaxLen), 10), true, nil - case TiDBCheckMb4ValueInUTF8: - return BoolToOnOff(config.GetGlobalConfig().CheckMb4ValueInUTF8), true, nil - case TiDBCapturePlanBaseline: - return CapturePlanBaseline.GetVal(), true, nil - case TiDBFoundInPlanCache: - return BoolToOnOff(s.PrevFoundInPlanCache), true, nil - case TiDBFoundInBinding: - return BoolToOnOff(s.PrevFoundInBinding), true, nil - case TiDBEnableCollectExecutionInfo: - return BoolToOnOff(config.GetGlobalConfig().EnableCollectExecutionInfo), true, nil - case TiDBTxnScope: - return s.TxnScope.GetVarValue(), true, nil - } - sVal, ok := s.GetSystemVar(key) - if ok { - return sVal, true, nil - } - if !sysVar.HasGlobalScope() { - // None-Global variable can use pre-defined default value. - return sysVar.Value, true, nil - } - return "", false, nil -} - -// GetGlobalSystemVar gets a global system variable. -func GetGlobalSystemVar(s *SessionVars, key string) (string, error) { - key = strings.ToLower(key) - gVal, ok, err := GetScopeNoneSystemVar(key) - if err != nil || ok { - return gVal, err + return sv.GetSessionFromHook(s) } - gVal, err = s.GlobalVarsAccessor.GetGlobalSysVar(key) - if err != nil { - return "", err - } - return gVal, nil + return sv.GetGlobalFromHook(s) } -// GetScopeNoneSystemVar checks the validation of `key`, -// and return the default value if its scope is `ScopeNone`. -func GetScopeNoneSystemVar(key string) (string, bool, error) { - sysVar := GetSysVar(key) - if sysVar == nil { - return "", false, ErrUnknownSystemVar.GenWithStackByArgs(key) - } - if sysVar.Scope == ScopeNone { - return sysVar.Value, true, nil +// GetGlobalSystemVar gets a global system variable. +func GetGlobalSystemVar(s *SessionVars, name string) (string, error) { + sv := GetSysVar(name) + if sv == nil { + return "", ErrUnknownSystemVar.GenWithStackByArgs(name) } - return "", false, nil + return sv.GetGlobalFromHook(s) } // epochShiftBits is used to reserve logical part of the timestamp. From 79669921ff2dab575fa3dba53d65c926481c03ff Mon Sep 17 00:00:00 2001 From: Howie Date: Tue, 25 May 2021 13:17:33 +0800 Subject: [PATCH 160/343] ddl: add region and split compatibility for temporary table (#24684) --- ddl/ddl_api.go | 6 ++++++ ddl/error.go | 3 +++ ddl/serial_test.go | 10 ++++++++++ errno/errcode.go | 1 + errno/errname.go | 1 + errors.toml | 10 ++++++++++ executor/executor_test.go | 17 +++++++++++++++++ planner/core/errors.go | 1 + planner/core/planbuilder.go | 11 +++++++++++ planner/core/preprocess.go | 15 +++++++++++++++ 10 files changed, 75 insertions(+) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index eaf73bccade73..9c3eca13c7fee 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1931,6 +1931,9 @@ func (d *ddl) CreateTableWithInfo( // preSplitAndScatter performs pre-split and scatter of the table's regions. // If `pi` is not nil, will only split region for `pi`, this is used when add partition. func (d *ddl) preSplitAndScatter(ctx sessionctx.Context, tbInfo *model.TableInfo, pi *model.PartitionInfo) { + if tbInfo.TempTableType != model.TempTableNone { + return + } sp, ok := d.store.(kv.SplittableStore) if !ok || atomic.LoadUint32(&EnableSplitTableRegion) == 0 { return @@ -2220,6 +2223,9 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err } tbInfo.MaxShardRowIDBits = tbInfo.ShardRowIDBits case ast.TableOptionPreSplitRegion: + if tbInfo.TempTableType != model.TempTableNone { + return errors.Trace(ErrOptOnTemporaryTable.GenWithStackByArgs("pre split regions")) + } tbInfo.PreSplitRegions = op.UintValue case ast.TableOptionCharset, ast.TableOptionCollate: // We don't handle charset and collate here since they're handled in `getCharsetAndCollateInTableOption`. diff --git a/ddl/error.go b/ddl/error.go index fb046f31c9f38..d587e9cb39143 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -279,6 +279,9 @@ var ( // ErrPartitionNoTemporary returns when partition at temporary mode ErrPartitionNoTemporary = dbterror.ClassDDL.NewStd(mysql.ErrPartitionNoTemporary) + // ErrOptOnTemporaryTable returns when exec unsupported opt at temporary mode + ErrOptOnTemporaryTable = dbterror.ClassDDL.NewStd(mysql.ErrOptOnTemporaryTable) + errUnsupportedOnCommitPreserve = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("TiDB doesn't support ON COMMIT PRESERVE ROWS for now", nil)) errUnsupportedEngineTemporary = dbterror.ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message("TiDB doesn't support this kind of engine for temporary table", nil)) ) diff --git a/ddl/serial_test.go b/ddl/serial_test.go index b891cce567392..f78ff3d16577b 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -524,6 +525,15 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) { tk.MustExec("drop database ctwl_db") tk.MustExec("drop database ctwl_db1") + + // Test create table like at temporary mode. + tk.MustExec("use test") + tk.MustExec("drop table if exists temporary_table;") + tk.MustExec("create global temporary table temporary_table (a int, b int,index(a)) on commit delete rows") + tk.MustExec("drop table if exists temporary_table_t1;") + _, err = tk.Exec("create table temporary_table_t1 like temporary_table") + c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error()) + tk.MustExec("drop table if exists temporary_table;") } // TestCancelAddIndex1 tests canceling ddl job when the add index worker is not started. diff --git a/errno/errcode.go b/errno/errcode.go index 1cb4889eccbc1..9947a8a726b71 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -922,6 +922,7 @@ const ( ErrAdminCheckTable = 8003 ErrTxnTooLarge = 8004 ErrWriteConflictInTiDB = 8005 + ErrOptOnTemporaryTable = 8006 ErrUnsupportedReloadPlugin = 8018 ErrUnsupportedReloadPluginVar = 8019 ErrTableLocked = 8020 diff --git a/errno/errname.go b/errno/errname.go index 98cbb17cd6b25..868e5dcc5231e 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -924,6 +924,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrMemExceedThreshold: mysql.Message("%s holds %dB memory, exceeds threshold %dB.%s", nil), ErrForUpdateCantRetry: mysql.Message("[%d] can not retry select for update statement", nil), ErrAdminCheckTable: mysql.Message("TiDB admin check table failed.", nil), + ErrOptOnTemporaryTable: mysql.Message("`%s` is unsupported on temporary tables.", nil), ErrTxnTooLarge: mysql.Message("Transaction is too large, size: %d", nil), ErrWriteConflictInTiDB: mysql.Message("Write conflict, txnStartTS %d is stale", nil), ErrInvalidPluginID: mysql.Message("Wrong plugin id: %s, valid plugin id is [name]-[version], both name and version should not contain '-'", nil), diff --git a/errors.toml b/errors.toml index 3d8b98f2368bb..ce6be545d6ab5 100644 --- a/errors.toml +++ b/errors.toml @@ -401,6 +401,11 @@ error = ''' Unknown SEQUENCE: '%-.300s' ''' +["ddl:8006"] +error = ''' +`%s` is unsupported on temporary tables. +''' + ["ddl:8200"] error = ''' Unsupported partition by range columns @@ -1136,6 +1141,11 @@ error = ''' Variable '%s' cannot be set using SET_VAR hint. ''' +["planner:8006"] +error = ''' +`%s` is unsupported on temporary tables. +''' + ["planner:8108"] error = ''' Unsupported type %T diff --git a/executor/executor_test.go b/executor/executor_test.go index 15594a73fe235..4ba3ea2e89e85 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5194,6 +5194,23 @@ func (s *testSplitTable) TestShowTableRegion(c *C) { // Test show table regions. tk.MustQuery(`split table t_regions between (-10000) and (10000) regions 4;`).Check(testkit.Rows("4 1")) re := tk.MustQuery("show table t_regions regions") + + // Test show table regions and split table on temporary table. + tk.MustExec("drop table if exists t_regions_temporary_table") + tk.MustExec("create global temporary table t_regions_temporary_table (a int key, b int, c int, index idx(b), index idx2(c)) ON COMMIT DELETE ROWS;") + // Test show table regions. + _, err = tk.Exec("show table t_regions_temporary_table regions") + c.Assert(err.Error(), Equals, plannercore.ErrOptOnTemporaryTable.GenWithStackByArgs("show table regions").Error()) + // Test split table. + _, err = tk.Exec("split table t_regions_temporary_table between (-10000) and (10000) regions 4;") + c.Assert(err.Error(), Equals, plannercore.ErrOptOnTemporaryTable.GenWithStackByArgs("split table").Error()) + _, err = tk.Exec("split partition table t_regions_temporary_table partition (p1,p2) index idx between (0) and (20000) regions 2;") + c.Assert(err.Error(), Equals, plannercore.ErrOptOnTemporaryTable.GenWithStackByArgs("split table").Error()) + tk.MustExec("drop table if exists t_regions_temporary_table") + // Test pre split regions + _, err = tk.Exec("create global temporary table temporary_table_pre_split(id int ) pre_split_regions=2 ON COMMIT DELETE ROWS;") + c.Assert(err.Error(), Equals, ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("pre split regions").Error()) + rows := re.Rows() // Table t_regions should have 5 regions now. // 4 regions to store record data. diff --git a/planner/core/errors.go b/planner/core/errors.go index 66d7c17e8a7a7..860a3444b5538 100644 --- a/planner/core/errors.go +++ b/planner/core/errors.go @@ -97,4 +97,5 @@ var ( ErrAccessDenied = dbterror.ClassOptimizer.NewStdErr(mysql.ErrAccessDenied, mysql.MySQLErrName[mysql.ErrAccessDeniedNoPassword]) ErrBadNull = dbterror.ClassOptimizer.NewStd(mysql.ErrBadNull) ErrNotSupportedWithSem = dbterror.ClassOptimizer.NewStd(mysql.ErrNotSupportedWithSem) + ErrOptOnTemporaryTable = dbterror.ClassOptimizer.NewStd(mysql.ErrOptOnTemporaryTable) ) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 2ae66a61602a5..f530908f8a6f7 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2207,6 +2207,14 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, err = ErrDBaccessDenied.GenWithStackByArgs(user.AuthUsername, user.AuthHostname, mysql.SystemDB) } b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, mysql.SystemDB, "", "", err) + case ast.ShowRegions: + tableInfo, err := b.is.TableByName(show.Table.Schema, show.Table.Name) + if err != nil { + return nil, err + } + if tableInfo.Meta().TempTableType != model.TempTableNone { + return nil, ErrOptOnTemporaryTable.GenWithStackByArgs("show table regions") + } } schema, names := buildShowSchema(show, isView, isSequence) p.SetSchema(schema) @@ -3102,6 +3110,9 @@ func (b *PlanBuilder) buildIndexAdvise(node *ast.IndexAdviseStmt) Plan { } func (b *PlanBuilder) buildSplitRegion(node *ast.SplitRegionStmt) (Plan, error) { + if node.Table.TableInfo.TempTableType != model.TempTableNone { + return nil, ErrOptOnTemporaryTable.GenWithStackByArgs("split table") + } if node.SplitSyntaxOpt != nil && node.SplitSyntaxOpt.HasPartition && node.Table.TableInfo.Partition == nil { return nil, ErrPartitionClauseOnNonpartitioned } diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index b5caf55e8de03..ca436189c0d41 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -585,6 +585,21 @@ func (p *preprocessor) checkAdminCheckTableGrammar(stmt *ast.AdminStmt) { } func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { + if stmt.ReferTable != nil { + schema := model.NewCIStr(p.ctx.GetSessionVars().CurrentDB) + if stmt.ReferTable.Schema.String() != "" { + schema = stmt.ReferTable.Schema + } + tableInfo, err := p.is.TableByName(schema, stmt.ReferTable.Name) + if err != nil { + p.err = err + return + } + if tableInfo.Meta().TempTableType != model.TempTableNone { + p.err = ErrOptOnTemporaryTable.GenWithStackByArgs("create table like") + return + } + } tName := stmt.Table.Name.String() if isIncorrectName(tName) { p.err = ddl.ErrWrongTableName.GenWithStackByArgs(tName) From 98f0d76bc770f661087b8eafaa3cbcb5a72df5ac Mon Sep 17 00:00:00 2001 From: JmPotato Date: Tue, 25 May 2021 14:25:33 +0800 Subject: [PATCH 161/343] *: clean up the deserted syntax of START TRANSACTION READ ONLY (#24813) --- executor/builder.go | 10 +- executor/simple.go | 104 ++--------------- executor/stale_txn_test.go | 136 +++++----------------- go.mod | 2 +- go.sum | 4 +- kv/fault_injection_test.go | 2 +- planner/core/common_plans.go | 4 +- planner/core/planbuilder.go | 5 +- session/session.go | 85 +++++--------- session/txn.go | 2 +- sessionctx/context.go | 13 +-- sessionctx/variable/varsutil.go | 12 +- store/tikv/kv.go | 27 ----- store/tikv/tests/2pc_test.go | 4 +- store/tikv/tests/extract_start_ts_test.go | 108 ----------------- store/tikv/txn.go | 86 ++------------ util/mock/context.go | 12 +- 17 files changed, 108 insertions(+), 508 deletions(-) delete mode 100644 store/tikv/tests/extract_start_ts_test.go diff --git a/executor/builder.go b/executor/builder.go index bf1ad7e46c9f0..2f08e604851a7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -706,11 +706,11 @@ func (b *executorBuilder) buildSimple(v *plannercore.Simple) Executor { base := newBaseExecutor(b.ctx, v.Schema(), v.ID()) base.initCap = chunk.ZeroCapacity e := &SimpleExec{ - baseExecutor: base, - Statement: v.Statement, - IsFromRemote: v.IsFromRemote, - is: b.is, - StalenessTxnOption: v.StalenessTxnOption, + baseExecutor: base, + Statement: v.Statement, + IsFromRemote: v.IsFromRemote, + is: b.is, + staleTxnStartTS: v.StaleTxnStartTS, } return e } diff --git a/executor/simple.go b/executor/simple.go index 9a0b4b919b1e3..2e9f8c89ea4be 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -39,10 +39,7 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" - "github.com/pingcap/tidb/types" - driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" @@ -75,8 +72,8 @@ type SimpleExec struct { done bool is infoschema.InfoSchema - // StalenessTxnOption is used to execute the staleness txn during a read-only begin statement. - StalenessTxnOption *sessionctx.StalenessTxnOption + // staleTxnStartTS is the StartTS that is used to execute the staleness txn during a read-only begin statement. + staleTxnStartTS uint64 } func (e *baseExecutor) getSysSession() (sessionctx.Context, error) { @@ -573,14 +570,18 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { // always create a new Txn instead of reusing it. if s.ReadOnly { enableNoopFuncs := e.ctx.GetSessionVars().EnableNoopFuncs - if !enableNoopFuncs && s.AsOf == nil && s.Bound == nil { + if !enableNoopFuncs && s.AsOf == nil { return expression.ErrFunctionsNoopImpl.GenWithStackByArgs("READ ONLY") } if s.AsOf != nil { - return e.executeStartTransactionReadOnlyWithBoundedStaleness(ctx, s) - } - if s.Bound != nil { - return e.executeStartTransactionReadOnlyWithTimestampBound(ctx, s) + if err := e.ctx.NewTxnWithStartTS(ctx, e.staleTxnStartTS); err != nil { + return err + } + // With START TRANSACTION, autocommit remains disabled until you end + // the transaction with COMMIT or ROLLBACK. The autocommit mode then + // reverts to its previous state. + e.ctx.GetSessionVars().SetInTxn(true) + return nil } } @@ -620,89 +621,6 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { return nil } -func (e *SimpleExec) executeStartTransactionReadOnlyWithBoundedStaleness(ctx context.Context, s *ast.BeginStmt) error { - if e.StalenessTxnOption == nil { - return errors.New("Failed to get timestamp during start transaction read only as of timestamp") - } - if err := e.ctx.NewTxnWithStalenessOption(ctx, *e.StalenessTxnOption); err != nil { - return err - } - - // With START TRANSACTION, autocommit remains disabled until you end - // the transaction with COMMIT or ROLLBACK. The autocommit mode then - // reverts to its previous state. - e.ctx.GetSessionVars().SetInTxn(true) - return nil -} - -// TODO: deprecate this syntax and only keep `AS OF TIMESTAMP` statement. -func (e *SimpleExec) executeStartTransactionReadOnlyWithTimestampBound(ctx context.Context, s *ast.BeginStmt) error { - opt := sessionctx.StalenessTxnOption{} - opt.Mode = s.Bound.Mode - switch s.Bound.Mode { - case ast.TimestampBoundReadTimestamp: - // TODO: support funcCallExpr in future - v, ok := s.Bound.Timestamp.(*driver.ValueExpr) - if !ok { - return errors.New("Invalid value for Bound Timestamp") - } - t, err := types.ParseTime(e.ctx.GetSessionVars().StmtCtx, v.GetString(), v.GetType().Tp, types.GetFsp(v.GetString())) - if err != nil { - return err - } - gt, err := t.GoTime(e.ctx.GetSessionVars().TimeZone) - if err != nil { - return err - } - opt.StartTS = oracle.GoTimeToTS(gt) - case ast.TimestampBoundExactStaleness: - // TODO: support funcCallExpr in future - v, ok := s.Bound.Timestamp.(*driver.ValueExpr) - if !ok { - return errors.New("Invalid value for Bound Timestamp") - } - d, err := types.ParseDuration(e.ctx.GetSessionVars().StmtCtx, v.GetString(), types.GetFsp(v.GetString())) - if err != nil { - return err - } - opt.PrevSec = uint64(d.Seconds()) - case ast.TimestampBoundMaxStaleness: - v, ok := s.Bound.Timestamp.(*driver.ValueExpr) - if !ok { - return errors.New("Invalid value for Bound Timestamp") - } - d, err := types.ParseDuration(e.ctx.GetSessionVars().StmtCtx, v.GetString(), types.GetFsp(v.GetString())) - if err != nil { - return err - } - opt.PrevSec = uint64(d.Seconds()) - case ast.TimestampBoundMinReadTimestamp: - v, ok := s.Bound.Timestamp.(*driver.ValueExpr) - if !ok { - return errors.New("Invalid value for Bound Timestamp") - } - t, err := types.ParseTime(e.ctx.GetSessionVars().StmtCtx, v.GetString(), v.GetType().Tp, types.GetFsp(v.GetString())) - if err != nil { - return err - } - gt, err := t.GoTime(e.ctx.GetSessionVars().TimeZone) - if err != nil { - return err - } - opt.StartTS = oracle.GoTimeToTS(gt) - } - err := e.ctx.NewTxnWithStalenessOption(ctx, opt) - if err != nil { - return err - } - - // With START TRANSACTION, autocommit remains disabled until you end - // the transaction with COMMIT or ROLLBACK. The autocommit mode then - // reverts to its previous state. - e.ctx.GetSessionVars().SetInTxn(true) - return nil -} - func (e *SimpleExec) executeRevokeRole(s *ast.RevokeRoleStmt) error { for _, role := range s.Roles { exists, err := userExists(e.ctx, role.Username, role.Hostname) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index a9336fbfac882..1f4f24a7f9f37 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -20,7 +20,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl/placement" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/testkit" ) @@ -32,66 +31,38 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { sql string IsStaleness bool expectPhysicalTS int64 - preSec int64 txnScope string zone string }{ { - name: "TimestampBoundExactStaleness", - preSQL: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:20';`, - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`, - IsStaleness: true, - expectPhysicalTS: 1599321600000, - txnScope: "local", - zone: "sh", - }, - { - name: "TimestampBoundReadTimestamp", + name: "AsOfTimestamp", preSQL: "begin", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`, + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`, IsStaleness: true, expectPhysicalTS: 1599321600000, txnScope: "local", - zone: "bj", - }, - { - name: "TimestampBoundExactStaleness", - preSQL: "begin", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:20';`, - IsStaleness: true, - preSec: 20, - txnScope: "local", - zone: "sh", - }, - { - name: "TimestampBoundExactStaleness", - preSQL: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`, - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:20';`, - IsStaleness: true, - preSec: 20, - txnScope: "local", - zone: "sz", + zone: "sh", }, { - name: "begin after TimestampBoundReadTimestamp", - preSQL: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`, + name: "begin after AsOfTimestamp", + preSQL: `START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`, sql: "begin", IsStaleness: false, - txnScope: kv.GlobalTxnScope, + txnScope: oracle.GlobalTxnScope, zone: "", }, { - name: "AsOfTimestamp", + name: "AsOfTimestamp with tidb_bounded_staleness", preSQL: "begin", - sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`, + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP tidb_bounded_staleness('2015-09-21 00:07:01', NOW());`, IsStaleness: true, - expectPhysicalTS: 1599321600000, + expectPhysicalTS: 1442765221000, txnScope: "local", - zone: "sh", + zone: "bj", }, { - name: "begin after AsOfTimestamp", - preSQL: `START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`, + name: "begin after AsOfTimestamp with tidb_bounded_staleness", + preSQL: `START TRANSACTION READ ONLY AS OF TIMESTAMP tidb_bounded_staleness('2015-09-21 00:07:01', NOW());`, sql: "begin", IsStaleness: false, txnScope: oracle.GlobalTxnScope, @@ -107,21 +78,15 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { tk.MustExec(fmt.Sprintf("set @@txn_scope=%v", testcase.txnScope)) tk.MustExec(testcase.preSQL) tk.MustExec(testcase.sql) + c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.IsStaleness) if testcase.expectPhysicalTS > 0 { c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, testcase.expectPhysicalTS) - } else if testcase.preSec > 0 { - curSec := time.Now().Unix() - startTS := oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS) - // exact stale txn tolerate 2 seconds deviation for startTS - c.Assert(startTS, Greater, (curSec-testcase.preSec-2)*1000) - c.Assert(startTS, Less, (curSec-testcase.preSec+2)*1000) } else if !testcase.IsStaleness { - curSec := time.Now().Unix() + curTS := oracle.ExtractPhysical(oracle.GoTimeToTS(time.Now())) startTS := oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS) - c.Assert(curSec*1000-startTS, Less, time.Second/time.Millisecond) - c.Assert(startTS-curSec*1000, Less, time.Second/time.Millisecond) + c.Assert(curTS-startTS, Less, time.Second.Milliseconds()) + c.Assert(startTS-curTS, Less, time.Second.Milliseconds()) } - c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.IsStaleness) tk.MustExec("commit") } failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") @@ -168,9 +133,6 @@ func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) tk.MustQuery(testcase.sql) tk.MustExec(`commit`) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) - tk.MustQuery(testcase.sql) - tk.MustExec(`commit`) } failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") failpoint.Disable("github.com/pingcap/tidb/store/tikv/assertStoreLabels") @@ -199,16 +161,6 @@ func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) tk.MustExec("commit") - // set @@tidb_snapshot before staleness txn - tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`) - c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) - tk.MustExec("commit") - // set @@tidb_snapshot during staleness txn - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`) - tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) - c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) - tk.MustExec("commit") } func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { @@ -217,7 +169,6 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key);") defer tk.MustExec(`drop table if exists t`) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`) testcases := []struct { name string sql string @@ -225,36 +176,6 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { // compareWithSafeTS will be 0 if StartTS==SafeTS, -1 if StartTS < SafeTS, and +1 if StartTS > SafeTS. compareWithSafeTS int }{ - { - name: "max 20 seconds ago, safeTS 10 secs ago", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:20'`, - injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)), - compareWithSafeTS: 0, - }, - { - name: "max 10 seconds ago, safeTS 20 secs ago", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`, - injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)), - compareWithSafeTS: 1, - }, - { - name: "max 20 seconds ago, safeTS 10 secs ago", - sql: func() string { - return fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MIN READ TIMESTAMP '%v'`, - time.Now().Add(-20*time.Second).Format("2006-01-02 15:04:05")) - }(), - injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)), - compareWithSafeTS: 0, - }, - { - name: "max 10 seconds ago, safeTS 20 secs ago", - sql: func() string { - return fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MIN READ TIMESTAMP '%v'`, - time.Now().Add(-10*time.Second).Format("2006-01-02 15:04:05")) - }(), - injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)), - compareWithSafeTS: 1, - }, { name: "20 seconds ago to now, safeTS 10 secs ago", sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP tidb_bounded_staleness(NOW() - INTERVAL 20 SECOND, NOW())`, @@ -273,6 +194,18 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-5 * time.Second)), compareWithSafeTS: -1, }, + { + name: "exact timestamp 5 seconds ago, safeTS 10 secs ago", + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP NOW() - INTERVAL 5 SECOND`, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)), + compareWithSafeTS: 1, + }, + { + name: "exact timestamp 10 seconds ago, safeTS 5 secs ago", + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP NOW() - INTERVAL 10 SECOND`, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-5 * time.Second)), + compareWithSafeTS: -1, + }, } for _, testcase := range testcases { c.Log(testcase.name) @@ -300,21 +233,12 @@ func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key);") - // test exact + // test as of schemaVer1 := tk.Se.GetInfoSchema().SchemaMetaVersion() time1 := time.Now() tk.MustExec("drop table if exists t") c.Assert(schemaVer1, Less, tk.Se.GetInfoSchema().SchemaMetaVersion()) - tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) + tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) tk.MustExec("commit") - - // test as of - schemaVer2 := tk.Se.GetInfoSchema().SchemaMetaVersion() - time2 := time.Now() - tk.MustExec("create table t (id int primary key);") - c.Assert(schemaVer2, Less, tk.Se.GetInfoSchema().SchemaMetaVersion()) - tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time2.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) - tk.MustExec("commit") } diff --git a/go.mod b/go.mod index 32d76211a667b..7b06ca2a8682d 100644 --- a/go.mod +++ b/go.mod @@ -37,7 +37,7 @@ require ( github.com/opentracing/opentracing-go v1.1.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 - github.com/pingcap/br v5.0.0-nightly.0.20210419090151-03762465b589+incompatible + github.com/pingcap/br v5.1.0-alpha.0.20210524083733-58e9e24fcb8e+incompatible github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3 github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd diff --git a/go.sum b/go.sum index c488db7c0347a..ac5cfe59f6b17 100644 --- a/go.sum +++ b/go.sum @@ -407,8 +407,8 @@ github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5e github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 h1:IXpGy7y9HyoShAFmzW2OPF0xCA5EOoSTyZHwsgYk9Ro= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= -github.com/pingcap/br v5.0.0-nightly.0.20210419090151-03762465b589+incompatible h1:VF2oZgvBqSIMmplEWXGGmktuQGdGGIGWwptmjJFhQbU= -github.com/pingcap/br v5.0.0-nightly.0.20210419090151-03762465b589+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= +github.com/pingcap/br v5.1.0-alpha.0.20210524083733-58e9e24fcb8e+incompatible h1:rRJG+3sGr4rbC3PUGpIhaO4uVB3ZssyJxZyLW8yFwBE= +github.com/pingcap/br v5.1.0-alpha.0.20210524083733-58e9e24fcb8e+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= diff --git a/kv/fault_injection_test.go b/kv/fault_injection_test.go index c5e203151fe63..09ee3a14dae3c 100644 --- a/kv/fault_injection_test.go +++ b/kv/fault_injection_test.go @@ -35,7 +35,7 @@ func (s testFaultInjectionSuite) TestFaultInjectionBasic(c *C) { storage := NewInjectedStore(newMockStorage(), &cfg) txn, err := storage.Begin() c.Assert(err, IsNil) - _, err = storage.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(GlobalTxnScope).SetStartTs(0)) + _, err = storage.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(GlobalTxnScope).SetStartTS(0)) c.Assert(err, IsNil) ver := Version{Ver: 1} snap := storage.GetSnapshot(ver) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index b42a84d926eeb..7170c991be051 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -724,8 +724,8 @@ type Simple struct { // Used for `global kill`. See https://github.com/pingcap/tidb/blob/master/docs/design/2020-06-01-global-kill.md. IsFromRemote bool - // StalenessTxnOption is the transaction option that will be built when planner builder calls buildSimple. - StalenessTxnOption *sessionctx.StalenessTxnOption + // StaleTxnStartTS is the StartTS that is used to build a staleness transaction by 'START TRANSACTION READ ONLY' statement. + StaleTxnStartTS uint64 } // PhysicalSimpleWrapper is a wrapper of `Simple` to implement physical plan interface. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index f530908f8a6f7..cac9dab16d8ab 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2340,10 +2340,7 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, if err != nil { return nil, err } - p.StalenessTxnOption = &sessionctx.StalenessTxnOption{ - Mode: ast.TimestampBoundReadTimestamp, - StartTS: startTS, - } + p.StaleTxnStartTS = startTS } } return p, nil diff --git a/session/session.go b/session/session.go index 9ef43bef3d7ed..a479ba956cf63 100644 --- a/session/session.go +++ b/session/session.go @@ -2000,6 +2000,35 @@ func (s *session) checkBeforeNewTxn(ctx context.Context) error { return nil } +// NewTxnWithStartTS create a transaction with the given StartTS. +func (s *session) NewTxnWithStartTS(ctx context.Context, startTS uint64) error { + if err := s.checkBeforeNewTxn(ctx); err != nil { + return err + } + txnScope := s.GetSessionVars().CheckAndGetTxnScope() + txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetStartTS(startTS)) + if err != nil { + return err + } + txn.SetVars(s.sessionVars.KVVars) + txn.SetOption(kv.IsStalenessReadOnly, true) + txn.SetOption(kv.TxnScope, txnScope) + s.txn.changeInvalidToValid(txn) + is, err := domain.GetDomain(s).GetSnapshotInfoSchema(txn.StartTS()) + if err != nil { + return errors.Trace(err) + } + s.sessionVars.TxnCtx = &variable.TransactionContext{ + InfoSchema: is, + CreateTime: time.Now(), + StartTS: txn.StartTS(), + ShardStep: int(s.sessionVars.ShardAllocateStep), + IsStaleness: true, + TxnScope: txnScope, + } + return nil +} + func (s *session) SetValue(key fmt.Stringer, value interface{}) { s.mu.Lock() s.mu.values[key] = value @@ -2648,7 +2677,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { } // no need to get txn from txnFutureCh since txn should init with startTs - txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTs(startTS)) + txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTS(startTS)) if err != nil { return err } @@ -2661,60 +2690,6 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { return nil } -// NewTxnWithStalenessOption create a transaction with Staleness option -func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionctx.StalenessTxnOption) error { - err := s.checkBeforeNewTxn(ctx) - if err != nil { - return err - } - var ( - txn kv.Transaction - txnScope = s.GetSessionVars().CheckAndGetTxnScope() - ) - switch option.Mode { - case ast.TimestampBoundReadTimestamp: - txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetStartTs(option.StartTS)) - if err != nil { - return err - } - case ast.TimestampBoundExactStaleness: - txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetPrevSec(option.PrevSec)) - if err != nil { - return err - } - case ast.TimestampBoundMaxStaleness: - txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetMaxPrevSec(option.PrevSec)) - if err != nil { - return err - } - case ast.TimestampBoundMinReadTimestamp: - txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetMinStartTS(option.StartTS)) - if err != nil { - return err - } - default: - // For unsupported staleness txn cases, fallback to NewTxn - return s.NewTxn(ctx) - } - txn.SetVars(s.sessionVars.KVVars) - txn.SetOption(kv.IsStalenessReadOnly, true) - txn.SetOption(kv.TxnScope, txnScope) - s.txn.changeInvalidToValid(txn) - is, err := domain.GetDomain(s).GetSnapshotInfoSchema(txn.StartTS()) - if err != nil { - return errors.Trace(err) - } - s.sessionVars.TxnCtx = &variable.TransactionContext{ - InfoSchema: is, - CreateTime: time.Now(), - StartTS: txn.StartTS(), - ShardStep: int(s.sessionVars.ShardAllocateStep), - IsStaleness: true, - TxnScope: txnScope, - } - return nil -} - // GetStore gets the store of session. func (s *session) GetStore() kv.Storage { return s.store diff --git a/session/txn.go b/session/txn.go index 12b2bd7069db2..df4f2d7a62bed 100644 --- a/session/txn.go +++ b/session/txn.go @@ -436,7 +436,7 @@ type txnFuture struct { func (tf *txnFuture) wait() (kv.Transaction, error) { startTS, err := tf.future.Wait() if err == nil { - return tf.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(tf.txnScope).SetStartTs(startTS)) + return tf.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(tf.txnScope).SetStartTS(startTS)) } else if config.GetGlobalConfig().Store == "unistore" { return nil, err } diff --git a/sessionctx/context.go b/sessionctx/context.go index 68767aef30e7c..96ede63f901ae 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -17,7 +17,6 @@ import ( "context" "fmt" - "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/owner" @@ -42,6 +41,8 @@ type Context interface { // If old transaction is valid, it is committed first. // It's used in BEGIN statement and DDL statements to commit old transaction. NewTxn(context.Context) error + // NewTxnWithStartTS initializes a transaction with the given StartTS. + NewTxnWithStartTS(ctx context.Context, startTS uint64) error // Txn returns the current transaction which is created before executing a statement. // The returned kv.Transaction is not nil, but it maybe pending or invalid. @@ -83,9 +84,6 @@ type Context interface { // It should be called right before we builds an executor. InitTxnWithStartTS(startTS uint64) error - // NewTxnWithStalenessOption initializes a transaction with StalenessTxnOption. - NewTxnWithStalenessOption(ctx context.Context, option StalenessTxnOption) error - // GetStore returns the store of session. GetStore() kv.Storage @@ -151,10 +149,3 @@ const ( // LastExecuteDDL is the key for whether the session execute a ddl command last time. LastExecuteDDL basicCtxType = 3 ) - -// StalenessTxnOption represents available options for the InitTxnWithStaleness -type StalenessTxnOption struct { - Mode ast.TimestampBoundMode - PrevSec uint64 - StartTS uint64 -} diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index e47b5d46b9aa7..d42128e73c612 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/timeutil" @@ -200,9 +201,6 @@ func GetGlobalSystemVar(s *SessionVars, name string) (string, error) { return sv.GetGlobalFromHook(s) } -// epochShiftBits is used to reserve logical part of the timestamp. -const epochShiftBits = 18 - // SetSessionSystemVar sets system variable and updates SessionVars states. func SetSessionSystemVar(vars *SessionVars, name string, value string) error { sysVar := GetSysVar(name) @@ -376,16 +374,10 @@ func setSnapshotTS(s *SessionVars, sVal string) error { } t1, err := t.GoTime(s.TimeZone) - s.SnapshotTS = GoTimeToTS(t1) + s.SnapshotTS = oracle.GoTimeToTS(t1) return err } -// GoTimeToTS converts a Go time to uint64 timestamp. -func GoTimeToTS(t time.Time) uint64 { - ts := (t.UnixNano() / int64(time.Millisecond)) << epochShiftBits - return uint64(ts) -} - // serverGlobalVariable is used to handle variables that acts in server and global scope. type serverGlobalVariable struct { sync.Mutex diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 622313f382abd..e7fe6a8309475 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -240,13 +240,6 @@ func (s *KVStore) CurrentTimestamp(txnScope string) (uint64, error) { } func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, error) { - failpoint.Inject("MockCurrentTimestamp", func(val failpoint.Value) { - if v, ok := val.(int); ok { - failpoint.Return(uint64(v), nil) - } else { - panic("MockCurrentTimestamp should be a number, try use this failpoint with \"return(ts)\"") - } - }) if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("TiKVStore.getTimestampWithRetry", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -275,26 +268,6 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, } } -func (s *KVStore) getStalenessTimestamp(bo *Backoffer, txnScope string, prevSec uint64) (uint64, error) { - failpoint.Inject("MockStalenessTimestamp", func(val failpoint.Value) { - if v, ok := val.(int); ok { - failpoint.Return(uint64(v), nil) - } else { - panic("MockStalenessTimestamp should be a number, try use this failpoint with \"return(ts)\"") - } - }) - for { - startTS, err := s.oracle.GetStaleTimestamp(bo.GetCtx(), txnScope, prevSec) - if err == nil { - return startTS, nil - } - err = bo.Backoff(retry.BoPDRPC, errors.Errorf("get staleness timestamp failed: %v", err)) - if err != nil { - return 0, errors.Trace(err) - } - } -} - func (s *KVStore) nextReplicaReadSeed() uint32 { return atomic.AddUint32(&s.replicaReadSeed, 1) } diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 8fea337bfd61b..d56436232e935 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -602,12 +602,12 @@ func (s *testCommitterSuite) TestRejectCommitTS(c *C) { // Use max.Uint64 to read the data and success. // That means the final commitTS > startTS+2, it's not the one we provide. // So we cover the rety commitTS logic. - txn1, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTs(committer.GetStartTS() + 2)) + txn1, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTS(committer.GetStartTS() + 2)) c.Assert(err, IsNil) _, err = txn1.Get(bo.GetCtx(), []byte("x")) c.Assert(tikverr.IsErrNotFound(err), IsTrue) - txn2, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTs(math.MaxUint64)) + txn2, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTS(math.MaxUint64)) c.Assert(err, IsNil) val, err := txn2.Get(bo.GetCtx(), []byte("x")) c.Assert(err, IsNil) diff --git a/store/tikv/tests/extract_start_ts_test.go b/store/tikv/tests/extract_start_ts_test.go deleted file mode 100644 index 82f37796dce8a..0000000000000 --- a/store/tikv/tests/extract_start_ts_test.go +++ /dev/null @@ -1,108 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tikv_test - -import ( - . "github.com/pingcap/check" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/store/mockstore/unistore" - "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/tikvrpc" -) - -type extractStartTsSuite struct { - store *tikv.KVStore -} - -var _ = SerialSuites(&extractStartTsSuite{}) - -func (s *extractStartTsSuite) SetUpTest(c *C) { - client, pdClient, cluster, err := unistore.New("") - c.Assert(err, IsNil) - unistore.BootstrapWithSingleStore(cluster) - store, err := tikv.NewTestTiKVStore(client, pdClient, nil, nil, 0) - c.Assert(err, IsNil) - probe := tikv.StoreProbe{KVStore: store} - probe.SetRegionCacheStore(2, tikvrpc.TiKV, 1, []*metapb.StoreLabel{ - { - Key: tikv.DCLabelKey, - Value: "local1", - }, - }) - probe.SetRegionCacheStore(3, tikvrpc.TiKV, 1, []*metapb.StoreLabel{ - { - Key: tikv.DCLabelKey, - Value: "Some Random Label", - }, - }) - probe.SetSafeTS(2, 102) - probe.SetSafeTS(3, 101) - s.store = probe.KVStore -} - -func (s *extractStartTsSuite) TestExtractStartTs(c *C) { - i := uint64(100) - // to prevent time change during test case execution - // we use failpoint to make it "fixed" - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockStalenessTimestamp", "return(200)"), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockCurrentTimestamp", `return(300)`), IsNil) - - cases := []struct { - expectedTS uint64 - option tikv.StartTSOption - }{ - // StartTS setted - {100, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, - // PrevSec setted - {200, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}}, - // MinStartTS setted, global - {101, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, - // MinStartTS setted, local - {102, tikv.StartTSOption{TxnScope: "local1", StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, - // MaxPrevSec setted - // however we need to add more cases to check the behavior when it fall backs to MinStartTS setted - // see `TestMaxPrevSecFallback` - {200, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, - // nothing setted - {300, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, - } - for _, cs := range cases { - expected := cs.expectedTS - result, _ := tikv.ExtractStartTs(s.store, cs.option) - c.Assert(result, Equals, expected) - } - - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockStalenessTimestamp"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockCurrentTimestamp"), IsNil) -} - -func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { - probe := tikv.StoreProbe{KVStore: s.store} - probe.SetSafeTS(2, 0x8000000000000002) - probe.SetSafeTS(3, 0x8000000000000001) - i := uint64(100) - cases := []struct { - expectedTS uint64 - option tikv.StartTSOption - }{ - {0x8000000000000001, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, - {0x8000000000000002, tikv.StartTSOption{TxnScope: "local1", StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, - } - for _, cs := range cases { - result, _ := tikv.ExtractStartTs(s.store, cs.option) - c.Assert(result, Equals, cs.expectedTS) - } -} diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 988f6501be553..78fd2d5277a82 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -30,14 +30,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" tikverr "github.com/pingcap/tidb/store/tikv/error" tikv "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" "go.uber.org/zap" @@ -58,11 +56,8 @@ type SchemaAmender interface { // `TxnScope` must be set for each object // Every other fields are optional, but currently at most one of them can be set type StartTSOption struct { - TxnScope string - StartTS *uint64 - PrevSec *uint64 - MinStartTS *uint64 - MaxPrevSec *uint64 + TxnScope string + StartTS *uint64 } // DefaultStartTSOption creates a default StartTSOption, ie. Work in GlobalTxnScope and get start ts when got used @@ -70,30 +65,12 @@ func DefaultStartTSOption() StartTSOption { return StartTSOption{TxnScope: oracle.GlobalTxnScope} } -// SetMaxPrevSec returns a new StartTSOption with MaxPrevSec set to maxPrevSec -func (to StartTSOption) SetMaxPrevSec(maxPrevSec uint64) StartTSOption { - to.MaxPrevSec = &maxPrevSec - return to -} - -// SetMinStartTS returns a new StartTSOption with MinStartTS set to minStartTS -func (to StartTSOption) SetMinStartTS(minStartTS uint64) StartTSOption { - to.MinStartTS = &minStartTS - return to -} - -// SetStartTs returns a new StartTSOption with StartTS set to startTS -func (to StartTSOption) SetStartTs(startTS uint64) StartTSOption { +// SetStartTS returns a new StartTSOption with StartTS set to the given startTS +func (to StartTSOption) SetStartTS(startTS uint64) StartTSOption { to.StartTS = &startTS return to } -// SetPrevSec returns a new StartTSOption with PrevSec set to prevSec -func (to StartTSOption) SetPrevSec(prevSec uint64) StartTSOption { - to.PrevSec = &prevSec - return to -} - // SetTxnScope returns a new StartTSOption with TxnScope set to txnScope func (to StartTSOption) SetTxnScope(txnScope string) StartTSOption { to.TxnScope = txnScope @@ -135,68 +112,29 @@ type KVTxn struct { kvFilter KVFilter } -// ExtractStartTs use `option` to get the proper startTS for a transaction -func ExtractStartTs(store *KVStore, option StartTSOption) (uint64, error) { - var startTs uint64 - var err error +// ExtractStartTS use `option` to get the proper startTS for a transaction. +func ExtractStartTS(store *KVStore, option StartTSOption) (uint64, error) { if option.StartTS != nil { - startTs = *option.StartTS - } else if option.PrevSec != nil { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTs, err = store.getStalenessTimestamp(bo, option.TxnScope, *option.PrevSec) - } else if option.MinStartTS != nil { - stores := make([]*Store, 0) - allStores := store.regionCache.getStoresByType(tikvrpc.TiKV) - if option.TxnScope != oracle.GlobalTxnScope { - for _, store := range allStores { - if store.IsLabelsMatch([]*metapb.StoreLabel{ - { - Key: DCLabelKey, - Value: option.TxnScope, - }, - }) { - stores = append(stores, store) - } - } - } else { - stores = allStores - } - safeTS := store.getMinSafeTSByStores(stores) - startTs = *option.MinStartTS - // If the safeTS is larger than the minStartTS, we will use safeTS as StartTS, otherwise we will use - // minStartTS directly. - if startTs < safeTS { - startTs = safeTS - } - } else if option.MaxPrevSec != nil { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - minStartTS, err := store.getStalenessTimestamp(bo, option.TxnScope, *option.MaxPrevSec) - if err != nil { - return 0, errors.Trace(err) - } - option.MinStartTS = &minStartTS - return ExtractStartTs(store, option) - } else { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTs, err = store.getTimestampWithRetry(bo, option.TxnScope) + return *option.StartTS, nil } - return startTs, err + bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + return store.getTimestampWithRetry(bo, option.TxnScope) } func newTiKVTxnWithOptions(store *KVStore, options StartTSOption) (*KVTxn, error) { if options.TxnScope == "" { options.TxnScope = oracle.GlobalTxnScope } - startTs, err := ExtractStartTs(store, options) + startTS, err := ExtractStartTS(store, options) if err != nil { return nil, errors.Trace(err) } - snapshot := newTiKVSnapshot(store, startTs, store.nextReplicaReadSeed()) + snapshot := newTiKVSnapshot(store, startTS, store.nextReplicaReadSeed()) newTiKVTxn := &KVTxn{ snapshot: snapshot, us: unionstore.NewUnionStore(snapshot), store: store, - startTS: startTs, + startTS: startTS, startTime: time.Now(), valid: true, vars: tikv.DefaultVars, diff --git a/util/mock/context.go b/util/mock/context.go index 4e9a9761f637a..9476d7295aa83 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -202,6 +202,11 @@ func (c *Context) NewTxn(context.Context) error { return nil } +// NewTxnWithStartTS implements the sessionctx.Context interface. +func (c *Context) NewTxnWithStartTS(ctx context.Context, startTS uint64) error { + return c.NewTxn(ctx) +} + // RefreshTxnCtx implements the sessionctx.Context interface. func (c *Context) RefreshTxnCtx(ctx context.Context) error { return errors.Trace(c.NewTxn(ctx)) @@ -218,7 +223,7 @@ func (c *Context) InitTxnWithStartTS(startTS uint64) error { return nil } if c.Store != nil { - txn, err := c.Store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(kv.GlobalTxnScope).SetStartTs(startTS)) + txn, err := c.Store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(kv.GlobalTxnScope).SetStartTS(startTS)) if err != nil { return errors.Trace(err) } @@ -227,11 +232,6 @@ func (c *Context) InitTxnWithStartTS(startTS uint64) error { return nil } -// NewTxnWithStalenessOption implements the sessionctx.Context interface. -func (c *Context) NewTxnWithStalenessOption(ctx context.Context, option sessionctx.StalenessTxnOption) error { - return c.NewTxn(ctx) -} - // GetStore gets the store of session. func (c *Context) GetStore() kv.Storage { return c.Store From 8c25a0074a62e6ebffe50ac765b2699b043f1f78 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 25 May 2021 15:23:33 +0800 Subject: [PATCH 162/343] */backoff: make backoff type as string instead of interface fmt.Stringer (#24810) --- store/tikv/retry/backoff.go | 12 ++++-- store/tikv/util/execdetails.go | 9 ++--- util/execdetails/execdetails.go | 5 +-- util/execdetails/execdetails_test.go | 44 ++++++---------------- util/stmtsummary/statement_summary.go | 8 ++-- util/stmtsummary/statement_summary_test.go | 25 ++++++------ 6 files changed, 41 insertions(+), 62 deletions(-) diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go index d07b9c4fdccae..a005f85bf698b 100644 --- a/store/tikv/retry/backoff.go +++ b/store/tikv/retry/backoff.go @@ -41,7 +41,7 @@ type Backoffer struct { maxSleep int totalSleep int errors []error - configs []fmt.Stringer + configs []*Config vars *kv.Variables noop bool @@ -172,7 +172,7 @@ func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err e } logutil.BgLogger().Warn(errMsg) // Use the first backoff type to generate a MySQL error. - return b.configs[0].(*Config).err + return b.configs[0].err } // Lazy initialize. @@ -265,8 +265,12 @@ func (b *Backoffer) GetTotalSleep() int { } // GetTypes returns type list. -func (b *Backoffer) GetTypes() []fmt.Stringer { - return b.configs +func (b *Backoffer) GetTypes() []string { + typs := make([]string, 0, len(b.configs)) + for _, cfg := range b.configs { + typs = append(typs, cfg.String()) + } + return typs } // GetCtx returns the binded context. diff --git a/store/tikv/util/execdetails.go b/store/tikv/util/execdetails.go index eeaaf92da6b27..97c88a3687319 100644 --- a/store/tikv/util/execdetails.go +++ b/store/tikv/util/execdetails.go @@ -15,7 +15,6 @@ package util import ( "bytes" - "fmt" "math" "strconv" "sync" @@ -50,7 +49,7 @@ type CommitDetails struct { CommitBackoffTime int64 Mu struct { sync.Mutex - BackoffTypes []fmt.Stringer + BackoffTypes []string } ResolveLockTime int64 WriteKeys int @@ -90,7 +89,7 @@ func (cd *CommitDetails) Clone() *CommitDetails { PrewriteRegionNum: cd.PrewriteRegionNum, TxnRetry: cd.TxnRetry, } - commit.Mu.BackoffTypes = append([]fmt.Stringer{}, cd.Mu.BackoffTypes...) + commit.Mu.BackoffTypes = append([]string{}, cd.Mu.BackoffTypes...) return commit } @@ -103,7 +102,7 @@ type LockKeysDetails struct { BackoffTime int64 Mu struct { sync.Mutex - BackoffTypes []fmt.Stringer + BackoffTypes []string } LockRPCTime int64 LockRPCCount int64 @@ -135,7 +134,7 @@ func (ld *LockKeysDetails) Clone() *LockKeysDetails { LockRPCCount: ld.LockRPCCount, RetryCount: ld.RetryCount, } - lock.Mu.BackoffTypes = append([]fmt.Stringer{}, ld.Mu.BackoffTypes...) + lock.Mu.BackoffTypes = append([]string{}, ld.Mu.BackoffTypes...) return lock } diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 676f0a241d489..16a17b656c1cc 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -892,14 +892,13 @@ func (e *RuntimeStatsWithCommit) String() string { return buf.String() } -func (e *RuntimeStatsWithCommit) formatBackoff(backoffTypes []fmt.Stringer) string { +func (e *RuntimeStatsWithCommit) formatBackoff(backoffTypes []string) string { if len(backoffTypes) == 0 { return "" } tpMap := make(map[string]struct{}) tpArray := []string{} - for _, tp := range backoffTypes { - tpStr := tp.String() + for _, tpStr := range backoffTypes { _, ok := tpMap[tpStr] if ok { continue diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index 371d06006051f..827410cb04350 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -14,7 +14,6 @@ package execdetails import ( - "fmt" "strconv" "sync" "testing" @@ -22,7 +21,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/store/tikv/util" - "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" ) @@ -43,14 +41,10 @@ func TestString(t *testing.T) { CommitBackoffTime: int64(time.Second), Mu: struct { sync.Mutex - BackoffTypes []fmt.Stringer - }{BackoffTypes: []fmt.Stringer{ - stringutil.MemoizeStr(func() string { - return "backoff1" - }), - stringutil.MemoizeStr(func() string { - return "backoff2" - }), + BackoffTypes []string + }{BackoffTypes: []string{ + "backoff1", + "backoff2", }}, ResolveLockTime: 1000000000, // 10^9 ns = 1s WriteKeys: 1, @@ -212,18 +206,8 @@ func TestRuntimeStatsWithCommit(t *testing.T) { CommitBackoffTime: int64(time.Second), Mu: struct { sync.Mutex - BackoffTypes []fmt.Stringer - }{BackoffTypes: []fmt.Stringer{ - stringutil.MemoizeStr(func() string { - return "backoff1" - }), - stringutil.MemoizeStr(func() string { - return "backoff2" - }), - stringutil.MemoizeStr(func() string { - return "backoff1" - }), - }}, + BackoffTypes []string + }{BackoffTypes: []string{"backoff1", "backoff2", "backoff1"}}, ResolveLockTime: int64(time.Second), WriteKeys: 3, WriteSize: 66, @@ -245,17 +229,11 @@ func TestRuntimeStatsWithCommit(t *testing.T) { BackoffTime: int64(time.Second * 3), Mu: struct { sync.Mutex - BackoffTypes []fmt.Stringer - }{BackoffTypes: []fmt.Stringer{ - stringutil.MemoizeStr(func() string { - return "backoff4" - }), - stringutil.MemoizeStr(func() string { - return "backoff5" - }), - stringutil.MemoizeStr(func() string { - return "backoff5" - }), + BackoffTypes []string + }{BackoffTypes: []string{ + "backoff4", + "backoff5", + "backoff5", }}, LockRPCTime: int64(time.Second * 5), LockRPCCount: 50, diff --git a/util/stmtsummary/statement_summary.go b/util/stmtsummary/statement_summary.go index f42089c9e7a01..3dac542bed4b8 100644 --- a/util/stmtsummary/statement_summary.go +++ b/util/stmtsummary/statement_summary.go @@ -172,7 +172,7 @@ type stmtSummaryByDigestElement struct { sumTxnRetry int64 maxTxnRetry int sumBackoffTimes int64 - backoffTypes map[fmt.Stringer]int + backoffTypes map[string]int authUsers map[string]struct{} // other sumMem int64 @@ -635,7 +635,7 @@ func newStmtSummaryByDigestElement(sei *StmtExecInfo, beginTime int64, intervalS minLatency: sei.TotalLatency, firstSeen: sei.StartTime, lastSeen: sei.StartTime, - backoffTypes: make(map[fmt.Stringer]int), + backoffTypes: make(map[string]int), authUsers: make(map[string]struct{}), planInCache: false, planCacheHits: 0, @@ -971,9 +971,9 @@ func formatSQL(sql string) string { } // Format the backoffType map to a string or nil. -func formatBackoffTypes(backoffMap map[fmt.Stringer]int) interface{} { +func formatBackoffTypes(backoffMap map[string]int) interface{} { type backoffStat struct { - backoffType fmt.Stringer + backoffType string count int } diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index fb4593e26e9b6..751a8e501f715 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" - "github.com/pingcap/tidb/util/stringutil" ) var _ = Suite(&testStmtSummarySuite{}) @@ -63,7 +62,7 @@ func TestT(t *testing.T) { } const ( - boTxnLockName = stringutil.StringerStr("txnlock") + boTxnLockName = "txnlock" ) // Test stmtSummaryByDigest.AddStatement. @@ -77,7 +76,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { // first statement stmtExecInfo1 := generateAnyExecInfo() - stmtExecInfo1.ExecDetail.CommitDetail.Mu.BackoffTypes = make([]fmt.Stringer, 0) + stmtExecInfo1.ExecDetail.CommitDetail.Mu.BackoffTypes = make([]string, 0) key := &stmtSummaryByDigestKey{ schemaName: stmtExecInfo1.SchemaName, digest: stmtExecInfo1.Digest, @@ -133,7 +132,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { maxPrewriteRegionNum: stmtExecInfo1.ExecDetail.CommitDetail.PrewriteRegionNum, sumTxnRetry: int64(stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry), maxTxnRetry: stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, - backoffTypes: make(map[fmt.Stringer]int), + backoffTypes: make(map[string]int), sumMem: stmtExecInfo1.MemMax, maxMem: stmtExecInfo1.MemMax, sumDisk: stmtExecInfo1.DiskMax, @@ -194,9 +193,9 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { CommitBackoffTime: 1000, Mu: struct { sync.Mutex - BackoffTypes []fmt.Stringer + BackoffTypes []string }{ - BackoffTypes: []fmt.Stringer{boTxnLockName}, + BackoffTypes: []string{boTxnLockName}, }, ResolveLockTime: 10000, WriteKeys: 100000, @@ -321,9 +320,9 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { CommitBackoffTime: 100, Mu: struct { sync.Mutex - BackoffTypes []fmt.Stringer + BackoffTypes []string }{ - BackoffTypes: []fmt.Stringer{boTxnLockName}, + BackoffTypes: []string{boTxnLockName}, }, ResolveLockTime: 1000, WriteKeys: 10000, @@ -577,9 +576,9 @@ func generateAnyExecInfo() *StmtExecInfo { CommitBackoffTime: 200, Mu: struct { sync.Mutex - BackoffTypes []fmt.Stringer + BackoffTypes []string }{ - BackoffTypes: []fmt.Stringer{boTxnLockName}, + BackoffTypes: []string{boTxnLockName}, }, ResolveLockTime: 2000, WriteKeys: 20000, @@ -961,12 +960,12 @@ func (s *testStmtSummarySuite) TestGetMoreThanOnceBindableStmt(c *C) { // Test `formatBackoffTypes`. func (s *testStmtSummarySuite) TestFormatBackoffTypes(c *C) { - backoffMap := make(map[fmt.Stringer]int) + backoffMap := make(map[string]int) c.Assert(formatBackoffTypes(backoffMap), IsNil) - bo1 := stringutil.StringerStr("pdrpc") + bo1 := "pdrpc" backoffMap[bo1] = 1 c.Assert(formatBackoffTypes(backoffMap), Equals, "pdrpc:1") - bo2 := stringutil.StringerStr("txnlock") + bo2 := "txnlock" backoffMap[bo2] = 2 c.Assert(formatBackoffTypes(backoffMap), Equals, "txnlock:2,pdrpc:1") From 2580240dde9cf3c19db090d055aa7b0c610aa138 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 25 May 2021 15:47:33 +0800 Subject: [PATCH 163/343] *: pass sql, plan digest down to KV request (#24854) --- bindinfo/bind_test.go | 9 +- bindinfo/handle.go | 12 +- bindinfo/session_handle.go | 6 +- config/config.go | 21 +++ distsql/request_builder.go | 10 ++ executor/adapter.go | 33 ++--- executor/analyze.go | 4 + executor/batch_point_get.go | 9 +- executor/checksum.go | 2 + executor/executor.go | 13 +- executor/executor_test.go | 121 ++++++++++++++++++ executor/insert.go | 1 + executor/partition_table_test.go | 8 +- executor/point_get.go | 1 + executor/replace.go | 1 + executor/update.go | 9 +- go.mod | 4 +- go.sum | 8 +- infoschema/tables_test.go | 4 +- kv/kv.go | 2 + kv/option.go | 2 + planner/core/cache.go | 5 +- planner/core/encode.go | 8 +- planner/core/plan_test.go | 10 +- planner/optimize.go | 6 +- session/session.go | 3 +- session/session_test.go | 2 +- sessionctx/stmtctx/stmtctx.go | 34 ++++- sessionctx/variable/session_test.go | 2 +- store/copr/batch_coprocessor.go | 13 +- store/copr/coprocessor.go | 13 +- store/driver/txn/snapshot.go | 2 + store/driver/txn/txn_driver.go | 2 + store/mockstore/unistore/rpc.go | 9 ++ store/tikv/2pc.go | 3 + store/tikv/cleanup.go | 2 +- store/tikv/commit.go | 2 +- store/tikv/prewrite.go | 2 +- store/tikv/scan.go | 14 +- store/tikv/snapshot.go | 21 ++- store/tikv/txn.go | 7 + tools/check/go.mod | 1 - util/deadlockhistory/deadlock_history.go | 2 +- util/deadlockhistory/deadlock_history_test.go | 16 ++- util/resourcegrouptag/resource_group_tag.go | 89 ++++--------- .../resource_group_tag_test.go | 111 ++++++++-------- 46 files changed, 432 insertions(+), 227 deletions(-) diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 22b60187f0a77..4175ddce77eb9 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -156,7 +156,8 @@ func normalizeWithDefaultDB(c *C, sql, db string) (string, string) { testParser := parser.New() stmt, err := testParser.ParseOneStmt(sql, "", "") c.Assert(err, IsNil) - return parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, "test", "")) + normalized, digest := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, "test", "")) + return normalized, digest.String() } func (s *testSuite) TestBindParse(c *C) { @@ -182,7 +183,7 @@ func (s *testSuite) TestBindParse(c *C) { c.Check(bindHandle.Size(), Equals, 1) sql, hash := parser.NormalizeDigest("select * from test . t") - bindData := bindHandle.GetBindRecord(hash, sql, "test") + bindData := bindHandle.GetBindRecord(hash.String(), sql, "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from `test` . `t`") bind := bindData.Bindings[0] @@ -656,7 +657,7 @@ func (s *testSuite) TestBindingSymbolList(c *C) { // Normalize sql, hash := parser.NormalizeDigest("select a, b from test . t where a = 1 limit 0, 1") - bindData := s.domain.BindHandle().GetBindRecord(hash, sql, "test") + bindData := s.domain.BindHandle().GetBindRecord(hash.String(), sql, "test") c.Assert(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select `a` , `b` from `test` . `t` where `a` = ? limit ...") bind := bindData.Bindings[0] @@ -776,7 +777,7 @@ func (s *testSuite) TestErrorBind(c *C) { c.Assert(err, IsNil, Commentf("err %v", err)) sql, hash := parser.NormalizeDigest("select * from test . t where i > ?") - bindData := s.domain.BindHandle().GetBindRecord(hash, sql, "test") + bindData := s.domain.BindHandle().GetBindRecord(hash.String(), sql, "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from `test` . `t` where `i` > ?") bind := bindData.Bindings[0] diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 6111910395d55..2281af3c88bd3 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -210,7 +210,7 @@ func (h *BindHandle) CreateBindRecord(sctx sessionctx.Context, record *BindRecor } sqlDigest := parser.DigestNormalized(record.OriginalSQL) - h.setBindRecord(sqlDigest, record) + h.setBindRecord(sqlDigest.String(), record) }() // Lock mysql.bind_info to synchronize with CreateBindRecord / AddBindRecord / DropBindRecord on other tidb instances. @@ -256,7 +256,7 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, record *BindRecord) } record.Db = strings.ToLower(record.Db) - oldRecord := h.GetBindRecord(parser.DigestNormalized(record.OriginalSQL), record.OriginalSQL, record.Db) + oldRecord := h.GetBindRecord(parser.DigestNormalized(record.OriginalSQL).String(), record.OriginalSQL, record.Db) var duplicateBinding *Binding if oldRecord != nil { binding := oldRecord.FindBinding(record.Bindings[0].ID) @@ -294,7 +294,7 @@ func (h *BindHandle) AddBindRecord(sctx sessionctx.Context, record *BindRecord) return } - h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL), record) + h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL).String(), record) }() // Lock mysql.bind_info to synchronize with CreateBindRecord / AddBindRecord / DropBindRecord on other tidb instances. @@ -367,7 +367,7 @@ func (h *BindHandle) DropBindRecord(originalSQL, db string, binding *Binding) (e if binding != nil { record.Bindings = append(record.Bindings, *binding) } - h.removeBindRecord(parser.DigestNormalized(originalSQL), record) + h.removeBindRecord(parser.DigestNormalized(originalSQL).String(), record) }() // Lock mysql.bind_info to synchronize with CreateBindRecord / AddBindRecord / DropBindRecord on other tidb instances. @@ -515,7 +515,7 @@ func (h *BindHandle) newBindRecord(row chunk.Row) (string, *BindRecord, error) { defer h.sctx.Unlock() h.sctx.GetSessionVars().CurrentDB = bindRecord.Db err := bindRecord.prepareHints(h.sctx.Context) - return hash, bindRecord, err + return hash.String(), bindRecord, err } // setBindRecord sets the BindRecord to the cache, if there already exists a BindRecord, @@ -624,7 +624,7 @@ func (h *BindHandle) CaptureBaselines() { } dbName := utilparser.GetDefaultDB(stmt, bindableStmt.Schema) normalizedSQL, digest := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, dbName, bindableStmt.Query)) - if r := h.GetBindRecord(digest, normalizedSQL, dbName); r != nil && r.HasUsingBinding() { + if r := h.GetBindRecord(digest.String(), normalizedSQL, dbName); r != nil && r.HasUsingBinding() { continue } bindSQL := GenerateBindSQL(context.TODO(), stmt, bindableStmt.PlanHint, true, dbName) diff --git a/bindinfo/session_handle.go b/bindinfo/session_handle.go index 2604d5b563f52..6b54aa9118f77 100644 --- a/bindinfo/session_handle.go +++ b/bindinfo/session_handle.go @@ -60,7 +60,7 @@ func (h *SessionHandle) CreateBindRecord(sctx sessionctx.Context, record *BindRe } // update the BindMeta to the cache. - h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL), record) + h.appendBindRecord(parser.DigestNormalized(record.OriginalSQL).String(), record) return nil } @@ -78,14 +78,14 @@ func (h *SessionHandle) DropBindRecord(originalSQL, db string, binding *Binding) } else { newRecord = record } - h.ch.setBindRecord(parser.DigestNormalized(record.OriginalSQL), newRecord) + h.ch.setBindRecord(parser.DigestNormalized(record.OriginalSQL).String(), newRecord) updateMetrics(metrics.ScopeSession, oldRecord, newRecord, false) return nil } // GetBindRecord return the BindMeta of the (normdOrigSQL,db) if BindMeta exist. func (h *SessionHandle) GetBindRecord(normdOrigSQL, db string) *BindRecord { - hash := parser.DigestNormalized(normdOrigSQL) + hash := parser.DigestNormalized(normdOrigSQL).String() bindRecords := h.ch[hash] for _, bindRecord := range bindRecords { if bindRecord.OriginalSQL == normdOrigSQL { diff --git a/config/config.go b/config/config.go index 664cb5bceaa7d..83490c345dae8 100644 --- a/config/config.go +++ b/config/config.go @@ -137,6 +137,7 @@ type Config struct { DelayCleanTableLock uint64 `toml:"delay-clean-table-lock" json:"delay-clean-table-lock"` SplitRegionMaxNum uint64 `toml:"split-region-max-num" json:"split-region-max-num"` StmtSummary StmtSummary `toml:"stmt-summary" json:"stmt-summary"` + TopSQL TopSQL `toml:"top-sql" json:"top-sql"` // RepairMode indicates that the TiDB is in the repair mode for table meta. RepairMode bool `toml:"repair-mode" json:"repair-mode"` RepairTableList []string `toml:"repair-table-list" json:"repair-table-list"` @@ -527,6 +528,16 @@ type StmtSummary struct { HistorySize int `toml:"history-size" json:"history-size"` } +// TopSQL is the config for top sql. +type TopSQL struct { + // Enable statement summary or not. + Enable bool `toml:"enable" json:"enable"` + // The refresh interval of statement summary. + RefreshInterval int `toml:"refresh-interval" json:"refresh-interval"` + // The maximum number of statements kept in memory. + MaxStmtCount uint `toml:"max-stmt-count" json:"max-stmt-count"` +} + // IsolationRead is the config for isolation read. type IsolationRead struct { // Engines filters tidb-server access paths by engine type. @@ -656,6 +667,11 @@ var defaultConf = Config{ RefreshInterval: 1800, HistorySize: 24, }, + TopSQL: TopSQL{ + Enable: true, + RefreshInterval: 1, + MaxStmtCount: 5000, + }, IsolationRead: IsolationRead{ Engines: []string{"tikv", "tiflash", "tidb"}, }, @@ -943,6 +959,11 @@ func TableLockEnabled() bool { return GetGlobalConfig().EnableTableLock } +// TopSQLEnabled uses to check whether enabled the top SQL feature. +func TopSQLEnabled() bool { + return GetGlobalConfig().TopSQL.Enable +} + // TableLockDelayClean uses to get the time of delay clean table lock. var TableLockDelayClean = func() uint64 { return GetGlobalConfig().DelayCleanTableLock diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 42a098cd05440..d34ecc9ab0c47 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -239,6 +240,7 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req }, } } + builder.SetResourceGroupTag(sv.StmtCtx) return builder } @@ -274,6 +276,14 @@ func (builder *RequestBuilder) SetFromInfoSchema(pis interface{}) *RequestBuilde return builder } +// SetResourceGroupTag sets the request resource group tag. +func (builder *RequestBuilder) SetResourceGroupTag(sc *stmtctx.StatementContext) *RequestBuilder { + if config.TopSQLEnabled() { + builder.Request.ResourceGroupTag = sc.GetResourceGroupTag() + } + return builder +} + func (builder *RequestBuilder) verifyTxnScope() error { if builder.txnScope == "" { builder.txnScope = kv.GlobalTxnScope diff --git a/executor/adapter.go b/executor/adapter.go index 15a823dd0f724..c5d9b0406602c 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -333,6 +333,8 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { return nil, err } + getPlanDigest(a.Ctx, a.Plan) + if err = e.Open(ctx); err != nil { terror.Call(e.Close) return nil, err @@ -919,11 +921,11 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { statsInfos := plannercore.GetStatsInfo(a.Plan) memMax := sessVars.StmtCtx.MemTracker.MaxConsumed() diskMax := sessVars.StmtCtx.DiskTracker.MaxConsumed() - _, planDigest := getPlanDigest(a.Ctx, a.Plan) + planDigest := getPlanDigest(a.Ctx, a.Plan) slowItems := &variable.SlowQueryLogItems{ TxnTS: txnTS, SQL: sql.String(), - Digest: digest, + Digest: digest.String(), TimeTotal: costTime, TimeParse: sessVars.DurationParse, TimeCompile: sessVars.DurationCompile, @@ -981,7 +983,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { } domain.GetDomain(a.Ctx).LogSlowQuery(&domain.SlowQueryInfo{ SQL: sql.String(), - Digest: digest, + Digest: digest.String(), Start: sessVars.StartTime, Duration: costTime, Detail: sessVars.StmtCtx.GetExecDetails(), @@ -1011,14 +1013,15 @@ func getPlanTree(sctx sessionctx.Context, p plannercore.Plan) string { } // getPlanDigest will try to get the select plan tree if the plan is select or the select plan of delete/update/insert statement. -func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (normalized, planDigest string) { - normalized, planDigest = sctx.GetSessionVars().StmtCtx.GetPlanDigest() - if len(normalized) > 0 { - return - } - normalized, planDigest = plannercore.NormalizePlan(p) - sctx.GetSessionVars().StmtCtx.SetPlanDigest(normalized, planDigest) - return +func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) string { + sc := sctx.GetSessionVars().StmtCtx + _, planDigest := sc.GetPlanDigest() + if planDigest != nil { + return planDigest.String() + } + normalized, planDigest := plannercore.NormalizePlan(p) + sc.SetPlanDigest(normalized, planDigest) + return planDigest.String() } // getEncodedPlan gets the encoded plan, and generates the hint string if indicated. @@ -1079,7 +1082,7 @@ func (a *ExecStmt) SummaryStmt(succ bool) { } prevSQL = sessVars.PrevStmt.String() } - sessVars.SetPrevStmtDigest(digest) + sessVars.SetPrevStmtDigest(digest.String()) // No need to encode every time, so encode lazily. planGenerator := func() (string, string) { @@ -1092,11 +1095,11 @@ func (a *ExecStmt) SummaryStmt(succ bool) { var planDigestGen func() string if a.Plan.TP() == plancodec.TypePointGet { planDigestGen = func() string { - _, planDigest := getPlanDigest(a.Ctx, a.Plan) + planDigest := getPlanDigest(a.Ctx, a.Plan) return planDigest } } else { - _, planDigest = getPlanDigest(a.Ctx, a.Plan) + planDigest = getPlanDigest(a.Ctx, a.Plan) } execDetail := stmtCtx.GetExecDetails() @@ -1120,7 +1123,7 @@ func (a *ExecStmt) SummaryStmt(succ bool) { Charset: charset, Collation: collation, NormalizedSQL: normalizedSQL, - Digest: digest, + Digest: digest.String(), PrevSQL: prevSQL, PrevSQLDigest: prevSQLDigest, PlanGenerator: planGenerator, diff --git a/executor/analyze.go b/executor/analyze.go index ed7463b2ac46f..a9aecb7547588 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -355,6 +355,7 @@ func (e *AnalyzeIndexExec) fetchAnalyzeResult(ranges []*ranger.Range, isNullRang } else { kvReqBuilder = builder.SetIndexRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.tableID.GetStatisticsID()}, e.idxInfo.ID, ranges) } + kvReqBuilder.SetResourceGroupTag(e.ctx.GetSessionVars().StmtCtx) kvReq, err := kvReqBuilder. SetAnalyzeRequest(e.analyzePB). SetStartTS(math.MaxUint64). @@ -653,6 +654,7 @@ func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectResult, error) { var builder distsql.RequestBuilder reqBuilder := builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.tableID.GetStatisticsID()}, e.handleCols != nil && !e.handleCols.IsInt(), ranges, nil) + builder.SetResourceGroupTag(e.ctx.GetSessionVars().StmtCtx) // Always set KeepOrder of the request to be true, in order to compute // correct `correlation` of columns. kvReq, err := reqBuilder. @@ -1323,6 +1325,7 @@ func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) (keysSize int, err if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } + setResourceGroupTagForTxn(e.ctx.GetSessionVars().StmtCtx, snapshot) for _, t := range e.scanTasks { iter, err := snapshot.Iter(kv.Key(t.StartKey), kv.Key(t.EndKey)) if err != nil { @@ -1343,6 +1346,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { snapshot.SetOption(kv.NotFillCache, true) snapshot.SetOption(kv.IsolationLevel, kv.RC) snapshot.SetOption(kv.Priority, kv.PriorityLow) + setResourceGroupTagForTxn(e.ctx.GetSessionVars().StmtCtx, snapshot) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 1e1fcd581cf61..c34eafd54c408 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -89,7 +89,9 @@ func (e *BatchPointGetExec) buildVirtualColumnInfo() { // Open implements the Executor interface. func (e *BatchPointGetExec) Open(context.Context) error { e.snapshotTS = e.startTS - txnCtx := e.ctx.GetSessionVars().TxnCtx + sessVars := e.ctx.GetSessionVars() + txnCtx := sessVars.TxnCtx + stmtCtx := sessVars.StmtCtx if e.lock { e.snapshotTS = txnCtx.GetForUpdateTS() } @@ -112,12 +114,12 @@ func (e *BatchPointGetExec) Open(context.Context) error { SnapshotRuntimeStats: snapshotStats, } snapshot.SetOption(kv.CollectRuntimeStats, snapshotStats) - e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) + stmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } - snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) + snapshot.SetOption(kv.TaskID, stmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness snapshot.SetOption(kv.IsStalenessReadOnly, isStaleness) if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != kv.GlobalTxnScope { @@ -128,6 +130,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { }, }) } + setResourceGroupTagForTxn(stmtCtx, snapshot) var batchGetter kv.BatchGetter = snapshot if txn.Valid() { lock := e.tblInfo.Lock diff --git a/executor/checksum.go b/executor/checksum.go index 63f622d2f8140..62543068820e9 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -240,6 +240,7 @@ func (c *checksumContext) buildTableRequest(ctx sessionctx.Context, tableID int6 } var builder distsql.RequestBuilder + builder.SetResourceGroupTag(ctx.GetSessionVars().StmtCtx) return builder.SetHandleRanges(ctx.GetSessionVars().StmtCtx, tableID, c.TableInfo.IsCommonHandle, ranges, nil). SetChecksumRequest(checksum). SetStartTS(c.StartTs). @@ -256,6 +257,7 @@ func (c *checksumContext) buildIndexRequest(ctx sessionctx.Context, tableID int6 ranges := ranger.FullRange() var builder distsql.RequestBuilder + builder.SetResourceGroupTag(ctx.GetSessionVars().StmtCtx) return builder.SetIndexRanges(ctx.GetSessionVars().StmtCtx, tableID, indexInfo.ID, ranges). SetChecksumRequest(checksum). SetStartTS(c.StartTs). diff --git a/executor/executor.go b/executor/executor.go index 2b9b8f0f52954..1d136bac8a2f9 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -29,6 +29,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" @@ -974,7 +975,11 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { } func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.LockCtx { + var planDigest *parser.Digest _, sqlDigest := seVars.StmtCtx.SQLDigest() + if config.TopSQLEnabled() { + _, planDigest = seVars.StmtCtx.GetPlanDigest() + } return &tikvstore.LockCtx{ Killed: &seVars.Killed, ForUpdateTS: seVars.TxnCtx.GetForUpdateTS(), @@ -984,7 +989,7 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.Loc LockKeysDuration: &seVars.StmtCtx.LockKeysDuration, LockKeysCount: &seVars.StmtCtx.LockKeysCount, LockExpired: &seVars.TxnCtx.LockExpire, - ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(sqlDigest), + ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(sqlDigest, planDigest), OnDeadlock: func(deadlock *tikverr.ErrDeadlock) { // TODO: Support collecting retryable deadlocks according to the config. if !deadlock.IsRetryable { @@ -1794,3 +1799,9 @@ func FillVirtualColumnValue(virtualRetTypes []*types.FieldType, virtualColumnInd } return nil } + +func setResourceGroupTagForTxn(sc *stmtctx.StatementContext, snapshot kv.Snapshot) { + if snapshot != nil && config.TopSQLEnabled() { + snapshot.SetOption(kv.ResourceGroupTag, sc.GetResourceGroupTag()) + } +} diff --git a/executor/executor_test.go b/executor/executor_test.go index 4ba3ea2e89e85..c50459cf2850b 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -31,6 +31,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser" "github.com/pingcap/parser/auth" @@ -58,6 +59,7 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/copr" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/oracle" @@ -146,6 +148,7 @@ var _ = SerialSuites(&testSerialSuite{&baseTestSuite{}}) var _ = SerialSuites(&testStaleTxnSerialSuite{&baseTestSuite{}}) var _ = SerialSuites(&testCoprCache{}) var _ = SerialSuites(&testPrepareSuite{}) +var _ = SerialSuites(&testResourceTagSuite{&baseTestSuite{}}) type testSuite struct{ *baseTestSuite } type testSuiteP1 struct{ *baseTestSuite } @@ -166,6 +169,7 @@ type testCoprCache struct { cls cluster.Cluster } type testPrepareSuite struct{ testData testutil.TestData } +type testResourceTagSuite struct{ *baseTestSuite } type baseTestSuite struct { cluster cluster.Cluster @@ -8311,3 +8315,120 @@ func (s testSerialSuite) TestExprBlackListForEnum(c *C) { rows = tk.MustQuery("desc format='brief' select * from t where b = 1 and a > 'a'").Rows() c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue) } + +func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, b int, unique index idx(a));") + tbInfo := testGetTableByName(c, tk.Se, "test", "t") + + // Enable Top SQL + cfg := config.GetGlobalConfig() + newCfg := *cfg + newCfg.TopSQL.Enable = true + config.StoreGlobalConfig(&newCfg) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook", `return(true)`), IsNil) + defer failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook") + + var sqlDigest, planDigest *parser.Digest + checkFn := func() {} + unistore.UnistoreRPCClientSendHook = func(req *tikvrpc.Request) { + var startKey []byte + var ctx *kvrpcpb.Context + switch req.Type { + case tikvrpc.CmdGet: + request := req.Get() + startKey = request.Key + ctx = request.Context + case tikvrpc.CmdBatchGet: + request := req.BatchGet() + startKey = request.Keys[0] + ctx = request.Context + case tikvrpc.CmdPrewrite: + request := req.Prewrite() + startKey = request.Mutations[0].Key + ctx = request.Context + case tikvrpc.CmdCommit: + request := req.Commit() + startKey = request.Keys[0] + ctx = request.Context + case tikvrpc.CmdCop: + request := req.Cop() + startKey = request.Ranges[0].Start + ctx = request.Context + case tikvrpc.CmdPessimisticLock: + request := req.PessimisticLock() + startKey = request.PrimaryLock + ctx = request.Context + } + tid := tablecodec.DecodeTableID(startKey) + if tid != tbInfo.Meta().ID { + return + } + if ctx == nil { + return + } + tag := &tipb.ResourceGroupTag{} + err := tag.Unmarshal(ctx.ResourceGroupTag) + c.Assert(err, IsNil) + sqlDigest = parser.NewDigest(tag.SqlDigest) + planDigest = parser.NewDigest(tag.PlanDigest) + checkFn() + } + + resetVars := func() { + sqlDigest = parser.NewDigest(nil) + planDigest = parser.NewDigest(nil) + } + + cases := []struct { + sql string + ignore bool + }{ + {sql: "insert into t values(1,1),(2,2),(3,3)"}, + {sql: "select * from t use index (idx) where a=1"}, + {sql: "select * from t use index (idx) where a in (1,2,3)"}, + {sql: "select * from t use index (idx) where a>1"}, + {sql: "select * from t where b>1"}, + {sql: "begin pessimistic", ignore: true}, + {sql: "insert into t values(4,4)"}, + {sql: "commit", ignore: true}, + {sql: "update t set a=5,b=5 where a=5"}, + {sql: "replace into t values(6,6)"}, + } + for _, ca := range cases { + resetVars() + commentf := Commentf("%v", ca.sql) + + _, expectSQLDigest := parser.NormalizeDigest(ca.sql) + var expectPlanDigest *parser.Digest + checkCnt := 0 + checkFn = func() { + if ca.ignore { + return + } + if expectPlanDigest == nil { + info := tk.Se.ShowProcess() + c.Assert(info, NotNil) + p, ok := info.Plan.(plannercore.Plan) + c.Assert(ok, IsTrue) + _, expectPlanDigest = plannercore.NormalizePlan(p) + } + c.Assert(sqlDigest.String(), Equals, expectSQLDigest.String(), commentf) + c.Assert(planDigest.String(), Equals, expectPlanDigest.String()) + checkCnt++ + } + + if strings.HasPrefix(ca.sql, "select") { + tk.MustQuery(ca.sql) + } else { + tk.MustExec(ca.sql) + } + if ca.ignore { + continue + } + c.Assert(checkCnt > 0, IsTrue, commentf) + } +} diff --git a/executor/insert.go b/executor/insert.go index 178aefed5fb8b..351f04c2ca5eb 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -63,6 +63,7 @@ func (e *InsertExec) exec(ctx context.Context, rows [][]types.Datum) error { if err != nil { return err } + setResourceGroupTagForTxn(sessVars.StmtCtx, txn) txnSize := txn.Size() sessVars.StmtCtx.AddRecordRows(uint64(len(rows))) // If you use the IGNORE keyword, duplicate-key error that occurs while executing the INSERT statement are ignored. diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index b9823a32a647e..82ec887ce2692 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -122,7 +122,7 @@ func (s *partitionTableSuite) TestPointGetwithRangeAndListPartitionTable(c *C) { tk.MustExec("set @@session.tidb_enable_list_partition = ON") // list partition table - tk.MustExec(`create table tlist(a int, b int, unique index idx_a(a), index idx_b(b)) partition by list(a)( + tk.MustExec(`create table tlist(a int, b int, unique index idx_a(a), index idx_b(b)) partition by list(a)( partition p0 values in (NULL, 1, 2, 3, 4), partition p1 values in (5, 6, 7, 8), partition p2 values in (9, 10, 11, 12));`) @@ -172,15 +172,15 @@ func (s *partitionTableSuite) TestPointGetwithRangeAndListPartitionTable(c *C) { } // test table dual - queryRange1 := fmt.Sprintf("select a from trange1 where a=200") + queryRange1 := "select a from trange1 where a=200" c.Assert(tk.HasPlan(queryRange1, "TableDual"), IsTrue) // check if TableDual is used tk.MustQuery(queryRange1).Check(testkit.Rows()) - queryRange2 := fmt.Sprintf("select a from trange2 where a=200") + queryRange2 := "select a from trange2 where a=200" c.Assert(tk.HasPlan(queryRange2, "TableDual"), IsTrue) // check if TableDual is used tk.MustQuery(queryRange2).Check(testkit.Rows()) - queryList := fmt.Sprintf("select a from tlist where a=200") + queryList := "select a from tlist where a=200" c.Assert(tk.HasPlan(queryList, "TableDual"), IsTrue) // check if TableDual is used tk.MustQuery(queryList).Check(testkit.Rows()) } diff --git a/executor/point_get.go b/executor/point_get.go index bc476794888fe..685d378f83d87 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -159,6 +159,7 @@ func (e *PointGetExecutor) Open(context.Context) error { }, }) } + setResourceGroupTagForTxn(e.ctx.GetSessionVars().StmtCtx, e.snapshot) return nil } diff --git a/executor/replace.go b/executor/replace.go index 8f35be4d05dbd..03dc4bfad0543 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -224,6 +224,7 @@ func (e *ReplaceExec) exec(ctx context.Context, newRows [][]types.Datum) error { defer snapshot.DelOption(kv.CollectRuntimeStats) } } + setResourceGroupTagForTxn(e.ctx.GetSessionVars().StmtCtx, txn) prefetchStart := time.Now() // Use BatchGet to fill cache. // It's an optimization and could be removed without affecting correctness. diff --git a/executor/update.go b/executor/update.go index 7c4b07ab8e6f6..954aa43c8067c 100644 --- a/executor/update.go +++ b/executor/update.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" @@ -258,11 +259,17 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { memUsageOfChk = chk.MemoryUsage() e.memTracker.Consume(memUsageOfChk) if e.collectRuntimeStatsEnabled() { - txn, err := e.ctx.Txn(false) + txn, err := e.ctx.Txn(true) if err == nil && txn.GetSnapshot() != nil { txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) } } + if config.TopSQLEnabled() { + txn, err := e.ctx.Txn(true) + if err == nil { + txn.SetOption(kv.ResourceGroupTag, e.ctx.GetSessionVars().StmtCtx.GetResourceGroupTag()) + } + } for rowIdx := 0; rowIdx < chk.NumRows(); rowIdx++ { chunkRow := chk.GetRow(rowIdx) datumRow := chunkRow.GetDatumRow(fields) diff --git a/go.mod b/go.mod index 7b06ca2a8682d..0879d5f157bf4 100644 --- a/go.mod +++ b/go.mod @@ -45,10 +45,10 @@ require ( github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 - github.com/pingcap/parser v0.0.0-20210518053259-92fa6fe07eb6 + github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307 github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible - github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 + github.com/pingcap/tipb v0.0.0-20210525032549-b80be13ddf6c github.com/prometheus/client_golang v1.5.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.9.1 diff --git a/go.sum b/go.sum index ac5cfe59f6b17..664a52bdca20d 100644 --- a/go.sum +++ b/go.sum @@ -440,16 +440,16 @@ github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIf github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 h1:ERrF0fTuIOnwfGbt71Ji3DKbOEaP189tjym50u8gpC8= github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= -github.com/pingcap/parser v0.0.0-20210518053259-92fa6fe07eb6 h1:wsH3psMH5ksDowsN9VUE9ZqSrX6oF4AYQQfOunkvSfU= -github.com/pingcap/parser v0.0.0-20210518053259-92fa6fe07eb6/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= +github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307 h1:v7SipssMu4X1tVQOe3PIVE73keJNHCFXe4Cza5uNDZ8= +github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307/go.mod h1:xZC8I7bug4GJ5KtHhgAikjTfU4kBv1Sbo3Pf1MZ6lVw= github.com/pingcap/sysutil v0.0.0-20200206130906-2bfa6dc40bcd/go.mod h1:EB/852NMQ+aRKioCpToQ94Wl7fktV+FNnxf3CX/TTXI= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 h1:A9KL9R+lWSVPH8IqUuH1QSTRJ5FGoY1bT2IcfPKsWD8= github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041UWP+NqYzrJ3fMgC/Hw9wnmQ/tUkp/JaHly8= github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:OzFN8H0EDMMqeulPhPMw2i2JaiZWOKFQ7zdRPhENNgo= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible h1:ceznmu/lLseGHP/jKyOa/3u/5H3wtLLLqkH2V3ssSjg= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1 h1:Kcp3jIcQrqG+pT1JQ0oWyRncVKQtDgnMFzRt3zJBaBo= -github.com/pingcap/tipb v0.0.0-20210422074242-57dd881b81b1/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= +github.com/pingcap/tipb v0.0.0-20210525032549-b80be13ddf6c h1:El3pMBpJHuSkItkHsnBqsaaHzJwFBNDt3Aul98AhREY= +github.com/pingcap/tipb v0.0.0-20210525032549-b80be13ddf6c/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 2d6506b56d5f4..761bb75fb76ed 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1517,7 +1517,7 @@ func (s *testTableSuite) TestTrx(c *C) { sm := &mockSessionManager{nil, make([]*txninfo.TxnInfo, 1)} sm.txnInfo[0] = &txninfo.TxnInfo{ StartTS: 424768545227014155, - CurrentSQLDigest: digest, + CurrentSQLDigest: digest.String(), State: txninfo.TxnRunningNormal, BlockStartTime: nil, EntriesCount: 1, @@ -1528,7 +1528,7 @@ func (s *testTableSuite) TestTrx(c *C) { } tk.Se.SetSessionManager(sm) tk.MustQuery("select * from information_schema.TIDB_TRX;").Check( - testkit.Rows("424768545227014155 2021-05-07 12:56:48 " + digest + " Normal 1 19 2 root test"), + testkit.Rows("424768545227014155 2021-05-07 12:56:48 " + digest.String() + " Normal 1 19 2 root test"), ) } diff --git a/kv/kv.go b/kv/kv.go index 0889106ba9fbf..be612512317db 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -288,6 +288,8 @@ type Request struct { IsStaleness bool // MatchStoreLabels indicates the labels the store should be matched MatchStoreLabels []*metapb.StoreLabel + // ResourceGroupTag indicates the kv request task group. + ResourceGroupTag []byte } // ResultSubset represents a result subset from a single storage unit. diff --git a/kv/option.go b/kv/option.go index dc0d700666d5a..de5a1d8834c40 100644 --- a/kv/option.go +++ b/kv/option.go @@ -59,6 +59,8 @@ const ( IsStalenessReadOnly // MatchStoreLabels indicates the labels the store should be matched MatchStoreLabels + // ResourceGroupTag indicates the resource group of the kv request. + ResourceGroupTag ) // ReplicaReadType is the type of replica to read data from diff --git a/planner/core/cache.go b/planner/core/cache.go index f97c207d189de..0e5a624b3d635 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -18,6 +18,7 @@ import ( "sync/atomic" "time" + "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -199,7 +200,7 @@ type CachedPrepareStmt struct { Executor interface{} NormalizedSQL string NormalizedPlan string - SQLDigest string - PlanDigest string + SQLDigest *parser.Digest + PlanDigest *parser.Digest ForUpdateRead bool } diff --git a/planner/core/encode.go b/planner/core/encode.go index d1cad479d52f8..8dc6ddeca9473 100644 --- a/planner/core/encode.go +++ b/planner/core/encode.go @@ -16,11 +16,11 @@ package core import ( "bytes" "crypto/sha256" - "fmt" "hash" "sync" "github.com/pingcap/failpoint" + "github.com/pingcap/parser" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/plancodec" ) @@ -120,10 +120,10 @@ type planDigester struct { } // NormalizePlan is used to normalize the plan and generate plan digest. -func NormalizePlan(p Plan) (normalized, digest string) { +func NormalizePlan(p Plan) (normalized string, digest *parser.Digest) { selectPlan := getSelectPlan(p) if selectPlan == nil { - return "", "" + return "", parser.NewDigest(nil) } d := digesterPool.Get().(*planDigester) defer digesterPool.Put(d) @@ -134,7 +134,7 @@ func NormalizePlan(p Plan) (normalized, digest string) { panic(err) } d.buf.Reset() - digest = fmt.Sprintf("%x", d.hasher.Sum(nil)) + digest = parser.NewDigest(d.hasher.Sum(nil)) d.hasher.Reset() return } diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 53f63f25fbc18..6c29eef90f5ae 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -171,12 +171,12 @@ func (s *testPlanNormalize) TestNormalizedPlanForDiffStore(c *C) { normalizedPlanRows := getPlanRows(normalizedPlan) c.Assert(err, IsNil) s.testData.OnRecord(func() { - output[i].Digest = digest + output[i].Digest = digest.String() output[i].Plan = normalizedPlanRows }) compareStringSlice(c, normalizedPlanRows, output[i].Plan) - c.Assert(digest != lastDigest, IsTrue) - lastDigest = digest + c.Assert(digest.String() != lastDigest, IsTrue) + lastDigest = digest.String() } } @@ -404,10 +404,10 @@ func testNormalizeDigest(tk *testkit.TestKit, c *C, sql1, sql2 string, isSame bo comment := Commentf("sql1: %v, sql2: %v\n%v !=\n%v\n", sql1, sql2, normalized1, normalized2) if isSame { c.Assert(normalized1, Equals, normalized2, comment) - c.Assert(digest1, Equals, digest2, comment) + c.Assert(digest1.String(), Equals, digest2.String(), comment) } else { c.Assert(normalized1 != normalized2, IsTrue, comment) - c.Assert(digest1 != digest2, IsTrue, comment) + c.Assert(digest1.String() != digest2.String(), IsTrue, comment) } } diff --git a/planner/optimize.go b/planner/optimize.go index ec9bfef67d0a7..c667d6b124ef9 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -306,7 +306,7 @@ func extractSelectAndNormalizeDigest(stmtNode ast.StmtNode, specifiledDB string) normalizeSQL := parser.Normalize(utilparser.RestoreWithDefaultDB(x.Stmt, specifiledDB, x.Text())) normalizeSQL = plannercore.EraseLastSemicolonInSQL(normalizeSQL) hash := parser.DigestNormalized(normalizeSQL) - return x.Stmt, normalizeSQL, hash, nil + return x.Stmt, normalizeSQL, hash.String(), nil case *ast.SetOprStmt: plannercore.EraseLastSemicolon(x) var normalizeExplainSQL string @@ -322,7 +322,7 @@ func extractSelectAndNormalizeDigest(stmtNode ast.StmtNode, specifiledDB string) } normalizeSQL := normalizeExplainSQL[idx:] hash := parser.DigestNormalized(normalizeSQL) - return x.Stmt, normalizeSQL, hash, nil + return x.Stmt, normalizeSQL, hash.String(), nil } case *ast.SelectStmt, *ast.SetOprStmt, *ast.DeleteStmt, *ast.UpdateStmt, *ast.InsertStmt: plannercore.EraseLastSemicolon(x) @@ -335,7 +335,7 @@ func extractSelectAndNormalizeDigest(stmtNode ast.StmtNode, specifiledDB string) return x, "", "", nil } normalizedSQL, hash := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(x, specifiledDB, x.Text())) - return x, normalizedSQL, hash, nil + return x, normalizedSQL, hash.String(), nil } return nil, "", "", nil } diff --git a/session/session.go b/session/session.go index a479ba956cf63..797d0326c4a48 100644 --- a/session/session.go +++ b/session/session.go @@ -1226,7 +1226,8 @@ func (s *session) SetProcessInfo(sql string, t time.Time, command byte, maxExecu if oldPi != nil && oldPi.Info == pi.Info { pi.Time = oldPi.Time } - _, pi.Digest = s.sessionVars.StmtCtx.SQLDigest() + _, digest := s.sessionVars.StmtCtx.SQLDigest() + pi.Digest = digest.String() // DO NOT reset the currentPlan to nil until this query finishes execution, otherwise reentrant calls // of SetProcessInfo would override Plan and PlanExplainRows to nil. if command == mysql.ComSleep { diff --git a/session/session_test.go b/session/session_test.go index 9d2d63cb02804..b8d8538193c7d 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4390,7 +4390,7 @@ func (s *testTxnStateSuite) TestBasic(c *C) { tk.MustExec("select * from t for update;") info = tk.Se.TxnInfo() _, expectedDigest := parser.NormalizeDigest("select * from t for update;") - c.Assert(info.CurrentSQLDigest, Equals, expectedDigest) + c.Assert(info.CurrentSQLDigest, Equals, expectedDigest.String()) c.Assert(info.State, Equals, txninfo.TxnRunningNormal) c.Assert(info.BlockStartTime, IsNil) // len and size will be covered in TestLenAndSize diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index d8a75aec48610..ea8bd70b8c0f2 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/util/disk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/resourcegrouptag" atomic2 "go.uber.org/atomic" "go.uber.org/zap" ) @@ -145,11 +146,11 @@ type StatementContext struct { digestMemo struct { sync.Once normalized string - digest string + digest *parser.Digest } // planNormalized use for cache the normalized plan, avoid duplicate builds. planNormalized string - planDigest string + planDigest *parser.Digest encodedPlan string planHint string planHintSet bool @@ -165,6 +166,8 @@ type StatementContext struct { // stmtCache is used to store some statement-related values. stmtCache map[StmtCacheKey]interface{} + // resourceGroupTag cache for the current statement resource group tag. + resourceGroupTag atomic.Value } // StmtHints are SessionVars related sql hints. @@ -229,7 +232,7 @@ func (sc *StatementContext) ResetStmtCache() { // SQLDigest gets normalized and digest for provided sql. // it will cache result after first calling. -func (sc *StatementContext) SQLDigest() (normalized, sqlDigest string) { +func (sc *StatementContext) SQLDigest() (normalized string, sqlDigest *parser.Digest) { sc.digestMemo.Do(func() { sc.digestMemo.normalized, sc.digestMemo.digest = parser.NormalizeDigest(sc.OriginalSQL) }) @@ -237,20 +240,37 @@ func (sc *StatementContext) SQLDigest() (normalized, sqlDigest string) { } // InitSQLDigest sets the normalized and digest for sql. -func (sc *StatementContext) InitSQLDigest(normalized, digest string) { +func (sc *StatementContext) InitSQLDigest(normalized string, digest *parser.Digest) { sc.digestMemo.Do(func() { sc.digestMemo.normalized, sc.digestMemo.digest = normalized, digest }) } // GetPlanDigest gets the normalized plan and plan digest. -func (sc *StatementContext) GetPlanDigest() (normalized, planDigest string) { +func (sc *StatementContext) GetPlanDigest() (normalized string, planDigest *parser.Digest) { return sc.planNormalized, sc.planDigest } +// GetResourceGroupTag gets the resource group of the statement. +func (sc *StatementContext) GetResourceGroupTag() []byte { + tag, _ := sc.resourceGroupTag.Load().([]byte) + if len(tag) > 0 { + return tag + } + normalized, sqlDigest := sc.SQLDigest() + if len(normalized) == 0 { + return nil + } + tag = resourcegrouptag.EncodeResourceGroupTag(sqlDigest, sc.planDigest) + sc.resourceGroupTag.Store(tag) + return tag +} + // SetPlanDigest sets the normalized plan and plan digest. -func (sc *StatementContext) SetPlanDigest(normalized, planDigest string) { - sc.planNormalized, sc.planDigest = normalized, planDigest +func (sc *StatementContext) SetPlanDigest(normalized string, planDigest *parser.Digest) { + if planDigest != nil { + sc.planNormalized, sc.planDigest = normalized, planDigest + } } // GetEncodedPlan gets the encoded plan, it is used to avoid repeated encode. diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index ee9030056bf05..00b728557d188 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -231,7 +231,7 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { logItems := &variable.SlowQueryLogItems{ TxnTS: txnTS, SQL: sql, - Digest: digest, + Digest: digest.String(), TimeTotal: costTime, TimeParse: time.Duration(10), TimeCompile: time.Duration(10), diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 8e45b546f10a9..7f954bb254051 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -514,12 +514,13 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *Backoffer, ta } req := tikvrpc.NewRequest(task.cmdType, &copReq, kvrpcpb.Context{ - IsolationLevel: isolationLevelToPB(b.req.IsolationLevel), - Priority: priorityToPB(b.req.Priority), - NotFillCache: b.req.NotFillCache, - RecordTimeStat: true, - RecordScanStat: true, - TaskId: b.req.TaskID, + IsolationLevel: isolationLevelToPB(b.req.IsolationLevel), + Priority: priorityToPB(b.req.Priority), + NotFillCache: b.req.NotFillCache, + RecordTimeStat: true, + RecordScanStat: true, + TaskId: b.req.TaskID, + ResourceGroupTag: b.req.ResourceGroupTag, }) req.StoreTp = tikvrpc.TiFlash diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index c66c3cda9af35..8834824432bfd 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -699,12 +699,13 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch } req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, options.GetTiKVReplicaReadType(worker.req.ReplicaRead), &worker.replicaReadSeed, kvrpcpb.Context{ - IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel), - Priority: priorityToPB(worker.req.Priority), - NotFillCache: worker.req.NotFillCache, - RecordTimeStat: true, - RecordScanStat: true, - TaskId: worker.req.TaskID, + IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel), + Priority: priorityToPB(worker.req.Priority), + NotFillCache: worker.req.NotFillCache, + RecordTimeStat: true, + RecordScanStat: true, + TaskId: worker.req.TaskID, + ResourceGroupTag: worker.req.ResourceGroupTag, }) req.StoreTp = getEndPointType(task.storeType) startTime := time.Now() diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 6692f45a749a3..892a85e9ccebb 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -88,6 +88,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { s.KVSnapshot.SetIsStatenessReadOnly(val.(bool)) case kv.MatchStoreLabels: s.KVSnapshot.SetMatchStoreLabels(val.([]*metapb.StoreLabel)) + case kv.ResourceGroupTag: + s.KVSnapshot.SetResourceGroupTag(val.([]byte)) } } diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 7f05f80139c12..a722557f8fc4e 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -170,6 +170,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { txn.KVTxn.GetSnapshot().SetIsStatenessReadOnly(val.(bool)) case kv.MatchStoreLabels: txn.KVTxn.GetSnapshot().SetMatchStoreLabels(val.([]*metapb.StoreLabel)) + case kv.ResourceGroupTag: + txn.KVTxn.SetResourceGroupTag(val.([]byte)) } } diff --git a/store/mockstore/unistore/rpc.go b/store/mockstore/unistore/rpc.go index 378b6b23b56a4..1cde24dd5ed94 100644 --- a/store/mockstore/unistore/rpc.go +++ b/store/mockstore/unistore/rpc.go @@ -59,6 +59,9 @@ type RPCClient struct { rpcCli Client } +// UnistoreRPCClientSendHook exports for test. +var UnistoreRPCClientSendHook func(*tikvrpc.Request) + // SendRequest sends a request to mock cluster. func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { failpoint.Inject("rpcServerBusy", func(val failpoint.Value) { @@ -67,6 +70,12 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R } }) + failpoint.Inject("unistoreRPCClientSendHook", func(val failpoint.Value) { + if val.(bool) && UnistoreRPCClientSendHook != nil { + UnistoreRPCClientSendHook(req) + } + }) + if req.StoreTp == tikvrpc.TiDB { return c.redirectRequestToRPCServer(ctx, addr, req, timeout) } diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index c01d97981dd09..a299ba357d3e8 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -107,6 +107,8 @@ type twoPhaseCommitter struct { doingAmend bool binlog BinlogExecutor + + resourceGroupTag []byte } type memBufferMutations struct { @@ -428,6 +430,7 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error { c.lockTTL = txnLockTTL(txn.startTime, size) c.priority = txn.priority.ToPB() c.syncLog = txn.syncLog + c.resourceGroupTag = txn.resourceGroupTag c.setDetail(commitDetail) return nil } diff --git a/store/tikv/cleanup.go b/store/tikv/cleanup.go index 0260d770cdd44..e21c1211af9bf 100644 --- a/store/tikv/cleanup.go +++ b/store/tikv/cleanup.go @@ -40,7 +40,7 @@ func (actionCleanup) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batc req := tikvrpc.NewRequest(tikvrpc.CmdBatchRollback, &pb.BatchRollbackRequest{ Keys: batch.mutations.GetKeys(), StartVersion: c.startTS, - }, pb.Context{Priority: c.priority, SyncLog: c.syncLog}) + }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) resp, err := c.store.SendReq(bo, req, batch.region, ReadTimeoutShort) if err != nil { return errors.Trace(err) diff --git a/store/tikv/commit.go b/store/tikv/commit.go index 10c60d9f6d4bd..8e876a6f11468 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -46,7 +46,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch StartVersion: c.startTS, Keys: keys, CommitVersion: c.commitTS, - }, pb.Context{Priority: c.priority, SyncLog: c.syncLog}) + }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index 49ddc1525b748..305806c931149 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -116,7 +116,7 @@ func (c *twoPhaseCommitter) buildPrewriteRequest(batch batchMutations, txnSize u req.TryOnePc = true } - return tikvrpc.NewRequest(tikvrpc.CmdPrewrite, req, pb.Context{Priority: c.priority, SyncLog: c.syncLog}) + return tikvrpc.NewRequest(tikvrpc.CmdPrewrite, req, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) } func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 94ece80ff067f..64b9a4728f551 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -192,9 +192,10 @@ func (s *Scanner) getData(bo *Backoffer) error { } sreq := &pb.ScanRequest{ Context: &pb.Context{ - Priority: s.snapshot.priority.ToPB(), - NotFillCache: s.snapshot.notFillCache, - IsolationLevel: s.snapshot.isolationLevel.ToPB(), + Priority: s.snapshot.priority.ToPB(), + NotFillCache: s.snapshot.notFillCache, + IsolationLevel: s.snapshot.isolationLevel.ToPB(), + ResourceGroupTag: s.snapshot.resourceGroupTag, }, StartKey: s.nextStartKey, EndKey: reqEndKey, @@ -210,9 +211,10 @@ func (s *Scanner) getData(bo *Backoffer) error { } s.snapshot.mu.RLock() req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdScan, sreq, s.snapshot.mu.replicaRead, &s.snapshot.replicaReadSeed, pb.Context{ - Priority: s.snapshot.priority.ToPB(), - NotFillCache: s.snapshot.notFillCache, - TaskId: s.snapshot.mu.taskID, + Priority: s.snapshot.priority.ToPB(), + NotFillCache: s.snapshot.notFillCache, + TaskId: s.snapshot.mu.taskID, + ResourceGroupTag: s.snapshot.resourceGroupTag, }) s.snapshot.mu.RUnlock() resp, err := sender.SendReq(bo, req, loc.Region, ReadTimeoutMedium) diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 180ac59369aca..9828537b7cb79 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -108,6 +108,8 @@ type KVSnapshot struct { matchStoreLabels []*metapb.StoreLabel } sampleStep uint32 + // resourceGroupTag is use to set the kv request resource group tag. + resourceGroupTag []byte } // newTiKVSnapshot creates a snapshot of an TiKV store. @@ -310,9 +312,10 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec Keys: pending, Version: s.version, }, s.mu.replicaRead, &s.replicaReadSeed, pb.Context{ - Priority: s.priority.ToPB(), - NotFillCache: s.notFillCache, - TaskId: s.mu.taskID, + Priority: s.priority.ToPB(), + NotFillCache: s.notFillCache, + TaskId: s.mu.taskID, + ResourceGroupTag: s.resourceGroupTag, }) isStaleness = s.mu.isStaleness matchStoreLabels = s.mu.matchStoreLabels @@ -462,9 +465,10 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, Key: k, Version: s.version, }, s.mu.replicaRead, &s.replicaReadSeed, pb.Context{ - Priority: s.priority.ToPB(), - NotFillCache: s.notFillCache, - TaskId: s.mu.taskID, + Priority: s.priority.ToPB(), + NotFillCache: s.notFillCache, + TaskId: s.mu.taskID, + ResourceGroupTag: s.resourceGroupTag, }) isStaleness = s.mu.isStaleness matchStoreLabels = s.mu.matchStoreLabels @@ -629,6 +633,11 @@ func (s *KVSnapshot) SetMatchStoreLabels(labels []*metapb.StoreLabel) { s.mu.matchStoreLabels = labels } +// SetResourceGroupTag sets resource group of the kv request. +func (s *KVSnapshot) SetResourceGroupTag(tag []byte) { + s.resourceGroupTag = tag +} + // SnapCacheHitCount gets the snapshot cache hit count. Only for test. func (s *KVSnapshot) SnapCacheHitCount() int { return int(atomic.LoadInt64(&s.mu.hitCnt)) diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 78fd2d5277a82..f2c5fc24449c3 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -110,6 +110,7 @@ type KVTxn struct { causalConsistency bool scope string kvFilter KVFilter + resourceGroupTag []byte } // ExtractStartTS use `option` to get the proper startTS for a transaction. @@ -231,6 +232,12 @@ func (txn *KVTxn) SetPriority(pri Priority) { txn.GetSnapshot().SetPriority(pri) } +// SetResourceGroupTag sets the resource tag for both write and read. +func (txn *KVTxn) SetResourceGroupTag(tag []byte) { + txn.resourceGroupTag = tag + txn.GetSnapshot().SetResourceGroupTag(tag) +} + // SetSchemaAmender sets an amender to update mutations after schema change. func (txn *KVTxn) SetSchemaAmender(sa SchemaAmender) { txn.schemaAmender = sa diff --git a/tools/check/go.mod b/tools/check/go.mod index f3827700f1589..90b34e7a296c4 100644 --- a/tools/check/go.mod +++ b/tools/check/go.mod @@ -18,7 +18,6 @@ require ( gopkg.in/alecthomas/gometalinter.v3 v3.0.0 // indirect gopkg.in/alecthomas/kingpin.v2 v2.2.6 // indirect gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20170321130658-9670b87a702e // indirect - gopkg.in/yaml.v2 v2.2.2 // indirect honnef.co/go/tools v0.0.0-20180920025451-e3ad64cb4ed3 ) diff --git a/util/deadlockhistory/deadlock_history.go b/util/deadlockhistory/deadlock_history.go index ddb78067ffe7c..c219442cf5bf1 100644 --- a/util/deadlockhistory/deadlock_history.go +++ b/util/deadlockhistory/deadlock_history.go @@ -183,7 +183,7 @@ func ErrDeadlockToDeadlockRecord(dl *tikverr.ErrDeadlock) *DeadlockRecord { } waitChain = append(waitChain, WaitChainItem{ TryLockTxn: rawItem.Txn, - SQLDigest: sqlDigest, + SQLDigest: hex.EncodeToString(sqlDigest), Key: rawItem.Key, AllSQLs: nil, TxnHoldingLock: rawItem.WaitForTxn, diff --git a/util/deadlockhistory/deadlock_history_test.go b/util/deadlockhistory/deadlock_history_test.go index 35cbb6c8513cd..dd9428a9f550a 100644 --- a/util/deadlockhistory/deadlock_history_test.go +++ b/util/deadlockhistory/deadlock_history_test.go @@ -20,9 +20,10 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/parser" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/resourcegrouptag" + "github.com/pingcap/tipb/go-tipb" ) type testDeadlockHistorySuite struct{} @@ -228,6 +229,11 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) { } func (s *testDeadlockHistorySuite) TestErrDeadlockToDeadlockRecord(c *C) { + digest1, digest2 := parser.NewDigest([]byte("aabbccdd")), parser.NewDigest([]byte("ddccbbaa")) + tag1 := tipb.ResourceGroupTag{SqlDigest: digest1.Bytes()} + tag2 := tipb.ResourceGroupTag{SqlDigest: digest2.Bytes()} + tag1Data, _ := tag1.Marshal() + tag2Data, _ := tag2.Marshal() err := &tikverr.ErrDeadlock{ Deadlock: &kvrpcpb.Deadlock{ LockTs: 101, @@ -238,13 +244,13 @@ func (s *testDeadlockHistorySuite) TestErrDeadlockToDeadlockRecord(c *C) { Txn: 100, WaitForTxn: 101, Key: []byte("k2"), - ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag("aabbccdd"), + ResourceGroupTag: tag1Data, }, { Txn: 101, WaitForTxn: 100, Key: []byte("k1"), - ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag("ddccbbaa"), + ResourceGroupTag: tag2Data, }, }, }, @@ -256,13 +262,13 @@ func (s *testDeadlockHistorySuite) TestErrDeadlockToDeadlockRecord(c *C) { WaitChain: []WaitChainItem{ { TryLockTxn: 100, - SQLDigest: "aabbccdd", + SQLDigest: digest1.String(), Key: []byte("k2"), TxnHoldingLock: 101, }, { TryLockTxn: 101, - SQLDigest: "ddccbbaa", + SQLDigest: digest2.String(), Key: []byte("k1"), TxnHoldingLock: 100, }, diff --git a/util/resourcegrouptag/resource_group_tag.go b/util/resourcegrouptag/resource_group_tag.go index cacbf574b91fb..03150a0393ea4 100644 --- a/util/resourcegrouptag/resource_group_tag.go +++ b/util/resourcegrouptag/resource_group_tag.go @@ -1,85 +1,40 @@ package resourcegrouptag import ( - "encoding/hex" - "github.com/pingcap/errors" - "github.com/pingcap/tidb/util/logutil" - "go.uber.org/zap" -) - -const ( - resourceGroupTagPrefixSQLDigest = byte(1) + "github.com/pingcap/parser" + "github.com/pingcap/tipb/go-tipb" ) -// EncodeResourceGroupTag encodes sqlDigest into resource group tag. -// A resource group tag can be carried in the Context field of TiKV requests, which is a byte array, and sent to TiKV as -// diagnostic information. Currently it contains only the SQL Digest, and the codec method is naive but extendable. -// This function doesn't return error. When there's some error, which can only be caused by unexpected format of the -// arguments, it simply returns an empty result. -// The format: -// +-----------+-----------------------+----------------------------+---------------+----------------+---- -// | version=1 | field1 prefix (1byte) | field1 content (var bytes) | field2 prefix | field2 content | ... -// +-----------+-----------------------+----------------------------+---------------+----------------+---- -// The `version` section marks the codec version, which makes it easier for changing the format in the future. -// Each field starts with a byte to mark what field it is, and the length of the content depends on the field's -// definition. -// Currently there's only one field (SQL Digest), and its content starts with a byte `B` describing it's length, and -// then follows by exactly `B` bytes. -func EncodeResourceGroupTag(sqlDigest string) []byte { - if len(sqlDigest) == 0 { - return nil - } - if len(sqlDigest) >= 512 { - logutil.BgLogger().Warn("failed to encode sql digest to resource group tag: length too long", zap.String("sqlDigest", sqlDigest)) +// EncodeResourceGroupTag encodes sql digest and plan digest into resource group tag. +func EncodeResourceGroupTag(sqlDigest, planDigest *parser.Digest) []byte { + if sqlDigest == nil && planDigest == nil { return nil } - res := make([]byte, 3+len(sqlDigest)/2) - - const encodingVersion = 1 - res[0] = encodingVersion - - res[1] = resourceGroupTagPrefixSQLDigest - // The SQL Digest is expected to be a hex string. Convert it back to bytes to save half of the memory. - res[2] = byte(len(sqlDigest) / 2) - _, err := hex.Decode(res[3:], []byte(sqlDigest)) + tag := &tipb.ResourceGroupTag{} + if sqlDigest != nil { + tag.SqlDigest = sqlDigest.Bytes() + } + if planDigest != nil { + tag.PlanDigest = planDigest.Bytes() + } + b, err := tag.Marshal() if err != nil { - logutil.BgLogger().Warn("failed to encode sql digest to resource group tag: invalid hex string", zap.String("sqlDigest", sqlDigest)) return nil } - - return res + return b } -// DecodeResourceGroupTag decodes a resource group tag into various information contained in it. Currently it contains -// only the SQL Digest. -func DecodeResourceGroupTag(data []byte) (sqlDigest string, err error) { +// DecodeResourceGroupTag decodes a resource group tag and return the sql digest. +func DecodeResourceGroupTag(data []byte) (sqlDigest []byte, err error) { if len(data) == 0 { - return "", nil - } - - encodingVersion := data[0] - if encodingVersion != 1 { - return "", errors.Errorf("unsupported resource group tag version %v", data[0]) + return nil, nil } - rem := data[1:] - - for len(rem) > 0 { - switch rem[0] { - case resourceGroupTagPrefixSQLDigest: - // There must be one more byte at rem[1] to represent the content's length, and the remaining bytes should - // not be shorter than the length specified by rem[1]. - if len(rem) < 2 || len(rem)-2 < int(rem[1]) { - return "", errors.Errorf("cannot parse resource group tag: field length mismatch, tag: %v", hex.EncodeToString(data)) - } - fieldLen := int(rem[1]) - sqlDigest = hex.EncodeToString(rem[2 : 2+fieldLen]) - rem = rem[2+fieldLen:] - default: - return "", errors.Errorf("resource group tag field not recognized, prefix: %v, tag: %v", rem[0], hex.EncodeToString(data)) - } + tag := &tipb.ResourceGroupTag{} + err = tag.Unmarshal(data) + if err != nil { + return nil, errors.Errorf("invalid resource group tag data %x", data) } - - return + return tag.SqlDigest, nil } diff --git a/util/resourcegrouptag/resource_group_tag_test.go b/util/resourcegrouptag/resource_group_tag_test.go index a979b92fce315..f5334aacbd17f 100644 --- a/util/resourcegrouptag/resource_group_tag_test.go +++ b/util/resourcegrouptag/resource_group_tag_test.go @@ -14,10 +14,14 @@ package resourcegrouptag import ( + "crypto/sha256" "math/rand" "testing" . "github.com/pingcap/check" + "github.com/pingcap/parser" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tipb/go-tipb" ) type testUtilsSuite struct{} @@ -29,83 +33,76 @@ func TestT(t *testing.T) { } func (s *testUtilsSuite) TestResourceGroupTagEncoding(c *C) { - sqlDigest := "" - tag := EncodeResourceGroupTag(sqlDigest) + sqlDigest := parser.NewDigest(nil) + tag := EncodeResourceGroupTag(sqlDigest, nil) c.Assert(len(tag), Equals, 0) decodedSQLDigest, err := DecodeResourceGroupTag(tag) c.Assert(err, IsNil) c.Assert(len(decodedSQLDigest), Equals, 0) - sqlDigest = "aa" - tag = EncodeResourceGroupTag(sqlDigest) + sqlDigest = parser.NewDigest([]byte{'a', 'a'}) + tag = EncodeResourceGroupTag(sqlDigest, nil) // version(1) + prefix(1) + length(1) + content(2hex -> 1byte) c.Assert(len(tag), Equals, 4) decodedSQLDigest, err = DecodeResourceGroupTag(tag) c.Assert(err, IsNil) - c.Assert(decodedSQLDigest, Equals, sqlDigest) + c.Assert(decodedSQLDigest, DeepEquals, sqlDigest.Bytes()) - sqlDigest = genRandHex(64) - tag = EncodeResourceGroupTag(sqlDigest) + sqlDigest = parser.NewDigest(genRandHex(64)) + tag = EncodeResourceGroupTag(sqlDigest, nil) decodedSQLDigest, err = DecodeResourceGroupTag(tag) c.Assert(err, IsNil) - c.Assert(decodedSQLDigest, Equals, sqlDigest) + c.Assert(decodedSQLDigest, DeepEquals, sqlDigest.Bytes()) - sqlDigest = genRandHex(510) - tag = EncodeResourceGroupTag(sqlDigest) + sqlDigest = parser.NewDigest(genRandHex(510)) + tag = EncodeResourceGroupTag(sqlDigest, nil) decodedSQLDigest, err = DecodeResourceGroupTag(tag) c.Assert(err, IsNil) - c.Assert(decodedSQLDigest, Equals, sqlDigest) - - // The max supported length is 255 bytes (510 hex digits). - sqlDigest = genRandHex(512) - tag = EncodeResourceGroupTag(sqlDigest) - c.Assert(len(tag), Equals, 0) - - // A hex string can't have odd length. - sqlDigest = genRandHex(15) - tag = EncodeResourceGroupTag(sqlDigest) - c.Assert(len(tag), Equals, 0) - - // Non-hexadecimal character is invalid - sqlDigest = "aabbccddgg" - tag = EncodeResourceGroupTag(sqlDigest) - c.Assert(len(tag), Equals, 0) - - // A tag should start with a supported version - tag = []byte("\x00") - _, err = DecodeResourceGroupTag(tag) - c.Assert(err, NotNil) - - // The fields should have format like `[prefix, length, content...]`, otherwise decoding it should returns error. - tag = []byte("\x01\x01") - _, err = DecodeResourceGroupTag(tag) - c.Assert(err, NotNil) - - tag = []byte("\x01\x01\x02") - _, err = DecodeResourceGroupTag(tag) - c.Assert(err, NotNil) - - tag = []byte("\x01\x01\x02AB") - decodedSQLDigest, err = DecodeResourceGroupTag(tag) - c.Assert(err, IsNil) - c.Assert(decodedSQLDigest, Equals, "4142") - - tag = []byte("\x01\x01\x00") - decodedSQLDigest, err = DecodeResourceGroupTag(tag) - c.Assert(err, IsNil) - c.Assert(len(decodedSQLDigest), Equals, 0) - - // Unsupported field - tag = []byte("\x01\x99") - _, err = DecodeResourceGroupTag(tag) - c.Assert(err, NotNil) + c.Assert(decodedSQLDigest, DeepEquals, sqlDigest.Bytes()) } -func genRandHex(length int) string { +func genRandHex(length int) []byte { const chars = "0123456789abcdef" res := make([]byte, length) for i := 0; i < length; i++ { res[i] = chars[rand.Intn(len(chars))] } - return string(res) + return res +} + +func genDigest(str string) []byte { + hasher := sha256.New() + hasher.Write(hack.Slice(str)) + return hasher.Sum(nil) +} + +func (s *testUtilsSuite) TestResourceGroupTagEncodingPB(c *C) { + digest1 := genDigest("abc") + digest2 := genDigest("abcdefg") + // Test for protobuf + resourceTag := &tipb.ResourceGroupTag{ + SqlDigest: digest1, + PlanDigest: digest2, + } + buf, err := resourceTag.Marshal() + c.Assert(err, IsNil) + c.Assert(len(buf), Equals, 68) + tag := &tipb.ResourceGroupTag{} + err = tag.Unmarshal(buf) + c.Assert(err, IsNil) + c.Assert(tag.SqlDigest, DeepEquals, digest1) + c.Assert(tag.PlanDigest, DeepEquals, digest2) + + // Test for protobuf sql_digest only + resourceTag = &tipb.ResourceGroupTag{ + SqlDigest: digest1, + } + buf, err = resourceTag.Marshal() + c.Assert(err, IsNil) + c.Assert(len(buf), Equals, 34) + tag = &tipb.ResourceGroupTag{} + err = tag.Unmarshal(buf) + c.Assert(err, IsNil) + c.Assert(tag.SqlDigest, DeepEquals, digest1) + c.Assert(tag.PlanDigest, IsNil) } From ab5cf857d3edd2c54b988d4ec38534e8a2d72844 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 25 May 2021 16:29:34 +0800 Subject: [PATCH 164/343] store/tikv: move client*.go into single package tikv/client (#24798) --- store/copr/batch_coprocessor.go | 4 +- store/copr/mpp.go | 2 +- store/driver/sql_fail_test.go | 1 - store/gcworker/gc_worker.go | 16 +- store/tikv/2pc.go | 3 +- store/tikv/cleanup.go | 3 +- store/tikv/client.go | 502 +---------------- store/tikv/client/client.go | 515 ++++++++++++++++++ store/tikv/{ => client}/client_batch.go | 4 +- store/tikv/{ => client}/client_collapse.go | 8 +- store/tikv/{ => client}/client_fail_test.go | 12 +- store/tikv/{ => client}/client_test.go | 27 +- .../{ => client}/mock_tikv_service_test.go | 2 +- store/tikv/commit.go | 3 +- store/tikv/delete_range.go | 3 +- store/tikv/kv.go | 7 +- store/tikv/lock_resolver.go | 15 +- store/tikv/pessimistic.go | 5 +- store/tikv/prewrite.go | 3 +- store/tikv/rawkv.go | 11 +- store/tikv/region_cache.go | 5 +- store/tikv/scan.go | 3 +- store/tikv/snapshot.go | 5 +- store/tikv/split_region.go | 3 +- 24 files changed, 623 insertions(+), 539 deletions(-) create mode 100644 store/tikv/client/client.go rename store/tikv/{ => client}/client_batch.go (99%) rename store/tikv/{ => client}/client_collapse.go (93%) rename store/tikv/{ => client}/client_fail_test.go (93%) rename store/tikv/{ => client}/client_test.go (95%) rename store/tikv/{ => client}/mock_tikv_service_test.go (99%) diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 7f954bb254051..1e6a1dddab2bc 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -491,6 +491,8 @@ func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *Backoffer, return buildBatchCopTasks(bo, b.store.GetRegionCache(), tikv.NewKeyRanges(ranges), b.req.StoreType) } +const readTimeoutUltraLong = 3600 * time.Second // For requests that may scan many regions for tiflash. + func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *Backoffer, task *batchCopTask) ([]*batchCopTask, error) { sender := tikv.NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) var regionInfos = make([]*coprocessor.RegionInfo, 0, len(task.regionInfos)) @@ -525,7 +527,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *Backoffer, ta req.StoreTp = tikvrpc.TiFlash logutil.BgLogger().Debug("send batch request to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.regionInfos))) - resp, retry, cancel, err := sender.SendReqToAddr(bo.TiKVBackoffer(), task.ctx, task.regionInfos, req, tikv.ReadTimeoutUltraLong) + resp, retry, cancel, err := sender.SendReqToAddr(bo.TiKVBackoffer(), task.ctx, task.regionInfos, req, readTimeoutUltraLong) // If there are store errors, we should retry for all regions. if retry { return b.retryBatchCopTask(ctx, bo, task) diff --git a/store/copr/mpp.go b/store/copr/mpp.go index 6d58e4ef732fe..ac2bbbb823cf2 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -310,7 +310,7 @@ func (m *mppIterator) establishMPPConns(bo *Backoffer, req *kv.MPPDispatchReques // Drain result from root task. // We don't need to process any special error. When we meet errors, just let it fail. - rpcResp, err := m.store.GetTiKVClient().SendRequest(bo.GetCtx(), req.Meta.GetAddress(), wrappedReq, tikv.ReadTimeoutUltraLong) + rpcResp, err := m.store.GetTiKVClient().SendRequest(bo.GetCtx(), req.Meta.GetAddress(), wrappedReq, readTimeoutUltraLong) if err != nil { logutil.BgLogger().Error("establish mpp connection meet error", zap.String("error", err.Error())) diff --git a/store/driver/sql_fail_test.go b/store/driver/sql_fail_test.go index 903dcedcb1878..dd0dcc6538c44 100644 --- a/store/driver/sql_fail_test.go +++ b/store/driver/sql_fail_test.go @@ -100,7 +100,6 @@ func (s *testSQLSerialSuite) TestFailBusyServerCop(c *C) { } func TestMain(m *testing.M) { - tikv.ReadTimeoutMedium = 2 * time.Second os.Exit(m.Run()) } diff --git a/store/gcworker/gc_worker.go b/store/gcworker/gc_worker.go index 72ae8bc34f0fa..5982f7e2dd0ca 100644 --- a/store/gcworker/gc_worker.go +++ b/store/gcworker/gc_worker.go @@ -185,6 +185,12 @@ var gcVariableComments = map[string]string{ gcScanLockModeKey: "Mode of scanning locks, \"physical\" or \"legacy\"", } +const ( + unsafeDestroyRangeTimeout = 5 * time.Minute + accessLockObserverTimeout = 10 * time.Second + gcTimeout = 5 * time.Minute +) + func (w *GCWorker) start(ctx context.Context, wg *sync.WaitGroup) { logutil.Logger(ctx).Info("[gc worker] start", zap.String("uuid", w.uuid)) @@ -808,7 +814,7 @@ func (w *GCWorker) doUnsafeDestroyRangeRequest(ctx context.Context, startKey []b go func() { defer wg.Done() - resp, err1 := w.tikvStore.GetTiKVClient().SendRequest(ctx, address, req, tikv.UnsafeDestroyRangeTimeout) + resp, err1 := w.tikvStore.GetTiKVClient().SendRequest(ctx, address, req, unsafeDestroyRangeTimeout) if err1 == nil { if resp == nil || resp.Resp == nil { err1 = errors.Errorf("unsafe destroy range returns nil response from store %v", storeID) @@ -1272,7 +1278,7 @@ func (w *GCWorker) registerLockObservers(ctx context.Context, safePoint uint64, for _, store := range stores { address := store.Address - resp, err := w.tikvStore.GetTiKVClient().SendRequest(ctx, address, req, tikv.AccessLockObserverTimeout) + resp, err := w.tikvStore.GetTiKVClient().SendRequest(ctx, address, req, accessLockObserverTimeout) if err != nil { return errors.Trace(err) } @@ -1312,7 +1318,7 @@ func (w *GCWorker) checkLockObservers(ctx context.Context, safePoint uint64, sto for _, store := range stores { address := store.Address - resp, err := w.tikvStore.GetTiKVClient().SendRequest(ctx, address, req, tikv.AccessLockObserverTimeout) + resp, err := w.tikvStore.GetTiKVClient().SendRequest(ctx, address, req, accessLockObserverTimeout) if err != nil { logError(store, err) continue @@ -1378,7 +1384,7 @@ func (w *GCWorker) removeLockObservers(ctx context.Context, safePoint uint64, st for _, store := range stores { address := store.Address - resp, err := w.tikvStore.GetTiKVClient().SendRequest(ctx, address, req, tikv.AccessLockObserverTimeout) + resp, err := w.tikvStore.GetTiKVClient().SendRequest(ctx, address, req, accessLockObserverTimeout) if err != nil { logError(store, err) continue @@ -1600,7 +1606,7 @@ func (w *GCWorker) doGCForRegion(bo *tikv.Backoffer, safePoint uint64, region ti SafePoint: safePoint, }) - resp, err := w.tikvStore.SendReq(bo, req, region, tikv.GCTimeout) + resp, err := w.tikvStore.SendReq(bo, req, region, gcTimeout) if err != nil { return nil, errors.Trace(err) } diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index a299ba357d3e8..64d3701eb7efd 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" @@ -794,7 +795,7 @@ func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, tt if err != nil { return 0, errors.Trace(err) } - resp, err := store.SendReq(bo, req, loc.Region, ReadTimeoutShort) + resp, err := store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) if err != nil { return 0, errors.Trace(err) } diff --git a/store/tikv/cleanup.go b/store/tikv/cleanup.go index e21c1211af9bf..553e00ff360a7 100644 --- a/store/tikv/cleanup.go +++ b/store/tikv/cleanup.go @@ -16,6 +16,7 @@ package tikv import ( "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/retry" @@ -41,7 +42,7 @@ func (actionCleanup) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batc Keys: batch.mutations.GetKeys(), StartVersion: c.startTS, }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) - resp, err := c.store.SendReq(bo, req, batch.region, ReadTimeoutShort) + resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/client.go b/store/tikv/client.go index 63b18496d04c6..cb446d4a300e0 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -1,4 +1,4 @@ -// Copyright 2016 PingCAP, Inc. +// Copyright 2021 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -11,509 +11,29 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package tikv provides tcp connection to kvserver. package tikv import ( - "context" - "fmt" - "io" - "math" - "runtime/trace" - "strconv" - "sync" - "sync/atomic" - "time" - - grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing" - "github.com/opentracing/opentracing-go" - "github.com/pingcap/errors" - "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/kvproto/pkg/debugpb" - "github.com/pingcap/kvproto/pkg/mpp" - "github.com/pingcap/kvproto/pkg/tikvpb" - "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/config" - "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" - "github.com/pingcap/tidb/store/tikv/tikvrpc" - "github.com/pingcap/tidb/store/tikv/util" - "github.com/prometheus/client_golang/prometheus" - "google.golang.org/grpc" - "google.golang.org/grpc/backoff" - "google.golang.org/grpc/connectivity" - "google.golang.org/grpc/credentials" - "google.golang.org/grpc/encoding/gzip" - "google.golang.org/grpc/keepalive" - "google.golang.org/grpc/metadata" ) -// MaxRecvMsgSize set max gRPC receive message size received from server. If any message size is larger than -// current value, an error will be reported from gRPC. -var MaxRecvMsgSize = math.MaxInt64 +// Client is a client that sends RPC. +// It should not be used after calling Close(). +type Client = client.Client // Timeout durations. -var ( - dialTimeout = 5 * time.Second - ReadTimeoutShort = 20 * time.Second // For requests that read/write several key-values. - ReadTimeoutMedium = 60 * time.Second // For requests that may need scan region. - ReadTimeoutLong = 150 * time.Second // For requests that may need scan region multiple times. - ReadTimeoutUltraLong = 3600 * time.Second // For requests that may scan many regions for tiflash. - GCTimeout = 5 * time.Minute - UnsafeDestroyRangeTimeout = 5 * time.Minute - AccessLockObserverTimeout = 10 * time.Second -) - const ( - grpcInitialWindowSize = 1 << 30 - grpcInitialConnWindowSize = 1 << 30 + ReadTimeoutMedium = client.ReadTimeoutMedium + ReadTimeoutShort = client.ReadTimeoutShort ) -// forwardMetadataKey is the key of gRPC metadata which represents a forwarded request. -const forwardMetadataKey = "tikv-forwarded-host" - -// Client is a client that sends RPC. -// It should not be used after calling Close(). -type Client interface { - // Close should release all data. - Close() error - // SendRequest sends Request. - SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) -} - -type connArray struct { - // The target host. - target string - - index uint32 - v []*grpc.ClientConn - // streamTimeout binds with a background goroutine to process coprocessor streaming timeout. - streamTimeout chan *tikvrpc.Lease - dialTimeout time.Duration - // batchConn is not null when batch is enabled. - *batchConn - done chan struct{} -} - -func newConnArray(maxSize uint, addr string, security config.Security, idleNotify *uint32, enableBatch bool, dialTimeout time.Duration) (*connArray, error) { - a := &connArray{ - index: 0, - v: make([]*grpc.ClientConn, maxSize), - streamTimeout: make(chan *tikvrpc.Lease, 1024), - done: make(chan struct{}), - dialTimeout: dialTimeout, - } - if err := a.Init(addr, security, idleNotify, enableBatch); err != nil { - return nil, err - } - return a, nil -} - -func (a *connArray) Init(addr string, security config.Security, idleNotify *uint32, enableBatch bool) error { - a.target = addr - - opt := grpc.WithInsecure() - if len(security.ClusterSSLCA) != 0 { - tlsConfig, err := security.ToTLSConfig() - if err != nil { - return errors.Trace(err) - } - opt = grpc.WithTransportCredentials(credentials.NewTLS(tlsConfig)) - } - - cfg := config.GetGlobalConfig() - var ( - unaryInterceptor grpc.UnaryClientInterceptor - streamInterceptor grpc.StreamClientInterceptor - ) - if cfg.OpenTracingEnable { - unaryInterceptor = grpc_opentracing.UnaryClientInterceptor() - streamInterceptor = grpc_opentracing.StreamClientInterceptor() - } - - allowBatch := (cfg.TiKVClient.MaxBatchSize > 0) && enableBatch - if allowBatch { - a.batchConn = newBatchConn(uint(len(a.v)), cfg.TiKVClient.MaxBatchSize, idleNotify) - a.pendingRequests = metrics.TiKVBatchPendingRequests.WithLabelValues(a.target) - a.batchSize = metrics.TiKVBatchRequests.WithLabelValues(a.target) - } - keepAlive := cfg.TiKVClient.GrpcKeepAliveTime - keepAliveTimeout := cfg.TiKVClient.GrpcKeepAliveTimeout - for i := range a.v { - ctx, cancel := context.WithTimeout(context.Background(), a.dialTimeout) - var callOptions []grpc.CallOption - callOptions = append(callOptions, grpc.MaxCallRecvMsgSize(MaxRecvMsgSize)) - if cfg.TiKVClient.GrpcCompressionType == gzip.Name { - callOptions = append(callOptions, grpc.UseCompressor(gzip.Name)) - } - conn, err := grpc.DialContext( - ctx, - addr, - opt, - grpc.WithInitialWindowSize(grpcInitialWindowSize), - grpc.WithInitialConnWindowSize(grpcInitialConnWindowSize), - grpc.WithUnaryInterceptor(unaryInterceptor), - grpc.WithStreamInterceptor(streamInterceptor), - grpc.WithDefaultCallOptions(callOptions...), - grpc.WithConnectParams(grpc.ConnectParams{ - Backoff: backoff.Config{ - BaseDelay: 100 * time.Millisecond, // Default was 1s. - Multiplier: 1.6, // Default - Jitter: 0.2, // Default - MaxDelay: 3 * time.Second, // Default was 120s. - }, - MinConnectTimeout: a.dialTimeout, - }), - grpc.WithKeepaliveParams(keepalive.ClientParameters{ - Time: time.Duration(keepAlive) * time.Second, - Timeout: time.Duration(keepAliveTimeout) * time.Second, - PermitWithoutStream: true, - }), - ) - cancel() - if err != nil { - // Cleanup if the initialization fails. - a.Close() - return errors.Trace(err) - } - a.v[i] = conn - - if allowBatch { - batchClient := &batchCommandsClient{ - target: a.target, - conn: conn, - forwardedClients: make(map[string]*batchCommandsStream), - batched: sync.Map{}, - epoch: 0, - closed: 0, - tikvClientCfg: cfg.TiKVClient, - tikvLoad: &a.tikvTransportLayerLoad, - dialTimeout: a.dialTimeout, - tryLock: tryLock{sync.NewCond(new(sync.Mutex)), false}, - } - a.batchCommandsClients = append(a.batchCommandsClients, batchClient) - } - } - go tikvrpc.CheckStreamTimeoutLoop(a.streamTimeout, a.done) - if allowBatch { - go a.batchSendLoop(cfg.TiKVClient) - } - - return nil -} - -func (a *connArray) Get() *grpc.ClientConn { - next := atomic.AddUint32(&a.index, 1) % uint32(len(a.v)) - return a.v[next] -} - -func (a *connArray) Close() { - if a.batchConn != nil { - a.batchConn.Close() - } - - for i, c := range a.v { - if c != nil { - err := c.Close() - terror.Log(errors.Trace(err)) - a.v[i] = nil - } - } - - close(a.done) -} - -// RPCClient is RPC client struct. -// TODO: Add flow control between RPC clients in TiDB ond RPC servers in TiKV. -// Since we use shared client connection to communicate to the same TiKV, it's possible -// that there are too many concurrent requests which overload the service of TiKV. -type RPCClient struct { - sync.RWMutex - - conns map[string]*connArray - security config.Security - - idleNotify uint32 - // recycleMu protect the conns from being modified during a connArray is taken out and used. - // That means recycleIdleConnArray() will wait until nobody doing sendBatchRequest() - recycleMu sync.RWMutex - // Periodically check whether there is any connection that is idle and then close and remove these connections. - // Implement background cleanup. - isClosed bool - dialTimeout time.Duration -} - -// NewRPCClient creates a client that manages connections and rpc calls with tikv-servers. -func NewRPCClient(security config.Security, opts ...func(c *RPCClient)) *RPCClient { - cli := &RPCClient{ - conns: make(map[string]*connArray), - security: security, - dialTimeout: dialTimeout, - } - for _, opt := range opts { - opt(cli) - } - return cli -} - // NewTestRPCClient is for some external tests. func NewTestRPCClient(security config.Security) Client { - return NewRPCClient(security) -} - -func (c *RPCClient) getConnArray(addr string, enableBatch bool, opt ...func(cfg *config.TiKVClient)) (*connArray, error) { - c.RLock() - if c.isClosed { - c.RUnlock() - return nil, errors.Errorf("rpcClient is closed") - } - array, ok := c.conns[addr] - c.RUnlock() - if !ok { - var err error - array, err = c.createConnArray(addr, enableBatch, opt...) - if err != nil { - return nil, err - } - } - return array, nil -} - -func (c *RPCClient) createConnArray(addr string, enableBatch bool, opts ...func(cfg *config.TiKVClient)) (*connArray, error) { - c.Lock() - defer c.Unlock() - array, ok := c.conns[addr] - if !ok { - var err error - client := config.GetGlobalConfig().TiKVClient - for _, opt := range opts { - opt(&client) - } - array, err = newConnArray(client.GrpcConnectionCount, addr, c.security, &c.idleNotify, enableBatch, c.dialTimeout) - if err != nil { - return nil, err - } - c.conns[addr] = array - } - return array, nil -} - -func (c *RPCClient) closeConns() { - c.Lock() - if !c.isClosed { - c.isClosed = true - // close all connections - for _, array := range c.conns { - array.Close() - } - } - c.Unlock() -} - -var sendReqHistCache sync.Map - -type sendReqHistCacheKey struct { - tp tikvrpc.CmdType - id uint64 -} - -func (c *RPCClient) updateTiKVSendReqHistogram(req *tikvrpc.Request, start time.Time) { - key := sendReqHistCacheKey{ - req.Type, - req.Context.GetPeer().GetStoreId(), - } - - v, ok := sendReqHistCache.Load(key) - if !ok { - reqType := req.Type.String() - storeID := strconv.FormatUint(req.Context.GetPeer().GetStoreId(), 10) - v = metrics.TiKVSendReqHistogram.WithLabelValues(reqType, storeID) - sendReqHistCache.Store(key, v) - } - - v.(prometheus.Observer).Observe(time.Since(start).Seconds()) -} - -// SendRequest sends a Request to server and receives Response. -func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { - if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { - span1 := span.Tracer().StartSpan(fmt.Sprintf("rpcClient.SendRequest, region ID: %d, type: %s", req.RegionId, req.Type), opentracing.ChildOf(span.Context())) - defer span1.Finish() - ctx = opentracing.ContextWithSpan(ctx, span1) - } - - start := time.Now() - defer func() { - stmtExec := ctx.Value(util.ExecDetailsKey) - if stmtExec != nil { - detail := stmtExec.(*util.ExecDetails) - atomic.AddInt64(&detail.WaitKVRespDuration, int64(time.Since(start))) - } - c.updateTiKVSendReqHistogram(req, start) - }() - - if atomic.CompareAndSwapUint32(&c.idleNotify, 1, 0) { - c.recycleMu.Lock() - c.recycleIdleConnArray() - c.recycleMu.Unlock() - } - - // TiDB will not send batch commands to TiFlash, to resolve the conflict with Batch Cop Request. - enableBatch := req.StoreTp != tikvrpc.TiDB && req.StoreTp != tikvrpc.TiFlash - c.recycleMu.RLock() - defer c.recycleMu.RUnlock() - connArray, err := c.getConnArray(addr, enableBatch) - if err != nil { - return nil, errors.Trace(err) - } - - // TiDB RPC server supports batch RPC, but batch connection will send heart beat, It's not necessary since - // request to TiDB is not high frequency. - if config.GetGlobalConfig().TiKVClient.MaxBatchSize > 0 && enableBatch { - if batchReq := req.ToBatchCommandsRequest(); batchReq != nil { - defer trace.StartRegion(ctx, req.Type.String()).End() - return sendBatchRequest(ctx, addr, req.ForwardedHost, connArray.batchConn, batchReq, timeout) - } - } - - clientConn := connArray.Get() - if state := clientConn.GetState(); state == connectivity.TransientFailure { - storeID := strconv.FormatUint(req.Context.GetPeer().GetStoreId(), 10) - metrics.TiKVGRPCConnTransientFailureCounter.WithLabelValues(addr, storeID).Inc() - } - - if req.IsDebugReq() { - client := debugpb.NewDebugClient(clientConn) - ctx1, cancel := context.WithTimeout(ctx, timeout) - defer cancel() - return tikvrpc.CallDebugRPC(ctx1, client, req) - } - - client := tikvpb.NewTikvClient(clientConn) - - // Set metadata for request forwarding. Needn't forward DebugReq. - if req.ForwardedHost != "" { - ctx = metadata.AppendToOutgoingContext(ctx, forwardMetadataKey, req.ForwardedHost) - } - switch req.Type { - case tikvrpc.CmdBatchCop: - return c.getBatchCopStreamResponse(ctx, client, req, timeout, connArray) - case tikvrpc.CmdCopStream: - return c.getCopStreamResponse(ctx, client, req, timeout, connArray) - case tikvrpc.CmdMPPConn: - return c.getMPPStreamResponse(ctx, client, req, timeout, connArray) - } - // Or else it's a unary call. - ctx1, cancel := context.WithTimeout(ctx, timeout) - defer cancel() - return tikvrpc.CallRPC(ctx1, client, req) -} - -func (c *RPCClient) getCopStreamResponse(ctx context.Context, client tikvpb.TikvClient, req *tikvrpc.Request, timeout time.Duration, connArray *connArray) (*tikvrpc.Response, error) { - // Coprocessor streaming request. - // Use context to support timeout for grpc streaming client. - ctx1, cancel := context.WithCancel(ctx) - // Should NOT call defer cancel() here because it will cancel further stream.Recv() - // We put it in copStream.Lease.Cancel call this cancel at copStream.Close - // TODO: add unit test for SendRequest. - resp, err := tikvrpc.CallRPC(ctx1, client, req) - if err != nil { - cancel() - return nil, errors.Trace(err) - } - - // Put the lease object to the timeout channel, so it would be checked periodically. - copStream := resp.Resp.(*tikvrpc.CopStreamResponse) - copStream.Timeout = timeout - copStream.Lease.Cancel = cancel - connArray.streamTimeout <- &copStream.Lease - - // Read the first streaming response to get CopStreamResponse. - // This can make error handling much easier, because SendReq() retry on - // region error automatically. - var first *coprocessor.Response - first, err = copStream.Recv() - if err != nil { - if errors.Cause(err) != io.EOF { - return nil, errors.Trace(err) - } - logutil.BgLogger().Debug("copstream returns nothing for the request.") - } - copStream.Response = first - return resp, nil - + return client.NewTestRPCClient(security) } -func (c *RPCClient) getBatchCopStreamResponse(ctx context.Context, client tikvpb.TikvClient, req *tikvrpc.Request, timeout time.Duration, connArray *connArray) (*tikvrpc.Response, error) { - // Coprocessor streaming request. - // Use context to support timeout for grpc streaming client. - ctx1, cancel := context.WithCancel(ctx) - // Should NOT call defer cancel() here because it will cancel further stream.Recv() - // We put it in copStream.Lease.Cancel call this cancel at copStream.Close - // TODO: add unit test for SendRequest. - resp, err := tikvrpc.CallRPC(ctx1, client, req) - if err != nil { - cancel() - return nil, errors.Trace(err) - } - - // Put the lease object to the timeout channel, so it would be checked periodically. - copStream := resp.Resp.(*tikvrpc.BatchCopStreamResponse) - copStream.Timeout = timeout - copStream.Lease.Cancel = cancel - connArray.streamTimeout <- &copStream.Lease - - // Read the first streaming response to get CopStreamResponse. - // This can make error handling much easier, because SendReq() retry on - // region error automatically. - var first *coprocessor.BatchResponse - first, err = copStream.Recv() - if err != nil { - if errors.Cause(err) != io.EOF { - return nil, errors.Trace(err) - } - logutil.BgLogger().Debug("batch copstream returns nothing for the request.") - } - copStream.BatchResponse = first - return resp, nil -} - -func (c *RPCClient) getMPPStreamResponse(ctx context.Context, client tikvpb.TikvClient, req *tikvrpc.Request, timeout time.Duration, connArray *connArray) (*tikvrpc.Response, error) { - // MPP streaming request. - // Use context to support timeout for grpc streaming client. - ctx1, cancel := context.WithCancel(ctx) - // Should NOT call defer cancel() here because it will cancel further stream.Recv() - // We put it in copStream.Lease.Cancel call this cancel at copStream.Close - // TODO: add unit test for SendRequest. - resp, err := tikvrpc.CallRPC(ctx1, client, req) - if err != nil { - cancel() - return nil, errors.Trace(err) - } - - // Put the lease object to the timeout channel, so it would be checked periodically. - copStream := resp.Resp.(*tikvrpc.MPPStreamResponse) - copStream.Timeout = timeout - copStream.Lease.Cancel = cancel - connArray.streamTimeout <- &copStream.Lease - - // Read the first streaming response to get CopStreamResponse. - // This can make error handling much easier, because SendReq() retry on - // region error automatically. - var first *mpp.MPPDataPacket - first, err = copStream.Recv() - if err != nil { - if errors.Cause(err) != io.EOF { - return nil, errors.Trace(err) - } - } - copStream.MPPDataPacket = first - return resp, nil -} - -// Close closes all connections. -func (c *RPCClient) Close() error { - // TODO: add a unit test for SendRequest After Closed - c.closeConns() - return nil +// NewRPCClient creates a client that manages connections and rpc calls with tikv-servers. +func NewRPCClient(security config.Security, opts ...func(c *client.RPCClient)) *client.RPCClient { + return client.NewRPCClient(security, opts...) } diff --git a/store/tikv/client/client.go b/store/tikv/client/client.go new file mode 100644 index 0000000000000..b2652aeabb977 --- /dev/null +++ b/store/tikv/client/client.go @@ -0,0 +1,515 @@ +// Copyright 2016 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package client provides tcp connection to kvserver. +package client + +import ( + "context" + "fmt" + "io" + "math" + "runtime/trace" + "strconv" + "sync" + "sync/atomic" + "time" + + grpc_opentracing "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing" + "github.com/opentracing/opentracing-go" + "github.com/pingcap/errors" + "github.com/pingcap/kvproto/pkg/coprocessor" + "github.com/pingcap/kvproto/pkg/debugpb" + "github.com/pingcap/kvproto/pkg/mpp" + "github.com/pingcap/kvproto/pkg/tikvpb" + "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/store/tikv/config" + "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/tikvrpc" + "github.com/pingcap/tidb/store/tikv/util" + "github.com/prometheus/client_golang/prometheus" + "google.golang.org/grpc" + "google.golang.org/grpc/backoff" + "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/credentials" + "google.golang.org/grpc/encoding/gzip" + "google.golang.org/grpc/keepalive" + "google.golang.org/grpc/metadata" +) + +// MaxRecvMsgSize set max gRPC receive message size received from server. If any message size is larger than +// current value, an error will be reported from gRPC. +var MaxRecvMsgSize = math.MaxInt64 + +// Timeout durations. +const ( + dialTimeout = 5 * time.Second + ReadTimeoutShort = 20 * time.Second // For requests that read/write several key-values. + ReadTimeoutMedium = 60 * time.Second // For requests that may need scan region. +) + +// Grpc window size +const ( + GrpcInitialWindowSize = 1 << 30 + GrpcInitialConnWindowSize = 1 << 30 +) + +// forwardMetadataKey is the key of gRPC metadata which represents a forwarded request. +const forwardMetadataKey = "tikv-forwarded-host" + +// Client is a client that sends RPC. +// It should not be used after calling Close(). +type Client interface { + // Close should release all data. + Close() error + // SendRequest sends Request. + SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) +} + +type connArray struct { + // The target host. + target string + + index uint32 + v []*grpc.ClientConn + // streamTimeout binds with a background goroutine to process coprocessor streaming timeout. + streamTimeout chan *tikvrpc.Lease + dialTimeout time.Duration + // batchConn is not null when batch is enabled. + *batchConn + done chan struct{} +} + +func newConnArray(maxSize uint, addr string, security config.Security, idleNotify *uint32, enableBatch bool, dialTimeout time.Duration) (*connArray, error) { + a := &connArray{ + index: 0, + v: make([]*grpc.ClientConn, maxSize), + streamTimeout: make(chan *tikvrpc.Lease, 1024), + done: make(chan struct{}), + dialTimeout: dialTimeout, + } + if err := a.Init(addr, security, idleNotify, enableBatch); err != nil { + return nil, err + } + return a, nil +} + +func (a *connArray) Init(addr string, security config.Security, idleNotify *uint32, enableBatch bool) error { + a.target = addr + + opt := grpc.WithInsecure() + if len(security.ClusterSSLCA) != 0 { + tlsConfig, err := security.ToTLSConfig() + if err != nil { + return errors.Trace(err) + } + opt = grpc.WithTransportCredentials(credentials.NewTLS(tlsConfig)) + } + + cfg := config.GetGlobalConfig() + var ( + unaryInterceptor grpc.UnaryClientInterceptor + streamInterceptor grpc.StreamClientInterceptor + ) + if cfg.OpenTracingEnable { + unaryInterceptor = grpc_opentracing.UnaryClientInterceptor() + streamInterceptor = grpc_opentracing.StreamClientInterceptor() + } + + allowBatch := (cfg.TiKVClient.MaxBatchSize > 0) && enableBatch + if allowBatch { + a.batchConn = newBatchConn(uint(len(a.v)), cfg.TiKVClient.MaxBatchSize, idleNotify) + a.pendingRequests = metrics.TiKVBatchPendingRequests.WithLabelValues(a.target) + a.batchSize = metrics.TiKVBatchRequests.WithLabelValues(a.target) + } + keepAlive := cfg.TiKVClient.GrpcKeepAliveTime + keepAliveTimeout := cfg.TiKVClient.GrpcKeepAliveTimeout + for i := range a.v { + ctx, cancel := context.WithTimeout(context.Background(), a.dialTimeout) + var callOptions []grpc.CallOption + callOptions = append(callOptions, grpc.MaxCallRecvMsgSize(MaxRecvMsgSize)) + if cfg.TiKVClient.GrpcCompressionType == gzip.Name { + callOptions = append(callOptions, grpc.UseCompressor(gzip.Name)) + } + conn, err := grpc.DialContext( + ctx, + addr, + opt, + grpc.WithInitialWindowSize(GrpcInitialWindowSize), + grpc.WithInitialConnWindowSize(GrpcInitialConnWindowSize), + grpc.WithUnaryInterceptor(unaryInterceptor), + grpc.WithStreamInterceptor(streamInterceptor), + grpc.WithDefaultCallOptions(callOptions...), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: backoff.Config{ + BaseDelay: 100 * time.Millisecond, // Default was 1s. + Multiplier: 1.6, // Default + Jitter: 0.2, // Default + MaxDelay: 3 * time.Second, // Default was 120s. + }, + MinConnectTimeout: a.dialTimeout, + }), + grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: time.Duration(keepAlive) * time.Second, + Timeout: time.Duration(keepAliveTimeout) * time.Second, + PermitWithoutStream: true, + }), + ) + cancel() + if err != nil { + // Cleanup if the initialization fails. + a.Close() + return errors.Trace(err) + } + a.v[i] = conn + + if allowBatch { + batchClient := &batchCommandsClient{ + target: a.target, + conn: conn, + forwardedClients: make(map[string]*batchCommandsStream), + batched: sync.Map{}, + epoch: 0, + closed: 0, + tikvClientCfg: cfg.TiKVClient, + tikvLoad: &a.tikvTransportLayerLoad, + dialTimeout: a.dialTimeout, + tryLock: tryLock{sync.NewCond(new(sync.Mutex)), false}, + } + a.batchCommandsClients = append(a.batchCommandsClients, batchClient) + } + } + go tikvrpc.CheckStreamTimeoutLoop(a.streamTimeout, a.done) + if allowBatch { + go a.batchSendLoop(cfg.TiKVClient) + } + + return nil +} + +func (a *connArray) Get() *grpc.ClientConn { + next := atomic.AddUint32(&a.index, 1) % uint32(len(a.v)) + return a.v[next] +} + +func (a *connArray) Close() { + if a.batchConn != nil { + a.batchConn.Close() + } + + for i, c := range a.v { + if c != nil { + err := c.Close() + terror.Log(errors.Trace(err)) + a.v[i] = nil + } + } + + close(a.done) +} + +// RPCClient is RPC client struct. +// TODO: Add flow control between RPC clients in TiDB ond RPC servers in TiKV. +// Since we use shared client connection to communicate to the same TiKV, it's possible +// that there are too many concurrent requests which overload the service of TiKV. +type RPCClient struct { + sync.RWMutex + + conns map[string]*connArray + security config.Security + + idleNotify uint32 + // recycleMu protect the conns from being modified during a connArray is taken out and used. + // That means recycleIdleConnArray() will wait until nobody doing sendBatchRequest() + recycleMu sync.RWMutex + // Periodically check whether there is any connection that is idle and then close and remove these connections. + // Implement background cleanup. + isClosed bool + dialTimeout time.Duration +} + +// NewRPCClient creates a client that manages connections and rpc calls with tikv-servers. +func NewRPCClient(security config.Security, opts ...func(c *RPCClient)) *RPCClient { + cli := &RPCClient{ + conns: make(map[string]*connArray), + security: security, + dialTimeout: dialTimeout, + } + for _, opt := range opts { + opt(cli) + } + return cli +} + +// NewTestRPCClient is for some external tests. +func NewTestRPCClient(security config.Security) Client { + return NewRPCClient(security) +} + +func (c *RPCClient) getConnArray(addr string, enableBatch bool, opt ...func(cfg *config.TiKVClient)) (*connArray, error) { + c.RLock() + if c.isClosed { + c.RUnlock() + return nil, errors.Errorf("rpcClient is closed") + } + array, ok := c.conns[addr] + c.RUnlock() + if !ok { + var err error + array, err = c.createConnArray(addr, enableBatch, opt...) + if err != nil { + return nil, err + } + } + return array, nil +} + +func (c *RPCClient) createConnArray(addr string, enableBatch bool, opts ...func(cfg *config.TiKVClient)) (*connArray, error) { + c.Lock() + defer c.Unlock() + array, ok := c.conns[addr] + if !ok { + var err error + client := config.GetGlobalConfig().TiKVClient + for _, opt := range opts { + opt(&client) + } + array, err = newConnArray(client.GrpcConnectionCount, addr, c.security, &c.idleNotify, enableBatch, c.dialTimeout) + if err != nil { + return nil, err + } + c.conns[addr] = array + } + return array, nil +} + +func (c *RPCClient) closeConns() { + c.Lock() + if !c.isClosed { + c.isClosed = true + // close all connections + for _, array := range c.conns { + array.Close() + } + } + c.Unlock() +} + +var sendReqHistCache sync.Map + +type sendReqHistCacheKey struct { + tp tikvrpc.CmdType + id uint64 +} + +func (c *RPCClient) updateTiKVSendReqHistogram(req *tikvrpc.Request, start time.Time) { + key := sendReqHistCacheKey{ + req.Type, + req.Context.GetPeer().GetStoreId(), + } + + v, ok := sendReqHistCache.Load(key) + if !ok { + reqType := req.Type.String() + storeID := strconv.FormatUint(req.Context.GetPeer().GetStoreId(), 10) + v = metrics.TiKVSendReqHistogram.WithLabelValues(reqType, storeID) + sendReqHistCache.Store(key, v) + } + + v.(prometheus.Observer).Observe(time.Since(start).Seconds()) +} + +// SendRequest sends a Request to server and receives Response. +func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { + if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { + span1 := span.Tracer().StartSpan(fmt.Sprintf("rpcClient.SendRequest, region ID: %d, type: %s", req.RegionId, req.Type), opentracing.ChildOf(span.Context())) + defer span1.Finish() + ctx = opentracing.ContextWithSpan(ctx, span1) + } + + start := time.Now() + defer func() { + stmtExec := ctx.Value(util.ExecDetailsKey) + if stmtExec != nil { + detail := stmtExec.(*util.ExecDetails) + atomic.AddInt64(&detail.WaitKVRespDuration, int64(time.Since(start))) + } + c.updateTiKVSendReqHistogram(req, start) + }() + + if atomic.CompareAndSwapUint32(&c.idleNotify, 1, 0) { + c.recycleMu.Lock() + c.recycleIdleConnArray() + c.recycleMu.Unlock() + } + + // TiDB will not send batch commands to TiFlash, to resolve the conflict with Batch Cop Request. + enableBatch := req.StoreTp != tikvrpc.TiDB && req.StoreTp != tikvrpc.TiFlash + c.recycleMu.RLock() + defer c.recycleMu.RUnlock() + connArray, err := c.getConnArray(addr, enableBatch) + if err != nil { + return nil, errors.Trace(err) + } + + // TiDB RPC server supports batch RPC, but batch connection will send heart beat, It's not necessary since + // request to TiDB is not high frequency. + if config.GetGlobalConfig().TiKVClient.MaxBatchSize > 0 && enableBatch { + if batchReq := req.ToBatchCommandsRequest(); batchReq != nil { + defer trace.StartRegion(ctx, req.Type.String()).End() + return sendBatchRequest(ctx, addr, req.ForwardedHost, connArray.batchConn, batchReq, timeout) + } + } + + clientConn := connArray.Get() + if state := clientConn.GetState(); state == connectivity.TransientFailure { + storeID := strconv.FormatUint(req.Context.GetPeer().GetStoreId(), 10) + metrics.TiKVGRPCConnTransientFailureCounter.WithLabelValues(addr, storeID).Inc() + } + + if req.IsDebugReq() { + client := debugpb.NewDebugClient(clientConn) + ctx1, cancel := context.WithTimeout(ctx, timeout) + defer cancel() + return tikvrpc.CallDebugRPC(ctx1, client, req) + } + + client := tikvpb.NewTikvClient(clientConn) + + // Set metadata for request forwarding. Needn't forward DebugReq. + if req.ForwardedHost != "" { + ctx = metadata.AppendToOutgoingContext(ctx, forwardMetadataKey, req.ForwardedHost) + } + switch req.Type { + case tikvrpc.CmdBatchCop: + return c.getBatchCopStreamResponse(ctx, client, req, timeout, connArray) + case tikvrpc.CmdCopStream: + return c.getCopStreamResponse(ctx, client, req, timeout, connArray) + case tikvrpc.CmdMPPConn: + return c.getMPPStreamResponse(ctx, client, req, timeout, connArray) + } + // Or else it's a unary call. + ctx1, cancel := context.WithTimeout(ctx, timeout) + defer cancel() + return tikvrpc.CallRPC(ctx1, client, req) +} + +func (c *RPCClient) getCopStreamResponse(ctx context.Context, client tikvpb.TikvClient, req *tikvrpc.Request, timeout time.Duration, connArray *connArray) (*tikvrpc.Response, error) { + // Coprocessor streaming request. + // Use context to support timeout for grpc streaming client. + ctx1, cancel := context.WithCancel(ctx) + // Should NOT call defer cancel() here because it will cancel further stream.Recv() + // We put it in copStream.Lease.Cancel call this cancel at copStream.Close + // TODO: add unit test for SendRequest. + resp, err := tikvrpc.CallRPC(ctx1, client, req) + if err != nil { + cancel() + return nil, errors.Trace(err) + } + + // Put the lease object to the timeout channel, so it would be checked periodically. + copStream := resp.Resp.(*tikvrpc.CopStreamResponse) + copStream.Timeout = timeout + copStream.Lease.Cancel = cancel + connArray.streamTimeout <- &copStream.Lease + + // Read the first streaming response to get CopStreamResponse. + // This can make error handling much easier, because SendReq() retry on + // region error automatically. + var first *coprocessor.Response + first, err = copStream.Recv() + if err != nil { + if errors.Cause(err) != io.EOF { + return nil, errors.Trace(err) + } + logutil.BgLogger().Debug("copstream returns nothing for the request.") + } + copStream.Response = first + return resp, nil + +} + +func (c *RPCClient) getBatchCopStreamResponse(ctx context.Context, client tikvpb.TikvClient, req *tikvrpc.Request, timeout time.Duration, connArray *connArray) (*tikvrpc.Response, error) { + // Coprocessor streaming request. + // Use context to support timeout for grpc streaming client. + ctx1, cancel := context.WithCancel(ctx) + // Should NOT call defer cancel() here because it will cancel further stream.Recv() + // We put it in copStream.Lease.Cancel call this cancel at copStream.Close + // TODO: add unit test for SendRequest. + resp, err := tikvrpc.CallRPC(ctx1, client, req) + if err != nil { + cancel() + return nil, errors.Trace(err) + } + + // Put the lease object to the timeout channel, so it would be checked periodically. + copStream := resp.Resp.(*tikvrpc.BatchCopStreamResponse) + copStream.Timeout = timeout + copStream.Lease.Cancel = cancel + connArray.streamTimeout <- &copStream.Lease + + // Read the first streaming response to get CopStreamResponse. + // This can make error handling much easier, because SendReq() retry on + // region error automatically. + var first *coprocessor.BatchResponse + first, err = copStream.Recv() + if err != nil { + if errors.Cause(err) != io.EOF { + return nil, errors.Trace(err) + } + logutil.BgLogger().Debug("batch copstream returns nothing for the request.") + } + copStream.BatchResponse = first + return resp, nil +} + +func (c *RPCClient) getMPPStreamResponse(ctx context.Context, client tikvpb.TikvClient, req *tikvrpc.Request, timeout time.Duration, connArray *connArray) (*tikvrpc.Response, error) { + // MPP streaming request. + // Use context to support timeout for grpc streaming client. + ctx1, cancel := context.WithCancel(ctx) + // Should NOT call defer cancel() here because it will cancel further stream.Recv() + // We put it in copStream.Lease.Cancel call this cancel at copStream.Close + // TODO: add unit test for SendRequest. + resp, err := tikvrpc.CallRPC(ctx1, client, req) + if err != nil { + cancel() + return nil, errors.Trace(err) + } + + // Put the lease object to the timeout channel, so it would be checked periodically. + copStream := resp.Resp.(*tikvrpc.MPPStreamResponse) + copStream.Timeout = timeout + copStream.Lease.Cancel = cancel + connArray.streamTimeout <- &copStream.Lease + + // Read the first streaming response to get CopStreamResponse. + // This can make error handling much easier, because SendReq() retry on + // region error automatically. + var first *mpp.MPPDataPacket + first, err = copStream.Recv() + if err != nil { + if errors.Cause(err) != io.EOF { + return nil, errors.Trace(err) + } + } + copStream.MPPDataPacket = first + return resp, nil +} + +// Close closes all connections. +func (c *RPCClient) Close() error { + // TODO: add a unit test for SendRequest After Closed + c.closeConns() + return nil +} diff --git a/store/tikv/client_batch.go b/store/tikv/client/client_batch.go similarity index 99% rename from store/tikv/client_batch.go rename to store/tikv/client/client_batch.go index 70f1cf27ccacc..10ebced212d6a 100644 --- a/store/tikv/client_batch.go +++ b/store/tikv/client/client_batch.go @@ -11,8 +11,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package tikv provides tcp connection to kvserver. -package tikv +// Package client provides tcp connection to kvserver. +package client import ( "context" diff --git a/store/tikv/client_collapse.go b/store/tikv/client/client_collapse.go similarity index 93% rename from store/tikv/client_collapse.go rename to store/tikv/client/client_collapse.go index e7f9cfadcf08b..159b3dd1ef50b 100644 --- a/store/tikv/client_collapse.go +++ b/store/tikv/client/client_collapse.go @@ -11,8 +11,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package tikv provides tcp connection to kvserver. -package tikv +// Package client provides tcp connection to kvserver. +package client import ( "context" @@ -32,6 +32,10 @@ type reqCollapse struct { Client } +// NewReqCollapse creates a reqCollapse. +func NewReqCollapse(client Client) Client { + return &reqCollapse{client} +} func (r reqCollapse) Close() error { if r.Client == nil { panic("client should not be nil") diff --git a/store/tikv/client_fail_test.go b/store/tikv/client/client_fail_test.go similarity index 93% rename from store/tikv/client_fail_test.go rename to store/tikv/client/client_fail_test.go index 25b7e36d2c95b..e1279886162a0 100644 --- a/store/tikv/client_fail_test.go +++ b/store/tikv/client/client_fail_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tikv +package client import ( "context" @@ -41,8 +41,8 @@ func (s testClientFailSuite) TearDownSuite(_ *C) { } func (s *testClientFailSuite) TestPanicInRecvLoop(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/panicInFailPendingRequests", `panic`), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/gotErrorInRecvLoop", `return("0")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/client/panicInFailPendingRequests", `panic`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/client/gotErrorInRecvLoop", `return("0")`), IsNil) server, port := startMockTikvService() c.Assert(port > 0, IsTrue) @@ -61,8 +61,8 @@ func (s *testClientFailSuite) TestPanicInRecvLoop(c *C) { _, err = rpcClient.SendRequest(context.Background(), addr, req, time.Second/2) c.Assert(err, NotNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/gotErrorInRecvLoop"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/panicInFailPendingRequests"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/client/gotErrorInRecvLoop"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/client/panicInFailPendingRequests"), IsNil) time.Sleep(time.Second * 2) req = tikvrpc.NewRequest(tikvrpc.CmdEmpty, &tikvpb.BatchCommandsEmptyRequest{}) @@ -115,7 +115,7 @@ func (s *testClientFailSuite) TestRecvErrorInMultipleRecvLoops(c *C) { } epoch := atomic.LoadUint64(&batchClient.epoch) - fp := "github.com/pingcap/tidb/store/tikv/gotErrorInRecvLoop" + fp := "github.com/pingcap/tidb/store/tikv/client/gotErrorInRecvLoop" // Send a request to each stream to trigger reconnection. for _, forwardedHost := range forwardedHosts { c.Assert(failpoint.Enable(fp, `1*return("0")`), IsNil) diff --git a/store/tikv/client_test.go b/store/tikv/client/client_test.go similarity index 95% rename from store/tikv/client_test.go rename to store/tikv/client/client_test.go index 3828422b02328..7b4b652f55c85 100644 --- a/store/tikv/client_test.go +++ b/store/tikv/client/client_test.go @@ -11,10 +11,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tikv +package client import ( "context" + "flag" "fmt" "sync" "sync/atomic" @@ -32,6 +33,30 @@ import ( "google.golang.org/grpc/metadata" ) +var ( + withTiKVGlobalLock sync.RWMutex + WithTiKV = flag.Bool("with-tikv", false, "run tests with TiKV cluster started. (not use the mock server)") +) + +// OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one. +type OneByOneSuite struct{} + +func (s *OneByOneSuite) SetUpSuite(c *C) { + if *WithTiKV { + withTiKVGlobalLock.Lock() + } else { + withTiKVGlobalLock.RLock() + } +} + +func (s *OneByOneSuite) TearDownSuite(c *C) { + if *WithTiKV { + withTiKVGlobalLock.Unlock() + } else { + withTiKVGlobalLock.RUnlock() + } +} + func TestT(t *testing.T) { CustomVerboseFlag = true TestingT(t) diff --git a/store/tikv/mock_tikv_service_test.go b/store/tikv/client/mock_tikv_service_test.go similarity index 99% rename from store/tikv/mock_tikv_service_test.go rename to store/tikv/client/mock_tikv_service_test.go index 5ad7b023b1e9d..b2b5682457818 100644 --- a/store/tikv/mock_tikv_service_test.go +++ b/store/tikv/client/mock_tikv_service_test.go @@ -1,4 +1,4 @@ -package tikv +package client import ( "context" diff --git a/store/tikv/commit.go b/store/tikv/commit.go index 8e876a6f11468..d5f88f4311e1c 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -19,6 +19,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/store/tikv/client" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -49,7 +50,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) - resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) + resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort) // If we fail to receive response for the request that commits primary key, it will be undetermined whether this // transaction has been successfully committed. diff --git a/store/tikv/delete_range.go b/store/tikv/delete_range.go index 6a826d718cba1..f42d5a94ccdae 100644 --- a/store/tikv/delete_range.go +++ b/store/tikv/delete_range.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/store/tikv/client" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/retry" @@ -115,7 +116,7 @@ func (t *DeleteRangeTask) sendReqOnRange(ctx context.Context, r kv.KeyRange) (Ra NotifyOnly: t.notifyOnly, }) - resp, err := t.store.SendReq(bo, req, loc.Region, ReadTimeoutMedium) + resp, err := t.store.SendReq(bo, req, loc.Region, client.ReadTimeoutMedium) if err != nil { return stat, errors.Trace(err) } diff --git a/store/tikv/kv.go b/store/tikv/kv.go index e7fe6a8309475..563f985f95083 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" @@ -128,7 +129,7 @@ func (s *KVStore) CheckVisibility(startTime uint64) error { } // NewKVStore creates a new TiKV store instance. -func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client) (*KVStore, error) { +func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, tikvclient Client) (*KVStore, error) { o, err := oracles.NewPdOracle(pdClient, time.Duration(oracleUpdateInterval)*time.Millisecond) if err != nil { return nil, errors.Trace(err) @@ -145,7 +146,7 @@ func NewKVStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Client closed: make(chan struct{}), replicaReadSeed: rand.Uint32(), } - store.clientMu.client = reqCollapse{client} + store.clientMu.client = client.NewReqCollapse(tikvclient) store.lockResolver = newLockResolver(store) go store.runSafePointChecker() @@ -413,7 +414,7 @@ func (s *KVStore) updateSafeTS(ctx context.Context) { resp, err := tikvClient.SendRequest(ctx, storeAddr, tikvrpc.NewRequest(tikvrpc.CmdStoreSafeTS, &kvrpcpb.StoreSafeTSRequest{KeyRange: &kvrpcpb.KeyRange{ StartKey: []byte(""), EndKey: []byte(""), - }}), ReadTimeoutShort) + }}), client.ReadTimeoutShort) storeIDStr := strconv.Itoa(int(storeID)) if err != nil { metrics.TiKVSafeTSUpdateCounter.WithLabelValues("fail", storeIDStr).Inc() diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index 0ed9ecb3fa471..7ff963113bb3b 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" @@ -95,7 +96,7 @@ func NewLockResolver(etcdAddrs []string, security config.Security, opts ...pd.Cl return nil, errors.Trace(err) } - s, err := NewKVStore(uuid, &CodecPDClient{pdCli}, spkv, NewRPCClient(security)) + s, err := NewKVStore(uuid, &CodecPDClient{pdCli}, spkv, client.NewRPCClient(security)) if err != nil { return nil, errors.Trace(err) } @@ -282,7 +283,7 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc Regi req := tikvrpc.NewRequest(tikvrpc.CmdResolveLock, &kvrpcpb.ResolveLockRequest{TxnInfos: listTxnInfos}) startTime = time.Now() - resp, err := lr.store.SendReq(bo, req, loc, ReadTimeoutShort) + resp, err := lr.store.SendReq(bo, req, loc, client.ReadTimeoutShort) if err != nil { return false, errors.Trace(err) } @@ -586,7 +587,7 @@ func (lr *LockResolver) getTxnStatus(bo *Backoffer, txnID uint64, primary []byte if err != nil { return status, errors.Trace(err) } - resp, err := lr.store.SendReq(bo, req, loc.Region, ReadTimeoutShort) + resp, err := lr.store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) if err != nil { return status, errors.Trace(err) } @@ -722,7 +723,7 @@ func (lr *LockResolver) checkSecondaries(bo *Backoffer, txnID uint64, curKeys [] } req := tikvrpc.NewRequest(tikvrpc.CmdCheckSecondaryLocks, checkReq) metrics.LockResolverCountWithQueryCheckSecondaryLocks.Inc() - resp, err := lr.store.SendReq(bo, req, curRegionID, ReadTimeoutShort) + resp, err := lr.store.SendReq(bo, req, curRegionID, client.ReadTimeoutShort) if err != nil { return errors.Trace(err) } @@ -852,7 +853,7 @@ func (lr *LockResolver) resolveRegionLocks(bo *Backoffer, l *Lock, region Region lreq.Keys = keys req := tikvrpc.NewRequest(tikvrpc.CmdResolveLock, lreq) - resp, err := lr.store.SendReq(bo, req, region, ReadTimeoutShort) + resp, err := lr.store.SendReq(bo, req, region, client.ReadTimeoutShort) if err != nil { return errors.Trace(err) } @@ -921,7 +922,7 @@ func (lr *LockResolver) resolveLock(bo *Backoffer, l *Lock, status TxnStatus, li lreq.Keys = [][]byte{l.Key} } req := tikvrpc.NewRequest(tikvrpc.CmdResolveLock, lreq) - resp, err := lr.store.SendReq(bo, req, loc.Region, ReadTimeoutShort) + resp, err := lr.store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) if err != nil { return errors.Trace(err) } @@ -972,7 +973,7 @@ func (lr *LockResolver) resolvePessimisticLock(bo *Backoffer, l *Lock, cleanRegi Keys: [][]byte{l.Key}, } req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticRollback, pessimisticRollbackReq) - resp, err := lr.store.SendReq(bo, req, loc.Region, ReadTimeoutShort) + resp, err := lr.store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/pessimistic.go b/store/tikv/pessimistic.go index 2da8e93dad946..37152652b1f0b 100644 --- a/store/tikv/pessimistic.go +++ b/store/tikv/pessimistic.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/store/tikv/client" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -118,7 +119,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * return &tikverr.ErrWriteConflict{WriteConflict: nil} }) startTime := time.Now() - resp, err := c.store.SendReq(bo, req, batch.region, ReadTimeoutShort) + resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort) if action.LockCtx.Stats != nil { atomic.AddInt64(&action.LockCtx.Stats.LockRPCTime, int64(time.Since(startTime))) atomic.AddInt64(&action.LockCtx.Stats.LockRPCCount, 1) @@ -220,7 +221,7 @@ func (actionPessimisticRollback) handleSingleBatch(c *twoPhaseCommitter, bo *Bac ForUpdateTs: c.forUpdateTS, Keys: batch.mutations.GetKeys(), }) - resp, err := c.store.SendReq(bo, req, batch.region, ReadTimeoutShort) + resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index 305806c931149..3dc0fd13a1e74 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" @@ -158,7 +159,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff req := c.buildPrewriteRequest(batch, txnSize) for { sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) - resp, err := sender.SendReq(bo, req, batch.region, ReadTimeoutShort) + resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort) // If we fail to receive response for async commit prewrite, it will be undetermined whether this // transaction has been successfully committed. diff --git a/store/tikv/rawkv.go b/store/tikv/rawkv.go index b96828df187fd..22c47a854f6f9 100644 --- a/store/tikv/rawkv.go +++ b/store/tikv/rawkv.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/metrics" @@ -65,7 +66,7 @@ func NewRawKVClient(pdAddrs []string, security config.Security, opts ...pd.Clien clusterID: pdCli.GetClusterID(context.TODO()), regionCache: NewRegionCache(pdCli), pdClient: pdCli, - rpcClient: NewRPCClient(security), + rpcClient: client.NewRPCClient(security), }, nil } @@ -366,7 +367,7 @@ func (c *RawKVClient) sendReq(key []byte, req *tikvrpc.Request, reverse bool) (* if err != nil { return nil, nil, errors.Trace(err) } - resp, err := sender.SendReq(bo, req, loc.Region, ReadTimeoutShort) + resp, err := sender.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) if err != nil { return nil, nil, errors.Trace(err) } @@ -446,7 +447,7 @@ func (c *RawKVClient) doBatchReq(bo *Backoffer, batch batch, cmdType tikvrpc.Cmd } sender := NewRegionRequestSender(c.regionCache, c.rpcClient) - resp, err := sender.SendReq(bo, req, batch.regionID, ReadTimeoutShort) + resp, err := sender.SendReq(bo, req, batch.regionID, client.ReadTimeoutShort) batchResp := singleBatchResp{} if err != nil { @@ -511,7 +512,7 @@ func (c *RawKVClient) sendDeleteRangeReq(startKey []byte, endKey []byte) (*tikvr EndKey: actualEndKey, }) - resp, err := sender.SendReq(bo, req, loc.Region, ReadTimeoutShort) + resp, err := sender.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) if err != nil { return nil, nil, errors.Trace(err) } @@ -616,7 +617,7 @@ func (c *RawKVClient) doBatchPut(bo *Backoffer, batch batch) error { req := tikvrpc.NewRequest(tikvrpc.CmdRawBatchPut, &kvrpcpb.RawBatchPutRequest{Pairs: kvPair}) sender := NewRegionRequestSender(c.regionCache, c.rpcClient) - resp, err := sender.SendReq(bo, req, batch.regionID, ReadTimeoutShort) + resp, err := sender.SendReq(bo, req, batch.regionID, client.ReadTimeoutShort) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 5dc27503465d4..1649e2b497cb7 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/config" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -2234,8 +2235,8 @@ func createKVHealthClient(ctx context.Context, addr string) (*grpc.ClientConn, h ctx, addr, opt, - grpc.WithInitialWindowSize(grpcInitialWindowSize), - grpc.WithInitialConnWindowSize(grpcInitialConnWindowSize), + grpc.WithInitialWindowSize(client.GrpcInitialWindowSize), + grpc.WithInitialConnWindowSize(client.GrpcInitialConnWindowSize), grpc.WithConnectParams(grpc.ConnectParams{ Backoff: backoff.Config{ BaseDelay: 100 * time.Millisecond, // Default was 1s. diff --git a/store/tikv/scan.go b/store/tikv/scan.go index 64b9a4728f551..eb94c77094472 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/tidb/store/tikv/client" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -217,7 +218,7 @@ func (s *Scanner) getData(bo *Backoffer) error { ResourceGroupTag: s.snapshot.resourceGroupTag, }) s.snapshot.mu.RUnlock() - resp, err := sender.SendReq(bo, req, loc.Region, ReadTimeoutMedium) + resp, err := sender.SendReq(bo, req, loc.Region, client.ReadTimeoutMedium) if err != nil { return errors.Trace(err) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 9828537b7cb79..743111827e85d 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/store/tikv/client" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -327,7 +328,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec if len(matchStoreLabels) > 0 { ops = append(ops, WithMatchLabels(matchStoreLabels)) } - resp, _, _, err := cli.SendReqCtx(bo, req, batch.region, ReadTimeoutMedium, tikvrpc.TiKV, "", ops...) + resp, _, _, err := cli.SendReqCtx(bo, req, batch.region, client.ReadTimeoutMedium, tikvrpc.TiKV, "", ops...) if err != nil { return errors.Trace(err) @@ -488,7 +489,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, if err != nil { return nil, errors.Trace(err) } - resp, _, _, err := cli.SendReqCtx(bo, req, loc.Region, ReadTimeoutShort, tikvrpc.TiKV, "", ops...) + resp, _, _, err := cli.SendReqCtx(bo, req, loc.Region, client.ReadTimeoutShort, tikvrpc.TiKV, "", ops...) if err != nil { return nil, errors.Trace(err) } diff --git a/store/tikv/split_region.go b/store/tikv/split_region.go index c33a89efc19be..045b4ccb9560d 100644 --- a/store/tikv/split_region.go +++ b/store/tikv/split_region.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" + "github.com/pingcap/tidb/store/tikv/client" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -124,7 +125,7 @@ func (s *KVStore) batchSendSingleRegion(bo *Backoffer, batch batch, scatter bool }) sender := NewRegionRequestSender(s.regionCache, s.GetTiKVClient()) - resp, err := sender.SendReq(bo, req, batch.regionID, ReadTimeoutShort) + resp, err := sender.SendReq(bo, req, batch.regionID, client.ReadTimeoutShort) batchResp := singleBatchResp{resp: resp} if err != nil { From 6e9d2750ae2042fb01acabb44de627ba92d7a1f4 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Tue, 25 May 2021 16:39:38 +0800 Subject: [PATCH 165/343] executor: implement set transaction read only as of transaction (#24766) --- executor/simple.go | 15 ++++++++++ executor/stale_txn_test.go | 48 ++++++++++++++++++++++++++++++++ sessionctx/variable/session.go | 3 ++ sessionctx/variable/sysvar.go | 3 ++ sessionctx/variable/tidb_vars.go | 3 ++ sessionctx/variable/varsutil.go | 17 +++++++++++ 6 files changed, 89 insertions(+) diff --git a/executor/simple.go b/executor/simple.go index 2e9f8c89ea4be..0eadfd5f85dd2 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -574,6 +574,10 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { return expression.ErrFunctionsNoopImpl.GenWithStackByArgs("READ ONLY") } if s.AsOf != nil { + // start transaction read only as of failed due to we set tx_read_ts before + if e.ctx.GetSessionVars().TxnReadTS > 0 { + return errors.New("start transaction read only as of is forbidden after set transaction read only as of") + } if err := e.ctx.NewTxnWithStartTS(ctx, e.staleTxnStartTS); err != nil { return err } @@ -584,6 +588,17 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { return nil } } + // When TxnReadTS is not 0, it indicates the transaction is staleness transaction + if e.ctx.GetSessionVars().TxnReadTS > 0 { + startTS := e.ctx.GetSessionVars().TxnReadTS + // clear TxnReadTS after we used it. + e.ctx.GetSessionVars().TxnReadTS = 0 + if err := e.ctx.NewTxnWithStartTS(ctx, startTS); err != nil { + return err + } + e.ctx.GetSessionVars().SetInTxn(true) + return nil + } // If BEGIN is the first statement in TxnCtx, we can reuse the existing transaction, without the // need to call NewTxn, which commits the existing transaction and begins a new one. diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 1f4f24a7f9f37..db4705f428d14 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testkit" ) @@ -242,3 +243,50 @@ func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) tk.MustExec("commit") } + +func (s *testStaleTxnSerialSuite) TestSetTransactionReadOnlyAsOf(c *C) { + t1, err := time.Parse(types.TimeFormat, "2016-09-21 09:53:04") + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, s.store) + testcases := []struct { + sql string + expectedTS uint64 + injectSafeTS uint64 + }{ + { + sql: `SET TRANSACTION READ ONLY as of timestamp '2021-04-21 00:42:12'`, + expectedTS: 424394603102208000, + injectSafeTS: 0, + }, + { + sql: `SET TRANSACTION READ ONLY as of timestamp tidb_bounded_staleness('2015-09-21 00:07:01', '2021-04-27 11:26:13')`, + expectedTS: oracle.GoTimeToTS(t1), + injectSafeTS: oracle.GoTimeToTS(t1), + }, + } + for _, testcase := range testcases { + if testcase.injectSafeTS > 0 { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/injectSafeTS", + fmt.Sprintf("return(%v)", testcase.injectSafeTS)), IsNil) + } + tk.MustExec(testcase.sql) + c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, testcase.expectedTS) + tk.MustExec("begin") + c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, uint64(0)) + c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.expectedTS) + tk.MustExec("commit") + tk.MustExec("begin") + c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Not(Equals), testcase.expectedTS) + + failpoint.Disable("github.com/pingcap/tidb/expression/injectSafeTS") + } + + err = tk.ExecToErr(`SET TRANSACTION READ ONLY as of timestamp tidb_bounded_staleness(invalid1, invalid2')`) + c.Assert(err, NotNil) + c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, uint64(0)) + + tk.MustExec(`SET TRANSACTION READ ONLY as of timestamp '2021-04-21 00:42:12'`) + err = tk.ExecToErr(`START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00'`) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "start transaction read only as of is forbidden after set transaction read only as of") +} diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index f2536e4edcb16..8bd2d6940306e 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -462,6 +462,9 @@ type SessionVars struct { // SnapshotTS is used for reading history data. For simplicity, SnapshotTS only supports distsql request. SnapshotTS uint64 + // TxnReadTS is used for staleness transaction, it provides next staleness transaction startTS. + TxnReadTS uint64 + // SnapshotInfoschema is used with SnapshotTS, when the schema version at snapshotTS less than current schema // version, we load an old version schema for query. SnapshotInfoschema interface{} diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index ee633e0d1afdf..31ee28de2b5a8 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -825,6 +825,9 @@ var defaultSysVars = []*SysVar{ }, GetSession: func(s *SessionVars) (string, error) { return s.TxnScope.GetVarValue(), nil }}, + {Scope: ScopeSession, Name: TiDBTxnReadTS, Value: "", SetSession: func(s *SessionVars, val string) error { + return setTxnReadTS(s, val) + }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "ENFORCE"}, SetSession: func(s *SessionVars, val string) error { s.allowMPPExecution = val return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 3fb99204ee2ea..5d7897ff9273a 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -203,6 +203,9 @@ const ( // TiDBTxnScope indicates whether using global transactions or local transactions. TiDBTxnScope = "txn_scope" + + // TiDBTxnReadTS indicates the next transaction should be staleness transaction and provide the startTS + TiDBTxnReadTS = "tx_read_ts" ) // TiDB system variable names that both in session and global scope. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index d42128e73c612..653d6e4bdd8ca 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -378,6 +378,23 @@ func setSnapshotTS(s *SessionVars, sVal string) error { return err } +func setTxnReadTS(s *SessionVars, sVal string) error { + if sVal == "" { + s.TxnReadTS = 0 + return nil + } + t, err := types.ParseTime(s.StmtCtx, sVal, mysql.TypeTimestamp, types.MaxFsp) + if err != nil { + return err + } + t1, err := t.GoTime(s.TimeZone) + if err != nil { + return err + } + s.TxnReadTS = oracle.GoTimeToTS(t1) + return err +} + // serverGlobalVariable is used to handle variables that acts in server and global scope. type serverGlobalVariable struct { sync.Mutex From ad72d38bde5166ab1c0a9b399b5fec5a0f7ec8f4 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 25 May 2021 17:05:34 +0800 Subject: [PATCH 166/343] store/tikv: move funcs split region/ranges by region cache into region_cache.go (#24861) --- ddl/backfilling.go | 2 +- store/copr/batch_coprocessor.go | 16 +++---- store/copr/coprocessor.go | 19 +++++---- store/tikv/key_ranges.go | 61 +++------------------------ store/tikv/region_cache.go | 74 +++++++++++++++++++++++++++++++++ 5 files changed, 99 insertions(+), 73 deletions(-) diff --git a/ddl/backfilling.go b/ddl/backfilling.go index ed279c68675cf..ec3b8a61f4990 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -333,7 +333,7 @@ func splitTableRanges(t table.PhysicalTable, store kv.Storage, startKey, endKey maxSleep := 10000 // ms bo := tikv.NewBackofferWithVars(context.Background(), maxSleep, nil) tikvRange := *(*tikvstore.KeyRange)(unsafe.Pointer(&kvRange)) - ranges, err := tikv.SplitRegionRanges(bo, s.GetRegionCache(), []tikvstore.KeyRange{tikvRange}) + ranges, err := s.GetRegionCache().SplitRegionRanges(bo, []tikvstore.KeyRange{tikvRange}) if err != nil { return nil, errors.Trace(err) } diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 1e6a1dddab2bc..18106649fb94d 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -251,21 +251,21 @@ func buildBatchCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.Key const cmdType = tikvrpc.CmdBatchCop rangesLen := ranges.Len() for { + + locations, err := cache.SplitKeyRangesByLocations(bo.TiKVBackoffer(), ranges) + if err != nil { + return nil, errors.Trace(err) + } var tasks []*copTask - appendTask := func(regionWithRangeInfo *tikv.KeyLocation, ranges *tikv.KeyRanges) { + for _, lo := range locations { tasks = append(tasks, &copTask{ - region: regionWithRangeInfo.Region, - ranges: ranges, + region: lo.Location.Region, + ranges: lo.Ranges, cmdType: cmdType, storeType: storeType, }) } - err := tikv.SplitKeyRanges(bo.TiKVBackoffer(), cache, ranges, appendTask) - if err != nil { - return nil, errors.Trace(err) - } - var batchTasks []*batchCopTask storeTaskMap := make(map[string]*batchCopTask) diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 8834824432bfd..5bdfd20537250 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -158,16 +158,22 @@ func buildCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRange } rangesLen := ranges.Len() + + locs, err := cache.SplitKeyRangesByLocations(bo.TiKVBackoffer(), ranges) + if err != nil { + return nil, errors.Trace(err) + } + var tasks []*copTask - appendTask := func(regionWithRangeInfo *tikv.KeyLocation, ranges *tikv.KeyRanges) { + for _, loc := range locs { // TiKV will return gRPC error if the message is too large. So we need to limit the length of the ranges slice // to make sure the message can be sent successfully. - rLen := ranges.Len() + rLen := loc.Ranges.Len() for i := 0; i < rLen; { nextI := mathutil.Min(i+rangesPerTask, rLen) tasks = append(tasks, &copTask{ - region: regionWithRangeInfo.Region, - ranges: ranges.Slice(i, nextI), + region: loc.Location.Region, + ranges: loc.Ranges.Slice(i, nextI), // Channel buffer is 2 for handling region split. // In a common case, two region split tasks will not be blocked. respChan: make(chan *copResponse, 2), @@ -178,11 +184,6 @@ func buildCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRange } } - err := tikv.SplitKeyRanges(bo.TiKVBackoffer(), cache, ranges, appendTask) - if err != nil { - return nil, errors.Trace(err) - } - if req.Desc { reverseTasks(tasks) } diff --git a/store/tikv/key_ranges.go b/store/tikv/key_ranges.go index e45364796fdf3..ec5e5dd92ac54 100644 --- a/store/tikv/key_ranges.go +++ b/store/tikv/key_ranges.go @@ -147,64 +147,15 @@ func (r *KeyRanges) ToPBRanges() []*coprocessor.KeyRange { func SplitRegionRanges(bo *Backoffer, cache *RegionCache, keyRanges []kv.KeyRange) ([]kv.KeyRange, error) { ranges := NewKeyRanges(keyRanges) - var ret []kv.KeyRange - appendRange := func(regionWithRangeInfo *KeyLocation, ranges *KeyRanges) { - for i := 0; i < ranges.Len(); i++ { - ret = append(ret, ranges.At(i)) - } - } - - err := SplitKeyRanges(bo, cache, ranges, appendRange) + locations, err := cache.SplitKeyRangesByLocations(bo, ranges) if err != nil { return nil, errors.Trace(err) } - return ret, nil -} - -// SplitKeyRanges splits KeyRanges by the regions info from cache. -func SplitKeyRanges(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, fn func(regionWithRangeInfo *KeyLocation, ranges *KeyRanges)) error { - for ranges.Len() > 0 { - loc, err := cache.LocateKey(bo, ranges.At(0).StartKey) - if err != nil { - return errors.Trace(err) - } - - // Iterate to the first range that is not complete in the region. - var i int - for ; i < ranges.Len(); i++ { - r := ranges.At(i) - if !(loc.Contains(r.EndKey) || bytes.Equal(loc.EndKey, r.EndKey)) { - break - } - } - // All rest ranges belong to the same region. - if i == ranges.Len() { - fn(loc, ranges) - break - } - - r := ranges.At(i) - if loc.Contains(r.StartKey) { - // Part of r is not in the region. We need to split it. - taskRanges := ranges.Slice(0, i) - taskRanges.last = &kv.KeyRange{ - StartKey: r.StartKey, - EndKey: loc.EndKey, - } - fn(loc, taskRanges) - - ranges = ranges.Slice(i+1, ranges.Len()) - ranges.first = &kv.KeyRange{ - StartKey: loc.EndKey, - EndKey: r.EndKey, - } - } else { - // rs[i] is not in the region. - taskRanges := ranges.Slice(0, i) - fn(loc, taskRanges) - ranges = ranges.Slice(i, ranges.Len()) + var ret []kv.KeyRange + for _, loc := range locations { + for i := 0; i < loc.Ranges.Len(); i++ { + ret = append(ret, loc.Ranges.At(i)) } } - - return nil + return ret, nil } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 1649e2b497cb7..2b0ddd99f6454 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -701,6 +701,80 @@ func (c *RegionCache) LocateEndKey(bo *Backoffer, key []byte) (*KeyLocation, err }, nil } +// SplitRegionRanges gets the split ranges from pd region. +func (c *RegionCache) SplitRegionRanges(bo *Backoffer, keyRanges []kv.KeyRange) ([]kv.KeyRange, error) { + ranges := NewKeyRanges(keyRanges) + + locations, err := c.SplitKeyRangesByLocations(bo, ranges) + if err != nil { + return nil, errors.Trace(err) + } + var ret []kv.KeyRange + for _, loc := range locations { + for i := 0; i < loc.Ranges.Len(); i++ { + ret = append(ret, loc.Ranges.At(i)) + } + } + return ret, nil +} + +// LocationKeyRanges wrapps a real Location in PD and its logical ranges info. +type LocationKeyRanges struct { + // Location is the real location in PD. + Location *KeyLocation + // Ranges is the logic ranges the current Location contains. + Ranges *KeyRanges +} + +// SplitKeyRangesByLocations splits the KeyRanges by logical info in the cache. +func (c *RegionCache) SplitKeyRangesByLocations(bo *Backoffer, ranges *KeyRanges) ([]*LocationKeyRanges, error) { + res := make([]*LocationKeyRanges, 0) + for ranges.Len() > 0 { + loc, err := c.LocateKey(bo, ranges.At(0).StartKey) + if err != nil { + return res, errors.Trace(err) + } + + // Iterate to the first range that is not complete in the region. + var i int + for ; i < ranges.Len(); i++ { + r := ranges.At(i) + if !(loc.Contains(r.EndKey) || bytes.Equal(loc.EndKey, r.EndKey)) { + break + } + } + // All rest ranges belong to the same region. + if i == ranges.Len() { + res = append(res, &LocationKeyRanges{Location: loc, Ranges: ranges}) + break + } + + r := ranges.At(i) + if loc.Contains(r.StartKey) { + // Part of r is not in the region. We need to split it. + taskRanges := ranges.Slice(0, i) + taskRanges.last = &kv.KeyRange{ + StartKey: r.StartKey, + EndKey: loc.EndKey, + } + res = append(res, &LocationKeyRanges{Location: loc, Ranges: taskRanges}) + + ranges = ranges.Slice(i+1, ranges.Len()) + ranges.first = &kv.KeyRange{ + StartKey: loc.EndKey, + EndKey: r.EndKey, + } + } else { + // rs[i] is not in the region. + taskRanges := ranges.Slice(0, i) + res = append(res, &LocationKeyRanges{Location: loc, Ranges: taskRanges}) + ranges = ranges.Slice(i, ranges.Len()) + } + } + + return res, nil +} + func (c *RegionCache) findRegionByKey(bo *Backoffer, key []byte, isEndKey bool) (r *Region, err error) { r = c.searchCachedRegion(key, isEndKey) if r == nil { From 0199b402e453028f814f87c33a4d727061891973 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 25 May 2021 17:23:34 +0800 Subject: [PATCH 167/343] server/http_handler: use func in helper instead of use tikvrpc directly (#24823) --- server/http_handler.go | 73 ++----------------------------------- server/http_handler_test.go | 14 +++---- store/helper/helper.go | 72 ++++++++++++++++++++++++++++++++++++ 3 files changed, 83 insertions(+), 76 deletions(-) diff --git a/server/http_handler.go b/server/http_handler.go index 093a57e45c1ab..4aaab275430c4 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -51,7 +51,6 @@ import ( "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" @@ -198,71 +197,7 @@ func (t *tikvHandlerTool) getHandle(tb table.PhysicalTable, params map[string]st return handle, nil } -func (t *tikvHandlerTool) getMvccByStartTs(startTS uint64, startKey, endKey kv.Key) (*mvccKV, error) { - bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil) - for { - curRegion, err := t.RegionCache.LocateKey(bo, startKey) - if err != nil { - logutil.BgLogger().Error("get MVCC by startTS failed", zap.Uint64("txnStartTS", startTS), - zap.Stringer("startKey", startKey), zap.Error(err)) - return nil, errors.Trace(err) - } - - tikvReq := tikvrpc.NewRequest(tikvrpc.CmdMvccGetByStartTs, &kvrpcpb.MvccGetByStartTsRequest{ - StartTs: startTS, - }) - tikvReq.Context.Priority = kvrpcpb.CommandPri_Low - kvResp, err := t.Store.SendReq(bo, tikvReq, curRegion.Region, time.Hour) - if err != nil { - logutil.BgLogger().Error("get MVCC by startTS failed", - zap.Uint64("txnStartTS", startTS), - zap.Stringer("startKey", startKey), - zap.Reflect("region", curRegion.Region), - zap.Stringer("curRegion", curRegion), - zap.Reflect("kvResp", kvResp), - zap.Error(err)) - return nil, errors.Trace(err) - } - data := kvResp.Resp.(*kvrpcpb.MvccGetByStartTsResponse) - if err := data.GetRegionError(); err != nil { - logutil.BgLogger().Warn("get MVCC by startTS failed", - zap.Uint64("txnStartTS", startTS), - zap.Stringer("startKey", startKey), - zap.Reflect("region", curRegion.Region), - zap.Stringer("curRegion", curRegion), - zap.Reflect("kvResp", kvResp), - zap.Stringer("error", err)) - continue - } - - if len(data.GetError()) > 0 { - logutil.BgLogger().Error("get MVCC by startTS failed", - zap.Uint64("txnStartTS", startTS), - zap.Stringer("startKey", startKey), - zap.Reflect("region", curRegion.Region), - zap.Stringer("curRegion", curRegion), - zap.Reflect("kvResp", kvResp), - zap.String("error", data.GetError())) - return nil, errors.New(data.GetError()) - } - - key := data.GetKey() - if len(key) > 0 { - resp := &kvrpcpb.MvccGetByKeyResponse{Info: data.Info, RegionError: data.RegionError, Error: data.Error} - return &mvccKV{Key: strings.ToUpper(hex.EncodeToString(key)), Value: resp, RegionID: curRegion.Region.GetID()}, nil - } - - if len(endKey) > 0 && curRegion.Contains(endKey) { - return nil, nil - } - if len(curRegion.EndKey) == 0 { - return nil, nil - } - startKey = kv.Key(curRegion.EndKey) - } -} - -func (t *tikvHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, idxCols []*model.ColumnInfo, handle kv.Handle) (*mvccKV, error) { +func (t *tikvHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, idxCols []*model.ColumnInfo, handle kv.Handle) (*helper.MvccKV, error) { sc := new(stmtctx.StatementContext) // HTTP request is not a database session, set timezone to UTC directly here. // See https://github.com/pingcap/tidb/blob/master/docs/tidb_http_api.md for more details. @@ -283,7 +218,7 @@ func (t *tikvHandlerTool) getMvccByIdxValue(idx table.Index, values url.Values, if err != nil { return nil, err } - return &mvccKV{strings.ToUpper(hex.EncodeToString(encodedKey)), regionID, data}, err + return &helper.MvccKV{Key: strings.ToUpper(hex.EncodeToString(encodedKey)), RegionID: regionID, Value: data}, err } // formValue2DatumRow converts URL query string to a Datum Row. @@ -1646,7 +1581,7 @@ func (h mvccTxnHandler) handleMvccGetByKey(params map[string]string, values url. if err != nil { return nil, err } - resp := &mvccKV{Key: strings.ToUpper(hex.EncodeToString(encodedKey)), Value: data, RegionID: regionID} + resp := &helper.MvccKV{Key: strings.ToUpper(hex.EncodeToString(encodedKey)), Value: data, RegionID: regionID} if len(values.Get("decode")) == 0 { return resp, nil } @@ -1713,7 +1648,7 @@ func (h *mvccTxnHandler) handleMvccGetByTxn(params map[string]string) (interface } startKey := tablecodec.EncodeTablePrefix(tableID) endKey := tablecodec.EncodeRowKeyWithHandle(tableID, kv.IntHandle(math.MaxInt64)) - return h.getMvccByStartTs(uint64(startTS), startKey, endKey) + return h.GetMvccByStartTs(uint64(startTS), startKey, endKey) } // serverInfo is used to report the servers info when do http request. diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 40ec398f8a650..208714ac082e1 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -559,7 +559,7 @@ partition by range (a) func decodeKeyMvcc(closer io.ReadCloser, c *C, valid bool) { decoder := json.NewDecoder(closer) - var data mvccKV + var data helper.MvccKV err := decoder.Decode(&data) c.Assert(err, IsNil) if valid { @@ -580,7 +580,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { resp, err := ts.fetchStatus("/mvcc/key/tidb/test/1") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) - var data mvccKV + var data helper.MvccKV err = decoder.Decode(&data) c.Assert(err, IsNil) c.Assert(data.Value, NotNil) @@ -601,7 +601,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { resp, err = ts.fetchStatus(fmt.Sprintf("/mvcc/txn/%d/tidb/test", startTs)) c.Assert(err, IsNil) - var p2 mvccKV + var p2 helper.MvccKV decoder = json.NewDecoder(resp.Body) err = decoder.Decode(&p2) c.Assert(err, IsNil) @@ -615,7 +615,7 @@ func (ts *HTTPHandlerTestSuite) TestGetTableMVCC(c *C) { resp, err = ts.fetchStatus("/mvcc/hex/" + hexKey) c.Assert(err, IsNil) decoder = json.NewDecoder(resp.Body) - var data2 mvccKV + var data2 helper.MvccKV err = decoder.Decode(&data2) c.Assert(err, IsNil) c.Assert(data2, DeepEquals, data) @@ -669,7 +669,7 @@ func (ts *HTTPHandlerTestSuite) TestGetMVCCNotFound(c *C) { resp, err := ts.fetchStatus("/mvcc/key/tidb/test/1234") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) - var data mvccKV + var data helper.MvccKV err = decoder.Decode(&data) c.Assert(err, IsNil) c.Assert(data.Value.Info.Lock, IsNil) @@ -974,14 +974,14 @@ func (ts *HTTPHandlerTestSuite) TestGetIndexMVCC(c *C) { resp, err = ts.fetchStatus("/mvcc/index/tidb/test/idx1/1?a=1") c.Assert(err, IsNil) decoder := json.NewDecoder(resp.Body) - var data1 mvccKV + var data1 helper.MvccKV err = decoder.Decode(&data1) c.Assert(err, NotNil) resp, err = ts.fetchStatus("/mvcc/index/tidb/test/idx2/1?a=1") c.Assert(err, IsNil) decoder = json.NewDecoder(resp.Body) - var data2 mvccKV + var data2 helper.MvccKV err = decoder.Decode(&data2) c.Assert(err, NotNil) diff --git a/store/helper/helper.go b/store/helper/helper.go index 8eb9b9d7db828..52100e472cbb2 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -109,6 +109,78 @@ func (h *Helper) GetMvccByEncodedKey(encodedKey kv.Key) (*kvrpcpb.MvccGetByKeyRe return kvResp.Resp.(*kvrpcpb.MvccGetByKeyResponse), nil } +// MvccKV wraps the key's mvcc info in tikv. +type MvccKV struct { + Key string `json:"key"` + RegionID uint64 `json:"region_id"` + Value *kvrpcpb.MvccGetByKeyResponse `json:"value"` +} + +// GetMvccByStartTs gets Mvcc info by startTS from tikv. +func (h *Helper) GetMvccByStartTs(startTS uint64, startKey, endKey kv.Key) (*MvccKV, error) { + bo := tikv.NewBackofferWithVars(context.Background(), 5000, nil) + for { + curRegion, err := h.RegionCache.LocateKey(bo, startKey) + if err != nil { + logutil.BgLogger().Error("get MVCC by startTS failed", zap.Uint64("txnStartTS", startTS), + zap.Stringer("startKey", startKey), zap.Error(err)) + return nil, errors.Trace(err) + } + + tikvReq := tikvrpc.NewRequest(tikvrpc.CmdMvccGetByStartTs, &kvrpcpb.MvccGetByStartTsRequest{ + StartTs: startTS, + }) + tikvReq.Context.Priority = kvrpcpb.CommandPri_Low + kvResp, err := h.Store.SendReq(bo, tikvReq, curRegion.Region, time.Hour) + if err != nil { + logutil.BgLogger().Error("get MVCC by startTS failed", + zap.Uint64("txnStartTS", startTS), + zap.Stringer("startKey", startKey), + zap.Reflect("region", curRegion.Region), + zap.Stringer("curRegion", curRegion), + zap.Reflect("kvResp", kvResp), + zap.Error(err)) + return nil, errors.Trace(err) + } + data := kvResp.Resp.(*kvrpcpb.MvccGetByStartTsResponse) + if err := data.GetRegionError(); err != nil { + logutil.BgLogger().Warn("get MVCC by startTS failed", + zap.Uint64("txnStartTS", startTS), + zap.Stringer("startKey", startKey), + zap.Reflect("region", curRegion.Region), + zap.Stringer("curRegion", curRegion), + zap.Reflect("kvResp", kvResp), + zap.Stringer("error", err)) + continue + } + + if len(data.GetError()) > 0 { + logutil.BgLogger().Error("get MVCC by startTS failed", + zap.Uint64("txnStartTS", startTS), + zap.Stringer("startKey", startKey), + zap.Reflect("region", curRegion.Region), + zap.Stringer("curRegion", curRegion), + zap.Reflect("kvResp", kvResp), + zap.String("error", data.GetError())) + return nil, errors.New(data.GetError()) + } + + key := data.GetKey() + if len(key) > 0 { + resp := &kvrpcpb.MvccGetByKeyResponse{Info: data.Info, RegionError: data.RegionError, Error: data.Error} + return &MvccKV{Key: strings.ToUpper(hex.EncodeToString(key)), Value: resp, RegionID: curRegion.Region.GetID()}, nil + } + + if len(endKey) > 0 && curRegion.Contains(endKey) { + return nil, nil + } + if len(curRegion.EndKey) == 0 { + return nil, nil + } + startKey = kv.Key(curRegion.EndKey) + } +} + // StoreHotRegionInfos records all hog region stores. // it's the response of PD. type StoreHotRegionInfos struct { From 681d8ccd16d2e44b41098485276167e94df7ca71 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 25 May 2021 17:37:33 +0800 Subject: [PATCH 168/343] executor: update memIndexLookUp to make it can read partition table correctly and add more cases about dynamic-mode with UnionScan (#24877) --- executor/mem_reader.go | 42 +++++++++++++++--- executor/partition_table_test.go | 75 ++++++++++++++++++++++++++++++++ 2 files changed, 112 insertions(+), 5 deletions(-) diff --git a/executor/mem_reader.go b/executor/mem_reader.go index f6023c93c5b1a..4a2ab2f4b191c 100644 --- a/executor/mem_reader.go +++ b/executor/mem_reader.go @@ -379,6 +379,11 @@ type memIndexLookUpReader struct { retFieldTypes []*types.FieldType idxReader *memIndexReader + + // partition mode + partitionMode bool // if it is accessing a partition table + partitionTables []table.PhysicalTable // partition tables to access + partitionKVRanges [][]kv.KeyRange // kv ranges for these partition tables } func buildMemIndexLookUpReader(us *UnionScanExec, idxLookUpReader *IndexLookUpExecutor) *memIndexLookUpReader { @@ -404,16 +409,43 @@ func buildMemIndexLookUpReader(us *UnionScanExec, idxLookUpReader *IndexLookUpEx conditions: us.conditions, retFieldTypes: retTypes(us), idxReader: memIdxReader, + + partitionMode: idxLookUpReader.partitionTableMode, + partitionKVRanges: idxLookUpReader.partitionKVRanges, + partitionTables: idxLookUpReader.prunedPartitions, } } func (m *memIndexLookUpReader) getMemRows() ([][]types.Datum, error) { - handles, err := m.idxReader.getMemRowsHandle() - if err != nil || len(handles) == 0 { - return nil, err + kvRanges := [][]kv.KeyRange{m.idxReader.kvRanges} + tbls := []table.Table{m.table} + if m.partitionMode { + m.idxReader.desc = false // keep-order if always false for IndexLookUp reading partitions so this parameter makes no sense + kvRanges = m.partitionKVRanges + tbls = tbls[:0] + for _, p := range m.partitionTables { + tbls = append(tbls, p) + } + } + + tblKVRanges := make([]kv.KeyRange, 0, 16) + numHandles := 0 + for i, tbl := range tbls { + m.idxReader.kvRanges = kvRanges[i] + handles, err := m.idxReader.getMemRowsHandle() + if err != nil { + return nil, err + } + if len(handles) == 0 { + continue + } + numHandles += len(handles) + tblKVRanges = append(tblKVRanges, distsql.TableHandlesToKVRanges(getPhysicalTableID(tbl), handles)...) + } + if numHandles == 0 { + return nil, nil } - tblKVRanges := distsql.TableHandlesToKVRanges(getPhysicalTableID(m.table), handles) colIDs := make(map[int64]int, len(m.columns)) for i, col := range m.columns { colIDs[col.ID] = i @@ -440,7 +472,7 @@ func (m *memIndexLookUpReader) getMemRows() ([][]types.Datum, error) { columns: m.columns, kvRanges: tblKVRanges, conditions: m.conditions, - addedRows: make([][]types.Datum, 0, len(handles)), + addedRows: make([][]types.Datum, 0, numHandles), retFieldTypes: m.retFieldTypes, colIDs: colIDs, pkColIDs: pkColIDs, diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 82ec887ce2692..c06289da77988 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -1737,6 +1737,81 @@ func (s *partitionTableSuite) TestParallelApply(c *C) { } } +func (s *partitionTableSuite) TestDirectReadingWithUnionScan(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_unionscan") + defer tk.MustExec(`drop database test_unionscan`) + tk.MustExec("use test_unionscan") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table trange(a int, b int, index idx_a(a)) partition by range(a) ( + partition p0 values less than (10), + partition p1 values less than (30), + partition p2 values less than (50))`) + tk.MustExec(`create table thash(a int, b int, index idx_a(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table tnormal(a int, b int, index idx_a(a))`) + + vals := make([]string, 0, 1000) + for i := 0; i < 1000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(50), rand.Intn(50))) + } + for _, tb := range []string{`trange`, `tnormal`, `thash`} { + sql := fmt.Sprintf(`insert into %v values `+strings.Join(vals, ", "), tb) + tk.MustExec(sql) + } + + randCond := func(col string) string { + la, ra := rand.Intn(50), rand.Intn(50) + if la > ra { + la, ra = ra, la + } + return fmt.Sprintf(`%v>=%v and %v<=%v`, col, la, col, ra) + } + + tk.MustExec(`begin`) + for i := 0; i < 1000; i++ { + if i == 0 || rand.Intn(2) == 0 { // insert some inflight rows + val := fmt.Sprintf("(%v, %v)", rand.Intn(50), rand.Intn(50)) + for _, tb := range []string{`trange`, `tnormal`, `thash`} { + sql := fmt.Sprintf(`insert into %v values `+val, tb) + tk.MustExec(sql) + } + } else { + var sql string + switch rand.Intn(3) { + case 0: // table scan + sql = `select * from %v ignore index(idx_a) where ` + randCond(`b`) + case 1: // index reader + sql = `select a from %v use index(idx_a) where ` + randCond(`a`) + case 2: // index lookup + sql = `select * from %v use index(idx_a) where ` + randCond(`a`) + ` and ` + randCond(`b`) + } + switch rand.Intn(2) { + case 0: // order by a + sql += ` order by a` + case 1: // order by b + sql += ` order by b` + } + + var result [][]interface{} + for _, tb := range []string{`trange`, `tnormal`, `thash`} { + q := fmt.Sprintf(sql, tb) + tk.HasPlan(q, `UnionScan`) + if result == nil { + result = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Sort().Check(result) + } + } + } + } + tk.MustExec(`rollback`) +} + func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 051524ce95dcab92320339ce748ad91f6089d770 Mon Sep 17 00:00:00 2001 From: disksing Date: Tue, 25 May 2021 17:53:34 +0800 Subject: [PATCH 169/343] kv: remove `DelOption` method (#24762) --- executor/batch_point_get.go | 2 +- executor/insert.go | 2 +- executor/insert_common.go | 2 +- executor/point_get.go | 2 +- executor/replace.go | 2 +- executor/update.go | 2 +- kv/interface_mock_test.go | 5 ----- kv/kv.go | 4 ---- kv/mock_test.go | 1 - store/driver/txn/snapshot.go | 13 +++++-------- store/driver/txn/txn_driver.go | 13 +++++-------- store/tikv/unionstore/mock.go | 1 - 12 files changed, 16 insertions(+), 33 deletions(-) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index c34eafd54c408..cd906019d284d 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -150,7 +150,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { // Close implements the Executor interface. func (e *BatchPointGetExec) Close() error { if e.runtimeStats != nil && e.snapshot != nil { - e.snapshot.DelOption(kv.CollectRuntimeStats) + e.snapshot.SetOption(kv.CollectRuntimeStats, nil) } e.inited = 0 e.index = 0 diff --git a/executor/insert.go b/executor/insert.go index 351f04c2ca5eb..c6195ccef34c9 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -216,7 +216,7 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D if e.collectRuntimeStatsEnabled() { if snapshot := txn.GetSnapshot(); snapshot != nil { snapshot.SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) - defer snapshot.DelOption(kv.CollectRuntimeStats) + defer snapshot.SetOption(kv.CollectRuntimeStats, nil) } } prefetchStart := time.Now() diff --git a/executor/insert_common.go b/executor/insert_common.go index 258e873db89db..5782c4189522a 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -1049,7 +1049,7 @@ func (e *InsertValues) batchCheckAndInsert(ctx context.Context, rows [][]types.D if e.collectRuntimeStatsEnabled() { if snapshot := txn.GetSnapshot(); snapshot != nil { snapshot.SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) - defer snapshot.DelOption(kv.CollectRuntimeStats) + defer snapshot.SetOption(kv.CollectRuntimeStats, nil) } } prefetchStart := time.Now() diff --git a/executor/point_get.go b/executor/point_get.go index 685d378f83d87..c0ff834c0d5c3 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -166,7 +166,7 @@ func (e *PointGetExecutor) Open(context.Context) error { // Close implements the Executor interface. func (e *PointGetExecutor) Close() error { if e.runtimeStats != nil && e.snapshot != nil { - e.snapshot.DelOption(kv.CollectRuntimeStats) + e.snapshot.SetOption(kv.CollectRuntimeStats, nil) } if e.idxInfo != nil && e.tblInfo != nil { actRows := int64(0) diff --git a/executor/replace.go b/executor/replace.go index 03dc4bfad0543..83df806489524 100644 --- a/executor/replace.go +++ b/executor/replace.go @@ -221,7 +221,7 @@ func (e *ReplaceExec) exec(ctx context.Context, newRows [][]types.Datum) error { if e.collectRuntimeStatsEnabled() { if snapshot := txn.GetSnapshot(); snapshot != nil { snapshot.SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) - defer snapshot.DelOption(kv.CollectRuntimeStats) + defer snapshot.SetOption(kv.CollectRuntimeStats, nil) } } setResourceGroupTagForTxn(e.ctx.GetSessionVars().StmtCtx, txn) diff --git a/executor/update.go b/executor/update.go index 954aa43c8067c..194f8e415a17a 100644 --- a/executor/update.go +++ b/executor/update.go @@ -414,7 +414,7 @@ func (e *UpdateExec) Close() error { if e.runtimeStats != nil && e.stats != nil { txn, err := e.ctx.Txn(false) if err == nil && txn.GetSnapshot() != nil { - txn.GetSnapshot().DelOption(kv.CollectRuntimeStats) + txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, nil) } } return e.children[0].Close() diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index 9e41832678294..0c471c6fb40e0 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -49,10 +49,6 @@ func (t *mockTxn) SetOption(opt int, val interface{}) { t.opts[opt] = val } -func (t *mockTxn) DelOption(opt int) { - delete(t.opts, opt) -} - func (t *mockTxn) GetOption(opt int) interface{} { return t.opts[opt] } @@ -259,4 +255,3 @@ func (s *mockSnapshot) IterReverse(k Key) (Iterator, error) { } func (s *mockSnapshot) SetOption(opt int, val interface{}) {} -func (s *mockSnapshot) DelOption(opt int) {} diff --git a/kv/kv.go b/kv/kv.go index be612512317db..fdcc7148247af 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -159,8 +159,6 @@ type Transaction interface { SetOption(opt int, val interface{}) // GetOption returns the option GetOption(opt int) interface{} - // DelOption deletes an option. - DelOption(opt int) // IsReadOnly checks if the transaction has only performed read operations. IsReadOnly() bool // StartTS returns the transaction start timestamp. @@ -322,8 +320,6 @@ type Snapshot interface { // SetOption sets an option with a value, when val is nil, uses the default // value of this option. Only ReplicaRead is supported for snapshot SetOption(opt int, val interface{}) - // DelOption deletes an option. - DelOption(opt int) } // BatchGetter is the interface for BatchGet. diff --git a/kv/mock_test.go b/kv/mock_test.go index e09c291d5de95..5efaa146920df 100644 --- a/kv/mock_test.go +++ b/kv/mock_test.go @@ -44,7 +44,6 @@ func (s testMockSuite) TestInterface(c *C) { mock.GetOption(23) } transaction.StartTS() - transaction.DelOption(23) if transaction.IsReadOnly() { _, err = transaction.Get(context.TODO(), Key("lock")) c.Check(err, IsNil) diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 892a85e9ccebb..257feb37938af 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -83,7 +83,11 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { case kv.TaskID: s.KVSnapshot.SetTaskID(val.(uint64)) case kv.CollectRuntimeStats: - s.KVSnapshot.SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) + if val == nil { + s.KVSnapshot.SetRuntimeStats(nil) + } else { + s.KVSnapshot.SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) + } case kv.IsStalenessReadOnly: s.KVSnapshot.SetIsStatenessReadOnly(val.(bool)) case kv.MatchStoreLabels: @@ -93,13 +97,6 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { } } -func (s *tikvSnapshot) DelOption(opt int) { - switch opt { - case kv.CollectRuntimeStats: - s.KVSnapshot.SetRuntimeStats(nil) - } -} - func toTiKVKeys(keys []kv.Key) [][]byte { bytesKeys := *(*[][]byte)(unsafe.Pointer(&keys)) return bytesKeys diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index a722557f8fc4e..886b0df39900a 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -151,7 +151,11 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { case kv.InfoSchema: txn.SetSchemaVer(val.(tikv.SchemaVer)) case kv.CollectRuntimeStats: - txn.KVTxn.GetSnapshot().SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) + if val == nil { + txn.KVTxn.GetSnapshot().SetRuntimeStats(nil) + } else { + txn.KVTxn.GetSnapshot().SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats)) + } case kv.SchemaAmender: txn.SetSchemaAmender(val.(tikv.SchemaAmender)) case kv.SampleStep: @@ -186,13 +190,6 @@ func (txn *tikvTxn) GetOption(opt int) interface{} { } } -func (txn *tikvTxn) DelOption(opt int) { - switch opt { - case kv.CollectRuntimeStats: - txn.KVTxn.GetSnapshot().SetRuntimeStats(nil) - } -} - // SetVars sets variables to the transaction. func (txn *tikvTxn) SetVars(vars interface{}) { if vs, ok := vars.(*tikv.Variables); ok { diff --git a/store/tikv/unionstore/mock.go b/store/tikv/unionstore/mock.go index 8a459a2170966..e62b1a4108147 100644 --- a/store/tikv/unionstore/mock.go +++ b/store/tikv/unionstore/mock.go @@ -55,4 +55,3 @@ func (s *mockSnapshot) IterReverse(k []byte) (Iterator, error) { } func (s *mockSnapshot) SetOption(opt int, val interface{}) {} -func (s *mockSnapshot) DelOption(opt int) {} From 11e7ac65f6b9f984dd351d77fa30556882e8fd8c Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Tue, 25 May 2021 04:01:34 -0600 Subject: [PATCH 170/343] session, variable, domain: optimize session sysvar cache (#24873) --- domain/sysvar_cache.go | 4 +++- session/session.go | 21 ++++----------------- sessionctx/variable/session.go | 3 +++ 3 files changed, 10 insertions(+), 18 deletions(-) diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index 23c9688ea2f81..8df217dd3ad42 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -123,7 +123,9 @@ func (svc *SysVarCache) RebuildSysVarCache(ctx sessionctx.Context) error { if _, ok := tableContents[sv.Name]; ok { sVal = tableContents[sv.Name] } - if sv.HasSessionScope() { + // session cache stores non-skippable variables, which essentially means session scope. + // for historical purposes there are some globals, but these should eventually be removed. + if !sv.SkipInit() { newSessionCache[sv.Name] = sVal } if sv.HasGlobalScope() { diff --git a/session/session.go b/session/session.go index 797d0326c4a48..c87f57f287b7c 100644 --- a/session/session.go +++ b/session/session.go @@ -2578,34 +2578,21 @@ func (s *session) loadCommonGlobalVariablesIfNeeded() error { vars.CommonGlobalLoaded = true // Deep copy sessionvar cache - // Eventually this whole map will be applied to systems[], which is a MySQL behavior. sessionCache, err := domain.GetDomain(s).GetSysVarCache().GetSessionCache(s) if err != nil { return err } - for varName, sv := range variable.GetSysVars() { - if sv.SkipInit() { - continue - } - // The item should be in the sessionCache, but due to a strange current behavior there are some Global-only - // vars that are in builtinGlobalVariable. For compatibility we need to fall back to the Global cache on these items. - // TODO: don't load these globals into the session! - var varVal string - var ok bool - if varVal, ok = sessionCache[varName]; !ok { - varVal, err = s.GetGlobalSysVar(varName) - if err != nil { - continue // skip variables that are not loaded. - } - } + for varName, varVal := range sessionCache { if _, ok := vars.GetSystemVar(varName); !ok { err = vars.SetSystemVarWithRelaxedValidation(varName, varVal) if err != nil { + if variable.ErrUnknownSystemVar.Equal(err) { + continue // sessionCache is stale; sysvar has likely been unregistered + } return err } } } - // when client set Capability Flags CLIENT_INTERACTIVE, init wait_timeout with interactive_timeout if vars.ClientCapability&mysql.ClientInteractive > 0 { if varVal, ok := vars.GetSystemVar(variable.InteractiveTimeout); ok { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 8bd2d6940306e..bac6acf9e7ec0 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1393,6 +1393,9 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { // Errors are not expected to be returned because this could cause upgrade issues. func (s *SessionVars) SetSystemVarWithRelaxedValidation(name string, val string) error { sv := GetSysVar(name) + if sv == nil { + return ErrUnknownSystemVar.GenWithStackByArgs(name) + } val = sv.ValidateWithRelaxedValidation(s, val, ScopeSession) return sv.SetSessionFromHook(s, val) } From 045e18f1c6c188ed9048d5fef9c9739432b85c2f Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 25 May 2021 20:07:34 +0800 Subject: [PATCH 171/343] executor: do not send snapshot request for (batch) point get on temporary table (#24804) --- executor/batch_point_get.go | 14 ++++++++++++++ executor/batch_point_get_test.go | 25 +++++++++++++++++++++++++ executor/point_get.go | 4 ++++ 3 files changed, 43 insertions(+) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index cd906019d284d..5d6bb5c172751 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -131,6 +131,10 @@ func (e *BatchPointGetExec) Open(context.Context) error { }) } setResourceGroupTagForTxn(stmtCtx, snapshot) + // Avoid network requests for the temporary table. + if e.tblInfo.TempTableType == model.TempTableGlobal { + snapshot = globalTemporaryTableSnapshot{snapshot} + } var batchGetter kv.BatchGetter = snapshot if txn.Valid() { lock := e.tblInfo.Lock @@ -147,6 +151,16 @@ func (e *BatchPointGetExec) Open(context.Context) error { return nil } +// Global temporary table would always be empty, so get the snapshot data of it is meanless. +// globalTemporaryTableSnapshot inherits kv.Snapshot and override the BatchGet methods to return empty. +type globalTemporaryTableSnapshot struct { + kv.Snapshot +} + +func (s globalTemporaryTableSnapshot) BatchGet(ctx context.Context, keys []kv.Key) (map[string][]byte, error) { + return make(map[string][]byte), nil +} + // Close implements the Executor interface. func (e *BatchPointGetExec) Close() error { if e.runtimeStats != nil && e.snapshot != nil { diff --git a/executor/batch_point_get_test.go b/executor/batch_point_get_test.go index 8f8c39d4b0eed..69a60535ce1ab 100644 --- a/executor/batch_point_get_test.go +++ b/executor/batch_point_get_test.go @@ -20,6 +20,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" @@ -320,3 +321,27 @@ func (s *testBatchPointGetSuite) TestBatchPointGetLockExistKey(c *C) { c.Assert(err, IsNil) } } + +func (s *testBatchPointGetSuite) TestPointGetForTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create global temporary table t1 (id int primary key, val int) on commit delete rows") + tk.MustExec("begin") + tk.MustExec("insert into t1 values (1,1)") + tk.MustQuery("explain format = 'brief' select * from t1 where id in (1, 2, 3)"). + Check(testkit.Rows("Batch_Point_Get 3.00 root table:t1 handle:[1 2 3], keep order:false, desc:false")) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy", "return(true)"), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy"), IsNil) + }() + + // Batch point get. + tk.MustQuery("select * from t1 where id in (1, 2, 3)").Check(testkit.Rows("1 1")) + tk.MustQuery("select * from t1 where id in (2, 3)").Check(testkit.Rows()) + + // Point get. + tk.MustQuery("select * from t1 where id = 1").Check(testkit.Rows("1 1")) + tk.MustQuery("select * from t1 where id = 2").Check(testkit.Rows()) +} diff --git a/executor/point_get.go b/executor/point_get.go index c0ff834c0d5c3..78f5553069063 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -369,6 +369,10 @@ func (e *PointGetExecutor) get(ctx context.Context, key kv.Key) ([]byte, error) // fallthrough to snapshot get. } + // Global temporary table is always empty, so no need to send the request. + if e.tblInfo.TempTableType == model.TempTableGlobal { + return nil, nil + } lock := e.tblInfo.Lock if lock != nil && (lock.Tp == model.TableLockRead || lock.Tp == model.TableLockReadOnly) { if e.ctx.GetSessionVars().EnablePointGetCache { From ced56153b8c525e8e3de23887c6f18c9272ad4e9 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 25 May 2021 20:33:34 +0800 Subject: [PATCH 172/343] executor,table: fix 'show create table' for the temporary table (#24857) --- executor/show.go | 22 ++++++++++++++++++++-- executor/show_test.go | 22 ++++++++++++++++++++++ table/tables/tables.go | 7 +++++-- 3 files changed, 47 insertions(+), 4 deletions(-) diff --git a/executor/show.go b/executor/show.go index 09e3d0c71e7b4..78f2e409b27e4 100644 --- a/executor/show.go +++ b/executor/show.go @@ -775,7 +775,13 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T } sqlMode := ctx.GetSessionVars().SQLMode - fmt.Fprintf(buf, "CREATE TABLE %s (\n", stringutil.Escape(tableInfo.Name.O, sqlMode)) + tableName := stringutil.Escape(tableInfo.Name.O, sqlMode) + switch tableInfo.TempTableType { + case model.TempTableGlobal: + fmt.Fprintf(buf, "CREATE GLOBAL TEMPORARY TABLE %s (\n", tableName) + default: + fmt.Fprintf(buf, "CREATE TABLE %s (\n", tableName) + } var pkCol *model.ColumnInfo var hasAutoIncID bool needAddComma := false @@ -952,7 +958,14 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T buf.WriteString("\n") - buf.WriteString(") ENGINE=InnoDB") + switch tableInfo.TempTableType { + case model.TempTableNone: + buf.WriteString(") ENGINE=InnoDB") + default: + // For now the only supported engine for temporary table is memory. + buf.WriteString(") ENGINE=memory") + } + // We need to explicitly set the default charset and collation // to make it work on MySQL server which has default collate utf8_general_ci. if len(tblCollate) == 0 || tblCollate == "binary" { @@ -1009,6 +1022,11 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T if len(tableInfo.Comment) > 0 { fmt.Fprintf(buf, " COMMENT='%s'", format.OutputFormat(tableInfo.Comment)) } + + if tableInfo.TempTableType == model.TempTableGlobal { + fmt.Fprintf(buf, " ON COMMIT DELETE ROWS") + } + // add partition info here. appendPartitionInfo(tableInfo.Partition, buf) return nil diff --git a/executor/show_test.go b/executor/show_test.go index ea6d6734159b6..57cec97990dc4 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -1303,3 +1303,25 @@ func (s *testSuite5) TestShowPerformanceSchema(c *C) { testkit.Rows("events_statements_summary_by_digest 0 SCHEMA_NAME 1 SCHEMA_NAME A 0 YES BTREE YES NULL NO", "events_statements_summary_by_digest 0 SCHEMA_NAME 2 DIGEST A 0 YES BTREE YES NULL NO")) } + +func (s *testSuite5) TestShowTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create global temporary table t1 (id int) on commit delete rows") + tk.MustExec("create global temporary table t3 (i int primary key, j int) on commit delete rows") + // For issue https://github.com/pingcap/tidb/issues/24752 + tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE GLOBAL TEMPORARY TABLE `t1` (\n" + + " `id` int(11) DEFAULT NULL\n" + + ") ENGINE=memory DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin ON COMMIT DELETE ROWS")) + // No panic, fix issue https://github.com/pingcap/tidb/issues/24788 + expect := "CREATE GLOBAL TEMPORARY TABLE `t3` (\n" + + " `i` int(11) NOT NULL,\n" + + " `j` int(11) DEFAULT NULL,\n" + + " PRIMARY KEY (`i`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=memory DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin ON COMMIT DELETE ROWS" + tk.MustQuery("show create table t3").Check(testkit.Rows("t3 " + expect)) + + // Verify that the `show create table` result can be used to build the table. + createTable := strings.ReplaceAll(expect, "t3", "t4") + tk.MustExec(createTable) +} diff --git a/table/tables/tables.go b/table/tables/tables.go index 4fb118c36c24d..003e2aa4081a3 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1373,8 +1373,11 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { } else if ctx.GetSessionVars().IDAllocator == nil { // Use an independent allocator for global temporary tables. if t.meta.TempTableType == model.TempTableGlobal { - alloc := ctx.GetSessionVars().GetTemporaryTable(t.meta).GetAutoIDAllocator() - return autoid.Allocators{alloc} + if alloc := ctx.GetSessionVars().GetTemporaryTable(t.meta).GetAutoIDAllocator(); alloc != nil { + return autoid.Allocators{alloc} + } + // If the session is not in a txn, for example, in "show create table", use the original allocator. + // Otherwise the would be a nil pointer dereference. } return t.allocs } From eeb66410469cd729eb14e030a2a74b3471aac49f Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Tue, 25 May 2021 20:57:34 +0800 Subject: [PATCH 173/343] *: fix missing reset for `DurationParse` (#24468) --- executor/adapter.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/executor/adapter.go b/executor/adapter.go index c5d9b0406602c..d0bcbdbbcacae 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -816,6 +816,7 @@ var ( // 2. record summary statement. // 3. record execute duration metric. // 4. update the `PrevStmt` in session variable. +// 5. reset `DurationParse` in session variable. func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, succ bool, hasMoreResults bool) { sessVars := a.Ctx.GetSessionVars() execDetail := sessVars.StmtCtx.GetExecDetails() @@ -853,6 +854,8 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, succ bool, hasMoreResults boo } else { sessionExecuteRunDurationGeneral.Observe(executeDuration.Seconds()) } + // Reset DurationParse due to the next statement may not need to be parsed (not a text protocol query). + sessVars.DurationParse = 0 } // CloseRecordSet will finish the execution of current statement and do some record work From f130a10f1c425ea20fdc9b8bb5edcf51b87a96e3 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Tue, 25 May 2021 21:09:34 +0800 Subject: [PATCH 174/343] grafana: fix KV transaction OPS (#24781) --- metrics/grafana/tidb.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index eb3434a541bc0..4053986976fcd 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -3923,7 +3923,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{tidb_cluster=\"$tidb_cluster\", type=\"commit\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, "legendFormat": "{{instance}}", From 66ad36ee102dfd6537f7f1b1cfb9147f7582559e Mon Sep 17 00:00:00 2001 From: Zhou Kunqin <25057648+time-and-fate@users.noreply.github.com> Date: Tue, 25 May 2021 23:57:34 +0800 Subject: [PATCH 175/343] planner, executor, statistics: support correlation calc for new sampling method (#24458) --- executor/analyze.go | 84 +++++++--- executor/builder.go | 2 +- planner/core/planbuilder.go | 14 +- statistics/builder.go | 37 ++++- statistics/handle/bootstrap.go | 2 +- statistics/handle/handle.go | 2 +- statistics/handle/handle_test.go | 75 ++++++++- statistics/handle/update.go | 1 + statistics/histogram.go | 12 +- statistics/row_sampler.go | 153 +----------------- statistics/sample_test.go | 2 +- statistics/statistics_test.go | 2 +- .../mockstore/unistore/cophandler/analyze.go | 12 +- util/sqlexec/restricted_sql_executor.go | 5 + 14 files changed, 191 insertions(+), 212 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index a9aecb7547588..180700bbc85b6 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -523,21 +523,13 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { } else { ranges = ranger.FullIntRange(false) } + collExtStats := colExec.ctx.GetSessionVars().EnableExtendedStats if colExec.analyzeVer == statistics.Version3 { - count, hists, topns, fmSketches, err := colExec.buildSamplingStats(ranges) + count, hists, topns, fmSketches, extStats, err := colExec.buildSamplingStats(ranges, collExtStats) if err != nil { return []analyzeResult{{Err: err, job: colExec.job}} } cLen := len(colExec.analyzePB.ColReq.ColumnsInfo) - colResult := analyzeResult{ - TableID: colExec.tableID, - Hist: hists[:cLen], - TopNs: topns[:cLen], - Fms: fmSketches[:cLen], - job: colExec.job, - StatsVer: colExec.analyzeVer, - Count: count, - } colGroupResult := analyzeResult{ TableID: colExec.tableID, Hist: hists[cLen:], @@ -548,9 +540,26 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { Count: count, IsIndex: 1, } + // Discard stats of _tidb_rowid. + // Because the process of analyzing will keep the order of results be the same as the colsInfo in the analyze task, + // and in `buildAnalyzeFullSamplingTask` we always place the _tidb_rowid at the last of colsInfo, so if there are + // stats for _tidb_rowid, it must be at the end of the column stats. + if hists[cLen-1].ID == -1 { + cLen -= 1 + } + colResult := analyzeResult{ + TableID: colExec.tableID, + Hist: hists[:cLen], + TopNs: topns[:cLen], + Fms: fmSketches[:cLen], + ExtStats: extStats, + job: colExec.job, + StatsVer: colExec.analyzeVer, + Count: count, + } + return []analyzeResult{colResult, colGroupResult} } - collExtStats := colExec.ctx.GetSessionVars().EnableExtendedStats hists, cms, topNs, fms, extStats, err := colExec.buildStats(ranges, collExtStats) if err != nil { return []analyzeResult{{Err: err, job: colExec.job}} @@ -608,7 +617,7 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { StatsVer: colExec.analyzeVer, } colResult.Count = int64(colResult.Hist[0].TotalRowCount()) - if colResult.StatsVer == statistics.Version2 { + if colResult.StatsVer >= statistics.Version2 { colResult.Count += int64(topNs[0].TotalCount()) } return append(result, colResult) @@ -674,15 +683,16 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe return result, nil } -func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range) ( +func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtStats bool) ( count int64, hists []*statistics.Histogram, topns []*statistics.TopN, fmSketches []*statistics.FMSketch, + extStats *statistics.ExtendedStatsColl, err error, ) { if err = e.open(ranges); err != nil { - return 0, nil, nil, nil, err + return 0, nil, nil, nil, nil, err } defer func() { if err1 := e.resultHandler.Close(); err1 != nil { @@ -704,7 +714,7 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range) ( for { data, err1 := e.resultHandler.nextRaw(context.TODO()) if err1 != nil { - return 0, nil, nil, nil, err1 + return 0, nil, nil, nil, nil, err1 } if data == nil { break @@ -712,7 +722,7 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range) ( colResp := &tipb.AnalyzeColumnsResp{} err = colResp.Unmarshal(data) if err != nil { - return 0, nil, nil, nil, err + return 0, nil, nil, nil, nil, err } subCollector := &statistics.RowSampleCollector{ MaxSampleSize: int(e.analyzePB.ColReq.SampleSize), @@ -725,24 +735,38 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range) ( for i := range sample.Columns { sample.Columns[i], err = tablecodec.DecodeColumnValue(sample.Columns[i].GetBytes(), &e.colsInfo[i].FieldType, sc.TimeZone) if err != nil { - return 0, nil, nil, nil, err + return 0, nil, nil, nil, nil, err } if sample.Columns[i].Kind() == types.KindBytes { sample.Columns[i].SetBytes(sample.Columns[i].GetBytes()) } } + // Calculate handle from the row data for each row. It will be used to sort the samples. + sample.Handle, err = e.handleCols.BuildHandleByDatums(sample.Columns) + if err != nil { + return 0, nil, nil, nil, nil, err + } } + + // The order of the samples are broken when merging samples from sub-collectors. + // So now we need to sort the samples according to the handle in order to calculate correlation. + sort.Slice(rootRowCollector.Samples, func(i, j int) bool { + return rootRowCollector.Samples[i].Handle.Compare(rootRowCollector.Samples[j].Handle) < 0 + }) + hists = make([]*statistics.Histogram, 0, len(e.colsInfo)) topns = make([]*statistics.TopN, 0, len(e.colsInfo)) fmSketches = make([]*statistics.FMSketch, 0, len(e.colsInfo)) + sampleCollectors := make([]*statistics.SampleCollector, 0, len(e.colsInfo)) for i, col := range e.colsInfo { sampleItems := make([]*statistics.SampleItem, 0, rootRowCollector.MaxSampleSize) - for _, row := range rootRowCollector.Samples { + for j, row := range rootRowCollector.Samples { if row.Columns[i].IsNull() { continue } sampleItems = append(sampleItems, &statistics.SampleItem{ - Value: row.Columns[i], + Value: row.Columns[i], + Ordinal: j, }) } collector := &statistics.SampleCollector{ @@ -752,9 +776,10 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range) ( FMSketch: rootRowCollector.FMSketches[i], TotalSize: rootRowCollector.TotalSizes[i], } - hg, topn, err := statistics.BuildHistAndTopNOnRowSample(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), col.ID, collector, &col.FieldType, true) + sampleCollectors = append(sampleCollectors, collector) + hg, topn, err := statistics.BuildHistAndTopN(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), col.ID, collector, &col.FieldType, true) if err != nil { - return 0, nil, nil, nil, err + return 0, nil, nil, nil, nil, err } hists = append(hists, hg) topns = append(topns, topn) @@ -771,7 +796,7 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range) ( for _, col := range idx.Columns { b, err = codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, b, row.Columns[col.Offset]) if err != nil { - return 0, nil, nil, nil, err + return 0, nil, nil, nil, nil, err } } sampleItems = append(sampleItems, &statistics.SampleItem{ @@ -785,15 +810,22 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range) ( FMSketch: rootRowCollector.FMSketches[colLen+i], TotalSize: rootRowCollector.TotalSizes[colLen+i], } - hg, topn, err := statistics.BuildHistAndTopNOnRowSample(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), idx.ID, collector, types.NewFieldType(mysql.TypeBlob), false) + hg, topn, err := statistics.BuildHistAndTopN(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), idx.ID, collector, types.NewFieldType(mysql.TypeBlob), false) if err != nil { - return 0, nil, nil, nil, err + return 0, nil, nil, nil, nil, err } hists = append(hists, hg) topns = append(topns, topn) fmSketches = append(fmSketches, rootRowCollector.FMSketches[colLen+i]) } count = rootRowCollector.Count + if needExtStats { + statsHandle := domain.GetDomain(e.ctx).StatsHandle() + extStats, err = statsHandle.BuildExtendedStats(e.tableID.GetStatisticsID(), e.colsInfo, sampleCollectors) + if err != nil { + return 0, nil, nil, nil, nil, err + } + } return } @@ -917,7 +949,7 @@ func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range, needExtStats boo if e.analyzeVer < 2 { hg, err = statistics.BuildColumn(e.ctx, int64(e.opts[ast.AnalyzeOptNumBuckets]), col.ID, collectors[i], &col.FieldType) } else { - hg, topn, err = statistics.BuildColumnHistAndTopN(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), col.ID, collectors[i], &col.FieldType) + hg, topn, err = statistics.BuildHistAndTopN(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), col.ID, collectors[i], &col.FieldType, true) topNs = append(topNs, topn) } if err != nil { @@ -1602,7 +1634,7 @@ func analyzeIndexIncremental(idxExec *analyzeIndexIncrementalExec) analyzeResult } cms.CalcDefaultValForAnalyze(uint64(hist.NDV)) } - if statsVer == statistics.Version2 { + if statsVer >= statistics.Version2 { poped := statistics.MergeTopNAndUpdateCMSketch(topN, idxExec.oldTopN, cms, uint32(idxExec.opts[ast.AnalyzeOptNumTopN])) hist.AddIdxVals(poped) } diff --git a/executor/builder.go b/executor/builder.go index 2f08e604851a7..7a66594222d09 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2022,7 +2022,7 @@ func (b *executorBuilder) buildAnalyzeIndexIncremental(task plannercore.AnalyzeI oldHist = idx.TruncateHistogram(bktID) } var oldTopN *statistics.TopN - if analyzeTask.idxExec.analyzePB.IdxReq.GetVersion() == statistics.Version2 { + if analyzeTask.idxExec.analyzePB.IdxReq.GetVersion() >= statistics.Version2 { oldTopN = idx.TopN.Copy() oldTopN.RemoveVal(oldHist.Bounds.GetRow(len(oldHist.Buckets)*2 - 1).GetBytes(0)) } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index cac9dab16d8ab..85a5cbded10ec 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1668,12 +1668,20 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( Incremental: as.Incremental, StatsVersion: version, } - taskSlice = append(taskSlice, AnalyzeColumnsTask{ + newTask := AnalyzeColumnsTask{ + HandleCols: BuildHandleColsForAnalyze(b.ctx, tbl.TableInfo), ColsInfo: tbl.TableInfo.Columns, analyzeInfo: info, TblInfo: tbl.TableInfo, Indexes: idxInfos, - }) + } + if newTask.HandleCols == nil { + extraCol := model.NewExtraHandleColInfo() + // Always place _tidb_rowid at the end of colsInfo, this is corresponding to logics in `analyzeColumnsPushdown`. + newTask.ColsInfo = append(newTask.ColsInfo, extraCol) + newTask.HandleCols = &IntHandleCols{col: colInfoToColumn(extraCol, len(newTask.ColsInfo)-1)} + } + taskSlice = append(taskSlice, newTask) } return taskSlice } @@ -1934,7 +1942,7 @@ func (b *PlanBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) (Plan, error) { return nil, errors.Errorf("Only support fast analyze in tikv storage.") } statsVersion := b.ctx.GetSessionVars().AnalyzeVersion - if b.ctx.GetSessionVars().EnableFastAnalyze && statsVersion == statistics.Version2 { + if b.ctx.GetSessionVars().EnableFastAnalyze && statsVersion >= statistics.Version2 { return nil, errors.Errorf("Fast analyze hasn't reached General Availability and only support analyze version 1 currently.") } for _, tbl := range as.TableNames { diff --git a/statistics/builder.go b/statistics/builder.go index 81c2f52dc84a9..6dad07b844e35 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -42,7 +42,7 @@ func NewSortedBuilder(sc *stmtctx.StatementContext, numBuckets, id int64, tp *ty numBuckets: numBuckets, valuesPerBucket: 1, hist: NewHistogram(id, 0, 0, 0, tp, int(numBuckets), 0), - needBucketNDV: statsVer == Version2, + needBucketNDV: statsVer >= Version2, } } @@ -210,8 +210,25 @@ func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *Sample return BuildColumnHist(ctx, numBuckets, id, collector, tp, collector.Count, collector.FMSketch.NDV(), collector.NullCount) } -// BuildColumnHistAndTopN build a histogram and TopN for a column from samples. -func BuildColumnHistAndTopN(ctx sessionctx.Context, numBuckets, numTopN int, id int64, collector *SampleCollector, tp *types.FieldType) (*Histogram, *TopN, error) { +// BuildHistAndTopN build a histogram and TopN for a column or an index from samples. +func BuildHistAndTopN( + ctx sessionctx.Context, + numBuckets, numTopN int, + id int64, + collector *SampleCollector, + tp *types.FieldType, + isColumn bool, +) (*Histogram, *TopN, error) { + var getComparedBytes func(datum types.Datum) ([]byte, error) + if isColumn { + getComparedBytes = func(datum types.Datum) ([]byte, error) { + return codec.EncodeKey(ctx.GetSessionVars().StmtCtx, nil, datum) + } + } else { + getComparedBytes = func(datum types.Datum) ([]byte, error) { + return datum.GetBytes(), nil + } + } count := collector.Count ndv := collector.FMSketch.NDV() nullCount := collector.NullCount @@ -237,7 +254,7 @@ func BuildColumnHistAndTopN(ctx sessionctx.Context, numBuckets, numTopN int, id // the topNList is always sorted by count from more to less topNList := make([]TopNMeta, 0, numTopN) - cur, err := codec.EncodeKey(ctx.GetSessionVars().StmtCtx, nil, samples[0].Value) + cur, err := getComparedBytes(samples[0].Value) if err != nil { return nil, nil, errors.Trace(err) } @@ -246,9 +263,11 @@ func BuildColumnHistAndTopN(ctx sessionctx.Context, numBuckets, numTopN int, id // Iterate through the samples for i := int64(0); i < sampleNum; i++ { - corrXYSum += float64(i) * float64(samples[i].Ordinal) + if isColumn { + corrXYSum += float64(i) * float64(samples[i].Ordinal) + } - sampleBytes, err := codec.EncodeKey(ctx.GetSessionVars().StmtCtx, nil, samples[i].Value) + sampleBytes, err := getComparedBytes(samples[i].Value) if err != nil { return nil, nil, errors.Trace(err) } @@ -286,7 +305,9 @@ func BuildColumnHistAndTopN(ctx sessionctx.Context, numBuckets, numTopN int, id } // Calc the correlation of the column between the handle column. - hg.Correlation = calcCorrelation(sampleNum, corrXYSum) + if isColumn { + hg.Correlation = calcCorrelation(sampleNum, corrXYSum) + } // Handle the counting for the last value. Basically equal to the case 2 above. // now topn is empty: append the "current" count directly @@ -310,7 +331,7 @@ func BuildColumnHistAndTopN(ctx sessionctx.Context, numBuckets, numTopN int, id // Step2: exclude topn from samples for i := int64(0); i < int64(len(samples)); i++ { - sampleBytes, err := codec.EncodeKey(ctx.GetSessionVars().StmtCtx, nil, samples[i].Value) + sampleBytes, err := getComparedBytes(samples[i].Value) if err != nil { return nil, nil, errors.Trace(err) } diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index 5fd48e096ca3f..9e9fc7af8cb87 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -131,7 +131,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, cache *stat var topnCount int64 // If this is stats of the Version2, we need to consider the topn's count as well. // See the comments of Version2 for more details. - if statsVer == statistics.Version2 { + if statsVer >= statistics.Version2 { var err error topnCount, err = h.initTopNCountSum(tblID, id) if err != nil { diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index fe3644739c1b8..0026c97bb0c7c 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1118,7 +1118,7 @@ func (h *Handle) columnCountFromStorage(reader *statsReader, tableID, colID, sta if err != nil { return 0, errors.Trace(err) } - if statsVer == statistics.Version2 { + if statsVer >= statistics.Version2 { // Before stats ver 2, histogram represents all data in this column. // In stats ver 2, histogram + TopN represent all data in this column. // So we need to add TopN total count here. diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 6d39e42b076e0..489be2167acf0 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -620,66 +620,89 @@ func (s *testStatsSuite) TestCorrelation(c *C) { testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") testKit.MustExec("create table t(c1 int primary key, c2 int)") + testKit.MustExec("select * from t where c1 > 10 and c2 > 10") testKit.MustExec("insert into t values(1,1),(3,12),(4,20),(2,7),(5,21)") + testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result := testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0") c.Assert(result.Rows()[1][9], Equals, "1") + testKit.MustExec("set @@session.tidb_analyze_version=3") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + c.Assert(len(result.Rows()), Equals, 2) + c.Assert(result.Rows()[0][9], Equals, "1") + c.Assert(result.Rows()[1][9], Equals, "1") testKit.MustExec("insert into t values(8,18)") + testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0") c.Assert(result.Rows()[1][9], Equals, "0.8285714285714286") - testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("set @@session.tidb_analyze_version=3") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "0") + c.Assert(result.Rows()[0][9], Equals, "1") c.Assert(result.Rows()[1][9], Equals, "0.8285714285714286") testKit.MustExec("truncate table t") - testKit.MustExec("set @@session.tidb_analyze_version=1") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 0) testKit.MustExec("insert into t values(1,21),(3,12),(4,7),(2,20),(5,1)") + testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0") c.Assert(result.Rows()[1][9], Equals, "-1") + testKit.MustExec("set @@session.tidb_analyze_version=3") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + c.Assert(len(result.Rows()), Equals, 2) + c.Assert(result.Rows()[0][9], Equals, "1") + c.Assert(result.Rows()[1][9], Equals, "-1") testKit.MustExec("insert into t values(8,4)") + testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0") c.Assert(result.Rows()[1][9], Equals, "-0.9428571428571428") - testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("set @@session.tidb_analyze_version=3") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) - c.Assert(result.Rows()[0][9], Equals, "0") + c.Assert(result.Rows()[0][9], Equals, "1") c.Assert(result.Rows()[1][9], Equals, "-0.9428571428571428") testKit.MustExec("truncate table t") - testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("insert into t values (1,1),(2,1),(3,1),(4,1),(5,1),(6,1),(7,1),(8,1),(9,1),(10,1),(11,1),(12,1),(13,1),(14,1),(15,1),(16,1),(17,1),(18,1),(19,1),(20,2),(21,2),(22,2),(23,2),(24,2),(25,2)") + testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0") c.Assert(result.Rows()[1][9], Equals, "1") + testKit.MustExec("set @@session.tidb_analyze_version=3") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + c.Assert(len(result.Rows()), Equals, 2) + c.Assert(result.Rows()[0][9], Equals, "1") + c.Assert(result.Rows()[1][9], Equals, "1") testKit.MustExec("drop table t") testKit.MustExec("create table t(c1 int, c2 int)") testKit.MustExec("insert into t values(1,1),(2,7),(3,12),(4,20),(5,21),(8,18)") + testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "1") c.Assert(result.Rows()[1][9], Equals, "0.8285714285714286") - testKit.MustExec("set @@session.tidb_analyze_version=2") + testKit.MustExec("set @@session.tidb_analyze_version=3") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) @@ -688,6 +711,13 @@ func (s *testStatsSuite) TestCorrelation(c *C) { testKit.MustExec("truncate table t") testKit.MustExec("insert into t values(1,1),(2,7),(3,12),(8,18),(4,20),(5,21)") + testKit.MustExec("set @@session.tidb_analyze_version=1") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() + c.Assert(len(result.Rows()), Equals, 2) + c.Assert(result.Rows()[0][9], Equals, "0.8285714285714286") + c.Assert(result.Rows()[1][9], Equals, "1") + testKit.MustExec("set @@session.tidb_analyze_version=3") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) @@ -697,6 +727,7 @@ func (s *testStatsSuite) TestCorrelation(c *C) { testKit.MustExec("drop table t") testKit.MustExec("create table t(c1 int primary key, c2 int, c3 int, key idx_c2(c2))") testKit.MustExec("insert into t values(1,1,1),(2,2,2),(3,3,3)") + testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 0").Sort() c.Assert(len(result.Rows()), Equals, 3) @@ -706,6 +737,16 @@ func (s *testStatsSuite) TestCorrelation(c *C) { result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 1").Sort() c.Assert(len(result.Rows()), Equals, 1) c.Assert(result.Rows()[0][9], Equals, "0") + testKit.MustExec("set @@tidb_analyze_version=3") + testKit.MustExec("analyze table t") + result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 0").Sort() + c.Assert(len(result.Rows()), Equals, 3) + c.Assert(result.Rows()[0][9], Equals, "1") + c.Assert(result.Rows()[1][9], Equals, "1") + c.Assert(result.Rows()[2][9], Equals, "1") + result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 1").Sort() + c.Assert(len(result.Rows()), Equals, 1) + c.Assert(result.Rows()[0][9], Equals, "0") } func (s *testStatsSuite) TestShowGlobalStats(c *C) { @@ -1846,6 +1887,12 @@ func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { c.Assert(statsTbl.ExtendedStats, NotNil) c.Assert(len(statsTbl.ExtendedStats.Stats), Equals, 0) + tk.MustExec("analyze table t") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( + "2 [1,2] 1.000000 1", + "2 [1,3] -1.000000 1", + )) + tk.MustExec("set @@session.tidb_analyze_version=3") tk.MustExec("analyze table t") tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( "2 [1,2] 1.000000 1", @@ -1875,12 +1922,26 @@ func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { // Check that table with NULLs won't cause panic tk.MustExec("delete from t") tk.MustExec("insert into t values(1,null,2), (2,null,null)") + tk.MustExec("set @@session.tidb_analyze_version=1") + tk.MustExec("analyze table t") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( + "2 [1,2] 0.000000 1", + "2 [1,3] 1.000000 1", + )) + tk.MustExec("set @@session.tidb_analyze_version=3") tk.MustExec("analyze table t") tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( "2 [1,2] 0.000000 1", "2 [1,3] 1.000000 1", )) tk.MustExec("insert into t values(3,3,3)") + tk.MustExec("set @@session.tidb_analyze_version=1") + tk.MustExec("analyze table t") + tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( + "2 [1,2] 1.000000 1", + "2 [1,3] 1.000000 1", + )) + tk.MustExec("set @@session.tidb_analyze_version=3") tk.MustExec("analyze table t") tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( "2 [1,2] 1.000000 1", diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 03df857ec4a5d..2251fd8a9bb32 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -1067,6 +1067,7 @@ var execOptionForAnalyze = map[int]sqlexec.OptionFuncAlias{ statistics.Version0: sqlexec.ExecOptionAnalyzeVer1, statistics.Version1: sqlexec.ExecOptionAnalyzeVer1, statistics.Version2: sqlexec.ExecOptionAnalyzeVer2, + statistics.Version3: sqlexec.ExecOptionAnalyzeVer3, } func (h *Handle) execAutoAnalyze(statsVer int, sql string, params ...interface{}) { diff --git a/statistics/histogram.go b/statistics/histogram.go index 37a78fe1b92de..29bdaed692084 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -814,7 +814,7 @@ func MergeHistograms(sc *stmtctx.StatementContext, lh *Histogram, rh *Histogram, rAvg *= 2 } for i := 0; i < rh.Len(); i++ { - if statsVer == Version2 { + if statsVer >= Version2 { lh.AppendBucketWithNDV(rh.GetLower(i), rh.GetUpper(i), rh.Buckets[i].Count+lCount-offset, rh.Buckets[i].Repeat, rh.Buckets[i].NDV) continue } @@ -925,14 +925,14 @@ func (c *Column) String() string { // TotalRowCount returns the total count of this column. func (c *Column) TotalRowCount() float64 { - if c.StatsVer == Version2 { + if c.StatsVer >= Version2 { return c.Histogram.TotalRowCount() + float64(c.TopN.TotalCount()) } return c.Histogram.TotalRowCount() } func (c *Column) notNullCount() float64 { - if c.StatsVer == Version2 { + if c.StatsVer >= Version2 { return c.Histogram.notNullCount() + float64(c.TopN.TotalCount()) } return c.Histogram.notNullCount() @@ -1140,7 +1140,7 @@ func (idx *Index) String() string { // TotalRowCount returns the total count of this index. func (idx *Index) TotalRowCount() float64 { - if idx.StatsVer == Version2 { + if idx.StatsVer >= Version2 { return idx.Histogram.TotalRowCount() + float64(idx.TopN.TotalCount()) } return idx.Histogram.TotalRowCount() @@ -1177,7 +1177,7 @@ func (idx *Index) equalRowCount(b []byte, modifyCount int64) float64 { return float64(idx.QueryBytes(b)) } // If it's version2, query the top-n first. - if idx.StatsVer == Version2 { + if idx.StatsVer >= Version2 { count, found := idx.TopN.QueryTopN(b) if found { return float64(count) @@ -1245,7 +1245,7 @@ func (idx *Index) GetRowCount(sc *stmtctx.StatementContext, coll *HistColl, inde expBackoffSuccess := false // Due to the limitation of calcFraction and convertDatumToScalar, the histogram actually won't estimate anything. // If the first column's range is point. - if rangePosition := GetOrdinalOfRangeCond(sc, indexRange); rangePosition > 0 && idx.StatsVer == Version2 && coll != nil { + if rangePosition := GetOrdinalOfRangeCond(sc, indexRange); rangePosition > 0 && idx.StatsVer >= Version2 && coll != nil { var expBackoffSel float64 expBackoffSel, expBackoffSuccess, err = idx.expBackoffEstimation(sc, coll, indexRange) if err != nil { diff --git a/statistics/row_sampler.go b/statistics/row_sampler.go index f4f40945af697..fce358e45ae24 100644 --- a/statistics/row_sampler.go +++ b/statistics/row_sampler.go @@ -14,13 +14,11 @@ package statistics import ( - "bytes" "container/heap" "context" "math/rand" - "github.com/pingcap/errors" - "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -52,6 +50,7 @@ type RowSampleCollector struct { type RowSampleItem struct { Columns []types.Datum Weight int64 + Handle kv.Handle } // WeightedRowSampleHeap implements the Heap interface. @@ -299,151 +298,3 @@ func RowSamplesToProto(samples WeightedRowSampleHeap) []*tipb.RowSample { } return rows } - -// BuildHistAndTopNOnRowSample build a histogram and TopN for a column from samples. -func BuildHistAndTopNOnRowSample( - ctx sessionctx.Context, - numBuckets, numTopN int, - id int64, - collector *SampleCollector, - tp *types.FieldType, - isColumn bool, -) (*Histogram, *TopN, error) { - var getComparedBytes func(datum types.Datum) ([]byte, error) - if isColumn { - getComparedBytes = func(datum types.Datum) ([]byte, error) { - return codec.EncodeKey(ctx.GetSessionVars().StmtCtx, nil, datum) - } - } else { - getComparedBytes = func(datum types.Datum) ([]byte, error) { - return datum.GetBytes(), nil - } - } - count := collector.Count - ndv := collector.FMSketch.NDV() - nullCount := collector.NullCount - if ndv > count { - ndv = count - } - if count == 0 || len(collector.Samples) == 0 { - return NewHistogram(id, ndv, nullCount, 0, tp, 0, collector.TotalSize), nil, nil - } - sc := ctx.GetSessionVars().StmtCtx - samples := collector.Samples - samples, err := SortSampleItems(sc, samples) - if err != nil { - return nil, nil, err - } - hg := NewHistogram(id, ndv, nullCount, 0, tp, numBuckets, collector.TotalSize) - - sampleNum := int64(len(samples)) - // As we use samples to build the histogram, the bucket number and repeat should multiply a factor. - sampleFactor := float64(count) / float64(len(samples)) - - // Step1: collect topn from samples - - // the topNList is always sorted by count from more to less - topNList := make([]TopNMeta, 0, numTopN) - cur, err := getComparedBytes(samples[0].Value) - if err != nil { - return nil, nil, errors.Trace(err) - } - curCnt := float64(0) - - // Iterate through the samples - for i := int64(0); i < sampleNum; i++ { - - sampleBytes, err := getComparedBytes(samples[i].Value) - if err != nil { - return nil, nil, errors.Trace(err) - } - // case 1, this value is equal to the last one: current count++ - if bytes.Equal(cur, sampleBytes) { - curCnt += 1 - continue - } - // case 2, meet a different value: counting for the "current" is complete - // case 2-1, now topn is empty: append the "current" count directly - if len(topNList) == 0 { - topNList = append(topNList, TopNMeta{Encoded: cur, Count: uint64(curCnt)}) - cur, curCnt = sampleBytes, 1 - continue - } - // case 2-2, now topn is full, and the "current" count is less than the least count in the topn: no need to insert the "current" - if len(topNList) >= numTopN && uint64(curCnt) <= topNList[len(topNList)-1].Count { - cur, curCnt = sampleBytes, 1 - continue - } - // case 2-3, now topn is not full, or the "current" count is larger than the least count in the topn: need to find a slot to insert the "current" - j := len(topNList) - for ; j > 0; j-- { - if uint64(curCnt) < topNList[j-1].Count { - break - } - } - topNList = append(topNList, TopNMeta{}) - copy(topNList[j+1:], topNList[j:]) - topNList[j] = TopNMeta{Encoded: cur, Count: uint64(curCnt)} - if len(topNList) > numTopN { - topNList = topNList[:numTopN] - } - cur, curCnt = sampleBytes, 1 - } - - // Handle the counting for the last value. Basically equal to the case 2 above. - // now topn is empty: append the "current" count directly - if len(topNList) == 0 { - topNList = append(topNList, TopNMeta{Encoded: cur, Count: uint64(curCnt)}) - } else if len(topNList) < numTopN || uint64(curCnt) > topNList[len(topNList)-1].Count { - // now topn is not full, or the "current" count is larger than the least count in the topn: need to find a slot to insert the "current" - j := len(topNList) - for ; j > 0; j-- { - if uint64(curCnt) < topNList[j-1].Count { - break - } - } - topNList = append(topNList, TopNMeta{}) - copy(topNList[j+1:], topNList[j:]) - topNList[j] = TopNMeta{Encoded: cur, Count: uint64(curCnt)} - if len(topNList) > numTopN { - topNList = topNList[:numTopN] - } - } - - // Step2: exclude topn from samples - for i := int64(0); i < int64(len(samples)); i++ { - sampleBytes, err := getComparedBytes(samples[i].Value) - if err != nil { - return nil, nil, errors.Trace(err) - } - for j := 0; j < len(topNList); j++ { - if bytes.Equal(sampleBytes, topNList[j].Encoded) { - // find the same value in topn: need to skip over this value in samples - copy(samples[i:], samples[uint64(i)+topNList[j].Count:]) - samples = samples[:uint64(len(samples))-topNList[j].Count] - i-- - continue - } - } - } - - for i := 0; i < len(topNList); i++ { - topNList[i].Count *= uint64(sampleFactor) - } - topn := &TopN{TopN: topNList} - - if uint64(count) <= topn.TotalCount() || int(hg.NDV) <= len(topn.TopN) { - // TopN includes all sample data - return hg, topn, nil - } - - // Step3: build histogram with the rest samples - if len(samples) > 0 { - _, err = buildHist(sc, hg, samples, count-int64(topn.TotalCount()), ndv-int64(len(topn.TopN)), int64(numBuckets)) - if err != nil { - return nil, nil, err - } - } - - return hg, topn, nil -} diff --git a/statistics/sample_test.go b/statistics/sample_test.go index 61ec41b85870a..3624b233395ae 100644 --- a/statistics/sample_test.go +++ b/statistics/sample_test.go @@ -297,7 +297,7 @@ func (s *testSampleSuite) TestBuildStatsOnRowSample(c *C) { TotalSize: int64(len(data)) * 8, } tp := types.NewFieldType(mysql.TypeLonglong) - hist, topN, err := BuildHistAndTopNOnRowSample(ctx, 5, 4, 1, collector, tp, true) + hist, topN, err := BuildHistAndTopN(ctx, 5, 4, 1, collector, tp, true) c.Assert(err, IsNil, Commentf("%+v", err)) topNStr, err := topN.DecodedString(ctx, []byte{tp.Tp}) c.Assert(err, IsNil) diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index e74e04f7596ac..7fd0bf64b0bf5 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -278,7 +278,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { count = col.lessRowCount(types.NewIntDatum(1)) c.Check(int(count), Equals, 5) - colv2, topnv2, err := BuildColumnHistAndTopN(ctx, int(bucketCount), topNCount, 2, collector, types.NewFieldType(mysql.TypeLonglong)) + colv2, topnv2, err := BuildHistAndTopN(ctx, int(bucketCount), topNCount, 2, collector, types.NewFieldType(mysql.TypeLonglong), true) c.Check(err, IsNil) c.Check(topnv2.TopN, NotNil) expectedTopNCount := []uint64{9990, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30} diff --git a/store/mockstore/unistore/cophandler/analyze.go b/store/mockstore/unistore/cophandler/analyze.go index 4f309027edc4a..c7ce4630fee85 100644 --- a/store/mockstore/unistore/cophandler/analyze.go +++ b/store/mockstore/unistore/cophandler/analyze.go @@ -107,7 +107,7 @@ func handleAnalyzeIndexReq(dbReader *dbreader.DBReader, rans []kv.KeyRange, anal return nil, err } } - if statsVer == statistics.Version2 { + if statsVer >= statistics.Version2 { if processor.topNCurValuePair.Count != 0 { processor.topNValuePairs = append(processor.topNValuePairs, processor.topNCurValuePair) } @@ -126,7 +126,7 @@ func handleAnalyzeIndexReq(dbReader *dbreader.DBReader, rans []kv.KeyRange, anal hg := statistics.HistogramToProto(processor.statsBuilder.Hist()) var cm *tipb.CMSketch if processor.cms != nil { - if statsVer == statistics.Version2 { + if statsVer >= statistics.Version2 { for _, valueCnt := range processor.topNValuePairs { h1, h2 := murmur3.Sum128(valueCnt.Encoded) processor.cms.SubValue(h1, h2, valueCnt.Count) @@ -207,7 +207,7 @@ func (p *analyzeIndexProcessor) Process(key, _ []byte) error { } } - if p.statsVer == statistics.Version2 { + if p.statsVer >= statistics.Version2 { if bytes.Equal(p.topNCurValuePair.Encoded, p.rowBuf) { p.topNCurValuePair.Count++ } else { @@ -547,7 +547,7 @@ func handleAnalyzeMixedReq(dbReader *dbreader.DBReader, rans []kv.KeyRange, anal colResp.Collectors = append(colResp.Collectors, statistics.SampleCollectorToProto(c)) } // common handle - if statsVer == statistics.Version2 { + if statsVer >= statistics.Version2 { if e.topNCurValuePair.Count != 0 { e.topNValuePairs = append(e.topNValuePairs, e.topNCurValuePair) } @@ -566,7 +566,7 @@ func handleAnalyzeMixedReq(dbReader *dbreader.DBReader, rans []kv.KeyRange, anal hg := statistics.HistogramToProto(e.statsBuilder.Hist()) var cm *tipb.CMSketch if e.cms != nil { - if statsVer == statistics.Version2 { + if statsVer >= statistics.Version2 { for _, valueCnt := range e.topNValuePairs { h1, h2 := murmur3.Sum128(valueCnt.Encoded) e.cms.SubValue(h1, h2, valueCnt.Count) @@ -623,7 +623,7 @@ func (e *analyzeMixedExec) Process(key, value []byte) error { } } - if e.statsVer == statistics.Version2 { + if e.statsVer >= statistics.Version2 { if bytes.Equal(e.topNCurValuePair.Encoded, e.rowBuf) { e.topNCurValuePair.Count++ } else { diff --git a/util/sqlexec/restricted_sql_executor.go b/util/sqlexec/restricted_sql_executor.go index e350118a77715..856ce6112bda3 100644 --- a/util/sqlexec/restricted_sql_executor.go +++ b/util/sqlexec/restricted_sql_executor.go @@ -72,6 +72,11 @@ var ExecOptionAnalyzeVer2 OptionFuncAlias = func(option *ExecOption) { option.AnalyzeVer = 2 } +// ExecOptionAnalyzeVer3 tells ExecRestrictedStmt to collect statistics with version3. +var ExecOptionAnalyzeVer3 OptionFuncAlias = func(option *ExecOption) { + option.AnalyzeVer = 3 +} + // ExecOptionWithSnapshot tells ExecRestrictedStmt to use a snapshot. func ExecOptionWithSnapshot(snapshot uint64) OptionFuncAlias { return func(option *ExecOption) { From 8a066999f136cf247a7e6d95e91449b19f86c131 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Tue, 25 May 2021 11:37:34 -0600 Subject: [PATCH 176/343] *: Remove incorrect global sysvar caching (#24699) --- expression/integration_test.go | 11 +++++++++++ planner/core/expression_rewriter.go | 6 +----- sessionctx/variable/varsutil.go | 1 - 3 files changed, 12 insertions(+), 6 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 22e401bab559f..d1db4e3cbbd69 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9504,6 +9504,17 @@ func (s *testIntegrationSuite) TestEnumIndex(c *C) { testkit.Rows("2")) } +// Previously global values were cached. This is incorrect. +// See: https://github.com/pingcap/tidb/issues/24368 +func (s *testIntegrationSuite) TestGlobalCacheCorrectness(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustQuery("SHOW VARIABLES LIKE 'max_connections'").Check(testkit.Rows("max_connections 151")) + tk.MustExec("SET GLOBAL max_connections=1234") + tk.MustQuery("SHOW VARIABLES LIKE 'max_connections'").Check(testkit.Rows("max_connections 1234")) + // restore + tk.MustExec("SET GLOBAL max_connections=151") +} + func (s *testIntegrationSuite) TestControlFunctionWithEnumOrSet(c *C) { defer s.cleanEnv(c) diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 423aef3376a77..29113fe64bfc9 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1239,11 +1239,7 @@ func (er *expressionRewriter) rewriteVariable(v *ast.VariableExpr) { var err error if sysVar.HasNoneScope() { val = sysVar.Value - } else if v.IsGlobal || !sysVar.HasSessionScope() { - // The condition "|| !sysVar.HasSessionScope()" is a workaround - // for issue https://github.com/pingcap/tidb/issues/24368 - // Where global values are cached incorrectly. When this issue closes, - // the if statement here can be simplified. + } else if v.IsGlobal { val, err = variable.GetGlobalSystemVar(sessionVars, name) } else { val, err = variable.GetSessionOrGlobalSystemVar(sessionVars, name) diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 653d6e4bdd8ca..0b0ca7d944eda 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -168,7 +168,6 @@ func GetSessionOrGlobalSystemVar(s *SessionVars, name string) (string, error) { return "", ErrUnknownSystemVar.GenWithStackByArgs(name) } if sv.HasNoneScope() { - s.systems[sv.Name] = sv.Value return sv.Value, nil } if sv.HasSessionScope() { From 95e9ea82acf9052b6b664c1dc7ca88088dad6048 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 26 May 2021 04:59:34 +0200 Subject: [PATCH 177/343] table: remove reading from non-specificed partitions in IODKU (#24872) --- planner/core/logical_plan_builder.go | 2 +- planner/core/planbuilder.go | 2 +- planner/core/point_get_plan.go | 2 +- table/tables/partition.go | 32 +++++++++++++++++++++------- table/tables/partition_test.go | 16 ++++++++++++++ 5 files changed, 43 insertions(+), 11 deletions(-) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 10f7ee31323f1..ec4a3c401fb53 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3630,7 +3630,7 @@ func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, as } pids[pid] = struct{}{} } - pt = tables.NewPartitionTableithGivenSets(pt, pids) + pt = tables.NewPartitionTableWithGivenSets(pt, pids) } b.partitionedTable = append(b.partitionedTable, pt) } else if len(tn.PartitionNames) != 0 { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 85a5cbded10ec..931221e576ddc 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2605,7 +2605,7 @@ func (b *PlanBuilder) buildInsert(ctx context.Context, insert *ast.InsertStmt) ( givenPartitionSets[id] = struct{}{} } pt := tableInPlan.(table.PartitionedTable) - insertPlan.Table = tables.NewPartitionTableithGivenSets(pt, givenPartitionSets) + insertPlan.Table = tables.NewPartitionTableWithGivenSets(pt, givenPartitionSets) } else if len(insert.PartitionNames) != 0 { return nil, ErrPartitionClauseOnNonpartitioned } diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index ec8c84e7b207e..2876533a250c8 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -1343,7 +1343,7 @@ func buildPointUpdatePlan(ctx sessionctx.Context, pointPlan PhysicalPlan, dbName } pids[pid] = struct{}{} } - pt = tables.NewPartitionTableithGivenSets(pt, pids) + pt = tables.NewPartitionTableWithGivenSets(pt, pids) } updatePlan.PartitionedTable = append(updatePlan.PartitionedTable, pt) } diff --git a/table/tables/partition.go b/table/tables/partition.go index d83373e9601cb..165f188866550 100644 --- a/table/tables/partition.go +++ b/table/tables/partition.go @@ -1081,6 +1081,18 @@ func (t *partitionedTable) GetPartitionByRow(ctx sessionctx.Context, r []types.D return t.partitions[pid], nil } +// GetPartitionByRow returns a Table, which is actually a Partition. +func (t *partitionTableWithGivenSets) GetPartitionByRow(ctx sessionctx.Context, r []types.Datum) (table.PhysicalTable, error) { + pid, err := t.locatePartition(ctx, t.Meta().GetPartitionInfo(), r) + if err != nil { + return nil, errors.Trace(err) + } + if _, ok := t.givenSetPartitions[pid]; !ok { + return nil, errors.WithStack(table.ErrRowDoesNotMatchGivenPartitionSet) + } + return t.partitions[pid], nil +} + // AddRecord implements the AddRecord method for the table.Table interface. func (t *partitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) { return partitionedTableAddRecord(ctx, t, r, nil, opts) @@ -1107,15 +1119,15 @@ func partitionedTableAddRecord(ctx sessionctx.Context, t *partitionedTable, r [] // checks the given partition set for AddRecord/UpdateRecord operations. type partitionTableWithGivenSets struct { *partitionedTable - partitions map[int64]struct{} + givenSetPartitions map[int64]struct{} } -// NewPartitionTableithGivenSets creates a new partition table from a partition table. -func NewPartitionTableithGivenSets(tbl table.PartitionedTable, partitions map[int64]struct{}) table.PartitionedTable { +// NewPartitionTableWithGivenSets creates a new partition table from a partition table. +func NewPartitionTableWithGivenSets(tbl table.PartitionedTable, partitions map[int64]struct{}) table.PartitionedTable { if raw, ok := tbl.(*partitionedTable); ok { return &partitionTableWithGivenSets{ - partitionedTable: raw, - partitions: partitions, + partitionedTable: raw, + givenSetPartitions: partitions, } } return tbl @@ -1123,12 +1135,12 @@ func NewPartitionTableithGivenSets(tbl table.PartitionedTable, partitions map[in // AddRecord implements the AddRecord method for the table.Table interface. func (t *partitionTableWithGivenSets) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) { - return partitionedTableAddRecord(ctx, t.partitionedTable, r, t.partitions, opts) + return partitionedTableAddRecord(ctx, t.partitionedTable, r, t.givenSetPartitions, opts) } func (t *partitionTableWithGivenSets) GetAllPartitionIDs() []int64 { ptIDs := make([]int64, 0, len(t.partitions)) - for id := range t.partitions { + for id := range t.givenSetPartitions { ptIDs = append(ptIDs, id) } return ptIDs @@ -1162,7 +1174,7 @@ func (t *partitionedTable) UpdateRecord(ctx context.Context, sctx sessionctx.Con } func (t *partitionTableWithGivenSets) UpdateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, currData, newData []types.Datum, touched []bool) error { - return partitionedTableUpdateRecord(ctx, sctx, t.partitionedTable, h, currData, newData, touched, t.partitions) + return partitionedTableUpdateRecord(ctx, sctx, t.partitionedTable, h, currData, newData, touched, t.givenSetPartitions) } func partitionedTableUpdateRecord(gctx context.Context, ctx sessionctx.Context, t *partitionedTable, h kv.Handle, currData, newData []types.Datum, touched []bool, partitionSelection map[int64]struct{}) error { @@ -1179,6 +1191,10 @@ func partitionedTableUpdateRecord(gctx context.Context, ctx sessionctx.Context, if _, ok := partitionSelection[to]; !ok { return errors.WithStack(table.ErrRowDoesNotMatchGivenPartitionSet) } + // Should not have been read from this partition! Checked already in GetPartitionByRow() + if _, ok := partitionSelection[from]; !ok { + return errors.WithStack(table.ErrRowDoesNotMatchGivenPartitionSet) + } } // The old and new data locate in different partitions. diff --git a/table/tables/partition_test.go b/table/tables/partition_test.go index a453c3b2ee967..dc7d9956f0c60 100644 --- a/table/tables/partition_test.go +++ b/table/tables/partition_test.go @@ -530,3 +530,19 @@ func (ts *testSuite) TestIssue21574(c *C) { tk.MustExec("drop table t_21574") tk.MustExec("create table t_21574 (`key` int, `table` int) partition by list columns (`key`,`table`) (partition p0 values in ((1,1)));") } + +func (ts *testSuite) TestIssue24746(c *C) { + tk := testkit.NewTestKitWithInit(c, ts.store) + tk.MustExec("use test") + tk.MustExec("drop tables if exists t_24746") + tk.MustExec("create table t_24746 (a int, b varchar(60), c int, primary key(a)) partition by range(a) (partition p0 values less than (5),partition p1 values less than (10), partition p2 values less than maxvalue)") + defer tk.MustExec("drop table t_24746") + err := tk.ExecToErr("insert into t_24746 partition (p1) values(4,'ERROR, not matching partition p1',4)") + c.Assert(table.ErrRowDoesNotMatchGivenPartitionSet.Equal(err), IsTrue) + tk.MustExec("insert into t_24746 partition (p0) values(4,'OK, first row in correct partition',4)") + err = tk.ExecToErr("insert into t_24746 partition (p0) values(4,'DUPLICATE, in p0',4) on duplicate key update a = a + 1, b = 'ERROR, not allowed to write to p1'") + c.Assert(table.ErrRowDoesNotMatchGivenPartitionSet.Equal(err), IsTrue) + // Actual bug, before the fix this was updating the row in p0 (deleting it in p0 and inserting in p1): + err = tk.ExecToErr("insert into t_24746 partition (p1) values(4,'ERROR, not allowed to read from partition p0',4) on duplicate key update a = a + 1, b = 'ERROR, not allowed to read from p0!'") + c.Assert(table.ErrRowDoesNotMatchGivenPartitionSet.Equal(err), IsTrue) +} From 2b3905b44a1f646dcfa97ab89c6bcdf0f87b3771 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 26 May 2021 11:25:34 +0800 Subject: [PATCH 178/343] *: avoid create new parser object in prepared exec (#24883) --- executor/prepared.go | 2 +- session/session.go | 2 ++ util/sqlexec/restricted_sql_executor.go | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/executor/prepared.go b/executor/prepared.go index 50d555cff3d34..a3d950ec0afc9 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -117,7 +117,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { ) if sqlParser, ok := e.ctx.(sqlexec.SQLParser); ok { // FIXME: ok... yet another parse API, may need some api interface clean. - stmts, err = sqlParser.ParseSQL(e.sqlText, charset, collation) + stmts, _, err = sqlParser.ParseSQL(ctx, e.sqlText, charset, collation) } else { p := parser.New() p.SetParserConfig(vars.BuildParserConfig()) diff --git a/session/session.go b/session/session.go index c87f57f287b7c..acbdc58fcbf75 100644 --- a/session/session.go +++ b/session/session.go @@ -1161,6 +1161,8 @@ func (s *session) getTiDBTableValue(name, val string) (string, error) { return validatedVal, nil } +var _ sqlexec.SQLParser = &session{} + func (s *session) ParseSQL(ctx context.Context, sql, charset, collation string) ([]ast.StmtNode, []error, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("session.ParseSQL", opentracing.ChildOf(span.Context())) diff --git a/util/sqlexec/restricted_sql_executor.go b/util/sqlexec/restricted_sql_executor.go index 856ce6112bda3..2a196262a9dec 100644 --- a/util/sqlexec/restricted_sql_executor.go +++ b/util/sqlexec/restricted_sql_executor.go @@ -101,7 +101,7 @@ type SQLExecutor interface { // But a session already has a parser bind in it, so we define this interface and use session as its implementation, // thus avoid allocating new parser. See session.SQLParser for more information. type SQLParser interface { - ParseSQL(sql, charset, collation string) ([]ast.StmtNode, error) + ParseSQL(ctx context.Context, sql, charset, collation string) ([]ast.StmtNode, []error, error) } // Statement is an interface for SQL execution. From f9c589427571390680c755a5c34548738d5e2ece Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Wed, 26 May 2021 11:33:34 +0800 Subject: [PATCH 179/343] planner: cleanup point update cache logic (#24755) --- planner/core/common_plans.go | 30 ------------------------------ 1 file changed, 30 deletions(-) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 7170c991be051..ad68fd111d617 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -448,18 +448,6 @@ func (e *Execute) tryCachePointPlan(ctx context.Context, sctx sessionctx.Context if err != nil { return err } - case *Update: - // Temporarily turn off the cache for UPDATE to solve #21884. - - //ok, err = IsPointUpdateByAutoCommit(sctx, p) - //if err != nil { - // return err - //} - //if ok { - // // make constant expression store paramMarker - // sctx.GetSessionVars().StmtCtx.PointExec = true - // p, names, err = OptimizeAstNode(ctx, sctx, prepared.Stmt, is) - //} } if ok { // just cache point plan now @@ -1358,21 +1346,3 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p Plan) (bo func IsAutoCommitTxn(ctx sessionctx.Context) bool { return ctx.GetSessionVars().IsAutocommit() && !ctx.GetSessionVars().InTxn() } - -// IsPointUpdateByAutoCommit checks if plan p is point update and is in autocommit context -func IsPointUpdateByAutoCommit(ctx sessionctx.Context, p Plan) (bool, error) { - if !IsAutoCommitTxn(ctx) { - return false, nil - } - - // check plan - updPlan, ok := p.(*Update) - if !ok { - return false, nil - } - if _, isFastSel := updPlan.SelectPlan.(*PointGetPlan); isFastSel { - return true, nil - } - - return false, nil -} From f79dc8b2180e546e1c77355c01c872e265bb017f Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 26 May 2021 12:41:34 +0800 Subject: [PATCH 180/343] executor: pessimistic lock on the temporary table should not be written to TiKV (#24737) --- executor/adapter.go | 1 + executor/executor.go | 23 ++++++++++- executor/executor_test.go | 38 +++++++++++++++++++ planner/core/integration_test.go | 26 ------------- planner/core/optimizer.go | 25 ------------ planner/core/point_get_plan.go | 10 +---- .../core/testdata/integration_suite_in.json | 9 ----- .../core/testdata/integration_suite_out.json | 33 ---------------- 8 files changed, 63 insertions(+), 102 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index d0bcbdbbcacae..64b0ac1f1e5b8 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -579,6 +579,7 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { if len(keys) == 0 { return nil } + keys = filterTemporaryTableKeys(sctx.GetSessionVars(), keys) seVars := sctx.GetSessionVars() lockCtx := newLockCtx(seVars, seVars.LockWaitTimeout) var lockKeyStats *util.LockKeysDetails diff --git a/executor/executor.go b/executor/executor.go index 1d136bac8a2f9..3474d7e33cd8f 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1005,7 +1005,8 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.Loc // locked by others. used for (select for update nowait) situation // except 0 means alwaysWait 1 means nowait func doLockKeys(ctx context.Context, se sessionctx.Context, lockCtx *tikvstore.LockCtx, keys ...kv.Key) error { - sctx := se.GetSessionVars().StmtCtx + sessVars := se.GetSessionVars() + sctx := sessVars.StmtCtx if !sctx.InUpdateStmt && !sctx.InDeleteStmt { atomic.StoreUint32(&se.GetSessionVars().TxnCtx.ForUpdate, 1) } @@ -1014,6 +1015,10 @@ func doLockKeys(ctx context.Context, se sessionctx.Context, lockCtx *tikvstore.L if err != nil { return err } + + // Skip the temporary table keys. + keys = filterTemporaryTableKeys(sessVars, keys) + var lockKeyStats *tikvutil.LockKeysDetails ctx = context.WithValue(ctx, tikvutil.LockKeysDetailCtxKey, &lockKeyStats) err = txn.LockKeys(tikvutil.SetSessionID(ctx, se.GetSessionVars().ConnectionID), lockCtx, keys...) @@ -1023,6 +1028,22 @@ func doLockKeys(ctx context.Context, se sessionctx.Context, lockCtx *tikvstore.L return err } +func filterTemporaryTableKeys(vars *variable.SessionVars, keys []kv.Key) []kv.Key { + txnCtx := vars.TxnCtx + if txnCtx == nil || txnCtx.GlobalTemporaryTables == nil { + return keys + } + + newKeys := keys[:] + for _, key := range keys { + tblID := tablecodec.DecodeTableID(key) + if _, ok := txnCtx.GlobalTemporaryTables[tblID]; !ok { + newKeys = append(newKeys, key) + } + } + return newKeys +} + // LimitExec represents limit executor // It ignores 'Offset' rows from src, then returns 'Count' rows at maximum. type LimitExec struct { diff --git a/executor/executor_test.go b/executor/executor_test.go index c50459cf2850b..65c60f392f689 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8240,6 +8240,44 @@ func (s *testSerialSuite) TestDeadlockTable(c *C) { )) } +func (s *testSuite1) TestTemporaryTableNoPessimisticLock(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create global temporary table t (a int primary key, b int) on commit delete rows") + tk.MustExec("insert into t values (1, 1)") + + // Do something on the temporary table, pessimistic transaction mode. + tk.MustExec("begin pessimistic") + tk.MustExec("insert into t values (2, 2)") + tk.MustExec("update t set b = b + 1 where a = 1") + tk.MustExec("delete from t where a > 1") + tk.MustQuery("select count(*) from t where b >= 2 for update") + + // Get the temporary table ID. + schema := tk.Se.GetInfoSchema().(infoschema.InfoSchema) + tbl, err := schema.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + meta := tbl.Meta() + c.Assert(meta.TempTableType, Equals, model.TempTableGlobal) + + // Scan the table range to check there is no lock. + // It's better to use the rawkv client, but the txnkv client should also works. + // If there is a lock, the txnkv client should have reported the lock error. + txn, err := s.store.Begin() + c.Assert(err, IsNil) + seekKey := tablecodec.EncodeTablePrefix(meta.ID) + endKey := tablecodec.EncodeTablePrefix(meta.ID + 1) + scanner, err := txn.Iter(seekKey, endKey) + c.Assert(err, IsNil) + for scanner.Valid() { + // No lock written to TiKV here. + c.FailNow() + } + + tk.MustExec("rollback") +} + func (s testSerialSuite) TestExprBlackListForEnum(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 8c5c105922536..3a2489a803cc8 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3846,29 +3846,3 @@ func (s *testIntegrationSerialSuite) TestEnforceMPP(c *C) { " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", " └─TableFullScan_19 10000.00 255020.00 batchCop[tiflash] table:t keep order:false, stats:pseudo")) } - -func (s *testIntegrationSuite) TestEliminateLockForTemporaryTable(c *C) { - tk := testkit.NewTestKit(c, s.store) - - tk.MustExec("use test;") - tk.MustExec("create global temporary table t1 (a int primary key, b int, c int, index i_b(b)) on commit delete rows;") - defer func() { - tk.MustExec("drop global temporary table if exists t1;") - }() - tk.MustExec("begin;") - tk.MustExec("insert t1 values (8,8,9);") - - var input []string - var output []struct { - SQL string - Plan []string - } - s.testData.GetTestCases(c, &input, &output) - for i, tt := range input { - s.testData.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows()) - }) - tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...)) - } -} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 0c1c4a668d3c8..59c228767171a 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" - "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" @@ -187,7 +186,6 @@ func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan { plan = InjectExtraProjection(plan) mergeContinuousSelections(plan) plan = eliminateUnionScanAndLock(sctx, plan) - plan = eliminateLockForTemporaryTable(plan) plan = enableParallelApply(sctx, plan) return plan } @@ -324,29 +322,6 @@ func eliminateUnionScanAndLock(sctx sessionctx.Context, p PhysicalPlan) Physical }) } -// eliminateLockForTemporaryTable eliminates lock for the temporary table. -func eliminateLockForTemporaryTable(p PhysicalPlan) PhysicalPlan { - iteratePhysicalPlan(p, func(p PhysicalPlan) bool { - if len(p.Children()) > 1 { - return false - } - switch x := p.(type) { - case *PointGetPlan: - if x.TblInfo.TempTableType != model.TempTableNone { - x.Lock = false - x.LockWaitTime = 0 - } - case *BatchPointGetPlan: - if x.TblInfo.TempTableType != model.TempTableNone { - x.Lock = false - x.LockWaitTime = 0 - } - } - return true - }) - return p -} - func iteratePhysicalPlan(p PhysicalPlan, f func(p PhysicalPlan) bool) { if !f(p) { return diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 2876533a250c8..ba780bdebde0f 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -463,10 +463,7 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) { if tidbutil.IsMemDB(fp.dbName) { return nil } - // ignore lock for temporary table. - if fp.TblInfo.TempTableType == model.TempTableNone { - fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) - } + fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) p = fp return } @@ -484,10 +481,7 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) { p = tableDual.Init(ctx, &property.StatsInfo{}, 0) return } - // ignore lock for temporary table. - if fp.TblInfo.TempTableType == model.TempTableNone { - fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) - } + fp.Lock, fp.LockWaitTime = getLockWaitTime(ctx, x.LockInfo) p = fp return } diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index f386f8d7f24e8..087b32110e18f 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -294,14 +294,5 @@ "select sum(1) from s1", "select count(1) as cnt from s1 union select count(1) as cnt from s2" ] - }, - { - "name": "TestEliminateLockForTemporaryTable", - "cases": [ - "select * from t1 where a = 2 for update", - "select * from t1 where a in (1,2) for update", - "select c + 1 from t1 where a = 2 and c = 2 for update", - "select c + 1 from t1 where a in (1,2) and c = 2 for update" - ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 902ff19276cc0..7c735fcb5657c 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1564,38 +1564,5 @@ ] } ] - }, - { - "Name": "TestEliminateLockForTemporaryTable", - "Cases": [ - { - "SQL": "select * from t1 where a = 2 for update", - "Plan": [ - "Point_Get 1.00 root table:t1 handle:2" - ] - }, - { - "SQL": "select * from t1 where a in (1,2) for update", - "Plan": [ - "Batch_Point_Get 2.00 root table:t1 handle:[1 2], keep order:false, desc:false" - ] - }, - { - "SQL": "select c + 1 from t1 where a = 2 and c = 2 for update", - "Plan": [ - "Projection 0.00 root plus(test.t1.c, 1)->Column#4", - "└─Selection 0.00 root eq(test.t1.c, 2)", - " └─Point_Get 1.00 root table:t1 handle:2" - ] - }, - { - "SQL": "select c + 1 from t1 where a in (1,2) and c = 2 for update", - "Plan": [ - "Projection 0.00 root plus(test.t1.c, 1)->Column#4", - "└─Selection 0.00 root eq(test.t1.c, 2)", - " └─Batch_Point_Get 2.00 root table:t1 handle:[1 2], keep order:false, desc:false" - ] - } - ] } ] From 5368be10432b3cd15de58c60c8bf3ea6d99e46c3 Mon Sep 17 00:00:00 2001 From: Shirly Date: Wed, 26 May 2021 13:27:34 +0800 Subject: [PATCH 181/343] store/tikv/backoff: use *Observer instead of Observer directly in Config (#24897) --- store/tikv/retry/backoff.go | 6 ++++-- store/tikv/retry/config.go | 28 ++++++++++++++-------------- 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go index a005f85bf698b..d57ab62a210ad 100644 --- a/store/tikv/retry/backoff.go +++ b/store/tikv/retry/backoff.go @@ -144,7 +144,7 @@ func (b *Backoffer) BackoffWithMaxSleep(typ int, maxSleepMs int, err error) erro case boMaxTsNotSynced: return b.BackoffWithCfgAndMaxSleep(BoMaxTsNotSynced, maxSleepMs, err) } - cfg := NewConfig("", metrics.BackoffHistogramEmpty, nil, tikverr.ErrUnknown) + cfg := NewConfig("", &metrics.BackoffHistogramEmpty, nil, tikverr.ErrUnknown) return b.BackoffWithCfgAndMaxSleep(cfg, maxSleepMs, err) } @@ -185,7 +185,9 @@ func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err e b.fn[cfg.name] = f } realSleep := f(b.ctx, maxSleepMs) - cfg.metric.Observe(float64(realSleep) / 1000) + if cfg.metric != nil { + (*cfg.metric).Observe(float64(realSleep) / 1000) + } b.totalSleep += realSleep if b.backoffSleepMS == nil { b.backoffSleepMS = make(map[string]int) diff --git a/store/tikv/retry/config.go b/store/tikv/retry/config.go index bd118cabd8028..6eb93eee81366 100644 --- a/store/tikv/retry/config.go +++ b/store/tikv/retry/config.go @@ -31,7 +31,7 @@ import ( // Config is the configuration of the Backoff function. type Config struct { name string - metric prometheus.Observer + metric *prometheus.Observer fnCfg *BackoffFnCfg err error } @@ -65,7 +65,7 @@ func NewBackoffFnCfg(base, cap, jitter int) *BackoffFnCfg { } // NewConfig creates a new Config for the Backoff operation. -func NewConfig(name string, metric prometheus.Observer, backoffFnCfg *BackoffFnCfg, err error) *Config { +func NewConfig(name string, metric *prometheus.Observer, backoffFnCfg *BackoffFnCfg, err error) *Config { return &Config{ name: name, metric: metric, @@ -80,22 +80,22 @@ func (c *Config) String() string { const txnLockFastName = "txnLockFast" -// Backoff Config samples. +// Backoff Config variables. var ( // TODO: distinguish tikv and tiflash in metrics - BoTiKVRPC = NewConfig("tikvRPC", metrics.BackoffHistogramRPC, NewBackoffFnCfg(100, 2000, EqualJitter), tikverr.ErrTiKVServerTimeout) - BoTiFlashRPC = NewConfig("tiflashRPC", metrics.BackoffHistogramRPC, NewBackoffFnCfg(100, 2000, EqualJitter), tikverr.ErrTiFlashServerTimeout) - BoTxnLock = NewConfig("txnLock", metrics.BackoffHistogramLock, NewBackoffFnCfg(200, 3000, EqualJitter), tikverr.ErrResolveLockTimeout) - BoPDRPC = NewConfig("pdRPC", metrics.BackoffHistogramPD, NewBackoffFnCfg(500, 3000, EqualJitter), tikverr.NewErrPDServerTimeout("")) + BoTiKVRPC = NewConfig("tikvRPC", &metrics.BackoffHistogramRPC, NewBackoffFnCfg(100, 2000, EqualJitter), tikverr.ErrTiKVServerTimeout) + BoTiFlashRPC = NewConfig("tiflashRPC", &metrics.BackoffHistogramRPC, NewBackoffFnCfg(100, 2000, EqualJitter), tikverr.ErrTiFlashServerTimeout) + BoTxnLock = NewConfig("txnLock", &metrics.BackoffHistogramLock, NewBackoffFnCfg(200, 3000, EqualJitter), tikverr.ErrResolveLockTimeout) + BoPDRPC = NewConfig("pdRPC", &metrics.BackoffHistogramPD, NewBackoffFnCfg(500, 3000, EqualJitter), tikverr.NewErrPDServerTimeout("")) // change base time to 2ms, because it may recover soon. - BoRegionMiss = NewConfig("regionMiss", metrics.BackoffHistogramRegionMiss, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrRegionUnavailable) - BoTiKVServerBusy = NewConfig("tikvServerBusy", metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiKVServerBusy) - BoTiFlashServerBusy = NewConfig("tiflashServerBusy", metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiFlashServerBusy) - BoTxnNotFound = NewConfig("txnNotFound", metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrResolveLockTimeout) - BoStaleCmd = NewConfig("staleCommand", metrics.BackoffHistogramStaleCmd, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrTiKVStaleCommand) - BoMaxTsNotSynced = NewConfig("maxTsNotSynced", metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrTiKVMaxTimestampNotSynced) + BoRegionMiss = NewConfig("regionMiss", &metrics.BackoffHistogramRegionMiss, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrRegionUnavailable) + BoTiKVServerBusy = NewConfig("tikvServerBusy", &metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiKVServerBusy) + BoTiFlashServerBusy = NewConfig("tiflashServerBusy", &metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiFlashServerBusy) + BoTxnNotFound = NewConfig("txnNotFound", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrResolveLockTimeout) + BoStaleCmd = NewConfig("staleCommand", &metrics.BackoffHistogramStaleCmd, NewBackoffFnCfg(2, 1000, NoJitter), tikverr.ErrTiKVStaleCommand) + BoMaxTsNotSynced = NewConfig("maxTsNotSynced", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrTiKVMaxTimestampNotSynced) // TxnLockFast's `base` load from vars.BackoffLockFast when create BackoffFn. - BoTxnLockFast = NewConfig(txnLockFastName, metrics.BackoffHistogramLockFast, NewBackoffFnCfg(2, 3000, EqualJitter), tikverr.ErrResolveLockTimeout) + BoTxnLockFast = NewConfig(txnLockFastName, &metrics.BackoffHistogramLockFast, NewBackoffFnCfg(2, 3000, EqualJitter), tikverr.ErrResolveLockTimeout) ) const ( From 59ee6f8d01e10306b8311e492c6b8383dc11b5d5 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 26 May 2021 14:19:34 +0800 Subject: [PATCH 182/343] executor: add more cases about dynamic-mode with union/DML/subquery (#24864) --- executor/partition_table_test.go | 178 +++++++++++++++++++++++++++++++ 1 file changed, 178 insertions(+) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index c06289da77988..fab6f9755ddea 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -1560,6 +1560,184 @@ func (s *partitionTableSuite) TestAddDropPartitions(c *C) { tk.MustPartition(`select * from t where a < 20`, "p1,p2,p3").Sort().Check(testkit.Rows("12", "15", "7")) } +func (s *partitionTableSuite) TestDML(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_DML") + defer tk.MustExec(`drop database test_DML`) + tk.MustExec("use test_DML") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table tinner (a int primary key, b int)`) + tk.MustExec(`create table thash (a int primary key, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int primary key, b int) partition by range(a) ( + partition p0 values less than(10000), + partition p1 values less than(20000), + partition p2 values less than(30000), + partition p3 values less than(40000), + partition p4 values less than MAXVALUE)`) + + vals := make([]string, 0, 50) + for i := 0; i < 50; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into tinner values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into trange values ` + strings.Join(vals, ", ")) + + // delete, insert, replace, update + for i := 0; i < 200; i++ { + var pattern string + switch rand.Intn(4) { + case 0: // delete + col := []string{"a", "b"}[rand.Intn(2)] + l := rand.Intn(40000) + r := l + rand.Intn(5000) + pattern = fmt.Sprintf(`delete from %%v where %v>%v and %v<%v`, col, l, col, r) + case 1: // insert + a, b := rand.Intn(40000), rand.Intn(40000) + pattern = fmt.Sprintf(`insert into %%v values (%v, %v)`, a, b) + case 2: // replace + a, b := rand.Intn(40000), rand.Intn(40000) + pattern = fmt.Sprintf(`replace into %%v(a, b) values (%v, %v)`, a, b) + case 3: // update + col := []string{"a", "b"}[rand.Intn(2)] + l := rand.Intn(40000) + r := l + rand.Intn(5000) + x := rand.Intn(1000) - 500 + pattern = fmt.Sprintf(`update %%v set %v=%v+%v where %v>%v and %v<%v`, col, col, x, col, l, col, r) + } + for _, tbl := range []string{"tinner", "thash", "trange"} { + tk.MustExec(fmt.Sprintf(pattern, tbl)) + } + + // check + r := tk.MustQuery(`select * from tinner`).Sort().Rows() + tk.MustQuery(`select * from thash`).Sort().Check(r) + tk.MustQuery(`select * from trange`).Sort().Check(r) + } +} + +func (s *partitionTableSuite) TestUnion(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_union") + defer tk.MustExec(`drop database test_union`) + tk.MustExec("use test_union") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table t(a int, b int, key(a))`) + tk.MustExec(`create table thash (a int, b int, key(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, key(a)) partition by range(a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + + vals := make([]string, 0, 1000) + for i := 0; i < 1000; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into t values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into trange values ` + strings.Join(vals, ", ")) + + randRange := func() (int, int) { + l, r := rand.Intn(40000), rand.Intn(40000) + if l > r { + l, r = r, l + } + return l, r + } + + for i := 0; i < 100; i++ { + a1l, a1r := randRange() + a2l, a2r := randRange() + b1l, b1r := randRange() + b2l, b2r := randRange() + for _, utype := range []string{"union all", "union distinct"} { + pattern := fmt.Sprintf(`select * from %%v where a>=%v and a<=%v and b>=%v and b<=%v + %v select * from %%v where a>=%v and a<=%v and b>=%v and b<=%v`, a1l, a1r, b1l, b1r, utype, a2l, a2r, b2l, b2r) + r := tk.MustQuery(fmt.Sprintf(pattern, "t", "t")).Sort().Rows() + tk.MustQuery(fmt.Sprintf(pattern, "thash", "thash")).Sort().Check(r) // hash + hash + tk.MustQuery(fmt.Sprintf(pattern, "trange", "trange")).Sort().Check(r) // range + range + tk.MustQuery(fmt.Sprintf(pattern, "trange", "thash")).Sort().Check(r) // range + hash + } + } +} + +func (s *partitionTableSuite) TestSubqueries(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_subquery") + defer tk.MustExec(`drop database test_subquery`) + tk.MustExec("use test_subquery") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table touter (a int, b int, index(a))`) + tk.MustExec(`create table tinner (a int, b int, c int, index(a))`) + tk.MustExec(`create table thash (a int, b int, c int, index(a)) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int, c int, index(a)) partition by range(a) ( + partition p0 values less than(10000), + partition p1 values less than(20000), + partition p2 values less than(30000), + partition p3 values less than(40000))`) + + outerVals := make([]string, 0, 100) + for i := 0; i < 100; i++ { + outerVals = append(outerVals, fmt.Sprintf(`(%v, %v)`, rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into touter values ` + strings.Join(outerVals, ", ")) + vals := make([]string, 0, 2000) + for i := 0; i < 2000; i++ { + vals = append(vals, fmt.Sprintf(`(%v, %v, %v)`, rand.Intn(40000), rand.Intn(40000), rand.Intn(40000))) + } + tk.MustExec(`insert into tinner values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into thash values ` + strings.Join(vals, ", ")) + tk.MustExec(`insert into trange values ` + strings.Join(vals, ", ")) + + // in + for i := 0; i < 50; i++ { + for _, op := range []string{"in", "not in"} { + x := rand.Intn(40000) + var r [][]interface{} + for _, t := range []string{"tinner", "thash", "trange"} { + q := fmt.Sprintf(`select * from touter where touter.a %v (select %v.b from %v where %v.a > touter.b and %v.c > %v)`, op, t, t, t, t, x) + if r == nil { + r = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Sort().Check(r) + } + } + } + } + + // exist + for i := 0; i < 50; i++ { + for _, op := range []string{"exists", "not exists"} { + x := rand.Intn(40000) + var r [][]interface{} + for _, t := range []string{"tinner", "thash", "trange"} { + q := fmt.Sprintf(`select * from touter where %v (select %v.b from %v where %v.a > touter.b and %v.c > %v)`, op, t, t, t, t, x) + if r == nil { + r = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Sort().Check(r) + } + } + } + } +} + func (s *partitionTableSuite) TestSplitRegion(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") From 88928adb0fd7342446799dacf704a24fb205fb83 Mon Sep 17 00:00:00 2001 From: tangenta Date: Wed, 26 May 2021 15:15:36 +0800 Subject: [PATCH 183/343] ddl, types: fix int to datetime conversion to avoid useless retry (#24874) --- ddl/column.go | 30 ++++++++++-------------------- ddl/column_type_change_test.go | 21 +++++++++++++++++++++ types/datum.go | 8 ++++++++ types/time.go | 20 +++++++++++--------- 4 files changed, 50 insertions(+), 29 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index e18c0c2d37e7b..17abe6aefbe08 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -40,7 +40,6 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" decoder "github.com/pingcap/tidb/util/rowDecoder" @@ -1023,19 +1022,18 @@ func (w *worker) doModifyColumnTypeWithData( // If timeout, we should return, check for the owner and re-wait job done. return ver, nil } - if needRollbackData(err) { - if err1 := t.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil { - logutil.BgLogger().Warn("[ddl] run modify column job failed, RemoveDDLReorgHandle failed, can't convert job to rollback", - zap.String("job", job.String()), zap.Error(err1)) - return ver, errors.Trace(err) - } - logutil.BgLogger().Warn("[ddl] run modify column job failed, convert job to rollback", zap.String("job", job.String()), zap.Error(err)) - // When encounter these error above, we change the job to rolling back job directly. - job.State = model.JobStateRollingback + if kv.IsTxnRetryableError(err) { + // Clean up the channel of notifyCancelReorgJob. Make sure it can't affect other jobs. + w.reorgCtx.cleanNotifyReorgCancel() + return ver, errors.Trace(err) + } + if err1 := t.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil { + logutil.BgLogger().Warn("[ddl] run modify column job failed, RemoveDDLReorgHandle failed, can't convert job to rollback", + zap.String("job", job.String()), zap.Error(err1)) return ver, errors.Trace(err) } - // Clean up the channel of notifyCancelReorgJob. Make sure it can't affect other jobs. - w.reorgCtx.cleanNotifyReorgCancel() + logutil.BgLogger().Warn("[ddl] run modify column job failed, convert job to rollback", zap.String("job", job.String()), zap.Error(err)) + job.State = model.JobStateRollingback return ver, errors.Trace(err) } // Clean up the channel of notifyCancelReorgJob. Make sure it can't affect other jobs. @@ -1082,14 +1080,6 @@ func (w *worker) doModifyColumnTypeWithData( return ver, errors.Trace(err) } -// needRollbackData indicates whether it needs to rollback data when specific error occurs. -func needRollbackData(err error) bool { - return kv.ErrKeyExists.Equal(err) || errCancelledDDLJob.Equal(err) || errCantDecodeRecord.Equal(err) || - types.ErrOverflow.Equal(err) || types.ErrDataTooLong.Equal(err) || types.ErrTruncated.Equal(err) || - json.ErrInvalidJSONText.Equal(err) || types.ErrBadNumber.Equal(err) || types.ErrInvalidYear.Equal(err) || - types.ErrWrongValue.Equal(err) -} - // BuildElements is exported for testing. func BuildElements(changingCol *model.ColumnInfo, changingIdxs []*model.IndexInfo) []*meta.Element { elements := make([]*meta.Element, 0, len(changingIdxs)+1) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 992631b4bd97b..0b6234a21d924 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -1716,6 +1716,27 @@ func (s *testColumnTypeChangeSuite) TestAlterPrimaryKeyToNull(c *C) { tk.MustGetErrCode("alter table t change column a a bigint null;", mysql.ErrPrimaryCantHaveNull) } +// Close https://github.com/pingcap/tidb/issues/24839. +func (s testColumnTypeChangeSuite) TestChangeUnsignedIntToDatetime(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.Se.GetSessionVars().EnableChangeColumnType = true + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int(10) unsigned default null, b bigint unsigned, c tinyint unsigned);") + tk.MustExec("insert into t values (1, 1, 1);") + tk.MustGetErrCode("alter table t modify column a datetime;", mysql.ErrTruncatedWrongValue) + tk.MustGetErrCode("alter table t modify column b datetime;", mysql.ErrTruncatedWrongValue) + tk.MustGetErrCode("alter table t modify column c datetime;", mysql.ErrTruncatedWrongValue) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int(10) unsigned default null, b bigint unsigned, c tinyint unsigned);") + tk.MustExec("insert into t values (4294967295, 18446744073709551615, 255);") + tk.MustGetErrCode("alter table t modify column a datetime;", mysql.ErrTruncatedWrongValue) + tk.MustGetErrCode("alter table t modify column b datetime;", mysql.ErrTruncatedWrongValue) + tk.MustGetErrCode("alter table t modify column c datetime;", mysql.ErrTruncatedWrongValue) +} + // Close issue #23202 func (s *testColumnTypeChangeSuite) TestDDLExitWhenCancelMeetPanic(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/types/datum.go b/types/datum.go index 195f1df36211d..a1f8c33416b4c 100644 --- a/types/datum.go +++ b/types/datum.go @@ -1182,6 +1182,14 @@ func (d *Datum) convertToMysqlTime(sc *stmtctx.StatementContext, target *FieldTy t, err = ParseTime(sc, d.GetString(), tp, fsp) case KindInt64: t, err = ParseTimeFromNum(sc, d.GetInt64(), tp, fsp) + case KindUint64: + intOverflow64 := d.GetInt64() < 0 + if intOverflow64 { + uNum := strconv.FormatUint(d.GetUint64(), 10) + t, err = ZeroDate, ErrWrongValue.GenWithStackByArgs(TimeStr, uNum) + } else { + t, err = ParseTimeFromNum(sc, d.GetInt64(), tp, fsp) + } case KindMysqlJSON: j := d.GetMysqlJSON() var s string diff --git a/types/time.go b/types/time.go index cae6123dd1f24..28a34b0284833 100644 --- a/types/time.go +++ b/types/time.go @@ -1799,7 +1799,7 @@ func splitDuration(t gotime.Duration) (int, int, int, int, int) { var maxDaysInMonth = []int{31, 29, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31} -func getTime(sc *stmtctx.StatementContext, num int64, tp byte) (Time, error) { +func getTime(sc *stmtctx.StatementContext, num, originNum int64, tp byte) (Time, error) { s1 := num / 1000000 s2 := num - s1*1000000 @@ -1815,7 +1815,8 @@ func getTime(sc *stmtctx.StatementContext, num int64, tp byte) (Time, error) { ct, ok := FromDateChecked(year, month, day, hour, minute, second, 0) if !ok { - return ZeroDatetime, errors.Trace(ErrWrongValue.GenWithStackByArgs(TimeStr, "")) + numStr := strconv.FormatInt(originNum, 10) + return ZeroDatetime, errors.Trace(ErrWrongValue.GenWithStackByArgs(TimeStr, numStr)) } t := NewTime(ct, tp, DefaultFsp) err := t.check(sc) @@ -1831,11 +1832,12 @@ func parseDateTimeFromNum(sc *stmtctx.StatementContext, num int64) (Time, error) if num == 0 { return t, nil } + originNum := num // Check datetime type. if num >= 10000101000000 { t.SetType(mysql.TypeDatetime) - return getTime(sc, num, t.Type()) + return getTime(sc, num, originNum, t.Type()) } // Check MMDD. @@ -1847,7 +1849,7 @@ func parseDateTimeFromNum(sc *stmtctx.StatementContext, num int64) (Time, error) // YYMMDD, year: 2000-2069 if num <= (70-1)*10000+1231 { num = (num + 20000000) * 1000000 - return getTime(sc, num, t.Type()) + return getTime(sc, num, originNum, t.Type()) } // Check YYMMDD. @@ -1859,13 +1861,13 @@ func parseDateTimeFromNum(sc *stmtctx.StatementContext, num int64) (Time, error) // YYMMDD, year: 1970-1999 if num <= 991231 { num = (num + 19000000) * 1000000 - return getTime(sc, num, t.Type()) + return getTime(sc, num, originNum, t.Type()) } // Adjust hour/min/second. if num <= 99991231 { num = num * 1000000 - return getTime(sc, num, t.Type()) + return getTime(sc, num, originNum, t.Type()) } // Check MMDDHHMMSS. @@ -1880,7 +1882,7 @@ func parseDateTimeFromNum(sc *stmtctx.StatementContext, num int64) (Time, error) // YYMMDDHHMMSS, 2000-2069 if num <= 69*10000000000+1231235959 { num = num + 20000000000000 - return getTime(sc, num, t.Type()) + return getTime(sc, num, originNum, t.Type()) } // Check YYYYMMDDHHMMSS. @@ -1892,10 +1894,10 @@ func parseDateTimeFromNum(sc *stmtctx.StatementContext, num int64) (Time, error) // YYMMDDHHMMSS, 1970-1999 if num <= 991231235959 { num = num + 19000000000000 - return getTime(sc, num, t.Type()) + return getTime(sc, num, originNum, t.Type()) } - return getTime(sc, num, t.Type()) + return getTime(sc, num, originNum, t.Type()) } // ParseTime parses a formatted string with type tp and specific fsp. From acf5e52ffc782b1919d519624bfdeaeeafb80025 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Wed, 26 May 2021 15:31:35 +0800 Subject: [PATCH 184/343] session, executor: fix misleading function name and move staleness test (#24902) --- executor/simple.go | 4 +- executor/stale_txn_test.go | 198 +++++++++++++++++++++++++++++++++++++ session/session.go | 4 +- session/session_test.go | 198 ------------------------------------- sessionctx/context.go | 4 +- util/mock/context.go | 4 +- 6 files changed, 206 insertions(+), 206 deletions(-) diff --git a/executor/simple.go b/executor/simple.go index 0eadfd5f85dd2..ec32f17150ebf 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -578,7 +578,7 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { if e.ctx.GetSessionVars().TxnReadTS > 0 { return errors.New("start transaction read only as of is forbidden after set transaction read only as of") } - if err := e.ctx.NewTxnWithStartTS(ctx, e.staleTxnStartTS); err != nil { + if err := e.ctx.NewStaleTxnWithStartTS(ctx, e.staleTxnStartTS); err != nil { return err } // With START TRANSACTION, autocommit remains disabled until you end @@ -593,7 +593,7 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { startTS := e.ctx.GetSessionVars().TxnReadTS // clear TxnReadTS after we used it. e.ctx.GetSessionVars().TxnReadTS = 0 - if err := e.ctx.NewTxnWithStartTS(ctx, startTS); err != nil { + if err := e.ctx.NewStaleTxnWithStartTS(ctx, startTS); err != nil { return err } e.ctx.GetSessionVars().SetInTxn(true) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index db4705f428d14..a36dd6073654e 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -290,3 +290,201 @@ func (s *testStaleTxnSerialSuite) TestSetTransactionReadOnlyAsOf(c *C) { c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "start transaction read only as of is forbidden after set transaction read only as of") } + +func (s *testStaleTxnSerialSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { + testcases := []struct { + name string + sql string + isValidate bool + }{ + { + name: "select statement", + sql: `select * from t;`, + isValidate: true, + }, + { + name: "explain statement", + sql: `explain insert into t (id) values (1);`, + isValidate: true, + }, + { + name: "explain analyze insert statement", + sql: `explain analyze insert into t (id) values (1);`, + isValidate: false, + }, + { + name: "explain analyze select statement", + sql: `explain analyze select * from t `, + isValidate: true, + }, + { + name: "execute insert statement", + sql: `EXECUTE stmt1;`, + isValidate: false, + }, + { + name: "execute select statement", + sql: `EXECUTE stmt2;`, + isValidate: true, + }, + { + name: "show statement", + sql: `show tables;`, + isValidate: true, + }, + { + name: "set union", + sql: `SELECT 1, 2 UNION SELECT 'a', 'b';`, + isValidate: true, + }, + { + name: "insert", + sql: `insert into t (id) values (1);`, + isValidate: false, + }, + { + name: "delete", + sql: `delete from t where id =1`, + isValidate: false, + }, + { + name: "update", + sql: "update t set id =2 where id =1", + isValidate: false, + }, + { + name: "point get", + sql: `select * from t where id = 1`, + isValidate: true, + }, + { + name: "batch point get", + sql: `select * from t where id in (1,2,3);`, + isValidate: true, + }, + { + name: "split table", + sql: `SPLIT TABLE t BETWEEN (0) AND (1000000000) REGIONS 16;`, + isValidate: true, + }, + { + name: "do statement", + sql: `DO SLEEP(1);`, + isValidate: true, + }, + { + name: "select for update", + sql: "select * from t where id = 1 for update", + isValidate: false, + }, + { + name: "select lock in share mode", + sql: "select * from t where id = 1 lock in share mode", + isValidate: true, + }, + { + name: "select for update union statement", + sql: "select * from t for update union select * from t;", + isValidate: false, + }, + { + name: "replace statement", + sql: "replace into t(id) values (1)", + isValidate: false, + }, + { + name: "load data statement", + sql: "LOAD DATA LOCAL INFILE '/mn/asa.csv' INTO TABLE t FIELDS TERMINATED BY x'2c' ENCLOSED BY b'100010' LINES TERMINATED BY '\r\n' IGNORE 1 LINES (id);", + isValidate: false, + }, + { + name: "update multi tables", + sql: "update t,t1 set t.id = 1,t1.id = 2 where t.1 = 2 and t1.id = 3;", + isValidate: false, + }, + { + name: "delete multi tables", + sql: "delete t from t1 where t.id = t1.id", + isValidate: false, + }, + { + name: "insert select", + sql: "insert into t select * from t1;", + isValidate: false, + }, + } + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("create table t (id int);") + tk.MustExec("create table t1 (id int);") + tk.MustExec(`PREPARE stmt1 FROM 'insert into t(id) values (5);';`) + tk.MustExec(`PREPARE stmt2 FROM 'select * from t';`) + tk.MustExec(`set @@tidb_enable_noop_functions=1;`) + for _, testcase := range testcases { + c.Log(testcase.name) + tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) + if testcase.isValidate { + _, err := tk.Exec(testcase.sql) + c.Assert(err, IsNil) + tk.MustExec("commit") + } else { + err := tk.ExecToErr(testcase.sql) + c.Assert(err, NotNil) + c.Assert(err.Error(), Matches, `.*only support read-only statement during read-only staleness transactions.*`) + } + } +} + +func (s *testStaleTxnSerialSuite) TestSpecialSQLInStalenessTxn(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + testcases := []struct { + name string + sql string + sameSession bool + }{ + { + name: "ddl", + sql: "create table t (id int, b int,INDEX(b));", + sameSession: false, + }, + { + name: "set global session", + sql: `SET GLOBAL sql_mode = 'STRICT_TRANS_TABLES,NO_AUTO_CREATE_USER';`, + sameSession: true, + }, + { + name: "analyze table", + sql: "analyze table t", + sameSession: true, + }, + { + name: "session binding", + sql: "CREATE SESSION BINDING FOR SELECT * FROM t WHERE b = 123 USING SELECT * FROM t IGNORE INDEX (b) WHERE b = 123;", + sameSession: true, + }, + { + name: "global binding", + sql: "CREATE GLOBAL BINDING FOR SELECT * FROM t WHERE b = 123 USING SELECT * FROM t IGNORE INDEX (b) WHERE b = 123;", + sameSession: true, + }, + { + name: "grant statements", + sql: "GRANT ALL ON test.* TO 'newuser';", + sameSession: false, + }, + { + name: "revoke statements", + sql: "REVOKE ALL ON test.* FROM 'newuser';", + sameSession: false, + }, + } + tk.MustExec("CREATE USER 'newuser' IDENTIFIED BY 'mypassword';") + for _, testcase := range testcases { + comment := Commentf(testcase.name) + tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) + c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, true, comment) + tk.MustExec(testcase.sql) + c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.sameSession, comment) + } +} diff --git a/session/session.go b/session/session.go index acbdc58fcbf75..bed1e6e2f7ae3 100644 --- a/session/session.go +++ b/session/session.go @@ -2003,8 +2003,8 @@ func (s *session) checkBeforeNewTxn(ctx context.Context) error { return nil } -// NewTxnWithStartTS create a transaction with the given StartTS. -func (s *session) NewTxnWithStartTS(ctx context.Context, startTS uint64) error { +// NewStaleTxnWithStartTS create a transaction with the given StartTS. +func (s *session) NewStaleTxnWithStartTS(ctx context.Context, startTS uint64) error { if err := s.checkBeforeNewTxn(ctx); err != nil { return err } diff --git a/session/session_test.go b/session/session_test.go index b8d8538193c7d..88f6a48b83a45 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -3978,204 +3978,6 @@ func (s *testSessionSerialSuite) TestIssue21943(c *C) { c.Assert(err.Error(), Equals, "[variable:1238]Variable 'last_plan_from_cache' is a read only variable") } -func (s *testSessionSerialSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { - testcases := []struct { - name string - sql string - isValidate bool - }{ - { - name: "select statement", - sql: `select * from t;`, - isValidate: true, - }, - { - name: "explain statement", - sql: `explain insert into t (id) values (1);`, - isValidate: true, - }, - { - name: "explain analyze insert statement", - sql: `explain analyze insert into t (id) values (1);`, - isValidate: false, - }, - { - name: "explain analyze select statement", - sql: `explain analyze select * from t `, - isValidate: true, - }, - { - name: "execute insert statement", - sql: `EXECUTE stmt1;`, - isValidate: false, - }, - { - name: "execute select statement", - sql: `EXECUTE stmt2;`, - isValidate: true, - }, - { - name: "show statement", - sql: `show tables;`, - isValidate: true, - }, - { - name: "set union", - sql: `SELECT 1, 2 UNION SELECT 'a', 'b';`, - isValidate: true, - }, - { - name: "insert", - sql: `insert into t (id) values (1);`, - isValidate: false, - }, - { - name: "delete", - sql: `delete from t where id =1`, - isValidate: false, - }, - { - name: "update", - sql: "update t set id =2 where id =1", - isValidate: false, - }, - { - name: "point get", - sql: `select * from t where id = 1`, - isValidate: true, - }, - { - name: "batch point get", - sql: `select * from t where id in (1,2,3);`, - isValidate: true, - }, - { - name: "split table", - sql: `SPLIT TABLE t BETWEEN (0) AND (1000000000) REGIONS 16;`, - isValidate: true, - }, - { - name: "do statement", - sql: `DO SLEEP(1);`, - isValidate: true, - }, - { - name: "select for update", - sql: "select * from t where id = 1 for update", - isValidate: false, - }, - { - name: "select lock in share mode", - sql: "select * from t where id = 1 lock in share mode", - isValidate: true, - }, - { - name: "select for update union statement", - sql: "select * from t for update union select * from t;", - isValidate: false, - }, - { - name: "replace statement", - sql: "replace into t(id) values (1)", - isValidate: false, - }, - { - name: "load data statement", - sql: "LOAD DATA LOCAL INFILE '/mn/asa.csv' INTO TABLE t FIELDS TERMINATED BY x'2c' ENCLOSED BY b'100010' LINES TERMINATED BY '\r\n' IGNORE 1 LINES (id);", - isValidate: false, - }, - { - name: "update multi tables", - sql: "update t,t1 set t.id = 1,t1.id = 2 where t.1 = 2 and t1.id = 3;", - isValidate: false, - }, - { - name: "delete multi tables", - sql: "delete t from t1 where t.id = t1.id", - isValidate: false, - }, - { - name: "insert select", - sql: "insert into t select * from t1;", - isValidate: false, - }, - } - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("create table t (id int);") - tk.MustExec("create table t1 (id int);") - tk.MustExec(`PREPARE stmt1 FROM 'insert into t(id) values (5);';`) - tk.MustExec(`PREPARE stmt2 FROM 'select * from t';`) - tk.MustExec(`set @@tidb_enable_noop_functions=1;`) - for _, testcase := range testcases { - c.Log(testcase.name) - tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) - if testcase.isValidate { - _, err := tk.Exec(testcase.sql) - c.Assert(err, IsNil) - tk.MustExec("commit") - } else { - err := tk.ExecToErr(testcase.sql) - c.Assert(err, NotNil) - c.Assert(err.Error(), Matches, `.*only support read-only statement during read-only staleness transactions.*`) - } - } -} - -func (s *testSessionSerialSuite) TestSpecialSQLInStalenessTxn(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - testcases := []struct { - name string - sql string - sameSession bool - }{ - { - name: "ddl", - sql: "create table t (id int, b int,INDEX(b));", - sameSession: false, - }, - { - name: "set global session", - sql: `SET GLOBAL sql_mode = 'STRICT_TRANS_TABLES,NO_AUTO_CREATE_USER';`, - sameSession: true, - }, - { - name: "analyze table", - sql: "analyze table t", - sameSession: true, - }, - { - name: "session binding", - sql: "CREATE SESSION BINDING FOR SELECT * FROM t WHERE b = 123 USING SELECT * FROM t IGNORE INDEX (b) WHERE b = 123;", - sameSession: true, - }, - { - name: "global binding", - sql: "CREATE GLOBAL BINDING FOR SELECT * FROM t WHERE b = 123 USING SELECT * FROM t IGNORE INDEX (b) WHERE b = 123;", - sameSession: true, - }, - { - name: "grant statements", - sql: "GRANT ALL ON test.* TO 'newuser';", - sameSession: false, - }, - { - name: "revoke statements", - sql: "REVOKE ALL ON test.* FROM 'newuser';", - sameSession: false, - }, - } - tk.MustExec("CREATE USER 'newuser' IDENTIFIED BY 'mypassword';") - for _, testcase := range testcases { - comment := Commentf(testcase.name) - tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) - c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, true, comment) - tk.MustExec(testcase.sql) - c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.sameSession, comment) - } -} - func (s *testSessionSerialSuite) TestRemovedSysVars(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) diff --git a/sessionctx/context.go b/sessionctx/context.go index 96ede63f901ae..bd568c7e85f4b 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -41,8 +41,8 @@ type Context interface { // If old transaction is valid, it is committed first. // It's used in BEGIN statement and DDL statements to commit old transaction. NewTxn(context.Context) error - // NewTxnWithStartTS initializes a transaction with the given StartTS. - NewTxnWithStartTS(ctx context.Context, startTS uint64) error + // NewStaleTxnWithStartTS initializes a staleness transaction with the given StartTS. + NewStaleTxnWithStartTS(ctx context.Context, startTS uint64) error // Txn returns the current transaction which is created before executing a statement. // The returned kv.Transaction is not nil, but it maybe pending or invalid. diff --git a/util/mock/context.go b/util/mock/context.go index 9476d7295aa83..0dea6bac0fb92 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -202,8 +202,8 @@ func (c *Context) NewTxn(context.Context) error { return nil } -// NewTxnWithStartTS implements the sessionctx.Context interface. -func (c *Context) NewTxnWithStartTS(ctx context.Context, startTS uint64) error { +// NewStaleTxnWithStartTS implements the sessionctx.Context interface. +func (c *Context) NewStaleTxnWithStartTS(ctx context.Context, startTS uint64) error { return c.NewTxn(ctx) } From 9b5d92ef242733c484faf1531e9acbe7e982e40a Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 26 May 2021 18:47:34 +0800 Subject: [PATCH 185/343] statistics: fix the unexpected estimation error on full sampling (#24623) --- statistics/cmsketch.go | 23 ++++++++--- statistics/handle/bootstrap.go | 2 +- statistics/handle/dump_test.go | 4 +- statistics/handle/handle_test.go | 2 +- statistics/histogram.go | 23 ++++++++--- statistics/selectivity_test.go | 36 +++++++++++++++++ statistics/testdata/stats_suite_in.json | 11 +++++ statistics/testdata/stats_suite_out.json | 51 ++++++++++++++++++++++++ 8 files changed, 138 insertions(+), 14 deletions(-) diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 68791a7a0787f..c510186b16c40 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -386,9 +386,13 @@ func CMSketchToProto(c *CMSketch, topn *TopN) *tipb.CMSketch { // CMSketchAndTopNFromProto converts CMSketch and TopN from its protobuf representation. func CMSketchAndTopNFromProto(protoSketch *tipb.CMSketch) (*CMSketch, *TopN) { - if protoSketch == nil || len(protoSketch.Rows) == 0 { + if protoSketch == nil { return nil, nil } + retTopN := TopNFromProto(protoSketch.TopN) + if len(protoSketch.Rows) == 0 { + return nil, retTopN + } c := NewCMSketch(int32(len(protoSketch.Rows)), int32(len(protoSketch.Rows[0].Counters))) for i, row := range protoSketch.Rows { c.count = 0 @@ -398,14 +402,14 @@ func CMSketchAndTopNFromProto(protoSketch *tipb.CMSketch) (*CMSketch, *TopN) { } } c.defaultValue = protoSketch.DefaultValue - if len(protoSketch.TopN) == 0 { - return c, nil - } - return c, TopNFromProto(protoSketch.TopN) + return c, retTopN } // TopNFromProto converts TopN from its protobuf representation. func TopNFromProto(protoTopN []*tipb.CMSketchTopN) *TopN { + if len(protoTopN) == 0 { + return nil + } topN := NewTopN(32) for _, e := range protoTopN { d := make([]byte, len(e.Data)) @@ -517,6 +521,15 @@ func (c *TopN) String() string { return builder.String() } +// Num returns the ndv of the TopN. +// TopN is declared directly in Histogram. So the Len is occupied by the Histogram. We use Num instead. +func (c *TopN) Num() int { + if c == nil { + return 0 + } + return len(c.TopN) +} + // DecodedString returns the value with decoded result. func (c *TopN) DecodedString(ctx sessionctx.Context, colTypes []byte) (string, error) { builder := &strings.Builder{} diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index 9e9fc7af8cb87..99189221a0444 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -185,7 +185,7 @@ func (h *Handle) initStatsTopN4Chunk(cache *statsCache, iter *chunk.Iterator4Chu continue } idx, ok := table.Indices[row.GetInt64(1)] - if !ok || idx.CMSketch == nil { + if !ok || (idx.CMSketch == nil && idx.StatsVer <= statistics.Version1) { continue } if idx.TopN == nil { diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index f0c7c2fb09110..58411e1669a60 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -287,10 +287,10 @@ func (s *testStatsSuite) TestDumpExtendedStats(c *C) { assertTableEqual(c, loadTblInStorage, tbl) } -func (s *testStatsSuite) TestDumpVer2Stats(c *C) { +func (s *testStatsSuite) TestDumpVer3Stats(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) - tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("set @@tidb_analyze_version = 3") tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b varchar(10))") diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 489be2167acf0..1518503fa73ba 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -523,7 +523,7 @@ func (s *testStatsSuite) TestInitStatsVer2(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.MustExec("set @@session.tidb_analyze_version=2") + tk.MustExec("set @@session.tidb_analyze_version=3") tk.MustExec("create table t(a int, b int, c int, index idx(a), index idxab(a, b))") tk.MustExec("insert into t values(1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (4, 4, 4), (4, 4, 4)") tk.MustExec("analyze table t with 2 topn, 3 buckets") diff --git a/statistics/histogram.go b/statistics/histogram.go index 29bdaed692084..027950b8326c7 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -1504,13 +1504,26 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta } func (idx *Index) outOfRange(val types.Datum) bool { - if idx.Histogram.Len() == 0 { + histEmpty, topNEmpty := idx.Histogram.Len() == 0, idx.TopN.Num() == 0 + // All empty. + if histEmpty && topNEmpty { return true } - withInLowBoundOrPrefixMatch := chunk.Compare(idx.Bounds.GetRow(0), 0, &val) <= 0 || - matchPrefix(idx.Bounds.GetRow(0), 0, &val) - withInHighBound := chunk.Compare(idx.Bounds.GetRow(idx.Bounds.NumRows()-1), 0, &val) >= 0 - return !withInLowBoundOrPrefixMatch || !withInHighBound + // TopN is not empty. Record found. + if !topNEmpty && idx.TopN.findTopN(val.GetBytes()) >= 0 { + return false + } + if !histEmpty { + withInLowBoundOrPrefixMatch := chunk.Compare(idx.Bounds.GetRow(0), 0, &val) <= 0 || + matchPrefix(idx.Bounds.GetRow(0), 0, &val) + withInHighBound := chunk.Compare(idx.Bounds.GetRow(idx.Bounds.NumRows()-1), 0, &val) >= 0 + // Hist is not empty. Record found. + if withInLowBoundOrPrefixMatch && withInHighBound { + return false + } + } + // No record found. Is out of range. + return true } // matchPrefix checks whether ad is the prefix of value diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 32ed9fcb9650b..546ca0b20a4ea 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -641,6 +641,42 @@ func (s *testStatsSuite) TestStatsVer2(c *C) { } } +func (s *testStatsSuite) TestTopNOutOfHist(c *C) { + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("set tidb_analyze_version=3") + + testKit.MustExec("drop table if exists topn_before_hist") + testKit.MustExec("create table topn_before_hist(a int, index idx(a))") + testKit.MustExec("insert into topn_before_hist values(1), (1), (1), (1), (2), (2), (3), (4), (5)") + testKit.MustExec("analyze table topn_before_hist with 2 topn, 3 buckets") + + testKit.MustExec("create table topn_after_hist(a int, index idx(a))") + testKit.MustExec("insert into topn_after_hist values(2), (2), (3), (4), (5), (6), (6), (6), (6)") + testKit.MustExec("analyze table topn_after_hist with 2 topn, 3 buckets") + + testKit.MustExec("create table topn_before_hist_no_index(a int)") + testKit.MustExec("insert into topn_before_hist_no_index values(1), (1), (1), (1), (2), (2), (3), (4), (5)") + testKit.MustExec("analyze table topn_before_hist_no_index with 2 topn, 3 buckets") + + testKit.MustExec("create table topn_after_hist_no_index(a int)") + testKit.MustExec("insert into topn_after_hist_no_index values(2), (2), (3), (4), (5), (6), (6), (6), (6)") + testKit.MustExec("analyze table topn_after_hist_no_index with 2 topn, 3 buckets") + + var ( + input []string + output [][]string + ) + s.testData.GetTestCases(c, &input, &output) + for i := range input { + s.testData.OnRecord(func() { + output[i] = s.testData.ConvertRowsToStrings(testKit.MustQuery(input[i]).Rows()) + }) + testKit.MustQuery(input[i]).Check(testkit.Rows(output[i]...)) + } +} + func (s *testStatsSuite) TestColumnIndexNullEstimation(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) diff --git a/statistics/testdata/stats_suite_in.json b/statistics/testdata/stats_suite_in.json index 6aef592ae4fa5..631e2aa6c60e2 100644 --- a/statistics/testdata/stats_suite_in.json +++ b/statistics/testdata/stats_suite_in.json @@ -66,6 +66,17 @@ "explain select * from ct2 where a=8 and b>=1 and b<=8" ] }, + { + "name": "TestTopNOutOfHist", + "cases": [ + "show stats_topn", + "show stats_buckets", + "explain select * from topn_before_hist where a = 1", + "explain select * from topn_after_hist where a = 6", + "explain select * from topn_after_hist_no_index where a = 6", + "explain select * from topn_before_hist_no_index where a = 1" + ] + }, { "name": "TestColumnIndexNullEstimation", "cases": [ diff --git a/statistics/testdata/stats_suite_out.json b/statistics/testdata/stats_suite_out.json index 80f4ad9e9e562..c25f082455c2f 100644 --- a/statistics/testdata/stats_suite_out.json +++ b/statistics/testdata/stats_suite_out.json @@ -422,6 +422,57 @@ ] ] }, + { + "Name": "TestTopNOutOfHist", + "Cases": [ + [ + "test topn_before_hist a 0 1 4", + "test topn_before_hist a 0 2 2", + "test topn_before_hist idx 1 1 4", + "test topn_before_hist idx 1 2 2", + "test topn_after_hist a 0 2 2", + "test topn_after_hist a 0 6 4", + "test topn_after_hist idx 1 2 2", + "test topn_after_hist idx 1 6 4", + "test topn_before_hist_no_index a 0 1 4", + "test topn_before_hist_no_index a 0 2 2", + "test topn_after_hist_no_index a 0 2 2", + "test topn_after_hist_no_index a 0 6 4" + ], + [ + "test topn_before_hist a 0 0 2 1 3 4 0", + "test topn_before_hist a 0 1 3 1 5 5 0", + "test topn_before_hist idx 1 0 2 1 3 4 0", + "test topn_before_hist idx 1 1 3 1 5 5 0", + "test topn_after_hist a 0 0 2 1 3 4 0", + "test topn_after_hist a 0 1 3 1 5 5 0", + "test topn_after_hist idx 1 0 2 1 3 4 0", + "test topn_after_hist idx 1 1 3 1 5 5 0", + "test topn_before_hist_no_index a 0 0 2 1 3 4 0", + "test topn_before_hist_no_index a 0 1 3 1 5 5 0", + "test topn_after_hist_no_index a 0 0 2 1 3 4 0", + "test topn_after_hist_no_index a 0 1 3 1 5 5 0" + ], + [ + "IndexReader_6 4.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 4.00 cop[tikv] table:topn_before_hist, index:idx(a) range:[1,1], keep order:false" + ], + [ + "IndexReader_6 4.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 4.00 cop[tikv] table:topn_after_hist, index:idx(a) range:[6,6], keep order:false" + ], + [ + "TableReader_7 4.00 root data:Selection_6", + "└─Selection_6 4.00 cop[tikv] eq(test.topn_after_hist_no_index.a, 6)", + " └─TableFullScan_5 9.00 cop[tikv] table:topn_after_hist_no_index keep order:false" + ], + [ + "TableReader_7 4.00 root data:Selection_6", + "└─Selection_6 4.00 cop[tikv] eq(test.topn_before_hist_no_index.a, 1)", + " └─TableFullScan_5 9.00 cop[tikv] table:topn_before_hist_no_index keep order:false" + ] + ] + }, { "Name": "TestColumnIndexNullEstimation", "Cases": [ From a8819a88eb29908a1e3a57137e9d955e158e9a8c Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Wed, 26 May 2021 19:25:35 +0800 Subject: [PATCH 186/343] docs: Rename some tables' column names in Lock View to reduce confusion (#24904) --- docs/design/2021-04-26-lock-view.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/design/2021-04-26-lock-view.md b/docs/design/2021-04-26-lock-view.md index 3ed0e5902c146..78faf9f6b61de 100644 --- a/docs/design/2021-04-26-lock-view.md +++ b/docs/design/2021-04-26-lock-view.md @@ -37,8 +37,8 @@ Several tables will be provided in `information_schema`. Some tables has both lo |------------|------------|---------| | `TRX_ID` | `unsigned bigint` | The transaction ID (aka. start ts) | | `TRX_STARTED`|`time`| Human readable start time of the transaction | -| `DIGEST`|`text`| The digest of the current executing SQL statement | -| `ALL_SQLS` | `text` | A list of all executed SQL statements' digests | +| `CURRENT_SQL_DIGEST`|`text`| The digest of the current executing SQL statement | +| `ALL_SQL_DIGESTS` | `text` | A list of all executed SQL statements' digests | | `STATE`| `enum('Running', 'Lock waiting', 'Committing', 'RollingBack')`| The state of the transaction | | `WAITING_START_TIME` | `time` | The elapsed time since the start of the current lock waiting (if any) | | `SCOPE` | `enum('Global', 'Local')` | The scope of the transaction | @@ -89,7 +89,7 @@ Several tables will be provided in `information_schema`. Some tables has both lo | `TRY_LOCK_TRX_ID` | `unsigned bigint` | The transaction ID (start ts) of the transaction that's trying to acquire the lock | | `CURRENT_SQL_DIGEST` | `text` | The SQL that's being blocked | | `KEY` | `varchar` | The key that's being locked, but locked by another transaction in the deadlock event | -| `ALL_SQLS` | `text` | A list of the digest of SQL statements that the transaction has executed | +| `ALL_SQL_DIGESTS` | `text` | A list of the digest of SQL statements that the transaction has executed | | `TRX_HOLDING_LOCK` | `unsigned bigint` | The transaction that's currently holding the lock. There will be another record in the table with the same `DEADLOCK_ID` for that transaction. | * Life span of rows: From f891c5a7cbbb2082ed281e1c63e3abb1890cea1c Mon Sep 17 00:00:00 2001 From: sylzd Date: Wed, 26 May 2021 21:29:35 +0800 Subject: [PATCH 187/343] planner: Fix runtime error occurs when illegal mix collate in subquery (#24887) --- expression/builtin.go | 7 ++++--- planner/core/expression_rewriter.go | 9 +++++++++ planner/core/expression_rewriter_test.go | 18 ++++++++++++++++++ 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/expression/builtin.go b/expression/builtin.go index a33650eef7b1f..8da5528b6a975 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -90,7 +90,7 @@ func newBaseBuiltinFunc(ctx sessionctx.Context, funcName string, args []Expressi if ctx == nil { return baseBuiltinFunc{}, errors.New("unexpected nil session ctx") } - if err := checkIllegalMixCollation(funcName, args, retType); err != nil { + if err := CheckIllegalMixCollation(funcName, args, retType); err != nil { return baseBuiltinFunc{}, err } derivedCharset, derivedCollate := DeriveCollationFromExprs(ctx, args...) @@ -112,7 +112,8 @@ var ( coerString = []string{"EXPLICIT", "NONE", "IMPLICIT", "SYSCONST", "COERCIBLE", "NUMERIC", "IGNORABLE"} ) -func checkIllegalMixCollation(funcName string, args []Expression, evalType types.EvalType) error { +// CheckIllegalMixCollation checks illegal mix collation with expressions +func CheckIllegalMixCollation(funcName string, args []Expression, evalType types.EvalType) error { if len(args) < 2 { return nil } @@ -169,7 +170,7 @@ func newBaseBuiltinFuncWithTp(ctx sessionctx.Context, funcName string, args []Ex } } - if err = checkIllegalMixCollation(funcName, args, retType); err != nil { + if err = CheckIllegalMixCollation(funcName, args, retType); err != nil { return } diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 29113fe64bfc9..7dd992243953e 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -536,6 +536,15 @@ func (er *expressionRewriter) handleCompareSubquery(ctx context.Context, v *ast. return v, true } } + + // Lexpr cannot compare with rexpr by different collate + opString := new(strings.Builder) + v.Op.Format(opString) + er.err = expression.CheckIllegalMixCollation(opString.String(), []expression.Expression{lexpr, rexpr}, 0) + if er.err != nil { + return v, true + } + switch v.Op { // Only EQ, NE and NullEQ can be composed with and. case opcode.EQ, opcode.NE, opcode.NullEQ: diff --git a/planner/core/expression_rewriter_test.go b/planner/core/expression_rewriter_test.go index 66bb860a52f0b..13b7355eb7ea4 100644 --- a/planner/core/expression_rewriter_test.go +++ b/planner/core/expression_rewriter_test.go @@ -395,3 +395,21 @@ func (s *testExpressionRewriterSuite) TestIssue22818(c *C) { tk.MustQuery("select * from t where a between \"23:22:22\" and \"23:22:22\"").Check( testkit.Rows("23:22:22")) } + +func (s *testExpressionRewriterSuite) TestIssue24705(c *C) { + defer testleak.AfterTest(c)() + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + defer func() { + dom.Close() + store.Close() + }() + + tk.MustExec("use test;") + tk.MustExec("drop table if exists t1,t2;") + tk.MustExec("create table t1 (c_int int, c_str varchar(40) character set utf8 collate utf8_general_ci);") + tk.MustExec("create table t2 (c_int int, c_str varchar(40) character set utf8 collate utf8_unicode_ci);") + err = tk.ExecToErr("select * from t1 where c_str < any (select c_str from t2 where c_int between 6 and 9);") + c.Assert(err.Error(), Equals, "[expression:1267]Illegal mix of collations (utf8_general_ci,IMPLICIT) and (utf8_unicode_ci,IMPLICIT) for operation '<'") +} From 1ae28529ab77255ba19945bbb55da3c379d2074a Mon Sep 17 00:00:00 2001 From: Ryan Leung Date: Thu, 27 May 2021 11:03:35 +0800 Subject: [PATCH 188/343] store/tikv: remove ServerMemoryQuota from tikv config (#24620) --- config/config.go | 1 - store/tikv/config/config.go | 2 -- 2 files changed, 3 deletions(-) diff --git a/config/config.go b/config/config.go index 83490c345dae8..cfadfe3fd94e7 100644 --- a/config/config.go +++ b/config/config.go @@ -199,7 +199,6 @@ func (c *Config) getTiKVConfig() *tikvcfg.Config { return &tikvcfg.Config{ CommitterConcurrency: c.Performance.CommitterConcurrency, MaxTxnTTL: c.Performance.MaxTxnTTL, - ServerMemoryQuota: defTiKVCfg.ServerMemoryQuota, TiKVClient: c.TiKVClient, Security: c.Security.ClusterSecurity(), PDClient: c.PDClient, diff --git a/store/tikv/config/config.go b/store/tikv/config/config.go index 6fe0c018498ea..09c559581076a 100644 --- a/store/tikv/config/config.go +++ b/store/tikv/config/config.go @@ -45,7 +45,6 @@ func init() { type Config struct { CommitterConcurrency int MaxTxnTTL uint64 - ServerMemoryQuota uint64 TiKVClient TiKVClient Security Security PDClient PDClient @@ -64,7 +63,6 @@ func DefaultConfig() Config { return Config{ CommitterConcurrency: 128, MaxTxnTTL: 60 * 60 * 1000, // 1hour - ServerMemoryQuota: 0, TiKVClient: DefaultTiKVClient(), PDClient: DefaultPDClient(), TxnLocalLatches: DefaultTxnLocalLatches(), From 2f96bc7502ac0dceac3e50b0a10eadf0a27e8925 Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Thu, 27 May 2021 11:45:35 +0800 Subject: [PATCH 189/343] expresssion: determine the field type of control function with enum type (#24830) --- expression/builtin_control.go | 6 ++++++ expression/integration_test.go | 13 +++++++++++++ 2 files changed, 19 insertions(+) diff --git a/expression/builtin_control.go b/expression/builtin_control.go index 13f539648e049..9ffc4bfe718a9 100644 --- a/expression/builtin_control.go +++ b/expression/builtin_control.go @@ -135,10 +135,16 @@ func InferType4ControlFuncs(lexp, rexp Expression) *types.FieldType { resultEvalType := resultFieldType.EvalType() if resultEvalType == types.ETInt { resultFieldType.Decimal = 0 + if resultFieldType.Tp == mysql.TypeEnum || resultFieldType.Tp == mysql.TypeSet { + resultFieldType.Tp = mysql.TypeLonglong + } } else if resultEvalType == types.ETString { if lhs.Tp != mysql.TypeNull || rhs.Tp != mysql.TypeNull { resultFieldType.Decimal = types.UnspecifiedLength } + if resultFieldType.Tp == mysql.TypeEnum || resultFieldType.Tp == mysql.TypeSet { + resultFieldType.Tp = mysql.TypeVarchar + } } return resultFieldType } diff --git a/expression/integration_test.go b/expression/integration_test.go index d1db4e3cbbd69..a4706629660a0 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9586,6 +9586,19 @@ func (s *testIntegrationSuite) TestControlFunctionWithEnumOrSet(c *C) { tk.MustQuery("select elt(1,s) = 'a' from s").Check(testkit.Rows("1")) tk.MustQuery("select elt(1,s) = 4 from s").Check(testkit.Rows("1")) tk.MustQuery("select s from s where elt(1,s)").Check(testkit.Rows("a")) + + // issue 24543 + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int,b enum(\"b\"),c enum(\"c\"));") + tk.MustExec("insert into t values(1,1,1),(2,1,1),(1,1,1),(2,1,1);") + tk.MustQuery("select if(A, null,b)=1 from t;").Check(testkit.Rows("", "", "", "")) + tk.MustQuery("select if(A, null,b)='a' from t;").Check(testkit.Rows("", "", "", "")) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int,b set(\"b\"),c set(\"c\"));") + tk.MustExec("insert into t values(1,1,1),(2,1,1),(1,1,1),(2,1,1);") + tk.MustQuery("select if(A, null,b)=1 from t;").Check(testkit.Rows("", "", "", "")) + tk.MustQuery("select if(A, null,b)='a' from t;").Check(testkit.Rows("", "", "", "")) } func (s *testIntegrationSuite) TestComplexShowVariables(c *C) { From 96ab4a3f0a586b0819946523cc6d5f2693468bb6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B7=B7=E6=B2=8CDM?= Date: Thu, 27 May 2021 13:25:35 +0800 Subject: [PATCH 190/343] store/helper, infoschema: fix the bug that cannot find down-peer (#24881) --- executor/infoschema_reader.go | 8 ++++---- store/helper/helper.go | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index ee5b4af84b574..d049fb3eb39e8 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1410,7 +1410,7 @@ func (e *memtableRetriever) setNewTiKVRegionPeersCols(region *helper.RegionInfo) } downPeerMap := make(map[int64]int64, len(region.DownPeers)) for _, peerStat := range region.DownPeers { - downPeerMap[peerStat.ID] = peerStat.DownSec + downPeerMap[peerStat.Peer.ID] = peerStat.DownSec } for _, peer := range region.Peers { row := make([]types.Datum, len(infoschema.TableTiKVRegionPeersCols)) @@ -1427,11 +1427,11 @@ func (e *memtableRetriever) setNewTiKVRegionPeersCols(region *helper.RegionInfo) } else { row[4].SetInt64(0) } - if pendingPeerIDSet.Exist(peer.ID) { - row[5].SetString(pendingPeer, mysql.DefaultCollationName) - } else if downSec, ok := downPeerMap[peer.ID]; ok { + if downSec, ok := downPeerMap[peer.ID]; ok { row[5].SetString(downPeer, mysql.DefaultCollationName) row[6].SetInt64(downSec) + } else if pendingPeerIDSet.Exist(peer.ID) { + row[5].SetString(pendingPeer, mysql.DefaultCollationName) } else { row[5].SetString(normalPeer, mysql.DefaultCollationName) } diff --git a/store/helper/helper.go b/store/helper/helper.go index 52100e472cbb2..53525f9687fa2 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -539,8 +539,8 @@ type RegionEpoch struct { // RegionPeerStat stores one field `DownSec` which indicates how long it's down than `RegionPeer`. type RegionPeerStat struct { - RegionPeer - DownSec int64 `json:"down_seconds"` + Peer RegionPeer `json:"peer"` + DownSec int64 `json:"down_seconds"` } // RegionInfo stores the information of one region. From a5400cc602d1f0af8464150abe661d32d03251bd Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Thu, 27 May 2021 15:47:35 +0800 Subject: [PATCH 191/343] test: Fix a misusing of testkit that runs two queries concurrently (#24923) --- session/pessimistic_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 58cf8a624fe54..b70a142cde171 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -2322,7 +2322,7 @@ func (s *testPessimisticSuite) TestAmendForUniqueIndex(c *C) { err = <-errCh c.Assert(err, Equals, nil) tk.MustExec("commit") - tk2.MustExec("admin check table t") + tk.MustExec("admin check table t") err = <-errCh c.Assert(err, Equals, nil) } From 415dae62edf872b9b5223e305667bf61ee342df4 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Thu, 27 May 2021 16:57:36 +0800 Subject: [PATCH 192/343] planner: unify name of datasource receiver (#24703) --- planner/core/explain.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/planner/core/explain.go b/planner/core/explain.go index 57334d172a0d6..a8f5609b43e9a 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -842,16 +842,16 @@ func (p *LogicalTableDual) ExplainInfo() string { } // ExplainInfo implements Plan interface. -func (p *DataSource) ExplainInfo() string { +func (ds *DataSource) ExplainInfo() string { buffer := bytes.NewBufferString("") - tblName := p.tableInfo.Name.O - if p.TableAsName != nil && p.TableAsName.O != "" { - tblName = p.TableAsName.O + tblName := ds.tableInfo.Name.O + if ds.TableAsName != nil && ds.TableAsName.O != "" { + tblName = ds.TableAsName.O } fmt.Fprintf(buffer, "table:%s", tblName) - if p.isPartition { - if pi := p.tableInfo.GetPartitionInfo(); pi != nil { - partitionName := pi.GetNameByID(p.physicalTableID) + if ds.isPartition { + if pi := ds.tableInfo.GetPartitionInfo(); pi != nil { + partitionName := pi.GetNameByID(ds.physicalTableID) fmt.Fprintf(buffer, ", partition:%s", partitionName) } } From e87d0359373f261d50106a29f0a9dd3d8de46214 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Thu, 27 May 2021 17:57:36 +0800 Subject: [PATCH 193/343] infoschema, executor, txn: implement DATA_LOCK_WAITS table (#24750) --- executor/builder.go | 3 +- executor/infoschema_reader.go | 69 ++++++++++++------- infoschema/tables.go | 11 +++ infoschema/tables_test.go | 58 ++++++++++++++++ kv/interface_mock_test.go | 13 ++++ kv/kv.go | 5 +- store/driver/tikv_driver.go | 23 +++++++ store/helper/helper.go | 2 + store/mockstore/mockstorage/storage.go | 21 ++++-- store/mockstore/unistore/tikv/detector.go | 8 +-- .../mockstore/unistore/tikv/detector_test.go | 4 +- store/mockstore/unistore/tikv/errors.go | 4 +- store/tikv/kv.go | 4 +- store/tikv/region_cache.go | 8 ++- store/tikv/tests/lock_test.go | 4 +- store/tikv/tikvrpc/tikvrpc.go | 10 +++ util/mock/store.go | 6 ++ 17 files changed, 208 insertions(+), 45 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 7a66594222d09..2f644a6eed2c4 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1537,7 +1537,8 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableTiDBTrx), strings.ToLower(infoschema.ClusterTableTiDBTrx), strings.ToLower(infoschema.TableDeadlocks), - strings.ToLower(infoschema.ClusterTableDeadlocks): + strings.ToLower(infoschema.ClusterTableDeadlocks), + strings.ToLower(infoschema.TableDataLockWaits): return &MemTableReaderExec{ baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), table: v.Table, diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index d049fb3eb39e8..3b4fc2e159e05 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -16,6 +16,7 @@ package executor import ( "bytes" "context" + "encoding/hex" "encoding/json" "fmt" "io" @@ -53,13 +54,16 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/deadlockhistory" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/pdapi" + "github.com/pingcap/tidb/util/resourcegrouptag" "github.com/pingcap/tidb/util/sem" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" "go.etcd.io/etcd/clientv3" + "go.uber.org/zap" ) type memtableRetriever struct { @@ -158,6 +162,8 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex err = e.setDataForDeadlock(sctx) case infoschema.ClusterTableDeadlocks: err = e.setDataForClusterDeadlock(sctx) + case infoschema.TableDataLockWaits: + err = e.setDataForTableDataLockWaits(sctx) } if err != nil { return nil, err @@ -1005,6 +1011,40 @@ func (e *memtableRetriever) dataForTiKVStoreStatus(ctx sessionctx.Context) (err return nil } +func hasPriv(ctx sessionctx.Context, priv mysql.PrivilegeType) bool { + if pm := privilege.GetPrivilegeManager(ctx); pm != nil { + return pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", priv) + } + return false +} + +func (e *memtableRetriever) setDataForTableDataLockWaits(ctx sessionctx.Context) error { + if !hasPriv(ctx, mysql.ProcessPriv) { + return plannercore.ErrSpecificAccessDenied.GenWithStackByArgs("PROCESS") + } + waits, err := ctx.GetStore().GetLockWaits() + if err != nil { + return err + } + for _, wait := range waits { + var digestStr interface{} + digest, err := resourcegrouptag.DecodeResourceGroupTag(wait.ResourceGroupTag) + if err != nil { + logutil.BgLogger().Warn("failed to decode resource group tag", zap.Error(err)) + digestStr = nil + } else { + digestStr = hex.EncodeToString(digest) + } + e.rows = append(e.rows, types.MakeDatums( + wait.Key, + wait.Txn, + wait.WaitForTxn, + digestStr, + )) + } + return nil +} + // DDLJobsReaderExec executes DDLJobs information retrieving. type DDLJobsReaderExec struct { baseExecutor @@ -1189,13 +1229,7 @@ func (e *memtableRetriever) setDataForProcessList(ctx sessionctx.Context) { } loginUser := ctx.GetSessionVars().User - var hasProcessPriv bool - if pm := privilege.GetPrivilegeManager(ctx); pm != nil { - if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) { - hasProcessPriv = true - } - } - + hasProcessPriv := hasPriv(ctx, mysql.ProcessPriv) pl := sm.ShowProcessList() records := make([][]types.Datum, 0, len(pl)) @@ -1946,13 +1980,8 @@ func (e *memtableRetriever) setDataForPlacementPolicy(ctx sessionctx.Context) er func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context, tableName string) error { // Seeing client errors should require the PROCESS privilege, with the exception of errors for your own user. // This is similar to information_schema.processlist, which is the closest comparison. - var hasProcessPriv bool + hasProcessPriv := hasPriv(ctx, mysql.ProcessPriv) loginUser := ctx.GetSessionVars().User - if pm := privilege.GetPrivilegeManager(ctx); pm != nil { - if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) { - hasProcessPriv = true - } - } var rows [][]types.Datum switch tableName { @@ -2026,12 +2055,7 @@ func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) { } loginUser := ctx.GetSessionVars().User - var hasProcessPriv bool - if pm := privilege.GetPrivilegeManager(ctx); pm != nil { - if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) { - hasProcessPriv = true - } - } + hasProcessPriv := hasPriv(ctx, mysql.ProcessPriv) infoList := sm.ShowTxnList() for _, info := range infoList { // If you have the PROCESS privilege, you can see all running transactions. @@ -2054,12 +2078,7 @@ func (e *memtableRetriever) setDataForClusterTiDBTrx(ctx sessionctx.Context) err } func (e *memtableRetriever) setDataForDeadlock(ctx sessionctx.Context) error { - hasPriv := false - if pm := privilege.GetPrivilegeManager(ctx); pm != nil { - hasPriv = pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) - } - - if !hasPriv { + if !hasPriv(ctx, mysql.ProcessPriv) { return plannercore.ErrSpecificAccessDenied.GenWithStackByArgs("PROCESS") } diff --git a/infoschema/tables.go b/infoschema/tables.go index 04988cb07c2b7..fec9378a491b3 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -167,6 +167,8 @@ const ( TableTiDBTrx = "TIDB_TRX" // TableDeadlocks is the string constatnt of deadlock table. TableDeadlocks = "DEADLOCKS" + // TableDataLockWaits is current lock waiting status table. + TableDataLockWaits = "DATA_LOCK_WAITS" ) var tableIDMap = map[string]int64{ @@ -243,6 +245,7 @@ var tableIDMap = map[string]int64{ ClusterTableTiDBTrx: autoid.InformationSchemaDBID + 71, TableDeadlocks: autoid.InformationSchemaDBID + 72, ClusterTableDeadlocks: autoid.InformationSchemaDBID + 73, + TableDataLockWaits: autoid.InformationSchemaDBID + 74, } type columnInfo struct { @@ -1368,6 +1371,13 @@ var tableDeadlocksCols = []columnInfo{ {name: "TRX_HOLDING_LOCK", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The transaction ID (start ts) of the transaction that's currently holding the lock"}, } +var tableDataLockWaitsCols = []columnInfo{ + {name: "KEY", tp: mysql.TypeVarchar, size: 64, flag: mysql.NotNullFlag, comment: "The key that's being waiting on"}, + {name: "TRX_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Current transaction that's waiting for the lock"}, + {name: "CURRENT_HOLDING_TRX_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "The transaction that's holding the lock and blocks the current transaction"}, + {name: "SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the SQL that's trying to acquire the lock"}, +} + // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. // The returned description string may be: // - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. @@ -1739,6 +1749,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableClientErrorsSummaryByHost: tableClientErrorsSummaryByHostCols, TableTiDBTrx: tableTiDBTrxCols, TableDeadlocks: tableDeadlocksCols, + TableDataLockWaits: tableDataLockWaitsCols, } func createInfoSchemaTable(_ autoid.Allocators, meta *model.TableInfo) (table.Table, error) { diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 761bb75fb76ed..65b5a37a4ba2a 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -28,6 +28,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/fn" + "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/parser" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" @@ -46,9 +47,13 @@ import ( "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/mockstore/mockstorage" + "github.com/pingcap/tidb/store/mockstore/unistore" + "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/pdapi" + "github.com/pingcap/tidb/util/resourcegrouptag" "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" @@ -57,12 +62,17 @@ import ( ) var _ = Suite(&testTableSuite{&testTableSuiteBase{}}) +var _ = Suite(&testDataLockWaitSuite{&testTableSuiteBase{}}) var _ = SerialSuites(&testClusterTableSuite{testTableSuiteBase: &testTableSuiteBase{}}) type testTableSuite struct { *testTableSuiteBase } +type testDataLockWaitSuite struct { + *testTableSuiteBase +} + type testTableSuiteBase struct { store kv.Storage dom *domain.Domain @@ -1552,3 +1562,51 @@ func (s *testTableSuite) TestInfoschemaDeadlockPrivilege(c *C) { }, nil, nil), IsTrue) _ = tk.MustQuery("select * from information_schema.deadlocks") } + +func (s *testDataLockWaitSuite) SetUpSuite(c *C) { + testleak.BeforeTest() + + client, pdClient, cluster, err := unistore.New("") + c.Assert(err, IsNil) + unistore.BootstrapWithSingleStore(cluster) + kvstore, err := tikv.NewTestTiKVStore(client, pdClient, nil, nil, 0) + c.Assert(err, IsNil) + _, digest1 := parser.NormalizeDigest("select * from t1 for update;") + _, digest2 := parser.NormalizeDigest("update t1 set f1=1 where id=2;") + s.store, err = mockstorage.NewMockStorageWithLockWaits(kvstore, []*deadlock.WaitForEntry{ + {Txn: 1, WaitForTxn: 2, KeyHash: 3, Key: []byte("a"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(digest1, nil)}, + {Txn: 4, WaitForTxn: 5, KeyHash: 6, Key: []byte("b"), ResourceGroupTag: resourcegrouptag.EncodeResourceGroupTag(digest2, nil)}, + }) + c.Assert(err, IsNil) + session.DisableStats4Test() + s.dom, err = session.BootstrapSession(s.store) + c.Assert(err, IsNil) +} + +func (s *testDataLockWaitSuite) TestDataLockWait(c *C) { + _, digest1 := parser.NormalizeDigest("select * from t1 for update;") + _, digest2 := parser.NormalizeDigest("update t1 set f1=1 where id=2;") + tk := s.newTestKitWithRoot(c) + tk.MustQuery("select * from information_schema.DATA_LOCK_WAITS;").Check(testkit.Rows("a 1 2 "+digest1.String(), "b 4 5 "+digest2.String())) +} + +func (s *testDataLockWaitSuite) TestDataLockPrivilege(c *C) { + tk := s.newTestKitWithRoot(c) + tk.MustExec("create user 'testuser'@'localhost'") + c.Assert(tk.Se.Auth(&auth.UserIdentity{ + Username: "testuser", + Hostname: "localhost", + }, nil, nil), IsTrue) + err := tk.QueryToErr("select * from information_schema.DATA_LOCK_WAITS") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the PROCESS privilege(s) for this operation") + + tk = s.newTestKitWithRoot(c) + tk.MustExec("create user 'testuser2'@'localhost'") + tk.MustExec("grant process on *.* to 'testuser2'@'localhost'") + c.Assert(tk.Se.Auth(&auth.UserIdentity{ + Username: "testuser2", + Hostname: "localhost", + }, nil, nil), IsTrue) + _ = tk.MustQuery("select * from information_schema.DATA_LOCK_WAITS") +} diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index 0c471c6fb40e0..2e78b8725a7dc 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -16,6 +16,7 @@ package kv import ( "context" + deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" @@ -103,6 +104,10 @@ func (t *mockTxn) GetSnapshot() Snapshot { return nil } +func (t *mockTxn) GetUnionStore() UnionStore { + return nil +} + func (t *mockTxn) NewStagingBuffer() MemBuffer { return nil } @@ -210,6 +215,10 @@ func (s *mockStorage) GetMemCache() MemManager { return nil } +func (s *mockStorage) GetLockWaits() ([]*deadlockpb.WaitForEntry, error) { + return nil, nil +} + func (s *mockStorage) GetMinSafeTS(txnScope string) uint64 { return 0 } @@ -255,3 +264,7 @@ func (s *mockSnapshot) IterReverse(k Key) (Iterator, error) { } func (s *mockSnapshot) SetOption(opt int, val interface{}) {} + +func (s *mockSnapshot) GetLockWaits() []deadlockpb.WaitForEntry { + return nil +} diff --git a/kv/kv.go b/kv/kv.go index fdcc7148247af..d65258131d184 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -18,6 +18,7 @@ import ( "crypto/tls" "time" + deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/config" @@ -340,7 +341,7 @@ type Driver interface { type Storage interface { // Begin a global transaction Begin() (Transaction, error) - // Begin a transaction with given option + // BeginWithOption begins a transaction with given option BeginWithOption(option tikv.StartTSOption) (Transaction, error) // GetSnapshot gets a snapshot that is able to read any data which data is <= ver. // if ver is MaxVersion or > current max committed version, we will use current version for this snapshot. @@ -369,6 +370,8 @@ type Storage interface { GetMemCache() MemManager // GetMinSafeTS return the minimal SafeTS of the storage with given txnScope. GetMinSafeTS(txnScope string) uint64 + // GetLockWaits return all lock wait information + GetLockWaits() ([]*deadlockpb.WaitForEntry, error) } // EtcdBackend is used for judging a storage is a real TiKV. diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index 5f5471d8e7251..689c6ea170678 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -24,6 +24,8 @@ import ( "time" "github.com/pingcap/errors" + deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/copr" derr "github.com/pingcap/tidb/store/driver/error" @@ -31,6 +33,7 @@ import ( "github.com/pingcap/tidb/store/gcworker" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" + "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/util/logutil" pd "github.com/tikv/pd/client" @@ -331,3 +334,23 @@ func (s *tikvStore) CurrentVersion(txnScope string) (kv.Version, error) { func (s *tikvStore) ShowStatus(ctx context.Context, key string) (interface{}, error) { return nil, kv.ErrNotImplemented } + +// GetLockWaits get return lock waits info +func (s *tikvStore) GetLockWaits() ([]*deadlockpb.WaitForEntry, error) { + stores := s.GetRegionCache().GetStoresByType(tikvrpc.TiKV) + var result []*deadlockpb.WaitForEntry + for _, store := range stores { + resp, err := s.GetTiKVClient().SendRequest(context.TODO(), store.GetAddr(), tikvrpc.NewRequest(tikvrpc.CmdLockWaitInfo, &kvrpcpb.GetLockWaitInfoRequest{}), time.Second*30) + if err != nil { + logutil.BgLogger().Warn("query lock wait info failed", zap.Error(err)) + continue + } + if resp.Resp == nil { + logutil.BgLogger().Warn("lock wait info from store is nil") + continue + } + entries := resp.Resp.(*kvrpcpb.GetLockWaitInfoResponse).Entries + result = append(result, entries...) + } + return result, nil +} diff --git a/store/helper/helper.go b/store/helper/helper.go index 53525f9687fa2..533b1d66d576e 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -29,6 +29,7 @@ import ( "time" "github.com/pingcap/errors" + deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" "github.com/pingcap/parser/model" @@ -72,6 +73,7 @@ type Storage interface { GetTiKVClient() tikv.Client Closed() <-chan struct{} GetMinSafeTS(txnScope string) uint64 + GetLockWaits() ([]*deadlockpb.WaitForEntry, error) } // Helper is a middleware to get some information from tikv/pd. It can be used for TiDB's http api or mem table. diff --git a/store/mockstore/mockstorage/storage.go b/store/mockstore/mockstorage/storage.go index 7d78d1a9b7418..4fa049fc69f42 100644 --- a/store/mockstore/mockstorage/storage.go +++ b/store/mockstore/mockstorage/storage.go @@ -17,6 +17,7 @@ import ( "context" "crypto/tls" + deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/copr" driver "github.com/pingcap/tidb/store/driver/txn" @@ -28,22 +29,28 @@ import ( type mockStorage struct { *tikv.KVStore *copr.Store - memCache kv.MemManager + memCache kv.MemManager + LockWaits []*deadlockpb.WaitForEntry } // NewMockStorage wraps tikv.KVStore as kv.Storage. func NewMockStorage(tikvStore *tikv.KVStore) (kv.Storage, error) { + return NewMockStorageWithLockWaits(tikvStore, nil) +} + +// NewMockStorageWithLockWaits wraps tikv.KVStore as kv.Storage, with mock LockWaits. +func NewMockStorageWithLockWaits(tikvStore *tikv.KVStore, lockWaits []*deadlockpb.WaitForEntry) (kv.Storage, error) { coprConfig := config.DefaultConfig().TiKVClient.CoprCache coprStore, err := copr.NewStore(tikvStore, &coprConfig) if err != nil { return nil, err } return &mockStorage{ - KVStore: tikvStore, - Store: coprStore, - memCache: kv.NewCacheDB(), + KVStore: tikvStore, + Store: coprStore, + memCache: kv.NewCacheDB(), + LockWaits: lockWaits, }, nil - } func (s *mockStorage) EtcdAddrs() ([]string, error) { @@ -111,6 +118,10 @@ func newTiKVTxn(txn *tikv.KVTxn, err error) (kv.Transaction, error) { return driver.NewTiKVTxn(txn), nil } +func (s *mockStorage) GetLockWaits() ([]*deadlockpb.WaitForEntry, error) { + return s.LockWaits, nil +} + func (s *mockStorage) Close() error { s.Store.Close() return s.KVStore.Close() diff --git a/store/mockstore/unistore/tikv/detector.go b/store/mockstore/unistore/tikv/detector.go index 6a56a0d9fd73c..b86804696a670 100644 --- a/store/mockstore/unistore/tikv/detector.go +++ b/store/mockstore/unistore/tikv/detector.go @@ -30,7 +30,7 @@ import ( "sync" "time" - deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" + deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/log" "go.uber.org/zap" ) @@ -93,7 +93,7 @@ func (d *Detector) Detect(sourceTxn, waitForTxn, keyHash uint64, diagCtx diagnos j := len(err.WaitChain) - i - 1 err.WaitChain[i], err.WaitChain[j] = err.WaitChain[j], err.WaitChain[i] } - err.WaitChain = append(err.WaitChain, &deadlockPB.WaitForEntry{ + err.WaitChain = append(err.WaitChain, &deadlockpb.WaitForEntry{ Txn: sourceTxn, Key: diagCtx.key, KeyHash: keyHash, @@ -122,7 +122,7 @@ func (d *Detector) doDetect(nowTime time.Time, sourceTxn, waitForTxn uint64) *Er } if keyHashPair.txn == sourceTxn { return &ErrDeadlock{DeadlockKeyHash: keyHashPair.keyHash, - WaitChain: []*deadlockPB.WaitForEntry{ + WaitChain: []*deadlockpb.WaitForEntry{ { Txn: waitForTxn, Key: keyHashPair.diagCtx.key, @@ -134,7 +134,7 @@ func (d *Detector) doDetect(nowTime time.Time, sourceTxn, waitForTxn uint64) *Er } } if err := d.doDetect(nowTime, sourceTxn, keyHashPair.txn); err != nil { - err.WaitChain = append(err.WaitChain, &deadlockPB.WaitForEntry{ + err.WaitChain = append(err.WaitChain, &deadlockpb.WaitForEntry{ Txn: waitForTxn, Key: keyHashPair.diagCtx.key, KeyHash: keyHashPair.keyHash, diff --git a/store/mockstore/unistore/tikv/detector_test.go b/store/mockstore/unistore/tikv/detector_test.go index 189a8b00b8217..c47260f886275 100644 --- a/store/mockstore/unistore/tikv/detector_test.go +++ b/store/mockstore/unistore/tikv/detector_test.go @@ -30,7 +30,7 @@ import ( "time" . "github.com/pingcap/check" - deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" + deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" ) func TestT(t *testing.T) { @@ -48,7 +48,7 @@ func (s *testDeadlockSuite) TestDeadlock(c *C) { resourceGroupTag: []byte(resourceGroupTag), } } - checkWaitChainEntry := func(entry *deadlockPB.WaitForEntry, txn, waitForTxn uint64, key, resourceGroupTag string) { + checkWaitChainEntry := func(entry *deadlockpb.WaitForEntry, txn, waitForTxn uint64, key, resourceGroupTag string) { c.Assert(entry.Txn, Equals, txn) c.Assert(entry.WaitForTxn, Equals, waitForTxn) c.Assert(string(entry.Key), Equals, key) diff --git a/store/mockstore/unistore/tikv/errors.go b/store/mockstore/unistore/tikv/errors.go index 98a70951871d5..bce76319320c9 100644 --- a/store/mockstore/unistore/tikv/errors.go +++ b/store/mockstore/unistore/tikv/errors.go @@ -16,7 +16,7 @@ package tikv import ( "fmt" - deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" + deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/mockstore/unistore/tikv/mvcc" ) @@ -91,7 +91,7 @@ type ErrDeadlock struct { LockKey []byte LockTS uint64 DeadlockKeyHash uint64 - WaitChain []*deadlockPB.WaitForEntry + WaitChain []*deadlockpb.WaitForEntry } func (e ErrDeadlock) Error() string { diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 563f985f95083..8adf5c1f58782 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -336,7 +336,7 @@ func (s *KVStore) GetTiKVClient() (client Client) { // GetMinSafeTS return the minimal safeTS of the storage with given txnScope. func (s *KVStore) GetMinSafeTS(txnScope string) uint64 { stores := make([]*Store, 0) - allStores := s.regionCache.getStoresByType(tikvrpc.TiKV) + allStores := s.regionCache.GetStoresByType(tikvrpc.TiKV) if txnScope != oracle.GlobalTxnScope { for _, store := range allStores { if store.IsLabelsMatch([]*metapb.StoreLabel{ @@ -402,7 +402,7 @@ func (s *KVStore) safeTSUpdater() { } func (s *KVStore) updateSafeTS(ctx context.Context) { - stores := s.regionCache.getStoresByType(tikvrpc.TiKV) + stores := s.regionCache.GetStoresByType(tikvrpc.TiKV) tikvClient := s.GetTiKVClient() wg := &sync.WaitGroup{} wg.Add(len(stores)) diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 2b0ddd99f6454..287edeb4170f5 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -1268,8 +1268,9 @@ func (c *RegionCache) getRegionByIDFromCache(regionID uint64) *Region { return latestRegion } +// GetStoresByType gets stores by type `typ` // TODO: revise it by get store by closure. -func (c *RegionCache) getStoresByType(typ tikvrpc.EndpointType) []*Store { +func (c *RegionCache) GetStoresByType(typ tikvrpc.EndpointType) []*Store { c.storeMu.Lock() defer c.storeMu.Unlock() stores := make([]*Store, 0) @@ -2236,6 +2237,11 @@ func (s *Store) requestLiveness(bo *Backoffer, c *RegionCache) (l livenessState) return } +// GetAddr returns the address of the store +func (s *Store) GetAddr() string { + return s.addr +} + func invokeKVStatusAPI(addr string, timeout time.Duration) (l livenessState) { start := time.Now() defer func() { diff --git a/store/tikv/tests/lock_test.go b/store/tikv/tests/lock_test.go index d64c1d102e6d1..d52329c50cfa5 100644 --- a/store/tikv/tests/lock_test.go +++ b/store/tikv/tests/lock_test.go @@ -25,7 +25,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" - deadlockPB "github.com/pingcap/kvproto/pkg/deadlock" + deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" @@ -714,7 +714,7 @@ func (s *testLockSuite) TestDeadlockReportWaitChain(c *C) { } // Check the given WaitForEntry is caused by txn[i] waiting for txn[j]. - checkWaitChainEntry := func(txns []*txnWrapper, entry *deadlockPB.WaitForEntry, i, j int) { + checkWaitChainEntry := func(txns []*txnWrapper, entry *deadlockpb.WaitForEntry, i, j int) { c.Assert(entry.Txn, Equals, txns[i].StartTS()) c.Assert(entry.WaitForTxn, Equals, txns[j].StartTS()) c.Assert(entry.Key, BytesEquals, []byte{'k', byte(j)}) diff --git a/store/tikv/tikvrpc/tikvrpc.go b/store/tikv/tikvrpc/tikvrpc.go index ac213e0da8239..9f1b4fc0806aa 100644 --- a/store/tikv/tikvrpc/tikvrpc.go +++ b/store/tikv/tikvrpc/tikvrpc.go @@ -69,6 +69,7 @@ const ( CmdPhysicalScanLock CmdStoreSafeTS + CmdLockWaitInfo CmdCop CmdType = 512 + iota CmdCopStream @@ -168,6 +169,8 @@ func (t CmdType) String() string { return "TxnHeartBeat" case CmdStoreSafeTS: return "StoreSafeTS" + case CmdLockWaitInfo: + return "LockWaitInfo" } return "Unknown" } @@ -427,6 +430,11 @@ func (req *Request) StoreSafeTS() *kvrpcpb.StoreSafeTSRequest { return req.Req.(*kvrpcpb.StoreSafeTSRequest) } +// LockWaitInfo returns GetLockWaitInfoRequest in request. +func (req *Request) LockWaitInfo() *kvrpcpb.GetLockWaitInfoRequest { + return req.Req.(*kvrpcpb.GetLockWaitInfoRequest) +} + // ToBatchCommandsRequest converts the request to an entry in BatchCommands request. func (req *Request) ToBatchCommandsRequest() *tikvpb.BatchCommandsRequest_Request { switch req.Type { @@ -924,6 +932,8 @@ func CallRPC(ctx context.Context, client tikvpb.TikvClient, req *Request) (*Resp resp.Resp, err = client.KvTxnHeartBeat(ctx, req.TxnHeartBeat()) case CmdStoreSafeTS: resp.Resp, err = client.GetStoreSafeTS(ctx, req.StoreSafeTS()) + case CmdLockWaitInfo: + resp.Resp, err = client.GetLockWaitInfo(ctx, req.LockWaitInfo()) default: return nil, errors.Errorf("invalid request type: %v", req.Type) } diff --git a/util/mock/store.go b/util/mock/store.go index 3adba59e115e5..beefae3dc7171 100644 --- a/util/mock/store.go +++ b/util/mock/store.go @@ -16,6 +16,7 @@ package mock import ( "context" + deadlockpb "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" @@ -78,3 +79,8 @@ func (s *Store) ShowStatus(ctx context.Context, key string) (interface{}, error) func (s *Store) GetMinSafeTS(txnScope string) uint64 { return 0 } + +// GetLockWaits implements kv.Storage interface. +func (s *Store) GetLockWaits() ([]*deadlockpb.WaitForEntry, error) { + return nil, nil +} From c3a27c91ee6aafef976e7058fcb40f5c50a81f50 Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Thu, 27 May 2021 19:49:35 +0800 Subject: [PATCH 194/343] infoschema: Add ALL_SQL_DIGESTS field to the TIDB_TRX table (#24863) --- executor/executor_test.go | 8 +- infoschema/infoschema_test.go | 1 + infoschema/tables.go | 15 +- infoschema/tables_test.go | 20 +- session/session.go | 13 +- session/session_test.go | 209 ++++++++++++++++-- session/txn.go | 147 ++++++++---- session/txninfo/txn_info.go | 67 +++++- util/deadlockhistory/deadlock_history.go | 8 +- util/deadlockhistory/deadlock_history_test.go | 14 +- 10 files changed, 400 insertions(+), 102 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 65c60f392f689..47f070641e8d6 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8187,14 +8187,14 @@ func (s *testSerialSuite) TestDeadlockTable(c *C) { TryLockTxn: 101, SQLDigest: "aabbccdd", Key: []byte("k1"), - AllSQLs: nil, + AllSQLDigests: nil, TxnHoldingLock: 102, }, { TryLockTxn: 102, SQLDigest: "ddccbbaa", Key: []byte("k2"), - AllSQLs: []string{"sql1"}, + AllSQLDigests: []string{"sql1"}, TxnHoldingLock: 101, }, }, @@ -8208,12 +8208,12 @@ func (s *testSerialSuite) TestDeadlockTable(c *C) { WaitChain: []deadlockhistory.WaitChainItem{ { TryLockTxn: 201, - AllSQLs: []string{}, + AllSQLDigests: []string{}, TxnHoldingLock: 202, }, { TryLockTxn: 202, - AllSQLs: []string{"sql1", "sql2, sql3"}, + AllSQLDigests: []string{"sql1", "sql2, sql3"}, TxnHoldingLock: 203, }, { diff --git a/infoschema/infoschema_test.go b/infoschema/infoschema_test.go index 87276ef1452b9..61f34032942fe 100644 --- a/infoschema/infoschema_test.go +++ b/infoschema/infoschema_test.go @@ -298,6 +298,7 @@ func (*testSuite) TestInfoTables(c *C) { "COLLATION_CHARACTER_SET_APPLICABILITY", "PROCESSLIST", "TIDB_TRX", + "DEADLOCKS", } for _, t := range infoTables { tb, err1 := is.TableByName(util.InformationSchemaName, model.NewCIStr(t)) diff --git a/infoschema/tables.go b/infoschema/tables.go index fec9378a491b3..d4851c245d016 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1349,26 +1349,27 @@ var tableClientErrorsSummaryByHostCols = []columnInfo{ var tableTiDBTrxCols = []columnInfo{ {name: "ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.PriKeyFlag | mysql.NotNullFlag | mysql.UnsignedFlag}, - {name: "START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Start time of the transaction"}, - {name: "DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, + {name: "START_TIME", tp: mysql.TypeTimestamp, decimal: 6, size: 26, comment: "Start time of the transaction"}, + {name: "CURRENT_SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, {name: "STATE", tp: mysql.TypeEnum, enumElems: txninfo.TxnRunningStateStrs, comment: "Current running state of the transaction"}, - {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, size: 26, comment: "Current lock waiting's start time"}, + {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, decimal: 6, size: 26, comment: "Current lock waiting's start time"}, {name: "LEN", tp: mysql.TypeLonglong, size: 64, comment: "How many entries are in MemDB"}, {name: "SIZE", tp: mysql.TypeLonglong, size: 64, comment: "MemDB used memory"}, {name: "SESSION_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.UnsignedFlag, comment: "Which session this transaction belongs to"}, {name: "USER", tp: mysql.TypeVarchar, size: 16, comment: "The user who open this session"}, {name: "DB", tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"}, + {name: "ALL_SQL_DIGESTS", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the digests of SQL statements that the transaction has executed"}, } var tableDeadlocksCols = []columnInfo{ - {name: "DEADLOCK_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The ID to dinstinguish different deadlock events"}, + {name: "DEADLOCK_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The ID to distinguish different deadlock events"}, {name: "OCCUR_TIME", tp: mysql.TypeTimestamp, decimal: 6, size: 26, comment: "The physical time when the deadlock occurs"}, {name: "RETRYABLE", tp: mysql.TypeTiny, size: 1, flag: mysql.NotNullFlag, comment: "Whether the deadlock is retryable. Retryable deadlocks are usually not reported to the client"}, - {name: "TRY_LOCK_TRX_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The transaction ID (start ts) of the transaction that's trying to acquire the lock"}, + {name: "TRY_LOCK_TRX_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "The transaction ID (start ts) of the transaction that's trying to acquire the lock"}, {name: "CURRENT_SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "The digest of the SQL that's being blocked"}, {name: "KEY", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "The key on which a transaction is waiting for another"}, - {name: "ALL_SQLS", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the digests of SQL statements that the transaction has executed"}, - {name: "TRX_HOLDING_LOCK", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag, comment: "The transaction ID (start ts) of the transaction that's currently holding the lock"}, + {name: "ALL_SQL_DIGESTS", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the digests of SQL statements that the transaction has executed"}, + {name: "TRX_HOLDING_LOCK", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "The transaction ID (start ts) of the transaction that's currently holding the lock"}, } var tableDataLockWaitsCols = []columnInfo{ diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 65b5a37a4ba2a..33394e57a204c 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -23,6 +23,7 @@ import ( "runtime" "strings" "time" + "unsafe" "github.com/gorilla/mux" . "github.com/pingcap/check" @@ -1524,7 +1525,7 @@ func (s *testTableSuite) TestInfoschemaClientErrors(c *C) { func (s *testTableSuite) TestTrx(c *C) { tk := s.newTestKitWithRoot(c) _, digest := parser.NormalizeDigest("select * from trx for update;") - sm := &mockSessionManager{nil, make([]*txninfo.TxnInfo, 1)} + sm := &mockSessionManager{nil, make([]*txninfo.TxnInfo, 2)} sm.txnInfo[0] = &txninfo.TxnInfo{ StartTS: 424768545227014155, CurrentSQLDigest: digest.String(), @@ -1536,10 +1537,21 @@ func (s *testTableSuite) TestTrx(c *C) { Username: "root", CurrentDB: "test", } + blockTime2 := time.Date(2021, 05, 20, 13, 18, 30, 123456000, time.UTC) + sm.txnInfo[1] = &txninfo.TxnInfo{ + StartTS: 425070846483628033, + CurrentSQLDigest: "", + AllSQLDigests: []string{"sql1", "sql2"}, + State: txninfo.TxnLockWaiting, + BlockStartTime: unsafe.Pointer(&blockTime2), + ConnectionID: 10, + Username: "user1", + CurrentDB: "db1", + } tk.Se.SetSessionManager(sm) - tk.MustQuery("select * from information_schema.TIDB_TRX;").Check( - testkit.Rows("424768545227014155 2021-05-07 12:56:48 " + digest.String() + " Normal 1 19 2 root test"), - ) + tk.MustQuery("select * from information_schema.TIDB_TRX;").Check(testkit.Rows( + "424768545227014155 2021-05-07 04:56:48.001000 "+digest.String()+" Normal 1 19 2 root test []", + "425070846483628033 2021-05-20 13:16:35.778000 LockWaiting 2021-05-20 13:18:30.123456 0 0 10 user1 db1 [sql1, sql2]")) } func (s *testTableSuite) TestInfoschemaDeadlockPrivilege(c *C) { diff --git a/session/session.go b/session/session.go index bed1e6e2f7ae3..03860c6320b2e 100644 --- a/session/session.go +++ b/session/session.go @@ -450,7 +450,6 @@ func (s *session) TxnInfo() *txninfo.TxnInfo { return nil } processInfo := s.ShowProcess() - txnInfo.CurrentSQLDigest = processInfo.Digest txnInfo.ConnectionID = processInfo.ID txnInfo.Username = processInfo.User txnInfo.CurrentDB = processInfo.DB @@ -1502,6 +1501,9 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex // Uncorrelated subqueries will execute once when building plan, so we reset process info before building plan. cmd32 := atomic.LoadUint32(&s.GetSessionVars().CommandValue) s.SetProcessInfo(stmtNode.Text(), time.Now(), byte(cmd32), 0) + _, digest := s.sessionVars.StmtCtx.SQLDigest() + s.txn.onStmtStart(digest.String()) + defer s.txn.onStmtEnd() // Transform abstract syntax tree to a physical plan(stored in executor.ExecStmt). compiler := executor.Compiler{Ctx: s} @@ -1873,10 +1875,15 @@ func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args [ if err != nil { return nil, err } + s.txn.onStmtStart(preparedStmt.SQLDigest.String()) + var rs sqlexec.RecordSet if ok { - return s.cachedPlanExec(ctx, stmtID, preparedStmt, args) + rs, err = s.cachedPlanExec(ctx, stmtID, preparedStmt, args) + } else { + rs, err = s.preparedStmtExec(ctx, stmtID, preparedStmt, args) } - return s.preparedStmtExec(ctx, stmtID, preparedStmt, args) + s.txn.onStmtEnd() + return rs, err } func (s *session) DropPreparedStmt(stmtID uint32) error { diff --git a/session/session_test.go b/session/session_test.go index 88f6a48b83a45..f68f75f133da9 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -19,6 +19,7 @@ import ( "fmt" "os" "path" + "strconv" "strings" "sync" "sync/atomic" @@ -42,7 +43,7 @@ import ( plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" - txninfo "github.com/pingcap/tidb/session/txninfo" + "github.com/pingcap/tidb/session/txninfo" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tidb/sessionctx/variable" @@ -82,7 +83,7 @@ var _ = SerialSuites(&testSessionSerialSuite{}) var _ = SerialSuites(&testBackupRestoreSuite{}) var _ = Suite(&testClusteredSuite{}) var _ = SerialSuites(&testClusteredSerialSuite{}) -var _ = SerialSuites(&testTxnStateSuite{}) +var _ = SerialSuites(&testTxnStateSerialSuite{}) type testSessionSuiteBase struct { cluster cluster.Cluster @@ -4178,33 +4179,94 @@ func (s *testSessionSuite3) TestGlobalTemporaryTable(c *C) { tk.MustQuery("select * from g_tmp").Check(testkit.Rows()) } -type testTxnStateSuite struct { +type testTxnStateSerialSuite struct { testSessionSuiteBase } -func (s *testTxnStateSuite) TestBasic(c *C) { +func (s *testTxnStateSerialSuite) TestBasic(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t(a int);") tk.MustExec("insert into t(a) values (1);") info := tk.Se.TxnInfo() c.Assert(info, IsNil) + tk.MustExec("begin pessimistic;") - tk.MustExec("select * from t for update;") + startTSStr := tk.MustQuery("select @@tidb_current_ts;").Rows()[0][0].(string) + startTS, err := strconv.ParseUint(startTSStr, 10, 64) + c.Assert(err, IsNil) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePessimisticLock", "pause"), IsNil) + ch := make(chan interface{}) + go func() { + tk.MustExec("select * from t for update;") + ch <- nil + }() + time.Sleep(100 * time.Millisecond) info = tk.Se.TxnInfo() _, expectedDigest := parser.NormalizeDigest("select * from t for update;") c.Assert(info.CurrentSQLDigest, Equals, expectedDigest.String()) + c.Assert(info.State, Equals, txninfo.TxnLockWaiting) + c.Assert((*time.Time)(info.BlockStartTime), NotNil) + c.Assert(info.StartTS, Equals, startTS) + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePessimisticLock"), IsNil) + <-ch + + info = tk.Se.TxnInfo() + c.Assert(info.CurrentSQLDigest, Equals, "") c.Assert(info.State, Equals, txninfo.TxnRunningNormal) - c.Assert(info.BlockStartTime, IsNil) + c.Assert((*time.Time)(info.BlockStartTime), IsNil) + c.Assert(info.StartTS, Equals, startTS) + _, beginDigest := parser.NormalizeDigest("begin pessimistic;") + _, selectTSDigest := parser.NormalizeDigest("select @@tidb_current_ts;") + c.Assert(info.AllSQLDigests, DeepEquals, []string{beginDigest.String(), selectTSDigest.String(), expectedDigest.String()}) + // len and size will be covered in TestLenAndSize c.Assert(info.ConnectionID, Equals, tk.Se.GetSessionVars().ConnectionID) c.Assert(info.Username, Equals, "") c.Assert(info.CurrentDB, Equals, "test") - tk.MustExec("commit;") + c.Assert(info.StartTS, Equals, startTS) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePrewrite", "pause"), IsNil) + go func() { + tk.MustExec("commit;") + ch <- nil + }() + time.Sleep(100 * time.Millisecond) + _, commitDigest := parser.NormalizeDigest("commit;") + info = tk.Se.TxnInfo() + c.Assert(info.CurrentSQLDigest, Equals, commitDigest.String()) + c.Assert(info.State, Equals, txninfo.TxnCommitting) + c.Assert(info.AllSQLDigests, DeepEquals, []string{beginDigest.String(), selectTSDigest.String(), expectedDigest.String(), commitDigest.String()}) + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePrewrite"), IsNil) + <-ch + info = tk.Se.TxnInfo() + c.Assert(info, IsNil) + + // Test autocommit transaction + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePrewrite", "pause"), IsNil) + go func() { + tk.MustExec("insert into t values (2)") + ch <- nil + }() + time.Sleep(100 * time.Millisecond) + info = tk.Se.TxnInfo() + _, expectedDigest = parser.NormalizeDigest("insert into t values (2)") + c.Assert(info.CurrentSQLDigest, Equals, expectedDigest.String()) + c.Assert(info.State, Equals, txninfo.TxnCommitting) + c.Assert((*time.Time)(info.BlockStartTime), IsNil) + c.Assert(info.StartTS, Greater, startTS) + c.Assert(len(info.AllSQLDigests), Equals, 1) + c.Assert(info.AllSQLDigests[0], Equals, expectedDigest.String()) + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePrewrite"), IsNil) + <-ch info = tk.Se.TxnInfo() c.Assert(info, IsNil) } -func (s *testTxnStateSuite) TestEntriesCountAndSize(c *C) { +func (s *testTxnStateSerialSuite) TestEntriesCountAndSize(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t(a int);") tk.MustExec("begin pessimistic;") @@ -4219,7 +4281,7 @@ func (s *testTxnStateSuite) TestEntriesCountAndSize(c *C) { tk.MustExec("commit;") } -func (s *testTxnStateSuite) TestBlocked(c *C) { +func (s *testTxnStateSerialSuite) TestBlocked(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk2 := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t(a int);") @@ -4237,7 +4299,7 @@ func (s *testTxnStateSuite) TestBlocked(c *C) { tk.MustExec("commit;") } -func (s *testTxnStateSuite) TestCommitting(c *C) { +func (s *testTxnStateSerialSuite) TestCommitting(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk2 := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t(a int);") @@ -4249,8 +4311,10 @@ func (s *testTxnStateSuite) TestCommitting(c *C) { tk2.MustExec("begin pessimistic") c.Assert(tk2.Se.TxnInfo(), NotNil) tk2.MustExec("select * from t where a = 2 for update;") - failpoint.Enable("github.com/pingcap/tidb/session/mockSlowCommit", "sleep(200)") - defer failpoint.Disable("github.com/pingcap/tidb/session/mockSlowCommit") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockSlowCommit", "sleep(200)"), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockSlowCommit"), IsNil) + }() tk2.MustExec("commit;") ch <- struct{}{} }() @@ -4260,7 +4324,7 @@ func (s *testTxnStateSuite) TestCommitting(c *C) { <-ch } -func (s *testTxnStateSuite) TestRollbacking(c *C) { +func (s *testTxnStateSerialSuite) TestRollbacking(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t(a int);") tk.MustExec("insert into t(a) values (1), (2);") @@ -4278,6 +4342,125 @@ func (s *testTxnStateSuite) TestRollbacking(c *C) { <-ch } +func (s *testTxnStateSerialSuite) TestTxnInfoWithPreparedStmt(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t(a int)") + tk.MustExec("prepare s1 from 'insert into t values (?)'") + tk.MustExec("set @v = 1") + + tk.MustExec("begin pessimistic") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePessimisticLock", "pause"), IsNil) + ch := make(chan interface{}) + go func() { + tk.MustExec("execute s1 using @v") + ch <- nil + }() + time.Sleep(100 * time.Millisecond) + info := tk.Se.TxnInfo() + _, expectDigest := parser.NormalizeDigest("insert into t values (?)") + c.Assert(info.CurrentSQLDigest, Equals, expectDigest.String()) + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePessimisticLock"), IsNil) + <-ch + info = tk.Se.TxnInfo() + c.Assert(info.CurrentSQLDigest, Equals, "") + _, beginDigest := parser.NormalizeDigest("begin pessimistic") + c.Assert(info.AllSQLDigests, DeepEquals, []string{beginDigest.String(), expectDigest.String()}) + + tk.MustExec("rollback") +} + +func (s *testTxnStateSerialSuite) TestTxnInfoWithScalarSubquery(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t (a int, b int)") + tk.MustExec("insert into t values (1, 10), (2, 1)") + + tk.MustExec("begin pessimistic") + _, beginDigest := parser.NormalizeDigest("begin pessimistic") + tk.MustExec("select * from t where a = (select b from t where a = 2)") + _, s1Digest := parser.NormalizeDigest("select * from t where a = (select b from t where a = 2)") + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePessimisticLock", "pause"), IsNil) + ch := make(chan interface{}) + go func() { + tk.MustExec("update t set b = b + 1 where a = (select b from t where a = 2)") + ch <- nil + }() + _, s2Digest := parser.NormalizeDigest("update t set b = b + 1 where a = (select b from t where a = 1)") + time.Sleep(100 * time.Millisecond) + info := tk.Se.TxnInfo() + c.Assert(info.CurrentSQLDigest, Equals, s2Digest.String()) + c.Assert(info.AllSQLDigests, DeepEquals, []string{beginDigest.String(), s1Digest.String(), s2Digest.String()}) + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePessimisticLock"), IsNil) + <-ch + tk.MustExec("rollback") +} + +func (s *testTxnStateSerialSuite) TestTxnInfoWithPSProtocol(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t (a int primary key)") + + // Test autocommit transaction + + idInsert, _, _, err := tk.Se.PrepareStmt("insert into t values (?)") + c.Assert(err, IsNil) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePrewrite", "pause"), IsNil) + ch := make(chan interface{}) + go func() { + _, err := tk.Se.ExecutePreparedStmt(context.Background(), idInsert, types.MakeDatums(1)) + c.Assert(err, IsNil) + ch <- nil + }() + time.Sleep(100 * time.Millisecond) + _, digest := parser.NormalizeDigest("insert into t values (1)") + info := tk.Se.TxnInfo() + c.Assert(info, NotNil) + c.Assert(info.StartTS, Greater, uint64(0)) + c.Assert(info.State, Equals, txninfo.TxnCommitting) + c.Assert(info.CurrentSQLDigest, Equals, digest.String()) + c.Assert(info.AllSQLDigests, DeepEquals, []string{digest.String()}) + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePrewrite"), IsNil) + <-ch + info = tk.Se.TxnInfo() + c.Assert(info, IsNil) + + // Test non-autocommit transaction + + id1, _, _, err := tk.Se.PrepareStmt("select * from t where a = ?") + c.Assert(err, IsNil) + _, digest1 := parser.NormalizeDigest("select * from t where a = ?") + id2, _, _, err := tk.Se.PrepareStmt("update t set a = a + 1 where a = ?") + c.Assert(err, IsNil) + _, digest2 := parser.NormalizeDigest("update t set a = a + 1 where a = ?") + + tk.MustExec("begin pessimistic") + + _, err = tk.Se.ExecutePreparedStmt(context.Background(), id1, types.MakeDatums(1)) + c.Assert(err, IsNil) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/beforePessimisticLock", "pause"), IsNil) + go func() { + _, err := tk.Se.ExecutePreparedStmt(context.Background(), id2, types.MakeDatums(1)) + c.Assert(err, IsNil) + ch <- nil + }() + time.Sleep(100 * time.Millisecond) + info = tk.Se.TxnInfo() + c.Assert(info.StartTS, Greater, uint64(0)) + c.Assert(info.CurrentSQLDigest, Equals, digest2.String()) + c.Assert(info.State, Equals, txninfo.TxnLockWaiting) + c.Assert((*time.Time)(info.BlockStartTime), NotNil) + _, beginDigest := parser.NormalizeDigest("begin pessimistic") + c.Assert(info.AllSQLDigests, DeepEquals, []string{beginDigest.String(), digest1.String(), digest2.String()}) + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/beforePessimisticLock"), IsNil) + <-ch + tk.MustExec("rollback") +} + func (s *testSessionSuite) TestReadDMLBatchSize(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set global tidb_dml_batch_size=1000") diff --git a/session/txn.go b/session/txn.go index df4f2d7a62bed..bb00265044ddf 100644 --- a/session/txn.go +++ b/session/txn.go @@ -63,15 +63,9 @@ type LazyTxn struct { // we need these fields because kv.Transaction provides no thread safety promise // but we hope getting TxnInfo is a thread safe op - infoStartTS uint64 - // current executing state - State txninfo.TxnRunningState - // last trying to block start time - blockStartTime unsafe.Pointer // *time.Time, cannot use atomic.Value here because it is possible to be nil - // how many entries are there in the memBuffer, should be equal to self.(kv.Transaction).Len() - EntriesCount uint64 - // how many memory space do the entries in the memBuffer take, should be equal to self.(kv.Transaction).Size() - EntriesSize uint64 + // txnInfo provides information about the transaction in a thread-safe way. To atomically replace the struct, + // it's stored as an unsafe.Pointer. + txnInfo unsafe.Pointer } // GetTableInfo returns the cached index name. @@ -86,9 +80,9 @@ func (txn *LazyTxn) CacheTableInfo(id int64, info *model.TableInfo) { func (txn *LazyTxn) init() { txn.mutations = make(map[int64]*binlog.TableMutation) - atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) - atomic.StoreUint64(&txn.EntriesCount, 0) - atomic.StoreUint64(&txn.EntriesSize, 0) + txn.storeTxnInfo(&txninfo.TxnInfo{ + State: txninfo.TxnRunningNormal, + }) } func (txn *LazyTxn) initStmtBuf() { @@ -124,8 +118,37 @@ func (txn *LazyTxn) cleanupStmtBuf() { buf := txn.Transaction.GetMemBuffer() buf.Cleanup(txn.stagingHandle) txn.initCnt = buf.Len() - atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) - atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) + + txnInfo := txn.getTxnInfo() + atomic.StoreUint64(&txnInfo.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txnInfo.EntriesSize, uint64(txn.Transaction.Size())) +} + +func (txn *LazyTxn) storeTxnInfo(info *txninfo.TxnInfo) { + atomic.StorePointer(&txn.txnInfo, unsafe.Pointer(info)) +} + +func (txn *LazyTxn) recreateTxnInfo( + startTS uint64, + state txninfo.TxnRunningState, + entriesCount, + entriesSize uint64, + currentSQLDigest string, + allSQLDigests []string, +) { + info := &txninfo.TxnInfo{ + StartTS: startTS, + State: state, + EntriesCount: entriesCount, + EntriesSize: entriesSize, + CurrentSQLDigest: currentSQLDigest, + AllSQLDigests: allSQLDigests, + } + txn.storeTxnInfo(info) +} + +func (txn *LazyTxn) getTxnInfo() *txninfo.TxnInfo { + return (*txninfo.TxnInfo)(atomic.LoadPointer(&txn.txnInfo)) } // Size implements the MemBuffer interface. @@ -181,20 +204,20 @@ func (txn *LazyTxn) GoString() string { func (txn *LazyTxn) changeInvalidToValid(kvTxn kv.Transaction) { txn.Transaction = kvTxn - atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) - atomic.StoreUint64(&txn.infoStartTS, kvTxn.StartTS()) txn.initStmtBuf() - atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) - atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) + txn.recreateTxnInfo( + kvTxn.StartTS(), + txninfo.TxnRunningNormal, + uint64(txn.Transaction.Len()), + uint64(txn.Transaction.Size()), + "", + nil) txn.txnFuture = nil } func (txn *LazyTxn) changeInvalidToPending(future *txnFuture) { txn.Transaction = nil txn.txnFuture = future - atomic.StoreUint64(&txn.infoStartTS, 0) - atomic.StoreUint64(&txn.EntriesCount, uint64(0)) - atomic.StoreUint64(&txn.EntriesSize, uint64(0)) } func (txn *LazyTxn) changePendingToValid(ctx context.Context) error { @@ -212,11 +235,17 @@ func (txn *LazyTxn) changePendingToValid(ctx context.Context) error { return err } txn.Transaction = t - atomic.StoreInt32(&txn.State, txninfo.TxnRunningNormal) - atomic.StoreUint64(&txn.infoStartTS, t.StartTS()) txn.initStmtBuf() - atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) - atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) + + // The txnInfo may already recorded the first statement (usually "begin") when it's pending, so keep them. + txnInfo := txn.getTxnInfo() + txn.recreateTxnInfo( + t.StartTS(), + txninfo.TxnRunningNormal, + uint64(txn.Transaction.Len()), + uint64(txn.Transaction.Size()), + txnInfo.CurrentSQLDigest, + txnInfo.AllSQLDigests) return nil } @@ -227,9 +256,36 @@ func (txn *LazyTxn) changeToInvalid() { txn.stagingHandle = kv.InvalidStagingHandle txn.Transaction = nil txn.txnFuture = nil - atomic.StoreUint64(&txn.infoStartTS, 0) - atomic.StoreUint64(&txn.EntriesCount, 0) - atomic.StoreUint64(&txn.EntriesSize, 0) + + txn.recreateTxnInfo( + 0, + txninfo.TxnRunningNormal, + 0, + 0, + "", + nil) +} + +func (txn *LazyTxn) onStmtStart(currentSQLDigest string) { + if len(currentSQLDigest) == 0 { + return + } + + info := txn.getTxnInfo().ShallowClone() + info.CurrentSQLDigest = currentSQLDigest + // Keeps at most 50 history sqls to avoid consuming too much memory. + const maxTransactionStmtHistory int = 50 + if len(info.AllSQLDigests) < maxTransactionStmtHistory { + info.AllSQLDigests = append(info.AllSQLDigests, currentSQLDigest) + } + + txn.storeTxnInfo(info) +} + +func (txn *LazyTxn) onStmtEnd() { + info := txn.getTxnInfo().ShallowClone() + info.CurrentSQLDigest = "" + txn.storeTxnInfo(info) } var hasMockAutoIncIDRetry = int64(0) @@ -269,7 +325,7 @@ func (txn *LazyTxn) Commit(ctx context.Context) error { return errors.Trace(kv.ErrInvalidTxn) } - atomic.StoreInt32(&txn.State, txninfo.TxnCommitting) + atomic.StoreInt32(&txn.getTxnInfo().State, txninfo.TxnCommitting) failpoint.Inject("mockSlowCommit", func(_ failpoint.Value) {}) @@ -301,7 +357,7 @@ func (txn *LazyTxn) Commit(ctx context.Context) error { // Rollback overrides the Transaction interface. func (txn *LazyTxn) Rollback() error { defer txn.reset() - atomic.StoreInt32(&txn.State, txninfo.TxnRollingBack) + atomic.StoreInt32(&txn.getTxnInfo().State, txninfo.TxnRollingBack) // mockSlowRollback is used to mock a rollback which takes a long time failpoint.Inject("mockSlowRollback", func(_ failpoint.Value) {}) return txn.Transaction.Rollback() @@ -309,15 +365,15 @@ func (txn *LazyTxn) Rollback() error { // LockKeys Wrap the inner transaction's `LockKeys` to record the status func (txn *LazyTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { - originState := atomic.LoadInt32(&txn.State) - atomic.StoreInt32(&txn.State, txninfo.TxnLockWaiting) + txnInfo := txn.getTxnInfo() + originState := atomic.SwapInt32(&txnInfo.State, txninfo.TxnLockWaiting) t := time.Now() - atomic.StorePointer(&txn.blockStartTime, unsafe.Pointer(&t)) + atomic.StorePointer(&txnInfo.BlockStartTime, unsafe.Pointer(&t)) err := txn.Transaction.LockKeys(ctx, lockCtx, keys...) - atomic.StorePointer(&txn.blockStartTime, unsafe.Pointer(nil)) - atomic.StoreInt32(&txn.State, originState) - atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) - atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) + atomic.StorePointer(&txnInfo.BlockStartTime, unsafe.Pointer(nil)) + atomic.StoreInt32(&txnInfo.State, originState) + atomic.StoreUint64(&txnInfo.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txnInfo.EntriesSize, uint64(txn.Transaction.Size())) return err } @@ -376,17 +432,11 @@ func keyNeedToLock(k, v []byte, flags kv.KeyFlags) bool { // Info dump the TxnState to Datum for displaying in `TIDB_TRX` // This function is supposed to be thread safe func (txn *LazyTxn) Info() *txninfo.TxnInfo { - startTs := atomic.LoadUint64(&txn.infoStartTS) - if startTs == 0 { + info := txn.getTxnInfo().ShallowClone() + if info.StartTS == 0 { return nil } - return &txninfo.TxnInfo{ - StartTS: startTs, - State: atomic.LoadInt32(&txn.State), - BlockStartTime: (*time.Time)(atomic.LoadPointer(&txn.blockStartTime)), - EntriesCount: atomic.LoadUint64(&txn.EntriesCount), - EntriesSize: atomic.LoadUint64(&txn.EntriesSize), - } + return info } // UpdateEntriesCountAndSize updates the EntriesCount and EntriesSize @@ -394,8 +444,9 @@ func (txn *LazyTxn) Info() *txninfo.TxnInfo { // txn.Transaction can be changed during this function's execution if running parallel. func (txn *LazyTxn) UpdateEntriesCountAndSize() { if txn.Valid() { - atomic.StoreUint64(&txn.EntriesCount, uint64(txn.Transaction.Len())) - atomic.StoreUint64(&txn.EntriesSize, uint64(txn.Transaction.Size())) + txnInfo := txn.getTxnInfo() + atomic.StoreUint64(&txnInfo.EntriesCount, uint64(txn.Transaction.Len())) + atomic.StoreUint64(&txnInfo.EntriesSize, uint64(txn.Transaction.Size())) } } diff --git a/session/txninfo/txn_info.go b/session/txninfo/txn_info.go index 77a2d8c90cd05..acc52e985f0f9 100644 --- a/session/txninfo/txn_info.go +++ b/session/txninfo/txn_info.go @@ -14,7 +14,10 @@ package txninfo import ( + "strings" + "sync/atomic" "time" + "unsafe" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/store/tikv/oracle" @@ -43,19 +46,28 @@ var TxnRunningStateStrs = []string{ // TxnInfo is information about a running transaction // This is supposed to be the datasource of `TIDB_TRX` in infoschema type TxnInfo struct { + // The following fields are immutable and can be safely read across threads. + StartTS uint64 - // digest of SQL current running + // Digest of SQL currently running CurrentSQLDigest string - // current executing State + // Digests of all SQLs executed in the transaction. + AllSQLDigests []string + + // The following fields are mutable and needs to be read or written by atomic operations. But since only the + // transaction's thread can modify its value, it's ok for the transaction's thread to read it without atomic + // operations. + + // Current execution state of the transaction. State TxnRunningState - // last trying to block start time - BlockStartTime *time.Time + // Last trying to block start time. Invalid if State is not TxnLockWaiting. It's an unsafe pointer to time.Time or nil. + BlockStartTime unsafe.Pointer // How many entries are in MemDB EntriesCount uint64 // MemDB used memory EntriesSize uint64 - // the following fields will be filled in `session` instead of `LazyTxn` + // The following fields will be filled in `session` instead of `LazyTxn` // Which session this transaction belongs to ConnectionID uint64 @@ -65,24 +77,54 @@ type TxnInfo struct { CurrentDB string } -// ToDatum Converts the `TxnInfo` to `Datum` to show in the `TIDB_TRX` table +// ShallowClone shallow clones the TxnInfo. It's safe to call concurrently with the transaction. +// Note that this function doesn't do deep copy and some fields of the result may be unsafe to write. Use it at your own +// risk. +func (info *TxnInfo) ShallowClone() *TxnInfo { + return &TxnInfo{ + StartTS: info.StartTS, + CurrentSQLDigest: info.CurrentSQLDigest, + AllSQLDigests: info.AllSQLDigests, + State: atomic.LoadInt32(&info.State), + BlockStartTime: atomic.LoadPointer(&info.BlockStartTime), + EntriesCount: atomic.LoadUint64(&info.EntriesCount), + EntriesSize: atomic.LoadUint64(&info.EntriesSize), + ConnectionID: info.ConnectionID, + Username: info.Username, + CurrentDB: info.CurrentDB, + } +} + +// ToDatum Converts the `TxnInfo` to `Datum` to show in the `TIDB_TRX` table. func (info *TxnInfo) ToDatum() []types.Datum { - humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(info.StartTS)*1e6) + // TODO: The timezone represented to the user is not correct and it will be always UTC time. + humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(info.StartTS)*1e6).UTC() + + var currentDigest interface{} + if len(info.CurrentSQLDigest) != 0 { + currentDigest = info.CurrentSQLDigest + } + var blockStartTime interface{} - if info.BlockStartTime == nil { + if t := (*time.Time)(atomic.LoadPointer(&info.BlockStartTime)); t == nil { blockStartTime = nil } else { - blockStartTime = types.NewTime(types.FromGoTime(*info.BlockStartTime), mysql.TypeTimestamp, 0) + blockStartTime = types.NewTime(types.FromGoTime(*t), mysql.TypeTimestamp, types.MaxFsp) } + e, err := types.ParseEnumValue(TxnRunningStateStrs, uint64(info.State+1)) if err != nil { panic("this should never happen") } + + allSQLs := "[" + strings.Join(info.AllSQLDigests, ", ") + "]" + state := types.NewMysqlEnumDatum(e) + datums := types.MakeDatums( info.StartTS, - types.NewTime(types.FromGoTime(humanReadableStartTime), mysql.TypeTimestamp, 0), - info.CurrentSQLDigest, + types.NewTime(types.FromGoTime(humanReadableStartTime), mysql.TypeTimestamp, types.MaxFsp), + currentDigest, ) datums = append(datums, state) datums = append(datums, types.MakeDatums( @@ -91,6 +133,7 @@ func (info *TxnInfo) ToDatum() []types.Datum { info.EntriesSize, info.ConnectionID, info.Username, - info.CurrentDB)...) + info.CurrentDB, + allSQLs)...) return datums } diff --git a/util/deadlockhistory/deadlock_history.go b/util/deadlockhistory/deadlock_history.go index c219442cf5bf1..ee2fb496a2a58 100644 --- a/util/deadlockhistory/deadlock_history.go +++ b/util/deadlockhistory/deadlock_history.go @@ -32,7 +32,7 @@ type WaitChainItem struct { TryLockTxn uint64 SQLDigest string Key []byte - AllSQLs []string + AllSQLDigests []string TxnHoldingLock uint64 } @@ -149,8 +149,8 @@ func (d *DeadlockHistory) GetAllDatum() [][]types.Datum { } row[6] = nil - if item.AllSQLs != nil { - row[6] = "[" + strings.Join(item.AllSQLs, ", ") + "]" + if item.AllSQLDigests != nil { + row[6] = "[" + strings.Join(item.AllSQLDigests, ", ") + "]" } row[7] = item.TxnHoldingLock @@ -185,7 +185,7 @@ func ErrDeadlockToDeadlockRecord(dl *tikverr.ErrDeadlock) *DeadlockRecord { TryLockTxn: rawItem.Txn, SQLDigest: hex.EncodeToString(sqlDigest), Key: rawItem.Key, - AllSQLs: nil, + AllSQLDigests: nil, TxnHoldingLock: rawItem.WaitForTxn, }) } diff --git a/util/deadlockhistory/deadlock_history_test.go b/util/deadlockhistory/deadlock_history_test.go index dd9428a9f550a..398d2943ab996 100644 --- a/util/deadlockhistory/deadlock_history_test.go +++ b/util/deadlockhistory/deadlock_history_test.go @@ -148,7 +148,7 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) { TryLockTxn: 101, SQLDigest: "sql1", Key: []byte("k1"), - AllSQLs: []string{"sql1", "sql2"}, + AllSQLDigests: []string{"sql1", "sql2"}, TxnHoldingLock: 102, }, // It should work even some information are missing. @@ -164,12 +164,12 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) { WaitChain: []WaitChainItem{ { TryLockTxn: 201, - AllSQLs: []string{}, + AllSQLDigests: []string{}, TxnHoldingLock: 202, }, { TryLockTxn: 202, - AllSQLs: []string{"sql1"}, + AllSQLDigests: []string{"sql1"}, TxnHoldingLock: 201, }, }, @@ -201,7 +201,7 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) { c.Assert(res[0][3].GetValue(), Equals, uint64(101)) // TRY_LOCK_TRX_ID c.Assert(res[0][4].GetValue(), Equals, "sql1") // SQL_DIGEST c.Assert(res[0][5].GetValue(), Equals, "6B31") // KEY - c.Assert(res[0][6].GetValue(), Equals, "[sql1, sql2]") // ALL_SQLS + c.Assert(res[0][6].GetValue(), Equals, "[sql1, sql2]") // ALL_SQL_DIGESTS c.Assert(res[0][7].GetValue(), Equals, uint64(102)) // TRX_HOLDING_LOCK c.Assert(res[1][0].GetValue(), Equals, uint64(1)) // ID @@ -210,21 +210,21 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) { c.Assert(res[1][3].GetValue(), Equals, uint64(102)) // TRY_LOCK_TRX_ID c.Assert(res[1][4].GetValue(), Equals, nil) // SQL_DIGEST c.Assert(res[1][5].GetValue(), Equals, nil) // KEY - c.Assert(res[1][6].GetValue(), Equals, nil) // ALL_SQLS + c.Assert(res[1][6].GetValue(), Equals, nil) // ALL_SQL_DIGESTS c.Assert(res[1][7].GetValue(), Equals, uint64(101)) // TRX_HOLDING_LOCK c.Assert(res[2][0].GetValue(), Equals, uint64(2)) // ID c.Assert(toGoTime(res[2][1]), Equals, time2) // OCCUR_TIME c.Assert(res[2][2].GetValue(), Equals, int64(1)) // RETRYABLE c.Assert(res[2][3].GetValue(), Equals, uint64(201)) // TRY_LOCK_TRX_ID - c.Assert(res[2][6].GetValue(), Equals, "[]") // ALL_SQLS + c.Assert(res[2][6].GetValue(), Equals, "[]") // ALL_SQL_DIGESTS c.Assert(res[2][7].GetValue(), Equals, uint64(202)) // TRX_HOLDING_LOCK c.Assert(res[3][0].GetValue(), Equals, uint64(2)) // ID c.Assert(toGoTime(res[3][1]), Equals, time2) // OCCUR_TIME c.Assert(res[3][2].GetValue(), Equals, int64(1)) // RETRYABLE c.Assert(res[3][3].GetValue(), Equals, uint64(202)) // TRY_LOCK_TRX_ID - c.Assert(res[3][6].GetValue(), Equals, "[sql1]") // ALL_SQLS + c.Assert(res[3][6].GetValue(), Equals, "[sql1]") // ALL_SQL_DIGESTS c.Assert(res[3][7].GetValue(), Equals, uint64(201)) // TRX_HOLDING_LOCK } From f3557bb5918d1b23cf571fbb6b6ee53397056982 Mon Sep 17 00:00:00 2001 From: ShuNing Date: Thu, 27 May 2021 21:11:35 +0800 Subject: [PATCH 195/343] planner, executor: supports select statement with AS OF (#24613) --- executor/adapter.go | 36 +++++++-- executor/builder.go | 1 - executor/compiler.go | 10 +-- executor/executor_test.go | 12 +-- executor/metrics_reader_test.go | 9 +-- executor/prepared.go | 13 ++-- executor/stale_txn_test.go | 113 ++++++++++++++++++++++++++++ expression/integration_test.go | 6 +- expression/typeinfer_test.go | 7 +- go.mod | 3 +- go.sum | 11 +-- planner/core/cbo_test.go | 30 ++++---- planner/core/common_plans.go | 4 +- planner/core/errors.go | 1 + planner/core/indexmerge_test.go | 2 +- planner/core/logical_plan_test.go | 33 ++++---- planner/core/physical_plan_test.go | 4 +- planner/core/planbuilder.go | 10 +-- planner/core/point_get_plan_test.go | 6 +- planner/core/preprocess.go | 85 +++++++++++++++++++-- planner/core/preprocess_test.go | 2 +- planner/core/stats_test.go | 6 +- server/conn.go | 5 +- session/session.go | 2 +- statistics/selectivity_test.go | 20 ++--- util/ranger/ranger_test.go | 36 ++++----- 26 files changed, 337 insertions(+), 130 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 64b0ac1f1e5b8..df6e62751f90a 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -26,6 +26,7 @@ import ( "github.com/cznic/mathutil" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" @@ -43,6 +44,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -177,6 +179,9 @@ func (a *recordSet) OnFetchReturned() { type ExecStmt struct { // GoCtx stores parent go context.Context for a stmt. GoCtx context.Context + // SnapshotTS stores the timestamp for stale read. + // It is not equivalent to session variables's snapshot ts, it only use to build the executor. + SnapshotTS uint64 // InfoSchema stores a reference to the schema information. InfoSchema infoschema.InfoSchema // Plan stores a reference to the final physical plan. @@ -268,18 +273,19 @@ func (a *ExecStmt) IsReadOnly(vars *variable.SessionVars) bool { // RebuildPlan rebuilds current execute statement plan. // It returns the current information schema version that 'a' is using. func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { - is := a.Ctx.GetInfoSchema().(infoschema.InfoSchema) - a.InfoSchema = is - if err := plannercore.Preprocess(a.Ctx, a.StmtNode, is, plannercore.InTxnRetry); err != nil { + ret := &plannercore.PreprocessorReturn{} + if err := plannercore.Preprocess(a.Ctx, a.StmtNode, plannercore.InTxnRetry, plannercore.WithPreprocessorReturn(ret)); err != nil { return 0, err } - p, names, err := planner.Optimize(ctx, a.Ctx, a.StmtNode, is) + a.InfoSchema = ret.InfoSchema + a.SnapshotTS = ret.SnapshotTS + p, names, err := planner.Optimize(ctx, a.Ctx, a.StmtNode, a.InfoSchema) if err != nil { return 0, err } a.OutputNames = names a.Plan = p - return is.SchemaMetaVersion(), nil + return a.InfoSchema.SchemaMetaVersion(), nil } // Exec builds an Executor from a plan. If the Executor doesn't return result, @@ -305,6 +311,25 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { logutil.Logger(ctx).Error("execute sql panic", zap.String("sql", a.GetTextToLog()), zap.Stack("stack")) }() + failpoint.Inject("assertStaleTSO", func(val failpoint.Value) { + if n, ok := val.(int); ok { + startTS := oracle.ExtractPhysical(a.SnapshotTS) / 1000 + if n != int(startTS) { + panic("different tso") + } + failpoint.Return() + } + }) + failpoint.Inject("assertStaleTSOWithTolerance", func(val failpoint.Value) { + if n, ok := val.(int); ok { + // Convert to seconds + startTS := oracle.ExtractPhysical(a.SnapshotTS) / 1000 + if int(startTS) <= n-1 || n+1 <= int(startTS) { + panic("tso violate tolerance") + } + failpoint.Return() + } + }) sctx := a.Ctx ctx = util.SetSessionID(ctx, sctx.GetSessionVars().ConnectionID) if _, ok := a.Plan.(*plannercore.Analyze); ok && sctx.GetSessionVars().InRestrictedSQL { @@ -747,6 +772,7 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { } b := newExecutorBuilder(ctx, a.InfoSchema) + b.snapshotTS = a.SnapshotTS e := b.build(a.Plan) if b.err != nil { return nil, errors.Trace(b.err) diff --git a/executor/builder.go b/executor/builder.go index 2f644a6eed2c4..67a0c33cb1fcb 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -640,7 +640,6 @@ func (b *executorBuilder) buildPrepare(v *plannercore.Prepare) Executor { base.initCap = chunk.ZeroCapacity return &PrepareExec{ baseExecutor: base, - is: b.is, name: v.Name, sqlText: v.SQLText, } diff --git a/executor/compiler.go b/executor/compiler.go index 5a658878ca747..673f17f24f40e 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" @@ -53,13 +52,13 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm ctx = opentracing.ContextWithSpan(ctx, span1) } - infoSchema := c.Ctx.GetInfoSchema().(infoschema.InfoSchema) - if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil { + ret := &plannercore.PreprocessorReturn{} + if err := plannercore.Preprocess(c.Ctx, stmtNode, plannercore.WithPreprocessorReturn(ret)); err != nil { return nil, err } stmtNode = plannercore.TryAddExtraLimit(c.Ctx, stmtNode) - finalPlan, names, err := planner.Optimize(ctx, c.Ctx, stmtNode, infoSchema) + finalPlan, names, err := planner.Optimize(ctx, c.Ctx, stmtNode, ret.InfoSchema) if err != nil { return nil, err } @@ -71,7 +70,8 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm } return &ExecStmt{ GoCtx: ctx, - InfoSchema: infoSchema, + SnapshotTS: ret.SnapshotTS, + InfoSchema: ret.InfoSchema, Plan: finalPlan, LowerPriority: lowerPriority, Text: stmtNode.Text(), diff --git a/executor/executor_test.go b/executor/executor_test.go index 47f070641e8d6..0e7eab38ac9e8 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -2349,14 +2349,14 @@ func (s *testSuiteP2) TestIsPointGet(c *C) { "select * from help_topic where help_topic_id=1": true, "select * from help_topic where help_category_id=1": false, } - infoSchema := ctx.GetInfoSchema().(infoschema.InfoSchema) for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") c.Check(err, IsNil) - err = plannercore.Preprocess(ctx, stmtNode, infoSchema) + preprocessorReturn := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(ctx, stmtNode, plannercore.WithPreprocessorReturn(preprocessorReturn)) c.Check(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), ctx, stmtNode, infoSchema) + p, _, err := planner.Optimize(context.TODO(), ctx, stmtNode, preprocessorReturn.InfoSchema) c.Check(err, IsNil) ret, err := plannercore.IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx, p) c.Assert(err, IsNil) @@ -2381,13 +2381,13 @@ func (s *testSuiteP2) TestClusteredIndexIsPointGet(c *C) { "select * from t where a='x' and c='x'": true, "select * from t where a='x' and c='x' and b=1": false, } - infoSchema := ctx.GetInfoSchema().(infoschema.InfoSchema) for sqlStr, result := range tests { stmtNode, err := s.ParseOneStmt(sqlStr, "", "") c.Check(err, IsNil) - err = plannercore.Preprocess(ctx, stmtNode, infoSchema) + preprocessorReturn := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(ctx, stmtNode, plannercore.WithPreprocessorReturn(preprocessorReturn)) c.Check(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), ctx, stmtNode, infoSchema) + p, _, err := planner.Optimize(context.TODO(), ctx, stmtNode, preprocessorReturn.InfoSchema) c.Check(err, IsNil) ret, err := plannercore.IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx, p) c.Assert(err, IsNil) diff --git a/executor/metrics_reader_test.go b/executor/metrics_reader_test.go index 51bdc560ed6d0..8d75ac41fd96e 100644 --- a/executor/metrics_reader_test.go +++ b/executor/metrics_reader_test.go @@ -20,10 +20,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" - "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/testkit" ) @@ -62,10 +60,11 @@ func (s *testSuite7) TestStmtLabel(c *C) { for _, tt := range tests { stmtNode, err := parser.New().ParseOneStmt(tt.sql, "", "") c.Check(err, IsNil) - is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) - err = plannercore.Preprocess(tk.Se.(sessionctx.Context), stmtNode, is) + preprocessorReturn := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(tk.Se, stmtNode, plannercore.WithPreprocessorReturn(preprocessorReturn)) + c.Check(err, IsNil) c.Assert(err, IsNil) - _, _, err = planner.Optimize(context.TODO(), tk.Se, stmtNode, is) + _, _, err = planner.Optimize(context.TODO(), tk.Se, stmtNode, preprocessorReturn.InfoSchema) c.Assert(err, IsNil) c.Assert(executor.GetStmtLabel(stmtNode), Equals, tt.label) } diff --git a/executor/prepared.go b/executor/prepared.go index a3d950ec0afc9..f494ebac3dc9d 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -79,7 +79,6 @@ func (e *paramMarkerExtractor) Leave(in ast.Node) (ast.Node, bool) { type PrepareExec struct { baseExecutor - is infoschema.InfoSchema name string sqlText string @@ -89,12 +88,11 @@ type PrepareExec struct { } // NewPrepareExec creates a new PrepareExec. -func NewPrepareExec(ctx sessionctx.Context, is infoschema.InfoSchema, sqlTxt string) *PrepareExec { +func NewPrepareExec(ctx sessionctx.Context, sqlTxt string) *PrepareExec { base := newBaseExecutor(ctx, nil, 0) base.initCap = chunk.ZeroCapacity return &PrepareExec{ baseExecutor: base, - is: is, sqlText: sqlTxt, } } @@ -159,7 +157,8 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { return ErrPsManyParam } - err = plannercore.Preprocess(e.ctx, stmt, e.is, plannercore.InPrepare) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(e.ctx, stmt, plannercore.InPrepare, plannercore.WithPreprocessorReturn(ret)) if err != nil { return err } @@ -177,14 +176,14 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { Stmt: stmt, StmtType: GetStmtLabel(stmt), Params: sorter.markers, - SchemaVersion: e.is.SchemaMetaVersion(), + SchemaVersion: ret.InfoSchema.SchemaMetaVersion(), } if !plannercore.PreparedPlanCacheEnabled() { prepared.UseCache = false } else { if !e.ctx.GetSessionVars().UseDynamicPartitionPrune() { - prepared.UseCache = plannercore.Cacheable(stmt, e.is) + prepared.UseCache = plannercore.Cacheable(stmt, ret.InfoSchema) } else { prepared.UseCache = plannercore.Cacheable(stmt, nil) } @@ -199,7 +198,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { var p plannercore.Plan e.ctx.GetSessionVars().PlanID = 0 e.ctx.GetSessionVars().PlanColumnID = 0 - destBuilder, _ := plannercore.NewPlanBuilder(e.ctx, e.is, &hint.BlockHintProcessor{}) + destBuilder, _ := plannercore.NewPlanBuilder(e.ctx, ret.InfoSchema, &hint.BlockHintProcessor{}) p, err = destBuilder.Build(ctx, stmt) if err != nil { return err diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index a36dd6073654e..f39cec6f9c220 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -18,6 +18,7 @@ import ( "time" . "github.com/pingcap/check" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/store/tikv/oracle" @@ -93,6 +94,118 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") } +func (s *testStaleTxnSerialSuite) TestSelectAsOf(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec(`drop table if exists b`) + tk.MustExec("create table t (id int primary key);") + tk.MustExec("create table b (pid int primary key);") + defer func() { + tk.MustExec(`drop table if exists b`) + tk.MustExec(`drop table if exists t`) + }() + time.Sleep(2 * time.Second) + now := time.Now() + time.Sleep(2 * time.Second) + + testcases := []struct { + name string + sql string + expectPhysicalTS int64 + preSec int64 + // IsStaleness is auto cleanup in select stmt. + errorStr string + }{ + { + name: "TimestampExactRead1", + sql: fmt.Sprintf("select * from t as of timestamp '%s';", now.Format("2006-1-2 15:04:05")), + expectPhysicalTS: now.Unix(), + }, + { + name: "NomalRead", + sql: `select * from b;`, + preSec: 0, + }, + { + name: "TimestampExactRead2", + sql: fmt.Sprintf("select * from t as of timestamp TIMESTAMP('%s');", now.Format("2006-1-2 15:04:05")), + expectPhysicalTS: now.Unix(), + }, + { + name: "TimestampExactRead3", + sql: `select * from t as of timestamp NOW() - INTERVAL 2 SECOND;`, + preSec: 2, + }, + { + name: "TimestampExactRead4", + sql: `select * from t as of timestamp TIMESTAMP(NOW() - INTERVAL 2 SECOND);`, + preSec: 2, + }, + { + name: "TimestampExactRead5", + sql: `select * from t as of timestamp TIMESTAMP(NOW() - INTERVAL 1 SECOND), b as of timestamp TIMESTAMP(NOW() - INTERVAL 1 SECOND);`, + preSec: 1, + }, + { + name: "TimestampExactRead6", + sql: `select * from t as of timestamp TIMESTAMP(NOW() - INTERVAL 1 SECOND), b as of timestamp TIMESTAMP('2020-09-06 00:00:00');`, + errorStr: ".*can not set different time in the as of.*", + }, + { + name: "TimestampExactRead7", + sql: `select * from t as of timestamp TIMESTAMP(NOW() - INTERVAL 1 SECOND), b;`, + errorStr: ".*can not set different time in the as of.*", + }, + { + name: "TimestampExactRead8", + sql: `select * from t, b as of timestamp TIMESTAMP(NOW() - INTERVAL 1 SECOND);`, + errorStr: ".*can not set different time in the as of.*", + }, + { + name: "NomalRead", + sql: `select * from t, b;`, + preSec: 0, + }, + { + name: "TimestampExactRead9", + sql: `select * from (select * from t as of timestamp TIMESTAMP(NOW() - INTERVAL 1 SECOND), b as of timestamp TIMESTAMP(NOW() - INTERVAL 1 SECOND)) as c, b;`, + errorStr: ".*can not set different time in the as of.*", + }, + { + name: "TimestampExactRead10", + sql: `select * from (select * from t as of timestamp TIMESTAMP(NOW() - INTERVAL 2 SECOND), b as of timestamp TIMESTAMP(NOW() - INTERVAL 2 SECOND)) as c;`, + preSec: 2, + }, + // Cannot be supported the SubSelect + { + name: "TimestampExactRead11", + sql: `select * from (select * from t as of timestamp TIMESTAMP(NOW() - INTERVAL 20 SECOND), b as of timestamp TIMESTAMP(NOW() - INTERVAL 20 SECOND)) as c as of timestamp Now();`, + errorStr: ".*You have an error in your SQL syntax.*", + }, + } + + for _, testcase := range testcases { + c.Log(testcase.name) + if testcase.expectPhysicalTS > 0 { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSO", fmt.Sprintf(`return(%d)`, testcase.expectPhysicalTS)), IsNil) + } else if testcase.preSec > 0 { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSOWithTolerance", fmt.Sprintf(`return(%d)`, time.Now().Unix()-testcase.preSec)), IsNil) + } + _, err := tk.Exec(testcase.sql) + if len(testcase.errorStr) != 0 { + c.Assert(err, ErrorMatches, testcase.errorStr) + continue + } + c.Assert(err, IsNil, Commentf("sql:%s, error stack %v", testcase.sql, errors.ErrorStack(err))) + if testcase.expectPhysicalTS > 0 { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSO"), IsNil) + } else if testcase.preSec > 0 { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSOWithTolerance"), IsNil) + } + } +} + func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/expression/integration_test.go b/expression/integration_test.go index a4706629660a0..1d883964d996a 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -4850,10 +4850,10 @@ func (s *testIntegrationSuite) TestFilterExtractFromDNF(c *C) { stmts, err := session.Parse(sctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) - is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) + p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) conds := make([]expression.Expression, len(selection.Conditions)) diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index bce929c933852..853873db7d749 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -20,7 +20,6 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/domain" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" @@ -139,10 +138,10 @@ func (s *testInferTypeSuite) TestInferType(c *C) { err = se.NewTxn(context.Background()) c.Assert(err, IsNil) - is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmt, is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmt, plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, comment) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmt, is) + p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmt, ret.InfoSchema) c.Assert(err, IsNil, comment) tp := p.Schema().Columns[0].RetType diff --git a/go.mod b/go.mod index 0879d5f157bf4..01cc11c222859 100644 --- a/go.mod +++ b/go.mod @@ -66,7 +66,7 @@ require ( go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b go.uber.org/atomic v1.7.0 go.uber.org/automaxprocs v1.2.0 - go.uber.org/multierr v1.6.0 // indirect + go.uber.org/multierr v1.7.0 // indirect go.uber.org/zap v1.16.0 golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a // indirect golang.org/x/mod v0.4.2 // indirect @@ -79,7 +79,6 @@ require ( gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 gopkg.in/yaml.v2 v2.4.0 // indirect - gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 // indirect honnef.co/go/tools v0.1.4 // indirect modernc.org/mathutil v1.2.2 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 diff --git a/go.sum b/go.sum index 664a52bdca20d..0ae5bdaf51b9e 100644 --- a/go.sum +++ b/go.sum @@ -538,8 +538,9 @@ github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UV github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.5.1/go.mod h1:5W2xD1RspED5o8YsWQXVCued0rvSQ+mT+I5cxcmMvtA= github.com/stretchr/testify v1.6.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.6.1 h1:hDPOHmpOpP40lSULcqw7IrRb/u7w6RpDC9399XyoNd0= 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/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= @@ -623,8 +624,8 @@ go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/ go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.4.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.5.0/go.mod h1:FeouvMocqHpRaaGuG9EjoKcStLC43Zu/fmqdUMPcKYU= -go.uber.org/multierr v1.6.0 h1:y6IPFStTAIT5Ytl7/XYmHvzXQ7S3g/IeZW9hyZ5thw4= -go.uber.org/multierr v1.6.0/go.mod h1:cdWPpRnG4AhwMwsgIHip0KRBQjJy5kYEpYjJxpXp9iU= +go.uber.org/multierr v1.7.0 h1:zaiO/rmgFjbmCXdSYJWQcdvOCsthmdaHfr3Gm2Kx4Ec= +go.uber.org/multierr v1.7.0/go.mod h1:7EAYxJLBy9rStEaz58O2t4Uvip6FSURkq8/ppBp95ak= go.uber.org/tools v0.0.0-20190618225709-2cfd321de3ee/go.mod h1:vJERXedbb3MVM5f9Ejo0C68/HhF8uaILCdgjnY+goOA= go.uber.org/zap v1.8.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= @@ -919,8 +920,8 @@ gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776 h1:tQIYjPdBoyREyB9XMu+nnTclpTYkz2zFM+lzLJFO4gQ= -gopkg.in/yaml.v3 v3.0.0-20200615113413-eeeca48fe776/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b h1:h8qDotaEPuJATrMmW04NCwg7v22aHH28wwpauUhK9Oo= +gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index a46a48c7191ff..ffa0b7ccde75e 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -295,10 +295,10 @@ func (s *testAnalyzeSuite) TestIndexRead(c *C) { c.Assert(err, IsNil) c.Assert(stmts, HasLen, 1) stmt := stmts[0] - is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is) + ret := &core.PreprocessorReturn{} + err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) c.Assert(err, IsNil) planString := core.ToString(p) s.testData.OnRecord(func() { @@ -330,10 +330,10 @@ func (s *testAnalyzeSuite) TestEmptyTable(c *C) { c.Assert(err, IsNil) c.Assert(stmts, HasLen, 1) stmt := stmts[0] - is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is) + ret := &core.PreprocessorReturn{} + err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) c.Assert(err, IsNil) planString := core.ToString(p) s.testData.OnRecord(func() { @@ -402,10 +402,10 @@ func (s *testAnalyzeSuite) TestAnalyze(c *C) { stmt := stmts[0] err = executor.ResetContextOfStmt(ctx, stmt) c.Assert(err, IsNil) - is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is) + ret := &core.PreprocessorReturn{} + err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) c.Assert(err, IsNil) planString := core.ToString(p) s.testData.OnRecord(func() { @@ -491,10 +491,10 @@ func (s *testAnalyzeSuite) TestPreparedNullParam(c *C) { c.Assert(err, IsNil) stmt := stmts[0] - is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is, core.InPrepare) + ret := &core.PreprocessorReturn{} + err = core.Preprocess(ctx, stmt, core.InPrepare, core.WithPreprocessorReturn(ret)) c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) c.Assert(err, IsNil) c.Assert(core.ToString(p), Equals, best, Commentf("for %s", sql)) @@ -726,14 +726,14 @@ func BenchmarkOptimize(b *testing.B) { c.Assert(err, IsNil) c.Assert(stmts, HasLen, 1) stmt := stmts[0] - is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is) + ret := &core.PreprocessorReturn{} + err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) c.Assert(err, IsNil) b.Run(tt.sql, func(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - _, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) + _, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) c.Assert(err, IsNil) } b.ReportAllocs() diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index ad68fd111d617..a20e2bf12a359 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -265,7 +265,9 @@ func (e *Execute) OptimizePreparedPlan(ctx context.Context, sctx sessionctx.Cont preparedObj.Executor = 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. - err := Preprocess(sctx, prepared.Stmt, is, InPrepare) + // FIXME: compatible with prepare https://github.com/pingcap/tidb/issues/24932 + ret := &PreprocessorReturn{InfoSchema: is} + err := Preprocess(sctx, prepared.Stmt, InPrepare, WithPreprocessorReturn(ret)) if err != nil { return ErrSchemaChanged.GenWithStack("Schema change caused error: %s", err.Error()) } diff --git a/planner/core/errors.go b/planner/core/errors.go index 860a3444b5538..c1bb613cda005 100644 --- a/planner/core/errors.go +++ b/planner/core/errors.go @@ -97,5 +97,6 @@ var ( ErrAccessDenied = dbterror.ClassOptimizer.NewStdErr(mysql.ErrAccessDenied, mysql.MySQLErrName[mysql.ErrAccessDeniedNoPassword]) ErrBadNull = dbterror.ClassOptimizer.NewStd(mysql.ErrBadNull) ErrNotSupportedWithSem = dbterror.ClassOptimizer.NewStd(mysql.ErrNotSupportedWithSem) + ErrDifferentAsOf = dbterror.ClassOptimizer.NewStd(mysql.ErrUnknown) ErrOptOnTemporaryTable = dbterror.ClassOptimizer.NewStd(mysql.ErrOptOnTemporaryTable) ) diff --git a/planner/core/indexmerge_test.go b/planner/core/indexmerge_test.go index a78b91b54f889..f1ebf4ee8144b 100644 --- a/planner/core/indexmerge_test.go +++ b/planner/core/indexmerge_test.go @@ -90,7 +90,7 @@ func (s *testIndexMergeSuite) TestIndexMergePathGeneration(c *C) { comment := Commentf("case:%v sql:%s", i, tc) stmt, err := s.ParseOneStmt(tc, "", "") c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 56652983ff8f8..379a4c486c5dc 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -457,7 +457,7 @@ func (s *testPlanSuite) TestSubquery(c *C) { stmt, err := s.ParseOneStmt(ca, "", "") c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) @@ -483,7 +483,7 @@ func (s *testPlanSuite) TestPlanBuilder(c *C) { c.Assert(err, IsNil, comment) s.ctx.GetSessionVars().SetHashJoinConcurrency(1) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) @@ -848,7 +848,7 @@ func (s *testPlanSuite) TestValidate(c *C) { comment := Commentf("for %s", sql) stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) _, _, err = BuildLogicalPlan(ctx, s.ctx, stmt, s.is) if tt.err == nil { @@ -1309,7 +1309,7 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) // to fix, Table 'test.ttt' doesn't exist - _ = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) builder.ctx.GetSessionVars().SetHashJoinConcurrency(1) _, err = builder.Build(context.TODO(), stmt) @@ -1389,7 +1389,7 @@ func (s *testPlanSuite) TestUnion(c *C) { comment := Commentf("case:%v sql:%s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) plan, err := builder.Build(ctx, stmt) @@ -1422,7 +1422,7 @@ func (s *testPlanSuite) TestTopNPushDown(c *C) { comment := Commentf("case:%v sql:%s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) @@ -1496,7 +1496,7 @@ func (s *testPlanSuite) TestOuterJoinEliminator(c *C) { comment := Commentf("case:%v sql:%s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) @@ -1533,7 +1533,7 @@ func (s *testPlanSuite) TestSelectView(c *C) { comment := Commentf("case:%v sql:%s", i, tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) @@ -1605,7 +1605,7 @@ func (s *testPlanSuite) optimize(ctx context.Context, sql string) (PhysicalPlan, if err != nil { return nil, nil, err } - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) if err != nil { return nil, nil, err } @@ -1697,7 +1697,7 @@ func (s *testPlanSuite) TestSkylinePruning(c *C) { comment := Commentf("case:%v sql:%s", i, tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) @@ -1768,7 +1768,7 @@ func (s *testPlanSuite) TestFastPlanContextTables(c *C) { for _, tt := range tests { stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) s.ctx.GetSessionVars().StmtCtx.Tables = nil p := TryFastPlan(s.ctx, stmt) @@ -1800,7 +1800,7 @@ func (s *testPlanSuite) TestUpdateEQCond(c *C) { comment := Commentf("case:%v sql:%s", i, tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) @@ -1817,7 +1817,7 @@ func (s *testPlanSuite) TestConflictedJoinTypeHints(c *C) { ctx := context.TODO() stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) @@ -1838,7 +1838,7 @@ func (s *testPlanSuite) TestSimplyOuterJoinWithOnlyOuterExpr(c *C) { ctx := context.TODO() stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) @@ -1890,7 +1890,7 @@ func (s *testPlanSuite) TestResolvingCorrelatedAggregate(c *C) { comment := Commentf("case:%v sql:%s", i, tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - err = Preprocess(s.ctx, stmt, s.is) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil, comment) p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil, comment) @@ -1932,7 +1932,8 @@ func (s *testPlanSuite) TestFastPathInvalidBatchPointGet(c *C) { comment := Commentf("case:%v sql:%s", i, tc.sql) stmt, err := s.ParseOneStmt(tc.sql, "", "") c.Assert(err, IsNil, comment) - c.Assert(Preprocess(s.ctx, stmt, s.is), IsNil, comment) + err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + c.Assert(err, IsNil, comment) plan := TryFastPlan(s.ctx, stmt) if tc.fastPlan { c.Assert(plan, NotNil) diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 6a8993286e36a..6bfda74d6ab1d 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -249,7 +249,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderBasePhysicalPlan(c *C) { stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - err = core.Preprocess(se, stmt, s.is) + err = core.Preprocess(se, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) c.Assert(err, IsNil) @@ -1427,7 +1427,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderSplitAvg(c *C) { stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - err = core.Preprocess(se, stmt, s.is) + err = core.Preprocess(se, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) p, _, err := planner.Optimize(context.TODO(), se, stmt, s.is) c.Assert(err, IsNil, comment) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 931221e576ddc..178779cabdf25 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2344,7 +2344,7 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShutdownPriv, "", "", "", nil) case *ast.BeginStmt: if raw.AsOf != nil { - startTS, err := b.calculateTsExpr(raw.AsOf) + startTS, err := calculateTsExpr(b.ctx, raw.AsOf) if err != nil { return nil, err } @@ -2355,19 +2355,19 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, } // calculateTsExpr calculates the TsExpr of AsOfClause to get a StartTS. -func (b *PlanBuilder) calculateTsExpr(asOfClause *ast.AsOfClause) (uint64, error) { - tsVal, err := evalAstExpr(b.ctx, asOfClause.TsExpr) +func calculateTsExpr(sctx sessionctx.Context, asOfClause *ast.AsOfClause) (uint64, error) { + tsVal, err := evalAstExpr(sctx, asOfClause.TsExpr) if err != nil { return 0, err } toTypeTimestamp := types.NewFieldType(mysql.TypeTimestamp) // We need at least the millionsecond here, so set fsp to 3. toTypeTimestamp.Decimal = 3 - tsTimestamp, err := tsVal.ConvertTo(b.ctx.GetSessionVars().StmtCtx, toTypeTimestamp) + tsTimestamp, err := tsVal.ConvertTo(sctx.GetSessionVars().StmtCtx, toTypeTimestamp) if err != nil { return 0, err } - tsTime, err := tsTimestamp.GetMysqlTime().GoTime(b.ctx.GetSessionVars().TimeZone) + tsTime, err := tsTimestamp.GetMysqlTime().GoTime(sctx.GetSessionVars().TimeZone) if err != nil { return 0, err } diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index dfece394a5390..1306da4bed42c 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -316,10 +316,10 @@ func (s *testPointGetSuite) TestPointGetId(c *C) { c.Assert(err, IsNil) c.Assert(stmts, HasLen, 1) stmt := stmts[0] - is := domain.GetDomain(ctx).InfoSchema() - err = core.Preprocess(ctx, stmt, is) + ret := &core.PreprocessorReturn{} + err = core.Preprocess(ctx, stmt, core.WithPreprocessorReturn(ret)) c.Assert(err, IsNil) - p, _, err := planner.Optimize(context.TODO(), ctx, stmt, is) + p, _, err := planner.Optimize(context.TODO(), ctx, stmt, ret.InfoSchema) c.Assert(err, IsNil) // Test explain format = 'brief' result is useless, plan id will be reset when running `explain`. c.Assert(p.ID(), Equals, 1) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index ca436189c0d41..bfc596d20f975 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta/autoid" @@ -51,6 +52,13 @@ func InTxnRetry(p *preprocessor) { p.flag |= inTxnRetry } +// WithPreprocessorReturn returns a PreprocessOpt to initialize the PreprocesorReturn. +func WithPreprocessorReturn(ret *PreprocessorReturn) PreprocessOpt { + return func(p *preprocessor) { + p.PreprocessorReturn = ret + } +} + // TryAddExtraLimit trys to add an extra limit for SELECT or UNION statement when sql_select_limit is set. func TryAddExtraLimit(ctx sessionctx.Context, node ast.StmtNode) ast.StmtNode { if ctx.GetSessionVars().SelectLimit == math.MaxUint64 || ctx.GetSessionVars().InRestrictedSQL { @@ -82,12 +90,21 @@ func TryAddExtraLimit(ctx sessionctx.Context, node ast.StmtNode) ast.StmtNode { } // Preprocess resolves table names of the node, and checks some statements validation. -func Preprocess(ctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema, preprocessOpt ...PreprocessOpt) error { - v := preprocessor{is: is, ctx: ctx, tableAliasInJoin: make([]map[string]interface{}, 0)} +// prepreocssReturn used to extract the infoschema for the tableName and the timestamp from the asof clause. +func Preprocess(ctx sessionctx.Context, node ast.Node, preprocessOpt ...PreprocessOpt) error { + v := preprocessor{ctx: ctx, tableAliasInJoin: make([]map[string]interface{}, 0)} for _, optFn := range preprocessOpt { optFn(&v) } + // PreprocessorReturn must be non-nil before preprocessing + if v.PreprocessorReturn == nil { + v.PreprocessorReturn = &PreprocessorReturn{} + } node.Accept(&v) + // InfoSchema must be non-nil after preprocessing + if v.InfoSchema == nil { + v.ensureInfoSchema() + } return errors.Trace(v.err) } @@ -109,18 +126,26 @@ const ( inSequenceFunction ) +// PreprocessorReturn is used to retain information obtained in the preprocessor. +type PreprocessorReturn struct { + SnapshotTS uint64 + InfoSchema infoschema.InfoSchema +} + // preprocessor is an ast.Visitor that preprocess // ast Nodes parsed from parser. type preprocessor struct { - is infoschema.InfoSchema ctx sessionctx.Context - err error flag preprocessorFlag stmtTp byte // tableAliasInJoin is a stack that keeps the table alias names for joins. // len(tableAliasInJoin) may bigger than 1 because the left/right child of join may be subquery that contains `JOIN` tableAliasInJoin []map[string]interface{} + + // values that may be returned + *PreprocessorReturn + err error } func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { @@ -571,7 +596,7 @@ func (p *preprocessor) checkAdminCheckTableGrammar(stmt *ast.AdminStmt) { } sName := model.NewCIStr(currentDB) tName := table.Name - tableInfo, err := p.is.TableByName(sName, tName) + tableInfo, err := p.ensureInfoSchema().TableByName(sName, tName) if err != nil { p.err = err return @@ -590,7 +615,8 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { if stmt.ReferTable.Schema.String() != "" { schema = stmt.ReferTable.Schema } - tableInfo, err := p.is.TableByName(schema, stmt.ReferTable.Name) + // get the infoschema from the context. + tableInfo, err := p.ensureInfoSchema().TableByName(schema, stmt.ReferTable.Name) if err != nil { p.err = err return @@ -1161,6 +1187,7 @@ func (p *preprocessor) handleTableName(tn *ast.TableName) { } tn.Schema = model.NewCIStr(currentDB) } + if p.flag&inCreateOrDropTable > 0 { // The table may not exist in create table or drop table statement. if p.flag&inRepairTable > 0 { @@ -1180,7 +1207,12 @@ func (p *preprocessor) handleTableName(tn *ast.TableName) { return } - table, err := p.is.TableByName(tn.Schema, tn.Name) + p.handleAsOf(tn.AsOf) + if p.err != nil { + return + } + + table, err := p.ensureInfoSchema().TableByName(tn.Schema, tn.Name) if err != nil { // We should never leak that the table doesn't exist (i.e. attach ErrTableNotExists) // unless we know that the user has permissions to it, should it exist. @@ -1202,7 +1234,7 @@ func (p *preprocessor) handleTableName(tn *ast.TableName) { return } tableInfo := table.Meta() - dbInfo, _ := p.is.SchemaByName(tn.Schema) + dbInfo, _ := p.ensureInfoSchema().SchemaByName(tn.Schema) // tableName should be checked as sequence object. if p.flag&inSequenceFunction > 0 { if !tableInfo.IsSequence() { @@ -1327,3 +1359,40 @@ func (p *preprocessor) checkFuncCastExpr(node *ast.FuncCastExpr) { } } } + +// handleAsOf tries to validate the timestamp. +// If it is not nil, timestamp is used to get the history infoschema from the infocache. +func (p *preprocessor) handleAsOf(node *ast.AsOfClause) { + dom := domain.GetDomain(p.ctx) + ts := uint64(0) + if node != nil { + ts, p.err = calculateTsExpr(p.ctx, node) + if p.err != nil { + return + } + } + if ts != 0 && p.InfoSchema == nil { + is, err := dom.GetSnapshotInfoSchema(ts) + if err != nil { + p.err = err + return + } + p.SnapshotTS = ts + p.InfoSchema = is + } + if p.SnapshotTS != ts { + p.err = ErrDifferentAsOf.GenWithStack("can not set different time in the as of") + } +} + +// ensureInfoSchema get the infoschema from the preprecessor. +// there some situations: +// - the stmt specifies the schema version. +// - session variable +// - transcation context +func (p *preprocessor) ensureInfoSchema() infoschema.InfoSchema { + if p.InfoSchema == nil { + p.InfoSchema = p.ctx.GetInfoSchema().(infoschema.InfoSchema) + } + return p.InfoSchema +} diff --git a/planner/core/preprocess_test.go b/planner/core/preprocess_test.go index 14b006c836ca9..d9f053f509e92 100644 --- a/planner/core/preprocess_test.go +++ b/planner/core/preprocess_test.go @@ -67,7 +67,7 @@ func (s *testValidatorSuite) runSQL(c *C, sql string, inPrepare bool, terr error if inPrepare { opts = append(opts, core.InPrepare) } - err := core.Preprocess(s.ctx, stmt, s.is, opts...) + err := core.Preprocess(s.ctx, stmt, append(opts, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: s.is}))...) c.Assert(terror.ErrorEqual(err, terr), IsTrue, Commentf("sql: %s, err:%v", sql, err)) } diff --git a/planner/core/stats_test.go b/planner/core/stats_test.go index d74f1ba1df1d8..6767a9b910b7d 100644 --- a/planner/core/stats_test.go +++ b/planner/core/stats_test.go @@ -69,15 +69,15 @@ func (s *testStatsSuite) TestGroupNDVs(c *C) { AggInput string JoinInput string } - is := dom.InfoSchema() s.testData.GetTestCases(c, &input, &output) for i, tt := range input { comment := Commentf("case:%v sql: %s", i, tt) stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - err = core.Preprocess(tk.Se, stmt, is) + ret := &core.PreprocessorReturn{} + err = core.Preprocess(tk.Se, stmt, core.WithPreprocessorReturn(ret)) c.Assert(err, IsNil) - builder, _ := core.NewPlanBuilder(tk.Se, is, &hint.BlockHintProcessor{}) + builder, _ := core.NewPlanBuilder(tk.Se, ret.InfoSchema, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil, comment) p, err = core.LogicalOptimize(ctx, builder.GetOptFlag(), p.(core.LogicalPlan)) diff --git a/server/conn.go b/server/conn.go index 78cdd1a46c12d..46d0a7c023a27 100644 --- a/server/conn.go +++ b/server/conn.go @@ -63,7 +63,6 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" @@ -1621,11 +1620,11 @@ func (cc *clientConn) prefetchPointPlanKeys(ctx context.Context, stmts []ast.Stm pointPlans := make([]plannercore.Plan, len(stmts)) var idxKeys []kv.Key var rowKeys []kv.Key - is := domain.GetDomain(cc.ctx).InfoSchema() sc := vars.StmtCtx for i, stmt := range stmts { // TODO: the preprocess is run twice, we should find some way to avoid do it again. - if err = plannercore.Preprocess(cc.ctx, stmt, is); err != nil { + // TODO: handle the PreprocessorReturn. + if err = plannercore.Preprocess(cc.ctx, stmt); err != nil { return nil, err } p := plannercore.TryFastPlan(cc.ctx.Session, stmt) diff --git a/session/session.go b/session/session.go index 03860c6320b2e..46d3a09033837 100644 --- a/session/session.go +++ b/session/session.go @@ -1708,7 +1708,7 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields // So we have to call PrepareTxnCtx here. s.PrepareTxnCtx(ctx) s.PrepareTSFuture(ctx) - prepareExec := executor.NewPrepareExec(s, s.GetInfoSchema().(infoschema.InfoSchema), sql) + prepareExec := executor.NewPrepareExec(s, sql) err = prepareExec.Next(ctx, nil) if err != nil { return diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 546ca0b20a4ea..359e1d2db9585 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -237,7 +237,6 @@ func (s *testStatsSuite) TestSelectivity(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) statsTbl := s.prepareSelectivity(testKit, c) - is := s.do.InfoSchema() longExpr := "0 < a and a = 1 " for i := 1; i < 64; i++ { @@ -294,9 +293,10 @@ func (s *testStatsSuite) TestSelectivity(c *C) { c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprs)) c.Assert(stmts, HasLen, 1) - err = plannercore.Preprocess(sctx, stmts[0], is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, comment) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) + p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for building plan, expr %s", err, tt.exprs)) sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) @@ -542,7 +542,6 @@ func BenchmarkSelectivity(b *testing.B) { testKit := testkit.NewTestKit(c, s.store) statsTbl := s.prepareSelectivity(testKit, c) - is := s.do.InfoSchema() exprs := "a > 1 and b < 2 and c > 3 and d < 4 and e > 5" sql := "select * from t where " + exprs comment := Commentf("for %s", exprs) @@ -550,9 +549,10 @@ func BenchmarkSelectivity(b *testing.B) { stmts, err := session.Parse(sctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, exprs)) c.Assert(stmts, HasLen, 1) - err = plannercore.Preprocess(sctx, stmts[0], is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, comment) - p, _, err := plannercore.BuildLogicalPlan(context.Background(), sctx, stmts[0], is) + p, _, err := plannercore.BuildLogicalPlan(context.Background(), sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for building plan, expr %s", err, exprs)) file, err := os.Create("cpu.profile") @@ -792,9 +792,8 @@ func (s *testStatsSuite) TestDNFCondSelectivity(c *C) { testKit.MustExec(`analyze table t`) ctx := context.Background() - is := s.do.InfoSchema() h := s.do.StatsHandle() - tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + tb, err := s.do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) tblInfo := tb.Meta() statsTbl := h.GetTableStats(tblInfo) @@ -813,9 +812,10 @@ func (s *testStatsSuite) TestDNFCondSelectivity(c *C) { c.Assert(err, IsNil, Commentf("error %v, for sql %s", err, tt)) c.Assert(stmts, HasLen, 1) - err = plannercore.Preprocess(sctx, stmts[0], is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, Commentf("error %v, for sql %s", err, tt)) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) + p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for building plan, sql %s", err, tt)) sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index b0b66e8a469c1..3f4e59848b044 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -304,10 +304,10 @@ func (s *testRangerSuite) TestTableRange(c *C) { stmts, err := session.Parse(sctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) - is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) + p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) conds := make([]expression.Expression, len(selection.Conditions)) @@ -648,10 +648,10 @@ create table t( stmts, err := session.Parse(sctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) - is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) + p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() @@ -839,10 +839,10 @@ create table t( stmts, err := session.Parse(sctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) - is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) + p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() @@ -1203,10 +1203,10 @@ func (s *testRangerSuite) TestColumnRange(c *C) { stmts, err := session.Parse(sctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) - is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) + p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) ds, ok := sel.Children()[0].(*plannercore.DataSource) @@ -1627,10 +1627,10 @@ func (s *testRangerSuite) TestIndexRangeForYear(c *C) { stmts, err := session.Parse(sctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) - is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) + p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() @@ -1698,10 +1698,10 @@ func (s *testRangerSuite) TestPrefixIndexRangeScan(c *C) { stmts, err := session.Parse(sctx, sql) c.Assert(err, IsNil, Commentf("error %v, for expr %s", err, tt.exprStr)) c.Assert(stmts, HasLen, 1) - is := domain.GetDomain(sctx).InfoSchema() - err = plannercore.Preprocess(sctx, stmts[0], is) + ret := &plannercore.PreprocessorReturn{} + err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, Commentf("error %v, for resolve name, expr %s", err, tt.exprStr)) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], is) + p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for build plan, expr %s", err, tt.exprStr)) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() From b64f9007cfec31c4eb2455c1b89ae441a6c6b79c Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Thu, 27 May 2021 22:25:35 +0800 Subject: [PATCH 196/343] infoschema: Remove the unimplemented ALL_SQL_DIGESTS column from the DEADLOCKS table (#24939) --- executor/executor_test.go | 10 +++---- infoschema/tables.go | 3 ++- util/deadlockhistory/deadlock_history.go | 9 +++---- util/deadlockhistory/deadlock_history_test.go | 26 ++++++++----------- 4 files changed, 21 insertions(+), 27 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 0e7eab38ac9e8..e7f6adb0737d3 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8232,11 +8232,11 @@ func (s *testSerialSuite) TestDeadlockTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustQuery("select * from information_schema.deadlocks").Check( testutil.RowsWithSep("/", - id1+"/2021-05-10 01:02:03.456789/0/101/aabbccdd/6B31//102", - id1+"/2021-05-10 01:02:03.456789/0/102/ddccbbaa/6B32/[sql1]/101", - id2+"/2022-06-11 02:03:04.987654/1/201///[]/202", - id2+"/2022-06-11 02:03:04.987654/1/202///[sql1, sql2, sql3]/203", - id2+"/2022-06-11 02:03:04.987654/1/203////201", + id1+"/2021-05-10 01:02:03.456789/0/101/aabbccdd/6B31/102", + id1+"/2021-05-10 01:02:03.456789/0/102/ddccbbaa/6B32/101", + id2+"/2022-06-11 02:03:04.987654/1/201///202", + id2+"/2022-06-11 02:03:04.987654/1/202///203", + id2+"/2022-06-11 02:03:04.987654/1/203///201", )) } diff --git a/infoschema/tables.go b/infoschema/tables.go index d4851c245d016..f5ccd2af0af26 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1368,8 +1368,9 @@ var tableDeadlocksCols = []columnInfo{ {name: "TRY_LOCK_TRX_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "The transaction ID (start ts) of the transaction that's trying to acquire the lock"}, {name: "CURRENT_SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "The digest of the SQL that's being blocked"}, {name: "KEY", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "The key on which a transaction is waiting for another"}, - {name: "ALL_SQL_DIGESTS", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the digests of SQL statements that the transaction has executed"}, {name: "TRX_HOLDING_LOCK", tp: mysql.TypeLonglong, size: 21, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "The transaction ID (start ts) of the transaction that's currently holding the lock"}, + // TODO: Implement the ALL_SQL_DIGESTS column + // {name: "ALL_SQL_DIGESTS", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "A list of the digests of SQL statements that the transaction has executed"}, } var tableDataLockWaitsCols = []columnInfo{ diff --git a/util/deadlockhistory/deadlock_history.go b/util/deadlockhistory/deadlock_history.go index ee2fb496a2a58..ef80f0f5d7ead 100644 --- a/util/deadlockhistory/deadlock_history.go +++ b/util/deadlockhistory/deadlock_history.go @@ -129,7 +129,7 @@ func (d *DeadlockHistory) GetAllDatum() [][]types.Datum { rows := make([][]types.Datum, 0, rowsCount) - row := make([]interface{}, 8) + row := make([]interface{}, 7) for _, rec := range records { row[0] = rec.ID row[1] = types.NewTime(types.FromGoTime(rec.OccurTime), mysql.TypeTimestamp, types.MaxFsp) @@ -148,12 +148,9 @@ func (d *DeadlockHistory) GetAllDatum() [][]types.Datum { row[5] = strings.ToUpper(hex.EncodeToString(item.Key)) } - row[6] = nil - if item.AllSQLDigests != nil { - row[6] = "[" + strings.Join(item.AllSQLDigests, ", ") + "]" - } + row[6] = item.TxnHoldingLock - row[7] = item.TxnHoldingLock + // TODO: Implement the ALL_SQL_DIGESTS column for the deadlock table. rows = append(rows, types.MakeDatums(row...)) } diff --git a/util/deadlockhistory/deadlock_history_test.go b/util/deadlockhistory/deadlock_history_test.go index 398d2943ab996..b3307c91cc3c5 100644 --- a/util/deadlockhistory/deadlock_history_test.go +++ b/util/deadlockhistory/deadlock_history_test.go @@ -184,7 +184,7 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) { res := h.GetAllDatum() c.Assert(len(res), Equals, 4) for _, row := range res { - c.Assert(len(row), Equals, 8) + c.Assert(len(row), Equals, 7) } toGoTime := func(d types.Datum) time.Time { @@ -195,14 +195,13 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) { return t } - c.Assert(res[0][0].GetValue(), Equals, uint64(1)) // ID - c.Assert(toGoTime(res[0][1]), Equals, time1) // OCCUR_TIME - c.Assert(res[0][2].GetValue(), Equals, int64(0)) // RETRYABLE - c.Assert(res[0][3].GetValue(), Equals, uint64(101)) // TRY_LOCK_TRX_ID - c.Assert(res[0][4].GetValue(), Equals, "sql1") // SQL_DIGEST - c.Assert(res[0][5].GetValue(), Equals, "6B31") // KEY - c.Assert(res[0][6].GetValue(), Equals, "[sql1, sql2]") // ALL_SQL_DIGESTS - c.Assert(res[0][7].GetValue(), Equals, uint64(102)) // TRX_HOLDING_LOCK + c.Assert(res[0][0].GetValue(), Equals, uint64(1)) // ID + c.Assert(toGoTime(res[0][1]), Equals, time1) // OCCUR_TIME + c.Assert(res[0][2].GetValue(), Equals, int64(0)) // RETRYABLE + c.Assert(res[0][3].GetValue(), Equals, uint64(101)) // TRY_LOCK_TRX_ID + c.Assert(res[0][4].GetValue(), Equals, "sql1") // SQL_DIGEST + c.Assert(res[0][5].GetValue(), Equals, "6B31") // KEY + c.Assert(res[0][6].GetValue(), Equals, uint64(102)) // TRX_HOLDING_LOCK c.Assert(res[1][0].GetValue(), Equals, uint64(1)) // ID c.Assert(toGoTime(res[1][1]), Equals, time1) // OCCUR_TIME @@ -210,22 +209,19 @@ func (s *testDeadlockHistorySuite) TestGetDatum(c *C) { c.Assert(res[1][3].GetValue(), Equals, uint64(102)) // TRY_LOCK_TRX_ID c.Assert(res[1][4].GetValue(), Equals, nil) // SQL_DIGEST c.Assert(res[1][5].GetValue(), Equals, nil) // KEY - c.Assert(res[1][6].GetValue(), Equals, nil) // ALL_SQL_DIGESTS - c.Assert(res[1][7].GetValue(), Equals, uint64(101)) // TRX_HOLDING_LOCK + c.Assert(res[1][6].GetValue(), Equals, uint64(101)) // TRX_HOLDING_LOCK c.Assert(res[2][0].GetValue(), Equals, uint64(2)) // ID c.Assert(toGoTime(res[2][1]), Equals, time2) // OCCUR_TIME c.Assert(res[2][2].GetValue(), Equals, int64(1)) // RETRYABLE c.Assert(res[2][3].GetValue(), Equals, uint64(201)) // TRY_LOCK_TRX_ID - c.Assert(res[2][6].GetValue(), Equals, "[]") // ALL_SQL_DIGESTS - c.Assert(res[2][7].GetValue(), Equals, uint64(202)) // TRX_HOLDING_LOCK + c.Assert(res[2][6].GetValue(), Equals, uint64(202)) // TRX_HOLDING_LOCK c.Assert(res[3][0].GetValue(), Equals, uint64(2)) // ID c.Assert(toGoTime(res[3][1]), Equals, time2) // OCCUR_TIME c.Assert(res[3][2].GetValue(), Equals, int64(1)) // RETRYABLE c.Assert(res[3][3].GetValue(), Equals, uint64(202)) // TRY_LOCK_TRX_ID - c.Assert(res[3][6].GetValue(), Equals, "[sql1]") // ALL_SQL_DIGESTS - c.Assert(res[3][7].GetValue(), Equals, uint64(201)) // TRX_HOLDING_LOCK + c.Assert(res[3][6].GetValue(), Equals, uint64(201)) // TRX_HOLDING_LOCK } func (s *testDeadlockHistorySuite) TestErrDeadlockToDeadlockRecord(c *C) { From 165c67f0e7c39b7868abd51608ba91bb51a461bb Mon Sep 17 00:00:00 2001 From: ClSlaid Date: Thu, 27 May 2021 23:39:35 +0800 Subject: [PATCH 197/343] planner/core/logical_plan_test.go: add error check (#24942) --- planner/core/logical_plan_test.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 379a4c486c5dc..05f51d517393f 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1308,8 +1308,10 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { comment := Commentf("for %s", tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - // to fix, Table 'test.ttt' doesn't exist - err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + + // TODO: to fix, Table 'test.ttt' doesn't exist + _ = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) + builder, _ := NewPlanBuilder(MockContext(), s.is, &hint.BlockHintProcessor{}) builder.ctx.GetSessionVars().SetHashJoinConcurrency(1) _, err = builder.Build(context.TODO(), stmt) From 348abf01f90eb19db09d72261cd83c37e563a813 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 28 May 2021 12:07:35 +0800 Subject: [PATCH 198/343] telemetry,session: add telemetry for the temporary table feature (#24866) --- session/session.go | 15 +++++++++++++++ telemetry/data_feature_usage.go | 14 ++++++++++++-- telemetry/data_feature_usage_test.go | 13 +++++++++++++ telemetry/util_test.go | 4 ++++ 4 files changed, 44 insertions(+), 2 deletions(-) diff --git a/session/session.go b/session/session.go index 46d3a09033837..dc64f5fa1dc9b 100644 --- a/session/session.go +++ b/session/session.go @@ -2849,6 +2849,21 @@ func (s *session) GetTxnWriteThroughputSLI() *sli.TxnWriteThroughputSLI { return &s.txn.writeSLI } +var _ telemetry.TemporaryTableFeatureChecker = &session{} + +// TemporaryTableExists is used by the telemetry package to avoid circle dependency. +func (s *session) TemporaryTableExists() bool { + is := domain.GetDomain(s).InfoSchema() + for _, dbInfo := range is.AllSchemas() { + for _, tbInfo := range is.SchemaTables(dbInfo.Name) { + if tbInfo.Meta().TempTableType != model.TempTableNone { + return true + } + } + } + return false +} + // GetInfoSchema returns snapshotInfoSchema if snapshot schema is set. // Transaction infoschema is returned if inside an explicit txn. // Otherwise the latest infoschema is returned. diff --git a/telemetry/data_feature_usage.go b/telemetry/data_feature_usage.go index ff3e3c672cff0..8e9b2e6cfb69b 100644 --- a/telemetry/data_feature_usage.go +++ b/telemetry/data_feature_usage.go @@ -23,8 +23,9 @@ import ( ) type featureUsage struct { - Txn *TxnUsage `json:"txn"` - ClusterIndex map[string]bool `json:"clusterIndex"` + Txn *TxnUsage `json:"txn"` + ClusterIndex map[string]bool `json:"clusterIndex"` + TemporaryTable bool `json:"temporaryTable"` } func getFeatureUsage(ctx sessionctx.Context) (*featureUsage, error) { @@ -62,9 +63,18 @@ func getFeatureUsage(ctx sessionctx.Context) (*featureUsage, error) { // transaction related feature usageInfo.Txn = GetTxnUsageInfo(ctx) + // Avoid the circle dependency. + usageInfo.TemporaryTable = ctx.(TemporaryTableFeatureChecker).TemporaryTableExists() + return &usageInfo, nil } +// TemporaryTableFeatureChecker is defined to avoid package circle dependency. +// The session struct implements this interface. +type TemporaryTableFeatureChecker interface { + TemporaryTableExists() bool +} + // TxnUsage records the usage info of transaction related features, including // async-commit, 1PC and counters of transactions committed with different protocols. type TxnUsage struct { diff --git a/telemetry/data_feature_usage_test.go b/telemetry/data_feature_usage_test.go index 9164591e09224..790a24779f234 100644 --- a/telemetry/data_feature_usage_test.go +++ b/telemetry/data_feature_usage_test.go @@ -90,3 +90,16 @@ func (s *testFeatureInfoSuite) TestTxnUsageInfo(c *C) { c.Assert(txnUsage.TxnCommitCounter.OnePC, Greater, int64(0)) c.Assert(txnUsage.TxnCommitCounter.TwoPC, Greater, int64(0)) } + +func (s *testFeatureInfoSuite) TestTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + usage, err := telemetry.GetFeatureUsage(tk.Se) + c.Assert(err, IsNil) + c.Assert(usage.TemporaryTable, IsFalse) + + tk.MustExec("create global temporary table t (id int) on commit delete rows") + usage, err = telemetry.GetFeatureUsage(tk.Se) + c.Assert(err, IsNil) + c.Assert(usage.TemporaryTable, IsTrue) +} diff --git a/telemetry/util_test.go b/telemetry/util_test.go index 035a8f9a3b2f8..f480bef6de62c 100644 --- a/telemetry/util_test.go +++ b/telemetry/util_test.go @@ -17,6 +17,7 @@ import ( "testing" . "github.com/pingcap/check" + "github.com/pingcap/tidb/sessionctx" ) var _ = Suite(&testUtilSuite{}) @@ -62,3 +63,6 @@ func (s *testUtilSuite) TestParseAddress(c *C) { c.Assert(port, Equals, tt.expectedPort) } } + +// GetFeatureUsage exports getFeatureUsage for testing. +var GetFeatureUsage func(ctx sessionctx.Context) (*featureUsage, error) = getFeatureUsage From ec12617da749e53224a30b48fcf5bdd72febaf59 Mon Sep 17 00:00:00 2001 From: disksing Date: Fri, 28 May 2021 14:17:35 +0800 Subject: [PATCH 199/343] *: update BR (#24905) --- go.mod | 2 +- go.sum | 4 ++-- kv/interface_mock_test.go | 4 ---- kv/union_store.go | 20 -------------------- store/tikv/kv/option.go | 22 ---------------------- 5 files changed, 3 insertions(+), 49 deletions(-) delete mode 100644 kv/union_store.go delete mode 100644 store/tikv/kv/option.go diff --git a/go.mod b/go.mod index 01cc11c222859..d29f89bfade55 100644 --- a/go.mod +++ b/go.mod @@ -37,7 +37,7 @@ require ( github.com/opentracing/opentracing-go v1.1.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 - github.com/pingcap/br v5.1.0-alpha.0.20210524083733-58e9e24fcb8e+incompatible + github.com/pingcap/br v5.1.0-alpha.0.20210526054934-d5f5f9df24f5+incompatible github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3 github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd diff --git a/go.sum b/go.sum index 0ae5bdaf51b9e..991423501bec3 100644 --- a/go.sum +++ b/go.sum @@ -407,8 +407,8 @@ github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5e github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 h1:IXpGy7y9HyoShAFmzW2OPF0xCA5EOoSTyZHwsgYk9Ro= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= -github.com/pingcap/br v5.1.0-alpha.0.20210524083733-58e9e24fcb8e+incompatible h1:rRJG+3sGr4rbC3PUGpIhaO4uVB3ZssyJxZyLW8yFwBE= -github.com/pingcap/br v5.1.0-alpha.0.20210524083733-58e9e24fcb8e+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= +github.com/pingcap/br v5.1.0-alpha.0.20210526054934-d5f5f9df24f5+incompatible h1:JXkYrdHpoW0Ht6fI+pl9SC7OcNpYvJg8hc/6i+V60Eo= +github.com/pingcap/br v5.1.0-alpha.0.20210526054934-d5f5f9df24f5+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= diff --git a/kv/interface_mock_test.go b/kv/interface_mock_test.go index 2e78b8725a7dc..2c01cbbb80deb 100644 --- a/kv/interface_mock_test.go +++ b/kv/interface_mock_test.go @@ -104,10 +104,6 @@ func (t *mockTxn) GetSnapshot() Snapshot { return nil } -func (t *mockTxn) GetUnionStore() UnionStore { - return nil -} - func (t *mockTxn) NewStagingBuffer() MemBuffer { return nil } diff --git a/kv/union_store.go b/kv/union_store.go deleted file mode 100644 index 1d12e0a7fd37b..0000000000000 --- a/kv/union_store.go +++ /dev/null @@ -1,20 +0,0 @@ -// Copyright 2015 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv - -// UnionStore is a store that wraps a snapshot for read and a MemBuffer for buffered write. -// Also, it provides some transaction related utilities. -// TODO: Remove after upgrading BR. -type UnionStore interface { -} diff --git a/store/tikv/kv/option.go b/store/tikv/kv/option.go deleted file mode 100644 index 7bd36733a568d..0000000000000 --- a/store/tikv/kv/option.go +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright 2021 PingCAP, Inc. - -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kv - -// Priority value for transaction priority. -// TODO: remove after BR update. -const ( - PriorityNormal = iota - PriorityLow - PriorityHigh -) From 138692f594c52319abcdeaa6177d19530287842f Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Fri, 28 May 2021 15:09:35 +0800 Subject: [PATCH 200/343] store/tikv: set execution timeout for general TiKV requests (#24936) --- store/tikv/region_request.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 36b297c580102..b8fcc837c1bc6 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -259,6 +259,12 @@ func (s *RegionRequestSender) SendReqCtx( } }) + // If the MaxExecutionDurationMs is not set yet, we set it to be the RPC timeout duration + // so TiKV can give up the requests whose response TiDB cannot receive due to timeout. + if req.Context.MaxExecutionDurationMs == 0 { + req.Context.MaxExecutionDurationMs = uint64(timeout.Milliseconds()) + } + tryTimes := 0 for { if (tryTimes > 0) && (tryTimes%1000 == 0) { From 0367c54698b4e9d4a2b1c08ef395b59069319d53 Mon Sep 17 00:00:00 2001 From: ClSlaid Date: Fri, 28 May 2021 15:19:35 +0800 Subject: [PATCH 201/343] inforschema, executor, util/kvcache, util/statement_summary : Add STATEMENTS_SUMMARY_EVICTED into information_schema (#24513) --- executor/builder.go | 1 + executor/infoschema_reader.go | 6 + infoschema/tables.go | 10 + infoschema/tables_test.go | 27 +++ util/kvcache/simple_lru.go | 3 + util/kvcache/simple_lru_test.go | 38 ++- util/stmtsummary/evicted.go | 192 +++++++++++++++ util/stmtsummary/evicted_test.go | 337 ++++++++++++++++++++++++++ util/stmtsummary/statement_summary.go | 16 +- 9 files changed, 615 insertions(+), 15 deletions(-) create mode 100644 util/stmtsummary/evicted.go create mode 100644 util/stmtsummary/evicted_test.go diff --git a/executor/builder.go b/executor/builder.go index 67a0c33cb1fcb..34c5e0c599f5a 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1527,6 +1527,7 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo strings.ToLower(infoschema.TableTiKVStoreStatus), strings.ToLower(infoschema.TableStatementsSummary), strings.ToLower(infoschema.TableStatementsSummaryHistory), + strings.ToLower(infoschema.TableStatementsSummaryEvicted), strings.ToLower(infoschema.ClusterTableStatementsSummary), strings.ToLower(infoschema.ClusterTableStatementsSummaryHistory), strings.ToLower(infoschema.TablePlacementPolicy), diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 3b4fc2e159e05..b2d733851c998 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -148,6 +148,8 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex infoschema.ClusterTableStatementsSummary, infoschema.ClusterTableStatementsSummaryHistory: err = e.setDataForStatementsSummary(sctx, e.table.Name.O) + case infoschema.TableStatementsSummaryEvicted: + e.setDataForStatementsSummaryEvicted(sctx) case infoschema.TablePlacementPolicy: err = e.setDataForPlacementPolicy(sctx) case infoschema.TableClientErrorsSummaryGlobal, @@ -2099,6 +2101,10 @@ func (e *memtableRetriever) setDataForClusterDeadlock(ctx sessionctx.Context) er return nil } +func (e *memtableRetriever) setDataForStatementsSummaryEvicted(ctx sessionctx.Context) { + e.rows = stmtsummary.StmtSummaryByDigestMap.ToEvictedCountDatum() +} + type hugeMemTableRetriever struct { dummyCloser table *model.TableInfo diff --git a/infoschema/tables.go b/infoschema/tables.go index f5ccd2af0af26..d320c633d2398 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -149,6 +149,8 @@ const ( TableStatementsSummary = "STATEMENTS_SUMMARY" // TableStatementsSummaryHistory is the string constant of statements summary history table. TableStatementsSummaryHistory = "STATEMENTS_SUMMARY_HISTORY" + // TableStatementsSummaryEvicted is the string constant of statements summary evicted table. + TableStatementsSummaryEvicted = "STATEMENTS_SUMMARY_EVICTED" // TableStorageStats is a table that contains all tables disk usage TableStorageStats = "TABLE_STORAGE_STATS" // TableTiFlashTables is the string constant of tiflash tables table. @@ -246,6 +248,7 @@ var tableIDMap = map[string]int64{ TableDeadlocks: autoid.InformationSchemaDBID + 72, ClusterTableDeadlocks: autoid.InformationSchemaDBID + 73, TableDataLockWaits: autoid.InformationSchemaDBID + 74, + TableStatementsSummaryEvicted: autoid.InformationSchemaDBID + 75, } type columnInfo struct { @@ -1380,6 +1383,12 @@ var tableDataLockWaitsCols = []columnInfo{ {name: "SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the SQL that's trying to acquire the lock"}, } +var tableStatementsSummaryEvictedCols = []columnInfo{ + {name: "BEGIN_TIME", tp: mysql.TypeTimestamp, size: 26}, + {name: "END_TIME", tp: mysql.TypeTimestamp, size: 26}, + {name: "EVICTED_COUNT", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag}, +} + // GetShardingInfo returns a nil or description string for the sharding information of given TableInfo. // The returned description string may be: // - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified. @@ -1742,6 +1751,7 @@ var tableNameToColumns = map[string][]columnInfo{ TableSequences: tableSequencesCols, TableStatementsSummary: tableStatementsSummaryCols, TableStatementsSummaryHistory: tableStatementsSummaryCols, + TableStatementsSummaryEvicted: tableStatementsSummaryEvictedCols, TableStorageStats: tableStorageStatsCols, TableTiFlashTables: tableTableTiFlashTablesCols, TableTiFlashSegments: tableTableTiFlashSegmentsCols, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 33394e57a204c..c376116c61172 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1373,6 +1373,33 @@ func (s *testTableSuite) TestStmtSummarySensitiveQuery(c *C) { )) } +func (s *testTableSuite) TestSimpleStmtSummaryEvictedCount(c *C) { + now := time.Now().Unix() + interval := int64(1800) + beginTimeForCurInterval := now - now%interval + tk := s.newTestKitWithPlanCache(c) + tk.MustExec(fmt.Sprintf("set global tidb_stmt_summary_refresh_interval = %v", interval)) + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") + // first sql + tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 1") + // second sql + tk.MustQuery("show databases;") + // query `evicted table` is also a SQL, passing it leads to the eviction of the previous SQLs. + tk.MustQuery("select * from `information_schema`.`STATEMENTS_SUMMARY_EVICTED`;"). + Check(testkit.Rows( + fmt.Sprintf("%s %s %v", + time.Unix(beginTimeForCurInterval, 0).Format("2006-01-02 15:04:05"), + time.Unix(beginTimeForCurInterval+interval, 0).Format("2006-01-02 15:04:05"), + int64(2)), + )) + // TODO: Add more tests. + + // clean up side effects + tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 100") + tk.MustExec("set global tidb_stmt_summary_refresh_interval = 1800") +} + func (s *testTableSuite) TestPerformanceSchemaforPlanCache(c *C) { orgEnable := plannercore.PreparedPlanCacheEnabled() defer func() { diff --git a/util/kvcache/simple_lru.go b/util/kvcache/simple_lru.go index cee4b0363f39a..aec402adb09ed 100644 --- a/util/kvcache/simple_lru.go +++ b/util/kvcache/simple_lru.go @@ -118,6 +118,9 @@ func (l *SimpleLRUCache) Put(key Key, value Value) { if l.size > l.capacity { lru := l.cache.Back() l.cache.Remove(lru) + if l.onEvict != nil { + l.onEvict(lru.Value.(*cacheEntry).key, lru.Value.(*cacheEntry).value) + } delete(l.elements, string(lru.Value.(*cacheEntry).key.Hash())) l.size-- } diff --git a/util/kvcache/simple_lru_test.go b/util/kvcache/simple_lru_test.go index 9040bf60e7765..d8b38ad12fb3f 100644 --- a/util/kvcache/simple_lru_test.go +++ b/util/kvcache/simple_lru_test.go @@ -58,35 +58,47 @@ func (s *testLRUCacheSuite) TestPut(c *C) { maxMem, err := memory.MemTotal() c.Assert(err, IsNil) - lru := NewSimpleLRUCache(3, 0, maxMem) - c.Assert(lru.capacity, Equals, uint(3)) + lruMaxMem := NewSimpleLRUCache(3, 0, maxMem) + lruZeroQuota := NewSimpleLRUCache(3, 0, 0) + c.Assert(lruMaxMem.capacity, Equals, uint(3)) + c.Assert(lruZeroQuota.capacity, Equals, uint(3)) keys := make([]*mockCacheKey, 5) vals := make([]int64, 5) - droppedKv := make(map[Key]Value) + maxMemDroppedKv := make(map[Key]Value) + zeroQuotaDroppedKv := make(map[Key]Value) - lru.SetOnEvict(func(key Key, value Value) { - droppedKv[key] = value + // test onEvict function + lruMaxMem.SetOnEvict(func(key Key, value Value) { + maxMemDroppedKv[key] = value + }) + // test onEvict function on 0 value of quota + lruZeroQuota.SetOnEvict(func(key Key, value Value) { + zeroQuotaDroppedKv[key] = value }) for i := 0; i < 5; i++ { keys[i] = newMockHashKey(int64(i)) vals[i] = int64(i) - lru.Put(keys[i], vals[i]) + lruMaxMem.Put(keys[i], vals[i]) + lruZeroQuota.Put(keys[i], vals[i]) } - c.Assert(lru.size, Equals, lru.capacity) - c.Assert(lru.size, Equals, uint(3)) + c.Assert(lruMaxMem.size, Equals, lruMaxMem.capacity) + c.Assert(lruZeroQuota.size, Equals, lruZeroQuota.capacity) + c.Assert(lruMaxMem.size, Equals, uint(3)) + c.Assert(lruZeroQuota.size, Equals, lruMaxMem.size) // test for non-existent elements - c.Assert(len(droppedKv), Equals, 2) + c.Assert(len(maxMemDroppedKv), Equals, 2) for i := 0; i < 2; i++ { - element, exists := lru.elements[string(keys[i].Hash())] + element, exists := lruMaxMem.elements[string(keys[i].Hash())] c.Assert(exists, IsFalse) c.Assert(element, IsNil) - c.Assert(droppedKv[keys[i]], Equals, vals[i]) + c.Assert(maxMemDroppedKv[keys[i]], Equals, vals[i]) + c.Assert(zeroQuotaDroppedKv[keys[i]], Equals, vals[i]) } // test for existent elements - root := lru.cache.Front() + root := lruMaxMem.cache.Front() c.Assert(root, NotNil) for i := 4; i >= 2; i-- { entry, ok := root.Value.(*cacheEntry) @@ -98,7 +110,7 @@ func (s *testLRUCacheSuite) TestPut(c *C) { c.Assert(key, NotNil) c.Assert(key, Equals, keys[i]) - element, exists := lru.elements[string(keys[i].Hash())] + element, exists := lruMaxMem.elements[string(keys[i].Hash())] c.Assert(exists, IsTrue) c.Assert(element, NotNil) c.Assert(element, Equals, root) diff --git a/util/stmtsummary/evicted.go b/util/stmtsummary/evicted.go new file mode 100644 index 0000000000000..d3cf1ff0abc29 --- /dev/null +++ b/util/stmtsummary/evicted.go @@ -0,0 +1,192 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtsummary + +import ( + "container/list" + "time" + + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/types" +) + +// stmtSummaryByDigestEvicted contents digests evicted from stmtSummaryByDigestMap +type stmtSummaryByDigestEvicted struct { + // record evicted data in intervals + // latest history data is Back() + history *list.List +} + +// element being stored in stmtSummaryByDigestEvicted +type stmtSummaryByDigestEvictedElement struct { + // beginTime is the begin time of current interval + beginTime int64 + // endTime is the end time of current interval + endTime int64 + // *Kinds* of digest being evicted + digestKeyMap map[string]struct{} +} + +// spawn a new pointer to stmtSummaryByDigestEvicted +func newStmtSummaryByDigestEvicted() *stmtSummaryByDigestEvicted { + return &stmtSummaryByDigestEvicted{ + history: list.New(), + } +} + +// spawn a new pointer to stmtSummaryByDigestEvictedElement +func newStmtSummaryByDigestEvictedElement(beginTime int64, endTime int64) *stmtSummaryByDigestEvictedElement { + return &stmtSummaryByDigestEvictedElement{ + beginTime: beginTime, + endTime: endTime, + digestKeyMap: make(map[string]struct{}), + } +} + +// AddEvicted is used add an evicted record to stmtSummaryByDigestEvicted +func (ssbde *stmtSummaryByDigestEvicted) AddEvicted(evictedKey *stmtSummaryByDigestKey, evictedValue *stmtSummaryByDigest, historySize int) { + if evictedValue == nil { + return + } + + evictedValue.Lock() + defer evictedValue.Unlock() + + if evictedValue.history == nil { + return + } + for e, h := evictedValue.history.Back(), ssbde.history.Back(); e != nil; e = e.Prev() { + evictedElement := e.Value.(*stmtSummaryByDigestElement) + + // use closure to minimize time holding lock + func() { + evictedElement.Lock() + defer evictedElement.Unlock() + // no record in ssbde.history, direct insert + if ssbde.history.Len() == 0 && historySize != 0 { + + eBeginTime := evictedElement.beginTime + eEndTime := evictedElement.endTime + record := newStmtSummaryByDigestEvictedElement(eBeginTime, eEndTime) + record.addEvicted(evictedKey, evictedElement) + ssbde.history.PushFront(record) + + h = ssbde.history.Back() + return + } + + // look for matching history interval + MATCHING: + for ; h != nil; h = h.Prev() { + historyElement := h.Value.(*stmtSummaryByDigestEvictedElement) + + switch historyElement.matchAndAdd(evictedKey, evictedElement) { + case isMatch: + // automatically added + break MATCHING + // not matching, create a new record and insert + case isTooYoung: + { + eBeginTime := evictedElement.beginTime + eEndTime := evictedElement.endTime + record := newStmtSummaryByDigestEvictedElement(eBeginTime, eEndTime) + record.addEvicted(evictedKey, evictedElement) + ssbde.history.InsertAfter(record, h) + break MATCHING + } + default: // isTooOld + { + if h == ssbde.history.Front() { + // if digest older than all records in ssbde.history. + eBeginTime := evictedElement.beginTime + eEndTime := evictedElement.endTime + record := newStmtSummaryByDigestEvictedElement(eBeginTime, eEndTime) + record.addEvicted(evictedKey, evictedElement) + ssbde.history.PushFront(record) + break MATCHING + } + } + } + } + }() + + // prevent exceeding history size + for ssbde.history.Len() > historySize && ssbde.history.Len() > 0 { + ssbde.history.Remove(ssbde.history.Front()) + } + } +} + +// Clear up all records in stmtSummaryByDigestEvicted +func (ssbde *stmtSummaryByDigestEvicted) Clear() { + ssbde.history.Init() +} + +// add an evicted record to stmtSummaryByDigestEvictedElement +func (seElement *stmtSummaryByDigestEvictedElement) addEvicted(digestKey *stmtSummaryByDigestKey, digestValue *stmtSummaryByDigestElement) { + if digestKey != nil { + seElement.digestKeyMap[string(digestKey.Hash())] = struct{}{} + } +} + +const ( + isMatch = 0 + isTooOld = 1 + isTooYoung = 2 +) + +// matchAndAdd check time interval of seElement and digestValue. +// if matches, it will add the digest and return enum match +// if digest too old, it will return enum tooOld and do nothing +// if digest too young, it will return enum tooYoung and do nothing +func (seElement *stmtSummaryByDigestEvictedElement) matchAndAdd(digestKey *stmtSummaryByDigestKey, digestValue *stmtSummaryByDigestElement) (statement int) { + if seElement == nil || digestValue == nil { + return isTooYoung + } + sBeginTime, sEndTime := seElement.beginTime, seElement.endTime + eBeginTime, eEndTime := digestValue.beginTime, digestValue.endTime + if sBeginTime <= eBeginTime && eEndTime <= sEndTime { + seElement.addEvicted(digestKey, digestValue) + return isMatch + } else if eEndTime <= sBeginTime { + return isTooOld + } else { + return isTooYoung + } +} + +// ToEvictedCountDatum converts history evicted record to `evicted count` record's datum +func (ssbde *stmtSummaryByDigestEvicted) ToEvictedCountDatum() [][]types.Datum { + records := make([][]types.Datum, 0, ssbde.history.Len()) + for e := ssbde.history.Back(); e != nil; e = e.Prev() { + if record := e.Value.(*stmtSummaryByDigestEvictedElement).toEvictedCountDatum(); record != nil { + records = append(records, record) + } + } + return records +} + +// toEvictedCountDatum converts evicted record to `EvictedCount` record's datum +func (seElement *stmtSummaryByDigestEvictedElement) toEvictedCountDatum() []types.Datum { + datum := types.MakeDatums( + types.NewTime(types.FromGoTime(time.Unix(seElement.beginTime, 0)), mysql.TypeTimestamp, 0), + types.NewTime(types.FromGoTime(time.Unix(seElement.endTime, 0)), mysql.TypeTimestamp, 0), + int64(len(seElement.digestKeyMap)), + ) + return datum +} + +func (ssMap *stmtSummaryByDigestMap) ToEvictedCountDatum() [][]types.Datum { + return ssMap.other.ToEvictedCountDatum() +} diff --git a/util/stmtsummary/evicted_test.go b/util/stmtsummary/evicted_test.go new file mode 100644 index 0000000000000..36861eb4cfd1e --- /dev/null +++ b/util/stmtsummary/evicted_test.go @@ -0,0 +1,337 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package stmtsummary + +import ( + "bytes" + "container/list" + "fmt" + "time" + + . "github.com/pingcap/check" + "github.com/pingcap/log" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/types" +) + +func newInduceSsbd(beginTime int64, endTime int64) *stmtSummaryByDigest { + newSsbd := &stmtSummaryByDigest{ + history: list.New(), + } + newSsbd.history.PushBack(newInduceSsbde(beginTime, endTime)) + return newSsbd +} +func newInduceSsbde(beginTime int64, endTime int64) *stmtSummaryByDigestElement { + newSsbde := &stmtSummaryByDigestElement{ + beginTime: beginTime, + endTime: endTime, + minLatency: time.Duration.Round(1<<63-1, time.Nanosecond), + } + return newSsbde +} + +// generate new stmtSummaryByDigestKey and stmtSummaryByDigest +func (s *testStmtSummarySuite) generateStmtSummaryByDigestKeyValue(schema string, beginTime int64, endTime int64) (*stmtSummaryByDigestKey, *stmtSummaryByDigest) { + key := &stmtSummaryByDigestKey{ + schemaName: schema, + } + value := newInduceSsbd(beginTime, endTime) + return key, value +} + +// Test stmtSummaryByDigestMap.ToEvictedCountDatum +func (s *testStmtSummarySuite) TestMapToEvictedCountDatum(c *C) { + ssMap := newStmtSummaryByDigestMap() + ssMap.Clear() + now := time.Now().Unix() + interval := ssMap.refreshInterval() + ssMap.beginTimeForCurInterval = now + interval + + // set summaryMap capacity to 1. + err := ssMap.summaryMap.SetCapacity(1) + if err != nil { + log.Fatal(err.Error()) + } + ssMap.Clear() + + sei0 := generateAnyExecInfo() + sei1 := generateAnyExecInfo() + + sei0.SchemaName = "I'll occupy this cache! :(" + ssMap.AddStatement(sei0) + n := ssMap.beginTimeForCurInterval + sei1.SchemaName = "sorry, it's mine now. =)" + ssMap.AddStatement(sei1) + + expectedEvictedCount := []interface{}{ + types.NewTime(types.FromGoTime(time.Unix(n, 0)), mysql.TypeTimestamp, types.DefaultFsp), + types.NewTime(types.FromGoTime(time.Unix(n+interval, 0)), mysql.TypeTimestamp, types.DefaultFsp), + int64(1), + } + + // test stmtSummaryByDigestMap.toEvictedCountDatum + match(c, ssMap.ToEvictedCountDatum()[0], expectedEvictedCount...) + + // test multiple intervals + ssMap.Clear() + err = ssMap.SetRefreshInterval("60", false) + interval = ssMap.refreshInterval() + c.Assert(err, IsNil) + err = ssMap.SetMaxStmtCount("1", false) + c.Assert(err, IsNil) + err = ssMap.SetHistorySize("100", false) + c.Assert(err, IsNil) + + ssMap.beginTimeForCurInterval = now + interval + // insert one statement every other interval. + for i := 0; i < 50; i++ { + ssMap.AddStatement(generateAnyExecInfo()) + ssMap.beginTimeForCurInterval += interval * 2 + } + c.Assert(ssMap.summaryMap.Size(), Equals, 1) + val := ssMap.summaryMap.Values()[0] + c.Assert(val, NotNil) + digest := val.(*stmtSummaryByDigest) + c.Assert(digest.history.Len(), Equals, 50) + + err = ssMap.SetHistorySize("25", false) + c.Assert(err, IsNil) + // update begin time + ssMap.beginTimeForCurInterval += interval * 2 + banditSei := generateAnyExecInfo() + banditSei.SchemaName = "Kick you out >:(" + ssMap.AddStatement(banditSei) + + evictedCountDatums := ssMap.ToEvictedCountDatum() + c.Assert(len(evictedCountDatums), Equals, 25) + + // update begin time + banditSei.SchemaName = "Yet another kicker" + ssMap.AddStatement(banditSei) + + evictedCountDatums = ssMap.ToEvictedCountDatum() + // test young digest + c.Assert(len(evictedCountDatums), Equals, 25) + n = ssMap.beginTimeForCurInterval + newlyEvicted := evictedCountDatums[0] + expectedEvictedCount = []interface{}{ + types.NewTime(types.FromGoTime(time.Unix(n, 0)), mysql.TypeTimestamp, types.DefaultFsp), + types.NewTime(types.FromGoTime(time.Unix(n+interval, 0)), mysql.TypeTimestamp, types.DefaultFsp), + int64(1), + } + match(c, newlyEvicted, expectedEvictedCount...) +} + +// Test stmtSummaryByDigestEvicted +func (s *testStmtSummarySuite) TestSimpleStmtSummaryByDigestEvicted(c *C) { + ssbde := newStmtSummaryByDigestEvicted() + evictedKey, evictedValue := s.generateStmtSummaryByDigestKeyValue("a", 1, 2) + + // test NULL + ssbde.AddEvicted(nil, nil, 10) + c.Assert(ssbde.history.Len(), Equals, 0) + ssbde.Clear() + // passing NULL key is used as *refresh*. + ssbde.AddEvicted(nil, evictedValue, 10) + c.Assert(ssbde.history.Len(), Equals, 1) + ssbde.Clear() + ssbde.AddEvicted(evictedKey, nil, 10) + c.Assert(ssbde.history.Len(), Equals, 0) + ssbde.Clear() + + // test zero historySize + ssbde.AddEvicted(evictedKey, evictedValue, 0) + c.Assert(ssbde.history.Len(), Equals, 0) + + ssbde = newStmtSummaryByDigestEvicted() + ssbde.AddEvicted(evictedKey, evictedValue, 1) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 1, end: 2, count: 1}") + // test insert same *kind* of digest + ssbde.AddEvicted(evictedKey, evictedValue, 1) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 1, end: 2, count: 1}") + + evictedKey, evictedValue = s.generateStmtSummaryByDigestKeyValue("b", 1, 2) + ssbde.AddEvicted(evictedKey, evictedValue, 1) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 1, end: 2, count: 2}") + + evictedKey, evictedValue = s.generateStmtSummaryByDigestKeyValue("b", 5, 6) + ssbde.AddEvicted(evictedKey, evictedValue, 2) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 5, end: 6, count: 1}, {begin: 1, end: 2, count: 2}") + + evictedKey, evictedValue = s.generateStmtSummaryByDigestKeyValue("b", 3, 4) + ssbde.AddEvicted(evictedKey, evictedValue, 3) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 5, end: 6, count: 1}, {begin: 3, end: 4, count: 1}, {begin: 1, end: 2, count: 2}") + + // test evicted element with multi-time range value. + ssbde = newStmtSummaryByDigestEvicted() + evictedKey, evictedValue = s.generateStmtSummaryByDigestKeyValue("a", 1, 2) + evictedValue.history.PushBack(newInduceSsbde(2, 3)) + evictedValue.history.PushBack(newInduceSsbde(5, 6)) + evictedValue.history.PushBack(newInduceSsbde(8, 9)) + ssbde.AddEvicted(evictedKey, evictedValue, 3) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 8, end: 9, count: 1}, {begin: 5, end: 6, count: 1}, {begin: 2, end: 3, count: 1}") + + evictedKey = &stmtSummaryByDigestKey{schemaName: "b"} + ssbde.AddEvicted(evictedKey, evictedValue, 4) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 8, end: 9, count: 2}, {begin: 5, end: 6, count: 2}, {begin: 2, end: 3, count: 2}, {begin: 1, end: 2, count: 1}") + + evictedKey, evictedValue = s.generateStmtSummaryByDigestKeyValue("c", 4, 5) + evictedValue.history.PushBack(newInduceSsbde(5, 6)) + evictedValue.history.PushBack(newInduceSsbde(7, 8)) + ssbde.AddEvicted(evictedKey, evictedValue, 4) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 8, end: 9, count: 2}, {begin: 7, end: 8, count: 1}, {begin: 5, end: 6, count: 3}, {begin: 4, end: 5, count: 1}") + + evictedKey, evictedValue = s.generateStmtSummaryByDigestKeyValue("d", 7, 8) + ssbde.AddEvicted(evictedKey, evictedValue, 4) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 8, end: 9, count: 2}, {begin: 7, end: 8, count: 2}, {begin: 5, end: 6, count: 3}, {begin: 4, end: 5, count: 1}") + + // test for too old + evictedKey, evictedValue = s.generateStmtSummaryByDigestKeyValue("d", 0, 1) + evictedValue.history.PushBack(newInduceSsbde(1, 2)) + evictedValue.history.PushBack(newInduceSsbde(2, 3)) + evictedValue.history.PushBack(newInduceSsbde(4, 5)) + ssbde.AddEvicted(evictedKey, evictedValue, 4) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 8, end: 9, count: 2}, {begin: 7, end: 8, count: 2}, {begin: 5, end: 6, count: 3}, {begin: 4, end: 5, count: 2}") + + // test for too young + evictedKey, evictedValue = s.generateStmtSummaryByDigestKeyValue("d", 1, 2) + evictedValue.history.PushBack(newInduceSsbde(9, 10)) + ssbde.AddEvicted(evictedKey, evictedValue, 4) + c.Assert(getAllEvicted(ssbde), Equals, "{begin: 9, end: 10, count: 1}, {begin: 8, end: 9, count: 2}, {begin: 7, end: 8, count: 2}, {begin: 5, end: 6, count: 3}") +} + +// Test stmtSummaryByDigestEvictedElement.ToEvictedCountDatum +func (s *testStmtSummarySuite) TestStmtSummaryByDigestEvictedElement(c *C) { + record := newStmtSummaryByDigestEvictedElement(0, 1) + evictedKey, evictedValue := s.generateStmtSummaryByDigestKeyValue("alpha", 0, 1) + digestValue := evictedValue.history.Back().Value.(*stmtSummaryByDigestElement) + + // test poisoning will NULL key. + record.addEvicted(nil, nil) + c.Assert(getEvicted(record), Equals, "{begin: 0, end: 1, count: 0}") + record.addEvicted(nil, digestValue) + c.Assert(getEvicted(record), Equals, "{begin: 0, end: 1, count: 0}") + + // test add evictedKey and evicted stmtSummaryByDigestElement + record.addEvicted(evictedKey, digestValue) + c.Assert(getEvicted(record), Equals, "{begin: 0, end: 1, count: 1}") + + // test add same *kind* of values. + record.addEvicted(evictedKey, digestValue) + c.Assert(getEvicted(record), Equals, "{begin: 0, end: 1, count: 1}") + + // test add different *kind* of values. + evictedKey, evictedValue = s.generateStmtSummaryByDigestKeyValue("bravo", 0, 1) + digestValue = evictedValue.history.Back().Value.(*stmtSummaryByDigestElement) + record.addEvicted(evictedKey, digestValue) + c.Assert(getEvicted(record), Equals, "{begin: 0, end: 1, count: 2}") +} + +// test stmtSummaryByDigestEvicted.addEvicted +// test evicted count's detail +func (s *testStmtSummarySuite) TestEvictedCountDetailed(c *C) { + ssMap := newStmtSummaryByDigestMap() + ssMap.Clear() + err := ssMap.SetRefreshInterval("60", false) + c.Assert(err, IsNil) + err = ssMap.SetHistorySize("100", false) + c.Assert(err, IsNil) + now := time.Now().Unix() + interval := int64(60) + ssMap.beginTimeForCurInterval = now + interval + // set capacity to 1 + err = ssMap.summaryMap.SetCapacity(1) + c.Assert(err, IsNil) + + // test stmtSummaryByDigest's history length + for i := 0; i < 100; i++ { + if i == 0 { + c.Assert(ssMap.summaryMap.Size(), Equals, 0) + } else { + c.Assert(ssMap.summaryMap.Size(), Equals, 1) + val := ssMap.summaryMap.Values()[0] + c.Assert(val, NotNil) + digest := val.(*stmtSummaryByDigest) + c.Assert(digest.history.Len(), Equals, i) + } + ssMap.AddStatement(generateAnyExecInfo()) + ssMap.beginTimeForCurInterval += interval + } + ssMap.beginTimeForCurInterval -= interval + + banditSei := generateAnyExecInfo() + banditSei.SchemaName = "kick you out >:(" + ssMap.AddStatement(banditSei) + evictedCountDatums := ssMap.ToEvictedCountDatum() + n := ssMap.beginTimeForCurInterval + for _, evictedCountDatum := range evictedCountDatums { + expectedDatum := []interface{}{ + types.NewTime(types.FromGoTime(time.Unix(n, 0)), mysql.TypeTimestamp, types.DefaultFsp), + types.NewTime(types.FromGoTime(time.Unix(n+60, 0)), mysql.TypeTimestamp, types.DefaultFsp), + int64(1), + } + match(c, evictedCountDatum, expectedDatum...) + n -= 60 + } + + // test more than one eviction in single interval + banditSei.SchemaName = "Yet another kicker" + n = ssMap.beginTimeForCurInterval + expectedDatum := []interface{}{ + types.NewTime(types.FromGoTime(time.Unix(n, 0)), mysql.TypeTimestamp, types.DefaultFsp), + types.NewTime(types.FromGoTime(time.Unix(n+60, 0)), mysql.TypeTimestamp, types.DefaultFsp), + int64(2), + } + ssMap.AddStatement(banditSei) + evictedCountDatums = ssMap.ToEvictedCountDatum() + match(c, evictedCountDatums[0], expectedDatum...) + + ssMap.Clear() + other := ssMap.other + // test poisoning with empty-history digestValue + other.AddEvicted(new(stmtSummaryByDigestKey), new(stmtSummaryByDigest), 100) + c.Assert(other.history.Len(), Equals, 0) +} + +func (s *testStmtSummarySuite) TestNewStmtSummaryByDigestEvictedElement(c *C) { + now := time.Now().Unix() + end := now + 60 + stmtEvictedElement := newStmtSummaryByDigestEvictedElement(now, end) + c.Assert(stmtEvictedElement.beginTime, Equals, now) + c.Assert(stmtEvictedElement.endTime, Equals, end) + c.Assert(len(stmtEvictedElement.digestKeyMap), Equals, 0) +} + +func (s *testStmtSummarySuite) TestStmtSummaryByDigestEvicted(c *C) { + stmtEvicted := newStmtSummaryByDigestEvicted() + c.Assert(stmtEvicted.history.Len(), Equals, 0) +} + +func getAllEvicted(ssdbe *stmtSummaryByDigestEvicted) string { + buf := bytes.NewBuffer(nil) + for e := ssdbe.history.Back(); e != nil; e = e.Prev() { + if buf.Len() != 0 { + buf.WriteString(", ") + } + val := e.Value.(*stmtSummaryByDigestEvictedElement) + buf.WriteString(fmt.Sprintf("{begin: %v, end: %v, count: %v}", val.beginTime, val.endTime, len(val.digestKeyMap))) + } + return buf.String() +} + +func getEvicted(ssbdee *stmtSummaryByDigestEvictedElement) string { + buf := bytes.NewBuffer(nil) + buf.WriteString(fmt.Sprintf("{begin: %v, end: %v, count: %v}", ssbdee.beginTime, ssbdee.endTime, len(ssbdee.digestKeyMap))) + return buf.String() +} diff --git a/util/stmtsummary/statement_summary.go b/util/stmtsummary/statement_summary.go index 3dac542bed4b8..acfdaf405cd12 100644 --- a/util/stmtsummary/statement_summary.go +++ b/util/stmtsummary/statement_summary.go @@ -73,6 +73,9 @@ type stmtSummaryByDigestMap struct { // sysVars encapsulates system variables needed to control statement summary. sysVars *systemVars + + // other stores summary of evicted data. + other *stmtSummaryByDigestEvicted } // StmtSummaryByDigestMap is a global map containing all statement summaries. @@ -235,11 +238,20 @@ type StmtExecInfo struct { // newStmtSummaryByDigestMap creates an empty stmtSummaryByDigestMap. func newStmtSummaryByDigestMap() *stmtSummaryByDigestMap { sysVars := newSysVars() + + ssbde := newStmtSummaryByDigestEvicted() + maxStmtCount := uint(sysVars.getVariable(typeMaxStmtCount)) - return &stmtSummaryByDigestMap{ + newSsMap := &stmtSummaryByDigestMap{ summaryMap: kvcache.NewSimpleLRUCache(maxStmtCount, 0, 0), sysVars: sysVars, + other: ssbde, } + newSsMap.summaryMap.SetOnEvict(func(k kvcache.Key, v kvcache.Value) { + historySize := newSsMap.historySize() + newSsMap.other.AddEvicted(k.(*stmtSummaryByDigestKey), v.(*stmtSummaryByDigest), historySize) + }) + return newSsMap } // AddStatement adds a statement to StmtSummaryByDigestMap. @@ -291,7 +303,6 @@ func (ssMap *stmtSummaryByDigestMap) AddStatement(sei *StmtExecInfo) { summary.isInternal = summary.isInternal && sei.IsInternal return summary, beginTime }() - // Lock a single entry, not the whole cache. if summary != nil { summary.add(sei, beginTime, intervalSeconds, historySize) @@ -304,6 +315,7 @@ func (ssMap *stmtSummaryByDigestMap) Clear() { defer ssMap.Unlock() ssMap.summaryMap.DeleteAll() + ssMap.other.Clear() ssMap.beginTimeForCurInterval = 0 } From fb3a30ea04a30c33916dc29d62e6ad94006afd26 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Fri, 28 May 2021 15:27:35 +0800 Subject: [PATCH 202/343] =?UTF-8?q?expression:=20builtin=20function=20curr?= =?UTF-8?q?ent=5Fdate()=20and=20curdate()=20should=20return=20DATE=20t?= =?UTF-8?q?=E2=80=A6=20(#24950)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- expression/builtin_time.go | 1 + expression/typeinfer_test.go | 7 ++++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/expression/builtin_time.go b/expression/builtin_time.go index 67413dab11374..9fe7a39a3d80b 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -2108,6 +2108,7 @@ func (c *currentDateFunctionClass) getFunction(ctx sessionctx.Context, args []Ex return nil, err } bf.tp.Flen, bf.tp.Decimal = 10, 0 + bf.tp.Tp = mysql.TypeDate sig := &builtinCurrentDateSig{bf} return sig, nil } diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 853873db7d749..e0324123bbaae 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -1697,9 +1697,10 @@ func (s *testInferTypeSuite) createTestCase4TimeFuncs() []typeInferTestCase { {"utc_time(6) ", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 15, 6}, {"utc_time(7) ", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 6}, - {"utc_date() ", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 10, 0}, - {"curdate()", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 10, 0}, - {"sysdate(4)", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 19, 0}, + {"curdate() ", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 10, 0}, + {"current_date() ", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 10, 0}, + {"utc_date() ", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag | mysql.NotNullFlag, 10, 0}, + {"sysdate(4) ", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 19, 0}, {"date(c_int_d )", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, {"date(c_bigint_d )", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, From 1b048b167627275301b8b4f48fe4a79ea3fe3479 Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Fri, 28 May 2021 16:03:35 +0800 Subject: [PATCH 203/343] infoschema: Use system local time for tables of lock view (#24952) --- executor/executor_test.go | 4 ++-- infoschema/tables_test.go | 6 +++--- session/txninfo/txn_info.go | 3 +-- 3 files changed, 6 insertions(+), 7 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index e7f6adb0737d3..ea767d3d59f02 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8178,7 +8178,7 @@ func (s *testSerialSuite) TestIssue24210(c *C) { func (s *testSerialSuite) TestDeadlockTable(c *C) { deadlockhistory.GlobalDeadlockHistory.Clear() - occurTime := time.Date(2021, 5, 10, 1, 2, 3, 456789000, time.UTC) + occurTime := time.Date(2021, 5, 10, 1, 2, 3, 456789000, time.Local) rec := &deadlockhistory.DeadlockRecord{ OccurTime: occurTime, IsRetryable: false, @@ -8201,7 +8201,7 @@ func (s *testSerialSuite) TestDeadlockTable(c *C) { } deadlockhistory.GlobalDeadlockHistory.Push(rec) - occurTime2 := time.Date(2022, 6, 11, 2, 3, 4, 987654000, time.UTC) + occurTime2 := time.Date(2022, 6, 11, 2, 3, 4, 987654000, time.Local) rec2 := &deadlockhistory.DeadlockRecord{ OccurTime: occurTime2, IsRetryable: true, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index c376116c61172..3de42c28e1235 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1564,7 +1564,7 @@ func (s *testTableSuite) TestTrx(c *C) { Username: "root", CurrentDB: "test", } - blockTime2 := time.Date(2021, 05, 20, 13, 18, 30, 123456000, time.UTC) + blockTime2 := time.Date(2021, 05, 20, 13, 18, 30, 123456000, time.Local) sm.txnInfo[1] = &txninfo.TxnInfo{ StartTS: 425070846483628033, CurrentSQLDigest: "", @@ -1577,8 +1577,8 @@ func (s *testTableSuite) TestTrx(c *C) { } tk.Se.SetSessionManager(sm) tk.MustQuery("select * from information_schema.TIDB_TRX;").Check(testkit.Rows( - "424768545227014155 2021-05-07 04:56:48.001000 "+digest.String()+" Normal 1 19 2 root test []", - "425070846483628033 2021-05-20 13:16:35.778000 LockWaiting 2021-05-20 13:18:30.123456 0 0 10 user1 db1 [sql1, sql2]")) + "424768545227014155 2021-05-07 12:56:48.001000 "+digest.String()+" Normal 1 19 2 root test []", + "425070846483628033 2021-05-20 21:16:35.778000 LockWaiting 2021-05-20 13:18:30.123456 0 0 10 user1 db1 [sql1, sql2]")) } func (s *testTableSuite) TestInfoschemaDeadlockPrivilege(c *C) { diff --git a/session/txninfo/txn_info.go b/session/txninfo/txn_info.go index acc52e985f0f9..e7f5afc319c6b 100644 --- a/session/txninfo/txn_info.go +++ b/session/txninfo/txn_info.go @@ -97,8 +97,7 @@ func (info *TxnInfo) ShallowClone() *TxnInfo { // ToDatum Converts the `TxnInfo` to `Datum` to show in the `TIDB_TRX` table. func (info *TxnInfo) ToDatum() []types.Datum { - // TODO: The timezone represented to the user is not correct and it will be always UTC time. - humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(info.StartTS)*1e6).UTC() + humanReadableStartTime := time.Unix(0, oracle.ExtractPhysical(info.StartTS)*1e6) var currentDigest interface{} if len(info.CurrentSQLDigest) != 0 { From c287bcb48cb3698916fe94c1db047dd82b226ab8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20van=20Eeden?= Date: Fri, 28 May 2021 10:33:36 +0200 Subject: [PATCH 204/343] executor: Add plugin column to mysql.user (#24894) --- executor/grant.go | 2 +- executor/show.go | 11 ++++++++--- executor/show_test.go | 9 +++++++++ executor/simple.go | 8 ++++---- privilege/privileges/privileges.go | 15 ++++++++++----- session/bootstrap.go | 16 ++++++++++++++-- session/bootstrap_test.go | 4 ++-- 7 files changed, 48 insertions(+), 17 deletions(-) diff --git a/executor/grant.go b/executor/grant.go index 54b5b66b70612..2b5efcd0803eb 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -137,7 +137,7 @@ func (e *GrantExec) Next(ctx context.Context, req *chunk.Chunk) error { if !ok { return errors.Trace(ErrPasswordFormat) } - _, err := internalSession.(sqlexec.SQLExecutor).ExecuteInternal(ctx, `INSERT INTO %n.%n (Host, User, authentication_string) VALUES (%?, %?, %?);`, mysql.SystemDB, mysql.UserTable, user.User.Hostname, user.User.Username, pwd) + _, err := internalSession.(sqlexec.SQLExecutor).ExecuteInternal(ctx, `INSERT INTO %n.%n (Host, User, authentication_string, plugin) VALUES (%?, %?, %?, %?);`, mysql.SystemDB, mysql.UserTable, user.User.Hostname, user.User.Username, pwd, mysql.AuthNativePassword) if err != nil { return err } diff --git a/executor/show.go b/executor/show.go index 78f2e409b27e4..2c7d922981784 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1331,7 +1331,7 @@ func (e *ShowExec) fetchShowCreateUser() error { exec := e.ctx.(sqlexec.RestrictedSQLExecutor) - stmt, err := exec.ParseWithParams(context.TODO(), `SELECT * FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.UserTable, userName, hostName) + stmt, err := exec.ParseWithParams(context.TODO(), `SELECT plugin FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.UserTable, userName, hostName) if err != nil { return errors.Trace(err) } @@ -1346,6 +1346,11 @@ func (e *ShowExec) fetchShowCreateUser() error { fmt.Sprintf("'%s'@'%s'", e.User.Username, e.User.Hostname)) } + authplugin := mysql.AuthNativePassword + if len(rows) == 1 && rows[0].GetString(0) != "" { + authplugin = rows[0].GetString(0) + } + stmt, err = exec.ParseWithParams(context.TODO(), `SELECT Priv FROM %n.%n WHERE User=%? AND Host=%?`, mysql.SystemDB, mysql.GlobalPrivTable, userName, hostName) if err != nil { return errors.Trace(err) @@ -1366,8 +1371,8 @@ func (e *ShowExec) fetchShowCreateUser() error { require = privValue.RequireStr() } // FIXME: the returned string is not escaped safely - showStr := fmt.Sprintf("CREATE USER '%s'@'%s' IDENTIFIED WITH 'mysql_native_password' AS '%s' REQUIRE %s PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK", - e.User.Username, e.User.Hostname, checker.GetEncodedPassword(e.User.Username, e.User.Hostname), require) + showStr := fmt.Sprintf("CREATE USER '%s'@'%s' IDENTIFIED WITH '%s' AS '%s' REQUIRE %s PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK", + e.User.Username, e.User.Hostname, authplugin, checker.GetEncodedPassword(e.User.Username, e.User.Hostname), require) e.appendRow([]interface{}{showStr}) return nil } diff --git a/executor/show_test.go b/executor/show_test.go index 57cec97990dc4..b00379b72418e 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -433,6 +433,15 @@ func (s *testSuite5) TestShowCreateUser(c *C) { // "show create user" for current user doesn't check privileges. rows = tk1.MustQuery("show create user current_user") rows.Check(testkit.Rows("CREATE USER 'check_priv'@'127.0.0.1' IDENTIFIED WITH 'mysql_native_password' AS '' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK")) + + // Creating users with `IDENTIFIED WITH 'caching_sha2_password'` is not supported yet. So manually creating an entry for now. + // later this can be changed to test the full path once 'caching_sha2_password' support is completed. + tk.MustExec("CREATE USER 'sha_test'@'%' IDENTIFIED BY 'temp_passwd'") + tk.MustExec("UPDATE mysql.user SET plugin='caching_sha2_password', authentication_string=0x24412430303524532C06366D1D1E2B2F4437681A057B6807193D1C4B6E772F667A764663534E6C3978716C3057644D73427A787747674679687632644A384F337941704A542F WHERE user='sha_test' AND host='%'") + tk.MustExec("FLUSH PRIVILEGES") + + rows = tk.MustQuery("SHOW CREATE USER 'sha_test'@'%'") + rows.Check(testkit.Rows("CREATE USER 'sha_test'@'%' IDENTIFIED WITH 'caching_sha2_password' AS '$A$005$S,\x066m\x1d\x1e+/D7h\x1a\x05{h\a\x19=\x1cKnw/fzvFcSNl9xql0WdMsBzxwGgFyhv2dJ8O3yApJT/' REQUIRE NONE PASSWORD EXPIRE DEFAULT ACCOUNT UNLOCK")) } func (s *testSuite5) TestUnprivilegedShow(c *C) { diff --git a/executor/simple.go b/executor/simple.go index ec32f17150ebf..4bbb2c3903b0a 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -753,9 +753,9 @@ func (e *SimpleExec) executeCreateUser(ctx context.Context, s *ast.CreateUserStm sql := new(strings.Builder) if s.IsCreateRole { - sqlexec.MustFormatSQL(sql, `INSERT INTO %n.%n (Host, User, authentication_string, Account_locked) VALUES `, mysql.SystemDB, mysql.UserTable) + sqlexec.MustFormatSQL(sql, `INSERT INTO %n.%n (Host, User, authentication_string, plugin, Account_locked) VALUES `, mysql.SystemDB, mysql.UserTable) } else { - sqlexec.MustFormatSQL(sql, `INSERT INTO %n.%n (Host, User, authentication_string) VALUES `, mysql.SystemDB, mysql.UserTable) + sqlexec.MustFormatSQL(sql, `INSERT INTO %n.%n (Host, User, authentication_string, plugin) VALUES `, mysql.SystemDB, mysql.UserTable) } users := make([]*auth.UserIdentity, 0, len(s.Specs)) @@ -784,9 +784,9 @@ func (e *SimpleExec) executeCreateUser(ctx context.Context, s *ast.CreateUserStm return errors.Trace(ErrPasswordFormat) } if s.IsCreateRole { - sqlexec.MustFormatSQL(sql, `(%?, %?, %?, %?)`, spec.User.Hostname, spec.User.Username, pwd, "Y") + sqlexec.MustFormatSQL(sql, `(%?, %?, %?, %?, %?)`, spec.User.Hostname, spec.User.Username, pwd, mysql.AuthNativePassword, "Y") } else { - sqlexec.MustFormatSQL(sql, `(%?, %?, %?)`, spec.User.Hostname, spec.User.Username, pwd) + sqlexec.MustFormatSQL(sql, `(%?, %?, %?, %?)`, spec.User.Hostname, spec.User.Username, pwd, mysql.AuthNativePassword) } users = append(users, spec.User) } diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index cdaca2f634263..cb68345738396 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -171,11 +171,16 @@ func (p *UserPrivileges) GetEncodedPassword(user, host string) string { return "" } pwd := record.AuthenticationString - if len(pwd) != 0 && len(pwd) != mysql.PWDHashLen+1 { - logutil.BgLogger().Error("user password from system DB not like sha1sum", zap.String("user", user)) - return "" - } - return pwd + switch len(pwd) { + case 0: + return pwd + case mysql.PWDHashLen + 1: // mysql_native_password + return pwd + case 70: // caching_sha2_password + return pwd + } + logutil.BgLogger().Error("user password from system DB not like a known hash format", zap.String("user", user), zap.Int("hash_length", len(pwd))) + return "" } // GetAuthWithoutVerification implements the Manager interface. diff --git a/session/bootstrap.go b/session/bootstrap.go index 34d6748ae38c1..6c03e5ae73e34 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -55,6 +55,7 @@ const ( Host CHAR(64), User CHAR(32), authentication_string TEXT, + plugin CHAR(64), Select_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Insert_priv ENUM('N','Y') NOT NULL DEFAULT 'N', Update_priv ENUM('N','Y') NOT NULL DEFAULT 'N', @@ -489,11 +490,13 @@ const ( version68 = 68 // version69 adds mysql.global_grants for DYNAMIC privileges version69 = 69 + // version70 adds mysql.user.plugin to allow multiple authentication plugins + version70 = 70 ) // 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 = version69 +var currentBootstrapVersion int64 = version70 var ( bootstrapVersion = []func(Session, int64){ @@ -566,6 +569,7 @@ var ( upgradeToVer67, upgradeToVer68, upgradeToVer69, + upgradeToVer70, } ) @@ -1489,6 +1493,14 @@ func upgradeToVer69(s Session, ver int64) { doReentrantDDL(s, CreateGlobalGrantsTable) } +func upgradeToVer70(s Session, ver int64) { + if ver >= version70 { + return + } + doReentrantDDL(s, "ALTER TABLE mysql.user ADD COLUMN plugin CHAR(64) AFTER authentication_string", infoschema.ErrColumnExists) + mustExecute(s, "UPDATE HIGH_PRIORITY mysql.user SET plugin='mysql_native_password'") +} + 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= %?`, @@ -1577,7 +1589,7 @@ func doDMLWorks(s Session) { // Insert a default user with empty password. mustExecute(s, `INSERT HIGH_PRIORITY INTO mysql.user VALUES - ("%", "root", "", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y", "Y", "Y", "Y")`) + ("%", "root", "", "mysql_native_password", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y", "Y", "Y", "Y")`) // Init global system variables table. values := make([]string, 0, len(variable.GetSysVars())) diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index ffa9ba22e5723..a27dd7a3ea7ba 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -57,7 +57,7 @@ func (s *testBootstrapSuite) TestBootstrap(c *C) { c.Assert(err, IsNil) c.Assert(req.NumRows() == 0, IsFalse) datums := statistics.RowToDatums(req.GetRow(0), r.Fields()) - match(c, datums, `%`, "root", "", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y", "Y", "Y", "Y") + match(c, datums, `%`, "root", "", "mysql_native_password", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y", "Y", "Y", "Y") c.Assert(se.Auth(&auth.UserIdentity{Username: "root", Hostname: "anyhost"}, []byte(""), []byte("")), IsTrue) mustExecSQL(c, se, "USE test;") @@ -162,7 +162,7 @@ func (s *testBootstrapSuite) TestBootstrapWithError(c *C) { c.Assert(req.NumRows() == 0, IsFalse) row := req.GetRow(0) datums := statistics.RowToDatums(row, r.Fields()) - match(c, datums, `%`, "root", "", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y", "Y", "Y", "Y") + match(c, datums, `%`, "root", "", "mysql_native_password", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N", "Y", "Y", "Y", "Y", "Y", "Y", "Y") c.Assert(r.Close(), IsNil) mustExecSQL(c, se, "USE test;") From 963288a68c72de60db2612b3147f806709ac29f3 Mon Sep 17 00:00:00 2001 From: wjHuang Date: Fri, 28 May 2021 16:47:35 +0800 Subject: [PATCH 205/343] planner: build plan for CTE (#24792) --- expression/integration_test.go | 42 +++ planner/core/exhaust_physical_plans.go | 109 +++---- planner/core/expression_rewriter.go | 9 + planner/core/find_best_task.go | 50 ++- planner/core/find_best_task_test.go | 8 +- planner/core/initialize.go | 26 ++ planner/core/logical_plan_builder.go | 409 ++++++++++++++++++++++++- planner/core/logical_plans.go | 40 +++ planner/core/physical_plans.go | 25 ++ planner/core/plan.go | 2 +- planner/core/planbuilder.go | 29 +- planner/core/preprocess.go | 12 +- util/plancodec/id.go | 4 + 13 files changed, 701 insertions(+), 64 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 1d883964d996a..061ef6ed81de2 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" @@ -9418,6 +9419,47 @@ func (s *testIntegrationSuite) TestIssue23925(c *C) { tk.MustQuery("select max(b) + 0 from t group by a;").Check(testkit.Rows("2")) } +func (s *testIntegrationSuite) TestCTEInvalidUsage(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + // A CTE can refer to CTEs defined earlier in the same WITH clause, but not those defined later. + tk.MustGetErrCode("with cte1 as (select * from cte2), cte2 as (select 1) select * from cte1;", errno.ErrNoSuchTable) + // A CTE in a given query block can refer to CTEs defined in query blocks at a more outer level, but not CTEs defined in query blocks at a more inner level. + // MySQL allows this statement, and it should be a bug of MySQL. PostgreSQL also reports an error. + tk.MustGetErrCode("with cte1 as (select * from cte2) select * from (with cte2 as (select 2) select * from cte1 ) q;", errno.ErrNoSuchTable) + // Aggregation function is not allowed in the recursive part. + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select sum(n) from cte group by n) select * from cte;", errno.ErrCTERecursiveForbidsAggregation) + // Window function is not allowed in the recursive part. + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select row_number() over(partition by n) from cte ) select * from cte;", errno.ErrCTERecursiveForbidsAggregation) + // Group by is not allowed in the recursive part. + tk.MustGetErrCode("with recursive cte(n) as (select 1 union (select * from cte order by n)) select * from cte;", errno.ErrNotSupportedYet) + tk.MustGetErrCode("with recursive cte(n) as (select 1 union (select * from cte order by n)) select * from cte;", errno.ErrNotSupportedYet) + // Distinct is not allowed in the recursive part. + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select distinct * from cte) select * from cte;", errno.ErrNotSupportedYet) + // Limit is not allowed in the recursive part. + tk.MustGetErrCode("with recursive cte(n) as (select 1 union (select * from cte limit 2)) select * from cte;", errno.ErrNotSupportedYet) + // The recursive SELECT part must reference the CTE only once and only in its FROM clause, not in any subquery. + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select * from cte, cte c1) select * from cte;", errno.ErrInvalidRequiresSingleReference) + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select * from (select * from cte) c1) select * from cte;", errno.ErrInvalidRequiresSingleReference) + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select * from cte where 1 in (select * from cte)) select * from cte;", errno.ErrInvalidRequiresSingleReference) + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select * from cte where exists (select * from cte)) select * from cte;", errno.ErrInvalidRequiresSingleReference) + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select * from cte where 1 > (select * from cte)) select * from cte;", errno.ErrInvalidRequiresSingleReference) + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select (select * from cte) c1) select * from cte;", errno.ErrInvalidRequiresSingleReference) + // The recursive part can reference tables other than the CTE and join them with the CTE. If used in a join like this, the CTE must not be on the right side of a LEFT JOIN. + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select * from t left join cte on t.a=cte.n) select * from cte;", errno.ErrCTERecursiveForbiddenJoinOrder) + // Recursive part containing non-recursive query is not allowed. + tk.MustGetErrCode("with recursive cte(n) as (select 1 intersect select 2 union select * from cte union select 1) select * from cte;", errno.ErrCTERecursiveRequiresNonRecursiveFirst) + tk.MustGetErrCode("with recursive cte(n) as (select * from cte union select * from cte) select * from cte;", errno.ErrCTERecursiveRequiresNonRecursiveFirst) + // Invalid use of intersect/except. + tk.MustGetErrCode("with recursive cte(n) as (select 1 intersect select * from cte) select * from cte;", errno.ErrNotSupportedYet) + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select 1 intersect select * from cte) select * from cte;", errno.ErrNotSupportedYet) + tk.MustGetErrCode("with recursive cte(n) as (select 1 except select * from cte) select * from cte;", errno.ErrNotSupportedYet) + tk.MustGetErrCode("with recursive cte(n) as (select 1 union select 1 except select * from cte) select * from cte;", errno.ErrNotSupportedYet) +} + func (s *testIntegrationSuite) TestIssue23889(c *C) { defer s.cleanEnv(c) tk := testkit.NewTestKit(c, s.store) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 7c7493eedd5c5..dcb4e991ebe33 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -39,16 +39,16 @@ import ( "go.uber.org/zap" ) -func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { if prop.IsFlashProp() { - return nil, true + return nil, true, nil } childProp := prop.CloneEssentialFields() us := PhysicalUnionScan{ Conditions: p.conditions, HandleCols: p.handleCols, }.Init(p.ctx, p.stats, p.blockOffset, childProp) - return []PhysicalPlan{us}, true + return []PhysicalPlan{us}, true, nil } func getMaxSortPrefix(sortCols, allCols []*expression.Column) []int { @@ -1647,19 +1647,19 @@ func (p *LogicalJoin) shouldUseMPPBCJ() bool { // Firstly we check the hint, if hint is figured by user, we force to choose the corresponding physical plan. // If the hint is not matched, it will get other candidates. // If the hint is not figured, we will pick all candidates. -func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { if val.(bool) { indexJoins, _ := p.tryToGetIndexJoin(prop) - failpoint.Return(indexJoins, true) + failpoint.Return(indexJoins, true, nil) } }) if prop.IsFlashProp() && ((p.preferJoinType&preferBCJoin) == 0 && p.preferJoinType > 0) { - return nil, false + return nil, false, nil } if prop.PartitionTp == property.BroadcastType { - return nil, false + return nil, false, nil } joins := make([]PhysicalPlan, 0, 8) canPushToTiFlash := p.canPushToCop(kv.TiFlash) @@ -1667,7 +1667,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P if p.shouldUseMPPBCJ() { mppJoins := p.tryToGetMppHashJoin(prop, true) if (p.preferJoinType & preferBCJoin) > 0 { - return mppJoins, true + return mppJoins, true, nil } joins = append(joins, mppJoins...) } else { @@ -1677,29 +1677,29 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } else if p.ctx.GetSessionVars().AllowBCJ && canPushToTiFlash { broadCastJoins := p.tryToGetBroadCastJoin(prop) if (p.preferJoinType & preferBCJoin) > 0 { - return broadCastJoins, true + return broadCastJoins, true, nil } joins = append(joins, broadCastJoins...) } if prop.IsFlashProp() { - return joins, true + return joins, true, nil } mergeJoins := p.GetMergeJoin(prop, p.schema, p.Stats(), p.children[0].statsInfo(), p.children[1].statsInfo()) if (p.preferJoinType&preferMergeJoin) > 0 && len(mergeJoins) > 0 { - return mergeJoins, true + return mergeJoins, true, nil } joins = append(joins, mergeJoins...) indexJoins, forced := p.tryToGetIndexJoin(prop) if forced { - return indexJoins, true + return indexJoins, true, nil } joins = append(joins, indexJoins...) hashJoins := p.getHashJoins(prop) if (p.preferJoinType&preferHashJoin) > 0 && len(hashJoins) > 0 { - return hashJoins, true + return hashJoins, true, nil } joins = append(joins, hashJoins...) @@ -1707,9 +1707,9 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P // If we reach here, it means we have a hint that doesn't work. // It might be affected by the required property, so we enforce // this property and try the hint again. - return joins, false + return joins, false, nil } - return joins, true + return joins, true, nil } func canExprsInJoinPushdown(p *LogicalJoin, storeType kv.StoreType) bool { @@ -1939,10 +1939,10 @@ func (p *LogicalProjection) TryToGetChildProp(prop *property.PhysicalProperty) ( return newProp, true } -func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { newProp, ok := p.TryToGetChildProp(prop) if !ok { - return nil, true + return nil, true, nil } proj := PhysicalProjection{ Exprs: p.Exprs, @@ -1950,7 +1950,7 @@ func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty AvoidColumnEvaluator: p.AvoidColumnEvaluator, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, newProp) proj.SetSchema(p.schema) - return []PhysicalPlan{proj}, true + return []PhysicalPlan{proj}, true, nil } func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPlan { @@ -2027,11 +2027,11 @@ func MatchItems(p *property.PhysicalProperty, items []*util.ByItems) bool { return true } -func (lt *LogicalTopN) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (lt *LogicalTopN) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { if MatchItems(prop, lt.ByItems) { - return append(lt.getPhysTopN(prop), lt.getPhysLimits(prop)...), true + return append(lt.getPhysTopN(prop), lt.getPhysLimits(prop)...), true, nil } - return nil, true + return nil, true, nil } // GetHashJoin is public for cascades planner. @@ -2039,9 +2039,9 @@ func (la *LogicalApply) GetHashJoin(prop *property.PhysicalProperty) *PhysicalHa return la.LogicalJoin.getHashJoin(prop, 1, false) } -func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { if !prop.AllColsFromSchema(la.children[0].Schema()) || prop.IsFlashProp() { // for convenient, we don't pass through any prop - return nil, true + return nil, true, nil } disableAggPushDownToCop(la.children[0]) join := la.GetHashJoin(prop) @@ -2074,7 +2074,7 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, SortItems: prop.SortItems}, &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64}) apply.SetSchema(la.schema) - return []PhysicalPlan{apply}, true + return []PhysicalPlan{apply}, true, nil } func disableAggPushDownToCop(p LogicalPlan) { @@ -2086,16 +2086,16 @@ func disableAggPushDownToCop(p LogicalPlan) { } } -func (p *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { if prop.IsFlashProp() { - return nil, true + return nil, true, nil } var byItems []property.SortItem byItems = append(byItems, p.PartitionBy...) byItems = append(byItems, p.OrderBy...) childProperty := &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, SortItems: byItems, CanAddEnforcer: true} if !prop.IsPrefix(childProperty) { - return nil, true + return nil, true, nil } window := PhysicalWindow{ WindowFuncDescs: p.WindowFuncDescs, @@ -2104,11 +2104,11 @@ func (p *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ Frame: p.Frame, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, childProperty) window.SetSchema(p.Schema()) - return []PhysicalPlan{window}, true + return []PhysicalPlan{window}, true, nil } // exhaustPhysicalPlans is only for implementing interface. DataSource and Dual generate task in `findBestTask` directly. -func (p *baseLogicalPlan) exhaustPhysicalPlans(_ *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *baseLogicalPlan) exhaustPhysicalPlans(_ *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { panic("baseLogicalPlan.exhaustPhysicalPlans() should never be called.") } @@ -2407,7 +2407,7 @@ func (la *LogicalAggregation) ResetHintIfConflicted() (preferHash bool, preferSt return } -func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { if la.aggHints.preferAggToCop { if !la.canPushToCop(kv.TiKV) { errMsg := "Optimizer Hint AGG_TO_COP is inapplicable" @@ -2421,12 +2421,12 @@ func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProper hashAggs := la.getHashAggs(prop) if hashAggs != nil && preferHash { - return hashAggs, true + return hashAggs, true, nil } streamAggs := la.getStreamAggs(prop) if streamAggs != nil && preferStream { - return streamAggs, true + return streamAggs, true, nil } aggs := append(hashAggs, streamAggs...) @@ -2437,20 +2437,20 @@ func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProper la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) } - return aggs, !(preferStream || preferHash) + return aggs, !(preferStream || preferHash), nil } -func (p *LogicalSelection) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *LogicalSelection) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { childProp := prop.CloneEssentialFields() sel := PhysicalSelection{ Conditions: p.Conditions, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, childProp) - return []PhysicalPlan{sel}, true + return []PhysicalPlan{sel}, true, nil } -func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { if !prop.IsEmpty() { - return nil, true + return nil, true, nil } if p.limitHints.preferLimitToCop { @@ -2476,12 +2476,12 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] limit.SetSchema(p.Schema()) ret = append(ret, limit) } - return ret, true + return ret, true, nil } -func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { if prop.IsFlashProp() { - return nil, true + return nil, true, nil } childProp := prop.CloneEssentialFields() lock := PhysicalLock{ @@ -2489,13 +2489,13 @@ func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P TblID2Handle: p.tblID2Handle, PartitionedTable: p.partitionedTable, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) - return []PhysicalPlan{lock}, true + return []PhysicalPlan{lock}, true, nil } -func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { // TODO: UnionAll can not pass any order, but we can change it to sort merge to keep order. if !prop.IsEmpty() || prop.IsFlashProp() { - return nil, true + return nil, true, nil } chReqProps := make([]*property.PhysicalProperty, 0, len(p.children)) for range p.children { @@ -2503,15 +2503,18 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) } ua := PhysicalUnionAll{}.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) ua.SetSchema(p.Schema()) - return []PhysicalPlan{ua}, true + return []PhysicalPlan{ua}, true, nil } -func (p *LogicalPartitionUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { - uas, flagHint := p.LogicalUnionAll.exhaustPhysicalPlans(prop) +func (p *LogicalPartitionUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { + uas, flagHint, err := p.LogicalUnionAll.exhaustPhysicalPlans(prop) + if err != nil { + return nil, false, err + } for _, ua := range uas { ua.(*PhysicalUnionAll).tp = plancodec.TypePartitionUnion } - return uas, flagHint + return uas, flagHint, nil } func (ls *LogicalSort) getPhysicalSort(prop *property.PhysicalProperty) *PhysicalSort { @@ -2530,7 +2533,7 @@ func (ls *LogicalSort) getNominalSort(reqProp *property.PhysicalProperty) *Nomin return ps } -func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { if MatchItems(prop, ls.ByItems) { ret := make([]PhysicalPlan, 0, 2) ret = append(ret, ls.getPhysicalSort(prop)) @@ -2538,15 +2541,15 @@ func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] if ns != nil { ret = append(ret, ns) } - return ret, true + return ret, true, nil } - return nil, true + return nil, true, nil } -func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { if !prop.IsEmpty() || prop.IsFlashProp() { - return nil, true + return nil, true, nil } mor := PhysicalMaxOneRow{}.Init(p.ctx, p.stats, p.blockOffset, &property.PhysicalProperty{ExpectedCnt: 2}) - return []PhysicalPlan{mor}, true + return []PhysicalPlan{mor}, true, nil } diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 7dd992243953e..c228ee957c90b 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -170,6 +170,7 @@ func (b *PlanBuilder) getExpressionRewriter(ctx context.Context, p LogicalPlan) rewriter.ctxStack = rewriter.ctxStack[:0] rewriter.ctxNameStk = rewriter.ctxNameStk[:0] rewriter.ctx = ctx + rewriter.err = nil return } @@ -497,6 +498,8 @@ func (er *expressionRewriter) buildSemiApplyFromEqualSubq(np LogicalPlan, l, r e } func (er *expressionRewriter) handleCompareSubquery(ctx context.Context, v *ast.CompareSubqueryExpr) (ast.Node, bool) { + ci := er.b.prepareCTECheckForSubQuery() + defer resetCTECheckForSubQuery(ci) v.L.Accept(er) if er.err != nil { return v, true @@ -780,6 +783,8 @@ func (er *expressionRewriter) handleEQAll(lexpr, rexpr expression.Expression, np } func (er *expressionRewriter) handleExistSubquery(ctx context.Context, v *ast.ExistsSubqueryExpr) (ast.Node, bool) { + ci := er.b.prepareCTECheckForSubQuery() + defer resetCTECheckForSubQuery(ci) subq, ok := v.Sel.(*ast.SubqueryExpr) if !ok { er.err = errors.Errorf("Unknown exists type %T.", v.Sel) @@ -845,6 +850,8 @@ out: } func (er *expressionRewriter) handleInSubquery(ctx context.Context, v *ast.PatternInExpr) (ast.Node, bool) { + ci := er.b.prepareCTECheckForSubQuery() + defer resetCTECheckForSubQuery(ci) asScalar := er.asScalar er.asScalar = true v.Expr.Accept(er) @@ -954,6 +961,8 @@ func (er *expressionRewriter) handleInSubquery(ctx context.Context, v *ast.Patte } func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, v *ast.SubqueryExpr) (ast.Node, bool) { + ci := er.b.prepareCTECheckForSubQuery() + defer resetCTECheckForSubQuery(ci) np, err := er.buildSubquery(ctx, v) if err != nil { er.err = err diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 8e8086b8061de..eede1b4788059 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -14,6 +14,7 @@ package core import ( + "context" "math" "github.com/pingcap/errors" @@ -311,7 +312,10 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun var hintWorksWithProp bool // Maybe the plan can satisfy the required property, // so we try to get the task without the enforced sort first. - plansFitsProp, hintWorksWithProp = p.self.exhaustPhysicalPlans(newProp) + plansFitsProp, hintWorksWithProp, err = p.self.exhaustPhysicalPlans(newProp) + if err != nil { + return nil, 0, err + } if !hintWorksWithProp && !newProp.IsEmpty() { // If there is a hint in the plan and the hint cannot satisfy the property, // we enforce this property and try to generate the PhysicalPlan again to @@ -327,7 +331,10 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun newProp.PartitionCols = nil newProp.PartitionTp = property.AnyType var hintCanWork bool - plansNeedEnforce, hintCanWork = p.self.exhaustPhysicalPlans(newProp) + plansNeedEnforce, hintCanWork, err = p.self.exhaustPhysicalPlans(newProp) + if err != nil { + return nil, 0, err + } if hintCanWork && !hintWorksWithProp { // If the hint can work with the empty property, but cannot work with // the required property, we give up `plansFitProp` to make sure the hint @@ -1932,3 +1939,42 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper is.cost = cost return is, cost, rowCount } + +func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (t task, cntPlan int64, err error) { + if !prop.IsEmpty() { + return invalidTask, 1, nil + } + if p.cte.cteTask != nil { + // Already built it. + return p.cte.cteTask, 1, nil + } + sp, _, err := DoOptimize(context.TODO(), p.ctx, p.cte.optFlag, p.cte.seedPartLogicalPlan) + if err != nil { + return nil, 1, err + } + + var rp PhysicalPlan + if p.cte.recursivePartLogicalPlan != nil { + rp, _, err = DoOptimize(context.TODO(), p.ctx, p.cte.optFlag, p.cte.recursivePartLogicalPlan) + if err != nil { + return nil, 1, err + } + } + + pcte := PhysicalCTE{SeedPlan: sp, RecurPlan: rp, CTE: p.cte}.Init(p.ctx, p.stats) + pcte.SetSchema(p.schema) + t = &rootTask{pcte, sp.statsInfo().RowCount} + p.cte.cteTask = t + return t, 1, nil +} + +func (p *LogicalCTETable) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (t task, cntPlan int64, err error) { + if !prop.IsEmpty() { + return nil, 1, nil + } + + pcteTable := PhysicalCTETable{IDForStorage: p.idForStorage}.Init(p.ctx, p.stats) + pcteTable.SetSchema(p.schema) + t = &rootTask{p: pcteTable} + return t, 1, nil +} diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go index dc83476d1cfac..c744282db0c45 100644 --- a/planner/core/find_best_task_test.go +++ b/planner/core/find_best_task_test.go @@ -96,14 +96,14 @@ func (p *mockLogicalPlan4Test) getPhysicalPlan2(prop *property.PhysicalProperty) return physicalPlan2 } -func (p *mockLogicalPlan4Test) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool) { +func (p *mockLogicalPlan4Test) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { plan1 := make([]PhysicalPlan, 0, 1) plan2 := make([]PhysicalPlan, 0, 1) if prop.IsEmpty() && p.canGeneratePlan2 { // Generate PhysicalPlan2 when the property is empty. plan2 = append(plan2, p.getPhysicalPlan2(prop)) if p.hasHintForPlan2 { - return plan2, true + return plan2, true, nil } } if all, _ := prop.AllSameOrder(); all { @@ -115,9 +115,9 @@ func (p *mockLogicalPlan4Test) exhaustPhysicalPlans(prop *property.PhysicalPrope if prop.IsEmpty() { p.ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("the hint is inapplicable for plan2")) } - return plan1, false + return plan1, false, nil } - return append(plan1, plan2...), true + return append(plan1, plan2...), true, nil } type mockPhysicalPlan4Test struct { diff --git a/planner/core/initialize.go b/planner/core/initialize.go index c63d4efa7ba31..f41340147abfe 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -528,3 +528,29 @@ func flattenPushDownPlan(p PhysicalPlan) []PhysicalPlan { } return plans } + +// Init only assigns type and context. +func (p LogicalCTE) Init(ctx sessionctx.Context, offset int) *LogicalCTE { + p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeCTE, &p, offset) + return &p +} + +// Init only assigns type and context. +func (p PhysicalCTE) Init(ctx sessionctx.Context, stats *property.StatsInfo) *PhysicalCTE { + p.basePlan = newBasePlan(ctx, plancodec.TypeCTE, 0) + p.stats = stats + return &p +} + +// Init only assigns type and context. +func (p LogicalCTETable) Init(ctx sessionctx.Context, offset int) *LogicalCTETable { + p.baseLogicalPlan = newBaseLogicalPlan(ctx, plancodec.TypeCTETable, &p, offset) + return &p +} + +// Init only assigns type and context. +func (p PhysicalCTETable) Init(ctx sessionctx.Context, stats *property.StatsInfo) *PhysicalCTETable { + p.basePlan = newBasePlan(ctx, plancodec.TypeCTETable, 0) + p.stats = stats + return &p +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index ec4a3c401fb53..ea529dae66423 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/opcode" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" @@ -43,6 +44,7 @@ import ( "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -305,6 +307,12 @@ func (b *PlanBuilder) buildTableRefs(ctx context.Context, from *ast.TableRefsCla p = b.buildTableDual() return } + defer func() { + // After build the resultSetNode, need to reset it so that it can be referenced by outer level. + for _, cte := range b.outerCTEs { + cte.recursiveRef = false + } + }() return b.buildResultSetNode(ctx, from.TableRefs) } @@ -316,8 +324,12 @@ func (b *PlanBuilder) buildResultSetNode(ctx context.Context, node ast.ResultSet var isTableName bool switch v := x.Source.(type) { case *ast.SelectStmt: + ci := b.prepareCTECheckForSubQuery() + defer resetCTECheckForSubQuery(ci) p, err = b.buildSelect(ctx, v) case *ast.SetOprStmt: + ci := b.prepareCTECheckForSubQuery() + defer resetCTECheckForSubQuery(ci) p, err = b.buildSetOpr(ctx, v) case *ast.TableName: p, err = b.buildDataSource(ctx, v, &x.AsName) @@ -669,6 +681,11 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (Logica return nil, err } + // The recursive part in CTE must not be on the right side of a LEFT JOIN. + if lc, ok := rightPlan.(*LogicalCTETable); ok && joinNode.Tp == ast.LeftJoin { + return nil, ErrCTERecursiveForbiddenJoinOrder.GenWithStackByArgs(lc.name) + } + handleMap1 := b.handleHelper.popMap() handleMap2 := b.handleHelper.popMap() b.handleHelper.mergeAndPush(handleMap1, handleMap2) @@ -1116,6 +1133,7 @@ func (b *PlanBuilder) buildProjectionField(ctx context.Context, p LogicalPlan, f UniqueID: b.ctx.GetSessionVars().AllocPlanColumnID(), RetType: expr.GetType(), } + newCol.SetCoercibility(expr.Coercibility()) return newCol, name, nil } @@ -1366,6 +1384,17 @@ func (b *PlanBuilder) buildProjection4Union(ctx context.Context, u *LogicalUnion } func (b *PlanBuilder) buildSetOpr(ctx context.Context, setOpr *ast.SetOprStmt) (LogicalPlan, error) { + if setOpr.With != nil { + l := len(b.outerCTEs) + defer func() { + b.outerCTEs = b.outerCTEs[:l] + }() + err := b.buildWith(ctx, setOpr.With) + if err != nil { + return nil, err + } + } + // Because INTERSECT has higher precedence than UNION and EXCEPT. We build it first. selectPlans := make([]LogicalPlan, 0, len(setOpr.SelectList.Selects)) afterSetOprs := make([]*ast.SetOprType, 0, len(setOpr.SelectList.Selects)) @@ -2252,6 +2281,16 @@ func (r *correlatedAggregateResolver) Enter(n ast.Node) (ast.Node, bool) { // ORDER BY, WHERE & GROUP BY. // Finally it restore the original SELECT stmt. func (r *correlatedAggregateResolver) resolveSelect(sel *ast.SelectStmt) (err error) { + if sel.With != nil { + l := len(r.b.outerCTEs) + defer func() { + r.b.outerCTEs = r.b.outerCTEs[:l] + }() + err := r.b.buildWith(r.ctx, sel.With) + if err != nil { + return err + } + } // collect correlated aggregate from sub-queries inside FROM clause. _, err = r.collectFromTableRefs(r.ctx, sel.From) if err != nil { @@ -3297,6 +3336,14 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L // table hints are only visible in the current SELECT statement. b.popTableHints() }() + if b.buildingRecursivePartForCTE { + if sel.Distinct || sel.OrderBy != nil || sel.Limit != nil { + return nil, ErrNotSupportedYet.GenWithStackByArgs("ORDER BY / LIMIT / SELECT DISTINCT in recursive query block of Common Table Expression") + } + if sel.GroupBy != nil { + return nil, ErrCTERecursiveForbidsAggregation.FastGenByArgs(b.genCTETableNameForError()) + } + } enableNoopFuncs := b.ctx.GetSessionVars().EnableNoopFuncs if sel.SelectStmtOpts != nil { if sel.SelectStmtOpts.CalcFoundRows && !enableNoopFuncs { @@ -3322,6 +3369,17 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L b.isForUpdateRead = true } + if sel.With != nil { + l := len(b.outerCTEs) + defer func() { + b.outerCTEs = b.outerCTEs[:l] + }() + err = b.buildWith(ctx, sel.With) + if err != nil { + return nil, err + } + } + // For sub-queries, the FROM clause may have already been built in outer query when resolving correlated aggregates. // If the ResultSetNode inside FROM clause has nothing to do with correlated aggregates, we can simply get the // existing ResultSetNode from the cache. @@ -3358,6 +3416,10 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L // For example: select id from t group by id WINDOW w AS (ORDER BY uids DESC) ORDER BY id; // We don't use the WINDOW w, but if the 'uids' column is not in the table t, we still need to report an error. if hasWindowFuncField || sel.WindowSpecs != nil { + if b.buildingRecursivePartForCTE { + return nil, ErrCTERecursiveForbidsAggregation.FastGenByArgs(b.genCTETableNameForError()) + } + windowAggMap, err = b.resolveWindowFunction(sel, p) if err != nil { return nil, err @@ -3406,16 +3468,21 @@ func (b *PlanBuilder) buildSelect(ctx context.Context, sel *ast.SelectStmt) (p L b.handleHelper.pushMap(nil) hasAgg := b.detectSelectAgg(sel) + needBuildAgg := hasAgg if hasAgg { + if b.buildingRecursivePartForCTE { + return nil, ErrCTERecursiveForbidsAggregation.GenWithStackByArgs(b.genCTETableNameForError()) + } + aggFuncs, totalMap = b.extractAggFuncsInSelectFields(sel.Fields.Fields) // len(aggFuncs) == 0 and sel.GroupBy == nil indicates that all the aggregate functions inside the SELECT fields // are actually correlated aggregates from the outer query, which have already been built in the outer query. // The only thing we need to do is to find them from b.correlatedAggMap in buildProjection. if len(aggFuncs) == 0 && sel.GroupBy == nil { - hasAgg = false + needBuildAgg = false } } - if hasAgg { + if needBuildAgg { var aggIndexMap map[int]int p, aggIndexMap, err = b.buildAggregation(ctx, p, aggFuncs, gbyCols, correlatedAggMap) if err != nil { @@ -3572,10 +3639,65 @@ func getStatsTable(ctx sessionctx.Context, tblInfo *model.TableInfo, pid int64) return statsTbl } +func (b *PlanBuilder) tryBuildCTE(ctx context.Context, tn *ast.TableName, asName *model.CIStr) (LogicalPlan, error) { + for i := len(b.outerCTEs) - 1; i >= 0; i-- { + cte := b.outerCTEs[i] + if cte.def.Name.L == tn.Name.L { + if cte.isBuilding { + if cte.nonRecursive { + // Can't see this CTE, try outer definition. + continue + } + + // Building the recursive part. + cte.useRecursive = true + if cte.seedLP == nil { + return nil, ErrCTERecursiveRequiresNonRecursiveFirst.FastGenByArgs(tn.Name.String()) + } + + if cte.enterSubquery || cte.recursiveRef { + return nil, ErrInvalidRequiresSingleReference.FastGenByArgs(tn.Name.String()) + } + + cte.recursiveRef = true + p := LogicalCTETable{name: cte.def.Name.String(), idForStorage: cte.storageID}.Init(b.ctx, b.getSelectOffset()) + p.SetSchema(getResultCTESchema(cte.seedLP.Schema(), b.ctx.GetSessionVars())) + p.SetOutputNames(cte.seedLP.OutputNames()) + return p, nil + } + + b.handleHelper.pushMap(nil) + var p LogicalPlan + lp := LogicalCTE{cte: &CTEClass{IsDistinct: cte.isDistinct, seedPartLogicalPlan: cte.seedLP, recursivePartLogicalPlan: cte.recurLP, IDForStorage: cte.storageID, optFlag: cte.optFlag}}.Init(b.ctx, b.getSelectOffset()) + lp.SetSchema(getResultCTESchema(cte.seedLP.Schema(), b.ctx.GetSessionVars())) + p = lp + p.SetOutputNames(cte.seedLP.OutputNames()) + if len(asName.String()) > 0 { + var on types.NameSlice + for _, name := range p.OutputNames() { + cpOn := *name + cpOn.TblName = *asName + on = append(on, &cpOn) + } + p.SetOutputNames(on) + } + return p, nil + } + } + + return nil, nil +} + func (b *PlanBuilder) buildDataSource(ctx context.Context, tn *ast.TableName, asName *model.CIStr) (LogicalPlan, error) { dbName := tn.Schema sessionVars := b.ctx.GetSessionVars() + if dbName.L == "" { + // Try CTE. + p, err := b.tryBuildCTE(ctx, tn, asName) + if err != nil || p != nil { + return p, err + } dbName = model.NewCIStr(sessionVars.CurrentDB) } @@ -5612,3 +5734,286 @@ func containDifferentJoinTypes(preferJoinType uint) bool { } return cnt > 1 } + +func (b *PlanBuilder) buildCte(ctx context.Context, cte *ast.CommonTableExpression, isRecursive bool) (p LogicalPlan, err error) { + if isRecursive { + // buildingRecursivePartForCTE likes a stack. We save it before building a recursive CTE and restore it after building. + // We need a stack because we need to handle the nested recursive CTE. And buildingRecursivePartForCTE indicates the innermost CTE. + saveCheck := b.buildingRecursivePartForCTE + b.buildingRecursivePartForCTE = false + err = b.buildRecursiveCTE(ctx, cte.Query.Query) + if err != nil { + return nil, err + } + b.buildingRecursivePartForCTE = saveCheck + } else { + p, err = b.buildResultSetNode(ctx, cte.Query.Query) + if err != nil { + return nil, err + } + + p, err = b.adjustCTEPlanOutputName(p, cte) + if err != nil { + return nil, err + } + + cInfo := b.outerCTEs[len(b.outerCTEs)-1] + cInfo.seedLP = p + } + return nil, nil +} + +// buildRecursiveCTE handles the with clause `with recursive xxx as xx`. +func (b *PlanBuilder) buildRecursiveCTE(ctx context.Context, cte ast.ResultSetNode) error { + cInfo := b.outerCTEs[len(b.outerCTEs)-1] + switch x := (cte).(type) { + case *ast.SetOprStmt: + // 1. Handle the WITH clause if exists. + if x.With != nil { + l := len(b.outerCTEs) + defer func() { + b.outerCTEs = b.outerCTEs[:l] + sw := x.With + x.With = sw + }() + err := b.buildWith(ctx, x.With) + if err != nil { + return err + } + } + // Set it to nil, so that when builds the seed part, it won't build again. Reset it in defer so that the AST doesn't change after this function. + x.With = nil + + // 2. Build plans for each part of SetOprStmt. + recursive := make([]LogicalPlan, 0) + tmpAfterSetOptsForRecur := []*ast.SetOprType{nil} + + expectSeed := true + for i := 0; i < len(x.SelectList.Selects); i++ { + var p LogicalPlan + var err error + + var afterOpr *ast.SetOprType + switch y := x.SelectList.Selects[i].(type) { + case *ast.SelectStmt: + p, err = b.buildSelect(ctx, y) + afterOpr = y.AfterSetOperator + case *ast.SetOprSelectList: + p, err = b.buildSetOpr(ctx, &ast.SetOprStmt{SelectList: y, With: y.With}) + afterOpr = y.AfterSetOperator + } + + if expectSeed { + if cInfo.useRecursive { + // 3. If it fail to build a plan, it may be the recursive part. Then we build the seed part plan, and rebuild it. + if i == 0 { + return ErrCTERecursiveRequiresNonRecursiveFirst.GenWithStackByArgs(cInfo.def.Name.String()) + } + + // It's the recursive part. Build the seed part, and build this recursive part again. + // Before we build the seed part, do some checks. + if x.OrderBy != nil { + return ErrNotSupportedYet.GenWithStackByArgs("ORDER BY over UNION in recursive Common Table Expression") + } + + // Check union type. + if afterOpr != nil { + if *afterOpr != ast.Union && *afterOpr != ast.UnionAll { + return ErrNotSupportedYet.GenWithStackByArgs(fmt.Sprintf("%s between seed part and recursive part, hint: The operator between seed part and recursive part must bu UNION[DISTINCT] or UNION ALL", afterOpr.String())) + } + cInfo.isDistinct = *afterOpr == ast.Union + } + + expectSeed = false + cInfo.useRecursive = false + + // Build seed part plan. + saveSelect := x.SelectList.Selects + x.SelectList.Selects = x.SelectList.Selects[:i] + p, err = b.buildSetOpr(ctx, x) + if err != nil { + return err + } + x.SelectList.Selects = saveSelect + p, err = b.adjustCTEPlanOutputName(p, cInfo.def) + if err != nil { + return err + } + cInfo.seedLP = p + + // Rebuild the plan. + i-- + b.buildingRecursivePartForCTE = true + continue + } + if err != nil { + return err + } + } else { + if err != nil { + return err + } + if afterOpr != nil { + if *afterOpr != ast.Union && *afterOpr != ast.UnionAll { + return ErrNotSupportedYet.GenWithStackByArgs(fmt.Sprintf("%s between recursive part's selects, hint: The operator between recursive part's selects must bu UNION[DISTINCT] or UNION ALL", afterOpr.String())) + } + } + if !cInfo.useRecursive { + return ErrCTERecursiveRequiresNonRecursiveFirst.GenWithStackByArgs(cInfo.def.Name.String()) + } + cInfo.useRecursive = false + recursive = append(recursive, p) + tmpAfterSetOptsForRecur = append(tmpAfterSetOptsForRecur, afterOpr) + } + } + + if len(recursive) == 0 { + // In this case, even if SQL specifies "WITH RECURSIVE", the CTE is non-recursive. + p, err := b.buildSetOpr(ctx, x) + if err != nil { + return err + } + p, err = b.adjustCTEPlanOutputName(p, cInfo.def) + if err != nil { + return err + } + cInfo.seedLP = p + return nil + } + + // Build the recursive part's logical plan. + recurPart, err := b.buildUnion(ctx, recursive, tmpAfterSetOptsForRecur) + if err != nil { + return err + } + recurPart, err = b.buildProjection4CTEUnion(ctx, cInfo.seedLP, recurPart) + if err != nil { + return err + } + // 4. Finally, we get the seed part plan and recursive part plan. + cInfo.recurLP = recurPart + return nil + default: + p, err := b.buildResultSetNode(ctx, x) + if err != nil { + // Refine the error message. + if errors.ErrorEqual(err, ErrCTERecursiveRequiresNonRecursiveFirst) { + err = ErrCTERecursiveRequiresUnion.GenWithStackByArgs(cInfo.def.Name.String()) + } + return err + } + p, err = b.adjustCTEPlanOutputName(p, cInfo.def) + if err != nil { + return err + } + cInfo.seedLP = p + return nil + } +} + +func (b *PlanBuilder) adjustCTEPlanOutputName(p LogicalPlan, def *ast.CommonTableExpression) (LogicalPlan, error) { + outPutNames := p.OutputNames() + for _, name := range outPutNames { + name.TblName = def.Name + name.DBName = model.NewCIStr(b.ctx.GetSessionVars().CurrentDB) + } + if len(def.ColNameList) > 0 { + if len(def.ColNameList) != len(p.OutputNames()) { + return nil, ddl.ErrViewWrongList + } + for i, n := range def.ColNameList { + outPutNames[i].ColName = n + } + } + p.SetOutputNames(outPutNames) + return p, nil +} + +// prepareCTECheckForSubQuery prepares the check that the recursive CTE can't be referenced in subQuery. It's used before building a subQuery. +// For example: with recursive cte(n) as (select 1 union select * from (select * from cte) c1) select * from cte; +func (b *PlanBuilder) prepareCTECheckForSubQuery() []*cteInfo { + modifiedCTE := make([]*cteInfo, 0) + for _, cte := range b.outerCTEs { + if cte.isBuilding && !cte.enterSubquery { + cte.enterSubquery = true + modifiedCTE = append(modifiedCTE, cte) + } + } + return modifiedCTE +} + +// resetCTECheckForSubQuery resets the related variable. It's used after leaving a subQuery. +func resetCTECheckForSubQuery(ci []*cteInfo) { + for _, cte := range ci { + cte.enterSubquery = false + } +} + +// genCTETableNameForError find the nearest CTE name. +func (b *PlanBuilder) genCTETableNameForError() string { + name := "" + for i := len(b.outerCTEs) - 1; i >= 0; i-- { + if b.outerCTEs[i].isBuilding { + name = b.outerCTEs[i].def.Name.String() + break + } + } + return name +} + +func (b *PlanBuilder) buildWith(ctx context.Context, w *ast.WithClause) error { + // Check CTE name must be unique. + nameMap := make(map[string]struct{}) + for _, cte := range w.CTEs { + if _, ok := nameMap[cte.Name.L]; ok { + return ErrNonUniqTable + } + nameMap[cte.Name.L] = struct{}{} + } + for _, cte := range w.CTEs { + b.outerCTEs = append(b.outerCTEs, &cteInfo{def: cte, nonRecursive: !w.IsRecursive, isBuilding: true, storageID: b.allocIDForCTEStorage}) + b.allocIDForCTEStorage++ + saveFlag := b.optFlag + // Init the flag to flagPrunColumns, otherwise it's missing. + b.optFlag = flagPrunColumns + _, err := b.buildCte(ctx, cte, w.IsRecursive) + if err != nil { + return err + } + b.outerCTEs[len(b.outerCTEs)-1].optFlag = b.optFlag + b.outerCTEs[len(b.outerCTEs)-1].isBuilding = false + b.optFlag = saveFlag + } + return nil +} + +func (b *PlanBuilder) buildProjection4CTEUnion(ctx context.Context, seed LogicalPlan, recur LogicalPlan) (LogicalPlan, error) { + if seed.Schema().Len() != recur.Schema().Len() { + return nil, ErrWrongNumberOfColumnsInSelect.GenWithStackByArgs() + } + exprs := make([]expression.Expression, len(seed.Schema().Columns)) + resSchema := getResultCTESchema(seed.Schema(), b.ctx.GetSessionVars()) + for i, col := range recur.Schema().Columns { + if !resSchema.Columns[i].RetType.Equal(col.RetType) { + exprs[i] = expression.BuildCastFunction4Union(b.ctx, col, resSchema.Columns[i].RetType) + } else { + exprs[i] = col + } + } + b.optFlag |= flagEliminateProjection + proj := LogicalProjection{Exprs: exprs, AvoidColumnEvaluator: true}.Init(b.ctx, b.getSelectOffset()) + proj.SetSchema(resSchema) + proj.SetChildren(recur) + return proj, nil +} + +// The recursive part/CTE's schema is nullable, and the UID should be unique. +func getResultCTESchema(seedSchema *expression.Schema, svar *variable.SessionVars) *expression.Schema { + res := seedSchema.Clone() + for _, col := range res.Columns { + col.RetType = col.RetType.Clone() + col.UniqueID = svar.AllocPlanColumnID() + col.RetType.Flag &= ^mysql.NotNullFlag + } + return res +} diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 09f7739e8017f..eac52f90da9c1 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -1169,3 +1169,43 @@ type LogicalShowDDLJobs struct { JobNumber int64 } + +// CTEClass holds the information and plan for a CTE. Most of the fields in this struct are the same as cteInfo. +// But the cteInfo is used when building the plan, and CTEClass is used also for building the executor. +type CTEClass struct { + // The union between seed part and recursive part is DISTINCT or DISTINCT ALL. + IsDistinct bool + // seedPartLogicalPlan and recursivePartLogicalPlan are the logical plans for the seed part and recursive part of this CTE. + seedPartLogicalPlan LogicalPlan + recursivePartLogicalPlan LogicalPlan + // cteTask is the physical plan for this CTE, is a wrapper of the PhysicalCTE. + cteTask task + // storageID for this CTE. + IDForStorage int + // optFlag is the optFlag for the whole CTE. + optFlag uint64 +} + +// LogicalCTE is for CTE. +type LogicalCTE struct { + logicalSchemaProducer + + cte *CTEClass +} + +// LogicalCTETable is for CTE table +type LogicalCTETable struct { + logicalSchemaProducer + + name string + idForStorage int +} + +// ExtractCorrelatedCols implements LogicalPlan interface. +func (p *LogicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn { + corCols := ExtractCorrelatedCols4LogicalPlan(p.cte.seedPartLogicalPlan) + if p.cte.recursivePartLogicalPlan != nil { + corCols = append(corCols, ExtractCorrelatedCols4LogicalPlan(p.cte.recursivePartLogicalPlan)...) + } + return corCols +} diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index f4e6769ee8d70..353fb2897b1f9 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1359,3 +1359,28 @@ func NewTableSampleInfo(node *ast.TableSample, fullSchema *expression.Schema, pt Partitions: pt, } } + +// PhysicalCTE is for CTE. +type PhysicalCTE struct { + physicalSchemaProducer + + SeedPlan PhysicalPlan + RecurPlan PhysicalPlan + CTE *CTEClass +} + +// PhysicalCTETable is for CTE table. +type PhysicalCTETable struct { + physicalSchemaProducer + + IDForStorage int +} + +// ExtractCorrelatedCols implements PhysicalPlan interface. +func (p *PhysicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn { + corCols := ExtractCorrelatedCols4PhysicalPlan(p.SeedPlan) + if p.RecurPlan != nil { + corCols = append(corCols, ExtractCorrelatedCols4PhysicalPlan(p.RecurPlan)...) + } + return corCols +} diff --git a/planner/core/plan.go b/planner/core/plan.go index ea3dc74cca180..dd7e41b77f7fa 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -282,7 +282,7 @@ type LogicalPlan interface { // It will return: // 1. All possible plans that can match the required property. // 2. Whether the SQL hint can work. Return true if there is no hint. - exhaustPhysicalPlans(*property.PhysicalProperty) (physicalPlans []PhysicalPlan, hintCanWork bool) + exhaustPhysicalPlans(*property.PhysicalProperty) (physicalPlans []PhysicalPlan, hintCanWork bool, err error) // ExtractCorrelatedCols extracts correlated columns inside the LogicalPlan. ExtractCorrelatedCols() []*expression.CorrelatedColumn diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 178779cabdf25..6640af5785b81 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -411,6 +411,29 @@ const ( renameView ) +type cteInfo struct { + def *ast.CommonTableExpression + // nonRecursive is used to decide if a CTE is visible. If a CTE start with `WITH RECURSIVE`, then nonRecursive is false, + // so it is visible in its definition. + nonRecursive bool + // useRecursive is used to record if a subSelect in CTE's definition refer to itself. This help us to identify the seed part and recursive part. + useRecursive bool + isBuilding bool + // isDistinct indicates if the union between seed part and recursive part is distinct or not. + isDistinct bool + // seedLP is the seed part's logical plan. + seedLP LogicalPlan + // recurLP is the recursive part's logical plan. + recurLP LogicalPlan + // storageID for this CTE. + storageID int + // optFlag is the optFlag for the whole CTE. + optFlag uint64 + // enterSubquery and recursiveRef are used to check "recursive table must be referenced only once, and not in any subquery". + enterSubquery bool + recursiveRef bool +} + // PlanBuilder builds Plan from an ast.Node. // It just builds the ast node straightforwardly. type PlanBuilder struct { @@ -418,6 +441,7 @@ type PlanBuilder struct { is infoschema.InfoSchema outerSchemas []*expression.Schema outerNames [][]*types.FieldName + outerCTEs []*cteInfo // colMapper stores the column that must be pre-resolved. colMapper map[*ast.ColumnNameExpr]int // visitInfo is used for privilege check. @@ -476,7 +500,9 @@ type PlanBuilder struct { // isForUpdateRead should be true in either of the following situations // 1. use `inside insert`, `update`, `delete` or `select for update` statement // 2. isolation level is RC - isForUpdateRead bool + isForUpdateRead bool + allocIDForCTEStorage int + buildingRecursivePartForCTE bool } type handleColHelper struct { @@ -584,6 +610,7 @@ func NewPlanBuilder(sctx sessionctx.Context, is infoschema.InfoSchema, processor return &PlanBuilder{ ctx: sctx, is: is, + outerCTEs: make([]*cteInfo, 0), colMapper: make(map[*ast.ColumnNameExpr]int), handleHelper: &handleColHelper{id2HandleMapStack: make([]map[int64][]HandleCols, 0)}, hintProcessor: processor, diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index bfc596d20f975..c23d4c2df170e 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -92,7 +92,7 @@ func TryAddExtraLimit(ctx sessionctx.Context, node ast.StmtNode) ast.StmtNode { // Preprocess resolves table names of the node, and checks some statements validation. // prepreocssReturn used to extract the infoschema for the tableName and the timestamp from the asof clause. func Preprocess(ctx sessionctx.Context, node ast.Node, preprocessOpt ...PreprocessOpt) error { - v := preprocessor{ctx: ctx, tableAliasInJoin: make([]map[string]interface{}, 0)} + v := preprocessor{ctx: ctx, tableAliasInJoin: make([]map[string]interface{}, 0), withName: make(map[string]interface{})} for _, optFn := range preprocessOpt { optFn(&v) } @@ -142,6 +142,7 @@ type preprocessor struct { // tableAliasInJoin is a stack that keeps the table alias names for joins. // len(tableAliasInJoin) may bigger than 1 because the left/right child of join may be subquery that contains `JOIN` tableAliasInJoin []map[string]interface{} + withName map[string]interface{} // values that may be returned *PreprocessorReturn @@ -242,6 +243,7 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { if node.FnName.L == ast.NextVal || node.FnName.L == ast.LastVal || node.FnName.L == ast.SetVal { p.flag |= inSequenceFunction } + case *ast.BRIEStmt: if node.Kind == ast.BRIEKindRestore { p.flag |= inCreateOrDropTable @@ -260,6 +262,10 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { } case *ast.GroupByClause: p.checkGroupBy(node) + case *ast.WithClause: + for _, cte := range node.CTEs { + p.withName[cte.Name.L] = struct{}{} + } default: p.flag &= ^parentIsJoin } @@ -1180,6 +1186,10 @@ func (p *preprocessor) stmtType() string { func (p *preprocessor) handleTableName(tn *ast.TableName) { if tn.Schema.L == "" { + if _, ok := p.withName[tn.Name.L]; ok { + return + } + currentDB := p.ctx.GetSessionVars().CurrentDB if currentDB == "" { p.err = errors.Trace(ErrNoDB) diff --git a/util/plancodec/id.go b/util/plancodec/id.go index 0cd4a1b1476be..626b8fc3b1dce 100644 --- a/util/plancodec/id.go +++ b/util/plancodec/id.go @@ -120,6 +120,10 @@ const ( TypeIndexFullScan = "IndexFullScan" // TypeIndexRangeScan is the type of IndexRangeScan. TypeIndexRangeScan = "IndexRangeScan" + // TypeCTETable is the type of TypeCTETable. + TypeCTETable = "CTETable" + // TypeCTE is the type of TypeCTE. + TypeCTE = "CTE" ) // plan id. From cea4e206890366cde1b4ab91d47aa01580579427 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 28 May 2021 20:33:35 +0800 Subject: [PATCH 206/343] executor: fix the unstable test TestDML (#24960) --- executor/partition_table_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index fab6f9755ddea..0d70bbd3c8eb4 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -1571,9 +1571,9 @@ func (s *partitionTableSuite) TestDML(c *C) { tk.MustExec("use test_DML") tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") - tk.MustExec(`create table tinner (a int primary key, b int)`) - tk.MustExec(`create table thash (a int primary key, b int) partition by hash(a) partitions 4`) - tk.MustExec(`create table trange (a int primary key, b int) partition by range(a) ( + tk.MustExec(`create table tinner (a int, b int)`) + tk.MustExec(`create table thash (a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int) partition by range(a) ( partition p0 values less than(10000), partition p1 values less than(20000), partition p2 values less than(30000), From 7a15d647037cdaf0c18d0a4e4ffc31bdbc00cac1 Mon Sep 17 00:00:00 2001 From: Shirly Date: Fri, 28 May 2021 20:53:36 +0800 Subject: [PATCH 207/343] =?UTF-8?q?store/copr:=20move=20tikv.keyranges=20a?= =?UTF-8?q?nd=20tikv.regioninfo=20to=20package=20store/=E2=80=A6=20(#24908?= =?UTF-8?q?)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- ddl/backfilling.go | 13 +- store/copr/batch_coprocessor.go | 37 ++-- store/{tikv => copr}/batch_request_sender.go | 26 +-- store/copr/coprocessor.go | 14 +- store/copr/coprocessor_test.go | 37 ++-- store/{tikv => copr}/key_ranges.go | 22 +-- store/{tikv => copr}/key_ranges_test.go | 19 +-- store/copr/mpp.go | 6 +- store/copr/region_cache.go | 138 +++++++++++++++ store/copr/store.go | 4 +- store/tikv/region_cache.go | 171 +++++-------------- store/tikv/region_cache_test.go | 4 +- store/tikv/region_request_test.go | 2 +- 13 files changed, 250 insertions(+), 243 deletions(-) rename store/{tikv => copr}/batch_request_sender.go (77%) rename store/{tikv => copr}/key_ranges.go (84%) rename store/{tikv => copr}/key_ranges_test.go (91%) create mode 100644 store/copr/region_cache.go diff --git a/ddl/backfilling.go b/ddl/backfilling.go index ec3b8a61f4990..8a5b7a837f552 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -20,7 +20,6 @@ import ( "strconv" "sync/atomic" "time" - "unsafe" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -32,8 +31,9 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/store/copr" + "github.com/pingcap/tidb/store/driver/backoff" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util" @@ -331,9 +331,9 @@ func splitTableRanges(t table.PhysicalTable, store kv.Storage, startKey, endKey } maxSleep := 10000 // ms - bo := tikv.NewBackofferWithVars(context.Background(), maxSleep, nil) - tikvRange := *(*tikvstore.KeyRange)(unsafe.Pointer(&kvRange)) - ranges, err := s.GetRegionCache().SplitRegionRanges(bo, []tikvstore.KeyRange{tikvRange}) + bo := backoff.NewBackofferWithVars(context.Background(), maxSleep, nil) + rc := copr.NewRegionCache(s.GetRegionCache()) + ranges, err := rc.SplitRegionRanges(bo, []kv.KeyRange{kvRange}) if err != nil { return nil, errors.Trace(err) } @@ -341,8 +341,7 @@ func splitTableRanges(t table.PhysicalTable, store kv.Storage, startKey, endKey errMsg := fmt.Sprintf("cannot find region in range [%s, %s]", startKey.String(), endKey.String()) return nil, errors.Trace(errInvalidSplitRegionRanges.GenWithStackByArgs(errMsg)) } - res := *(*[]kv.KeyRange)(unsafe.Pointer(&ranges)) - return res, nil + return ranges, nil } func (w *worker) waitTaskResults(workers []*backfillWorker, taskCnt int, diff --git a/store/copr/batch_coprocessor.go b/store/copr/batch_coprocessor.go index 18106649fb94d..ade644f411897 100644 --- a/store/copr/batch_coprocessor.go +++ b/store/copr/batch_coprocessor.go @@ -21,7 +21,6 @@ import ( "sync" "sync/atomic" "time" - "unsafe" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -33,7 +32,6 @@ import ( "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -46,7 +44,7 @@ type batchCopTask struct { cmdType tikvrpc.CmdType ctx *tikv.RPCContext - regionInfos []tikv.RegionInfo + regionInfos []RegionInfo } type batchCopResponse struct { @@ -109,7 +107,7 @@ func balanceBatchCopTask(originalTasks []*batchCopTask) []*batchCopTask { return originalTasks } storeTaskMap := make(map[uint64]*batchCopTask) - storeCandidateRegionMap := make(map[uint64]map[string]tikv.RegionInfo) + storeCandidateRegionMap := make(map[uint64]map[string]RegionInfo) totalRegionCandidateNum := 0 totalRemainingRegionNum := 0 @@ -119,7 +117,7 @@ func balanceBatchCopTask(originalTasks []*batchCopTask) []*batchCopTask { storeAddr: task.storeAddr, cmdType: task.cmdType, ctx: task.ctx, - regionInfos: []tikv.RegionInfo{task.regionInfos[0]}, + regionInfos: []RegionInfo{task.regionInfos[0]}, } storeTaskMap[taskStoreID] = batchTask } @@ -155,7 +153,7 @@ func balanceBatchCopTask(originalTasks []*batchCopTask) []*batchCopTask { continue } if _, ok := storeCandidateRegionMap[storeID]; !ok { - candidateMap := make(map[string]tikv.RegionInfo) + candidateMap := make(map[string]RegionInfo) storeCandidateRegionMap[storeID] = candidateMap } if _, duplicateRegion := storeCandidateRegionMap[storeID][taskKey]; duplicateRegion { @@ -210,7 +208,7 @@ func balanceBatchCopTask(originalTasks []*batchCopTask) []*batchCopTask { break } var key string - var ri tikv.RegionInfo + var ri RegionInfo for key, ri = range storeCandidateRegionMap[store] { // get the first region break @@ -246,13 +244,13 @@ func balanceBatchCopTask(originalTasks []*batchCopTask) []*batchCopTask { return ret } -func buildBatchCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { +func buildBatchCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, storeType kv.StoreType) ([]*batchCopTask, error) { start := time.Now() const cmdType = tikvrpc.CmdBatchCop rangesLen := ranges.Len() for { - locations, err := cache.SplitKeyRangesByLocations(bo.TiKVBackoffer(), ranges) + locations, err := cache.SplitKeyRangesByLocations(bo, ranges) if err != nil { return nil, errors.Trace(err) } @@ -288,13 +286,13 @@ func buildBatchCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.Key } allStores := cache.GetAllValidTiFlashStores(task.region, rpcCtx.Store) if batchCop, ok := storeTaskMap[rpcCtx.Addr]; ok { - batchCop.regionInfos = append(batchCop.regionInfos, tikv.RegionInfo{Region: task.region, Meta: rpcCtx.Meta, Ranges: task.ranges, AllStores: allStores}) + batchCop.regionInfos = append(batchCop.regionInfos, RegionInfo{Region: task.region, Meta: rpcCtx.Meta, Ranges: task.ranges, AllStores: allStores}) } else { batchTask := &batchCopTask{ storeAddr: rpcCtx.Addr, cmdType: cmdType, ctx: rpcCtx, - regionInfos: []tikv.RegionInfo{{Region: task.region, Meta: rpcCtx.Meta, Ranges: task.ranges, AllStores: allStores}}, + regionInfos: []RegionInfo{{Region: task.region, Meta: rpcCtx.Meta, Ranges: task.ranges, AllStores: allStores}}, } storeTaskMap[rpcCtx.Addr] = batchTask } @@ -346,7 +344,7 @@ func (c *CopClient) sendBatch(ctx context.Context, req *kv.Request, vars *tikv.V } ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs) bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) - ranges := toTiKVKeyRanges(req.KeyRanges) + ranges := NewKeyRanges(req.KeyRanges) tasks, err := buildBatchCopTasks(bo, c.store.GetRegionCache(), ranges, req.StoreType) if err != nil { return copErrorResponse{err} @@ -482,19 +480,19 @@ func (b *batchCopIterator) handleTask(ctx context.Context, bo *Backoffer, task * // Merge all ranges and request again. func (b *batchCopIterator) retryBatchCopTask(ctx context.Context, bo *Backoffer, batchTask *batchCopTask) ([]*batchCopTask, error) { - var ranges []tikvstore.KeyRange + var ranges []kv.KeyRange for _, ri := range batchTask.regionInfos { - ri.Ranges.Do(func(ran *tikvstore.KeyRange) { + ri.Ranges.Do(func(ran *kv.KeyRange) { ranges = append(ranges, *ran) }) } - return buildBatchCopTasks(bo, b.store.GetRegionCache(), tikv.NewKeyRanges(ranges), b.req.StoreType) + return buildBatchCopTasks(bo, b.store.GetRegionCache(), NewKeyRanges(ranges), b.req.StoreType) } const readTimeoutUltraLong = 3600 * time.Second // For requests that may scan many regions for tiflash. func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *Backoffer, task *batchCopTask) ([]*batchCopTask, error) { - sender := tikv.NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) + sender := NewRegionBatchRequestSender(b.store.GetRegionCache(), b.store.GetTiKVClient()) var regionInfos = make([]*coprocessor.RegionInfo, 0, len(task.regionInfos)) for _, ri := range task.regionInfos { regionInfos = append(regionInfos, &coprocessor.RegionInfo{ @@ -527,7 +525,7 @@ func (b *batchCopIterator) handleTaskOnce(ctx context.Context, bo *Backoffer, ta req.StoreTp = tikvrpc.TiFlash logutil.BgLogger().Debug("send batch request to ", zap.String("req info", req.String()), zap.Int("cop task len", len(task.regionInfos))) - resp, retry, cancel, err := sender.SendReqToAddr(bo.TiKVBackoffer(), task.ctx, task.regionInfos, req, readTimeoutUltraLong) + resp, retry, cancel, err := sender.SendReqToAddr(bo, task.ctx, task.regionInfos, req, readTimeoutUltraLong) // If there are store errors, we should retry for all regions. if retry { return b.retryBatchCopTask(ctx, bo, task) @@ -625,8 +623,3 @@ func (b *batchCopIterator) sendToRespCh(resp *batchCopResponse) (exit bool) { } return } - -func toTiKVKeyRanges(ranges []kv.KeyRange) *tikv.KeyRanges { - res := *(*[]tikvstore.KeyRange)(unsafe.Pointer(&ranges)) - return tikv.NewKeyRanges(res) -} diff --git a/store/tikv/batch_request_sender.go b/store/copr/batch_request_sender.go similarity index 77% rename from store/tikv/batch_request_sender.go rename to store/copr/batch_request_sender.go index 74a62dcfd781c..dea0c98148aa0 100644 --- a/store/tikv/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tikv +package copr import ( "context" @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -29,7 +30,7 @@ import ( // RegionInfo contains region related information for batchCopTask type RegionInfo struct { - Region RegionVerID + Region tikv.RegionVerID Meta *metapb.Region Ranges *KeyRanges AllStores []uint64 @@ -37,30 +38,30 @@ type RegionInfo struct { // RegionBatchRequestSender sends BatchCop requests to TiFlash server by stream way. type RegionBatchRequestSender struct { - *RegionRequestSender + *tikv.RegionRequestSender } // NewRegionBatchRequestSender creates a RegionBatchRequestSender object. -func NewRegionBatchRequestSender(cache *RegionCache, client Client) *RegionBatchRequestSender { +func NewRegionBatchRequestSender(cache *RegionCache, client tikv.Client) *RegionBatchRequestSender { return &RegionBatchRequestSender{ - RegionRequestSender: NewRegionRequestSender(cache, client), + RegionRequestSender: tikv.NewRegionRequestSender(cache.RegionCache, client), } } // SendReqToAddr send batch cop request -func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *RPCContext, regionInfos []RegionInfo, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { +func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *tikv.RPCContext, regionInfos []RegionInfo, req *tikvrpc.Request, timout time.Duration) (resp *tikvrpc.Response, retry bool, cancel func(), err error) { cancel = func() {} if e := tikvrpc.SetContext(req, rpcCtx.Meta, rpcCtx.Peer); e != nil { return nil, false, cancel, errors.Trace(e) } ctx := bo.GetCtx() - if rawHook := ctx.Value(RPCCancellerCtxKey{}); rawHook != nil { - ctx, cancel = rawHook.(*RPCCanceller).WithCancel(ctx) + if rawHook := ctx.Value(tikv.RPCCancellerCtxKey{}); rawHook != nil { + ctx, cancel = rawHook.(*tikv.RPCCanceller).WithCancel(ctx) } start := time.Now() resp, err = ss.GetClient().SendRequest(ctx, rpcCtx.Addr, req, timout) if ss.Stats != nil { - RecordRegionRequestRuntimeStats(ss.Stats, req.Type, time.Since(start)) + tikv.RecordRegionRequestRuntimeStats(ss.Stats, req.Type, time.Since(start)) } if err != nil { cancel() @@ -75,11 +76,11 @@ func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *RPCCont return } -func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx *RPCContext, regionInfos []RegionInfo, err error) error { +func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx *tikv.RPCContext, regionInfos []RegionInfo, err error) error { // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { return errors.Trace(err) - } else if atomic.LoadUint32(&ShuttingDown) > 0 { + } else if atomic.LoadUint32(&tikv.ShuttingDown) > 0 { return tikverr.ErrTiDBShuttingDown } @@ -88,7 +89,8 @@ func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx // will change. If tiflash's replica is more than two, the "reload region" will always be false. // Now that the batch cop and mpp has a relative low qps, it's reasonable to reload every time // when meeting io error. - ss.GetRegionCache().OnSendFailForBatchRegions(bo, ctx.Store, regionInfos, true, err) + rc := RegionCache{ss.GetRegionCache()} + rc.OnSendFailForBatchRegions(bo, ctx.Store, regionInfos, true, err) // Retry on send request failure when it's not canceled. // When a store is not available, the leader of related region should be elected quickly. diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 5bdfd20537250..50d7d6c0a1546 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -76,7 +76,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa } ctx = context.WithValue(ctx, tikv.TxnStartKey(), req.StartTs) bo := backoff.NewBackofferWithVars(ctx, copBuildTaskMaxBackoff, vars) - ranges := toTiKVKeyRanges(req.KeyRanges) + ranges := NewKeyRanges(req.KeyRanges) tasks, err := buildCopTasks(bo, c.store.GetRegionCache(), ranges, req) if err != nil { return copErrorResponse{err} @@ -130,7 +130,7 @@ func (c *CopClient) Send(ctx context.Context, req *kv.Request, variables interfa // copTask contains a related Region and KeyRange for a kv.Request. type copTask struct { region tikv.RegionVerID - ranges *tikv.KeyRanges + ranges *KeyRanges respChan chan *copResponse storeAddr string @@ -146,7 +146,7 @@ func (r *copTask) String() string { // rangesPerTask limits the length of the ranges slice sent in one copTask. const rangesPerTask = 25000 -func buildCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRanges, req *kv.Request) ([]*copTask, error) { +func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv.Request) ([]*copTask, error) { start := time.Now() cmdType := tikvrpc.CmdCop if req.Streaming { @@ -159,7 +159,7 @@ func buildCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRange rangesLen := ranges.Len() - locs, err := cache.SplitKeyRangesByLocations(bo.TiKVBackoffer(), ranges) + locs, err := cache.SplitKeyRangesByLocations(bo, ranges) if err != nil { return nil, errors.Trace(err) } @@ -197,7 +197,7 @@ func buildCopTasks(bo *Backoffer, cache *tikv.RegionCache, ranges *tikv.KeyRange return tasks, nil } -func buildTiDBMemCopTasks(ranges *tikv.KeyRanges, req *kv.Request) ([]*copTask, error) { +func buildTiDBMemCopTasks(ranges *KeyRanges, req *kv.Request) ([]*copTask, error) { servers, err := infosync.GetAllServerInfo(context.Background()) if err != nil { return nil, err @@ -905,7 +905,7 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.R if resp.pbResp.Range != nil { resp.startKey = resp.pbResp.Range.Start } else if task.ranges != nil && task.ranges.Len() > 0 { - resp.startKey = kv.Key(task.ranges.At(0).StartKey) + resp.startKey = task.ranges.At(0).StartKey } if resp.detail == nil { resp.detail = new(CopRuntimeStats) @@ -1033,7 +1033,7 @@ func (worker *copIteratorWorker) buildCopTasksFromRemain(bo *Backoffer, lastRang // split: [s1 --> s2) // In normal scan order, all data before s1 is consumed, so the remain ranges should be [s1 --> r2) [r3 --> r4) // In reverse scan order, all data after s2 is consumed, so the remain ranges should be [r1 --> r2) [r3 --> s2) -func (worker *copIteratorWorker) calculateRemain(ranges *tikv.KeyRanges, split *coprocessor.KeyRange, desc bool) *tikv.KeyRanges { +func (worker *copIteratorWorker) calculateRemain(ranges *KeyRanges, split *coprocessor.KeyRange, desc bool) *KeyRanges { if desc { left, _ := ranges.Split(split.End) return left diff --git a/store/copr/coprocessor_test.go b/store/copr/coprocessor_test.go index 3bd34f05d95f9..592eb33b0e5f3 100644 --- a/store/copr/coprocessor_test.go +++ b/store/copr/coprocessor_test.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/driver/backoff" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" ) @@ -41,7 +40,7 @@ func (s *testCoprocessorSuite) TestBuildTasks(c *C) { cluster := mocktikv.NewCluster(mocktikv.MustNewMVCCStore()) _, regionIDs, _ := mocktikv.BootstrapWithMultiRegions(cluster, []byte("g"), []byte("n"), []byte("t")) pdCli := &tikv.CodecPDClient{Client: mocktikv.NewPDClient(cluster)} - cache := tikv.NewRegionCache(pdCli) + cache := NewRegionCache(tikv.NewRegionCache(pdCli)) defer cache.Close() bo := backoff.NewBackofferWithVars(context.Background(), 3000, nil) @@ -158,48 +157,48 @@ func (s *testCoprocessorSuite) TestSplitRegionRanges(c *C) { cluster := mocktikv.NewCluster(mocktikv.MustNewMVCCStore()) mocktikv.BootstrapWithMultiRegions(cluster, []byte("g"), []byte("n"), []byte("t")) pdCli := &tikv.CodecPDClient{Client: mocktikv.NewPDClient(cluster)} - cache := tikv.NewRegionCache(pdCli) + cache := NewRegionCache(tikv.NewRegionCache(pdCli)) defer cache.Close() - bo := tikv.NewBackofferWithVars(context.Background(), 3000, nil) + bo := backoff.NewBackofferWithVars(context.Background(), 3000, nil) - ranges, err := tikv.SplitRegionRanges(bo, cache, buildKeyRanges("a", "c")) + ranges, err := cache.SplitRegionRanges(bo, buildKeyRanges("a", "c")) c.Assert(err, IsNil) c.Assert(ranges, HasLen, 1) s.rangeEqual(c, ranges, "a", "c") - ranges, err = tikv.SplitRegionRanges(bo, cache, buildKeyRanges("h", "y")) + ranges, err = cache.SplitRegionRanges(bo, buildKeyRanges("h", "y")) c.Assert(err, IsNil) c.Assert(len(ranges), Equals, 3) s.rangeEqual(c, ranges, "h", "n", "n", "t", "t", "y") - ranges, err = tikv.SplitRegionRanges(bo, cache, buildKeyRanges("s", "z")) + ranges, err = cache.SplitRegionRanges(bo, buildKeyRanges("s", "z")) c.Assert(err, IsNil) c.Assert(len(ranges), Equals, 2) s.rangeEqual(c, ranges, "s", "t", "t", "z") - ranges, err = tikv.SplitRegionRanges(bo, cache, buildKeyRanges("s", "s")) + ranges, err = cache.SplitRegionRanges(bo, buildKeyRanges("s", "s")) c.Assert(err, IsNil) c.Assert(len(ranges), Equals, 1) s.rangeEqual(c, ranges, "s", "s") - ranges, err = tikv.SplitRegionRanges(bo, cache, buildKeyRanges("t", "t")) + ranges, err = cache.SplitRegionRanges(bo, buildKeyRanges("t", "t")) c.Assert(err, IsNil) c.Assert(len(ranges), Equals, 1) s.rangeEqual(c, ranges, "t", "t") - ranges, err = tikv.SplitRegionRanges(bo, cache, buildKeyRanges("t", "u")) + ranges, err = cache.SplitRegionRanges(bo, buildKeyRanges("t", "u")) c.Assert(err, IsNil) c.Assert(len(ranges), Equals, 1) s.rangeEqual(c, ranges, "t", "u") - ranges, err = tikv.SplitRegionRanges(bo, cache, buildKeyRanges("u", "z")) + ranges, err = cache.SplitRegionRanges(bo, buildKeyRanges("u", "z")) c.Assert(err, IsNil) c.Assert(len(ranges), Equals, 1) s.rangeEqual(c, ranges, "u", "z") // min --> max - ranges, err = tikv.SplitRegionRanges(bo, cache, buildKeyRanges("a", "z")) + ranges, err = cache.SplitRegionRanges(bo, buildKeyRanges("a", "z")) c.Assert(err, IsNil) c.Assert(ranges, HasLen, 4) s.rangeEqual(c, ranges, "a", "g", "g", "n", "n", "t", "t", "z") @@ -211,7 +210,7 @@ func (s *testCoprocessorSuite) TestRebuild(c *C) { cluster := mocktikv.NewCluster(mocktikv.MustNewMVCCStore()) storeID, regionIDs, peerIDs := mocktikv.BootstrapWithMultiRegions(cluster, []byte("m")) pdCli := &tikv.CodecPDClient{Client: mocktikv.NewPDClient(cluster)} - cache := tikv.NewRegionCache(pdCli) + cache := NewRegionCache(tikv.NewRegionCache(pdCli)) defer cache.Close() bo := backoff.NewBackofferWithVars(context.Background(), 3000, nil) @@ -238,10 +237,10 @@ func (s *testCoprocessorSuite) TestRebuild(c *C) { s.taskEqual(c, tasks[0], regionIDs[2], "q", "z") } -func buildKeyRanges(keys ...string) []tikvstore.KeyRange { - var ranges []tikvstore.KeyRange +func buildKeyRanges(keys ...string) []kv.KeyRange { + var ranges []kv.KeyRange for i := 0; i < len(keys); i += 2 { - ranges = append(ranges, tikvstore.KeyRange{ + ranges = append(ranges, kv.KeyRange{ StartKey: []byte(keys[i]), EndKey: []byte(keys[i+1]), }) @@ -249,8 +248,8 @@ func buildKeyRanges(keys ...string) []tikvstore.KeyRange { return ranges } -func buildCopRanges(keys ...string) *tikv.KeyRanges { - return tikv.NewKeyRanges(buildKeyRanges(keys...)) +func buildCopRanges(keys ...string) *KeyRanges { + return NewKeyRanges(buildKeyRanges(keys...)) } func (s *testCoprocessorSuite) taskEqual(c *C, task *copTask, regionID uint64, keys ...string) { @@ -262,7 +261,7 @@ func (s *testCoprocessorSuite) taskEqual(c *C, task *copTask, regionID uint64, k } } -func (s *testCoprocessorSuite) rangeEqual(c *C, ranges []tikvstore.KeyRange, keys ...string) { +func (s *testCoprocessorSuite) rangeEqual(c *C, ranges []kv.KeyRange, keys ...string) { for i := 0; i < len(ranges); i++ { r := ranges[i] c.Assert(string(r.StartKey), Equals, keys[2*i]) diff --git a/store/tikv/key_ranges.go b/store/copr/key_ranges.go similarity index 84% rename from store/tikv/key_ranges.go rename to store/copr/key_ranges.go index ec5e5dd92ac54..6b26d1026c785 100644 --- a/store/tikv/key_ranges.go +++ b/store/copr/key_ranges.go @@ -11,16 +11,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tikv +package copr import ( "bytes" "fmt" "sort" - "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/kv" ) // KeyRanges is like []kv.KeyRange, but may has extra elements at head/tail. @@ -142,20 +141,3 @@ func (r *KeyRanges) ToPBRanges() []*coprocessor.KeyRange { }) return ranges } - -// SplitRegionRanges get the split ranges from pd region. -func SplitRegionRanges(bo *Backoffer, cache *RegionCache, keyRanges []kv.KeyRange) ([]kv.KeyRange, error) { - ranges := NewKeyRanges(keyRanges) - - locations, err := cache.SplitKeyRangesByLocations(bo, ranges) - if err != nil { - return nil, errors.Trace(err) - } - var ret []kv.KeyRange - for _, loc := range locations { - for i := 0; i < loc.Ranges.Len(); i++ { - ret = append(ret, loc.Ranges.At(i)) - } - } - return ret, nil -} diff --git a/store/tikv/key_ranges_test.go b/store/copr/key_ranges_test.go similarity index 91% rename from store/tikv/key_ranges_test.go rename to store/copr/key_ranges_test.go index 40f924392acd6..eb1755c53a32f 100644 --- a/store/tikv/key_ranges_test.go +++ b/store/copr/key_ranges_test.go @@ -11,11 +11,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -package tikv +package copr import ( . "github.com/pingcap/check" - "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/kv" ) type testKeyRangesSuite struct { @@ -126,18 +126,3 @@ func (s *testKeyRangesSuite) testSplit(c *C, ranges *KeyRanges, checkLeft bool, } } } - -func buildKeyRanges(keys ...string) []kv.KeyRange { - var ranges []kv.KeyRange - for i := 0; i < len(keys); i += 2 { - ranges = append(ranges, kv.KeyRange{ - StartKey: []byte(keys[i]), - EndKey: []byte(keys[i+1]), - }) - } - return ranges -} - -func buildCopRanges(keys ...string) *KeyRanges { - return NewKeyRanges(buildKeyRanges(keys...)) -} diff --git a/store/copr/mpp.go b/store/copr/mpp.go index ac2bbbb823cf2..db0aff7e22696 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -61,7 +61,7 @@ func (c *MPPClient) ConstructMPPTasks(ctx context.Context, req *kv.MPPBuildTasks if req.KeyRanges == nil { return c.selectAllTiFlashStore(), nil } - ranges := toTiKVKeyRanges(req.KeyRanges) + ranges := NewKeyRanges(req.KeyRanges) tasks, err := buildBatchCopTasks(bo, c.store.GetRegionCache(), ranges, kv.TiFlash) if err != nil { return nil, errors.Trace(err) @@ -214,8 +214,8 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req // Or else it's the task without region, which always happens in high layer task without table. // In that case if originalTask != nil { - sender := tikv.NewRegionBatchRequestSender(m.store.GetRegionCache(), m.store.GetTiKVClient()) - rpcResp, _, _, err = sender.SendReqToAddr(bo.TiKVBackoffer(), originalTask.ctx, originalTask.regionInfos, wrappedReq, tikv.ReadTimeoutMedium) + sender := NewRegionBatchRequestSender(m.store.GetRegionCache(), m.store.GetTiKVClient()) + rpcResp, _, _, err = sender.SendReqToAddr(bo, originalTask.ctx, originalTask.regionInfos, wrappedReq, tikv.ReadTimeoutMedium) // No matter what the rpc error is, we won't retry the mpp dispatch tasks. // TODO: If we want to retry, we must redo the plan fragment cutting and task scheduling. // That's a hard job but we can try it in the future. diff --git a/store/copr/region_cache.go b/store/copr/region_cache.go new file mode 100644 index 0000000000000..9d6bd4dc973c6 --- /dev/null +++ b/store/copr/region_cache.go @@ -0,0 +1,138 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package copr + +import ( + "bytes" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/pingcap/tidb/store/tikv/metrics" + "go.uber.org/zap" +) + +// RegionCache wraps tikv.RegionCache. +type RegionCache struct { + *tikv.RegionCache +} + +// NewRegionCache returns a new RegionCache. +func NewRegionCache(rc *tikv.RegionCache) *RegionCache { + return &RegionCache{rc} +} + +// SplitRegionRanges gets the split ranges from pd region. +func (c *RegionCache) SplitRegionRanges(bo *Backoffer, keyRanges []kv.KeyRange) ([]kv.KeyRange, error) { + ranges := NewKeyRanges(keyRanges) + + locations, err := c.SplitKeyRangesByLocations(bo, ranges) + if err != nil { + return nil, errors.Trace(err) + } + var ret []kv.KeyRange + for _, loc := range locations { + for i := 0; i < loc.Ranges.Len(); i++ { + ret = append(ret, loc.Ranges.At(i)) + } + } + return ret, nil +} + +// LocationKeyRanges wrapps a real Location in PD and its logical ranges info. +type LocationKeyRanges struct { + // Location is the real location in PD. + Location *tikv.KeyLocation + // Ranges is the logic ranges the current Location contains. + Ranges *KeyRanges +} + +// SplitKeyRangesByLocations splits the KeyRanges by logical info in the cache. +func (c *RegionCache) SplitKeyRangesByLocations(bo *Backoffer, ranges *KeyRanges) ([]*LocationKeyRanges, error) { + res := make([]*LocationKeyRanges, 0) + for ranges.Len() > 0 { + loc, err := c.LocateKey(bo.TiKVBackoffer(), ranges.At(0).StartKey) + if err != nil { + return res, errors.Trace(err) + } + + // Iterate to the first range that is not complete in the region. + var i int + for ; i < ranges.Len(); i++ { + r := ranges.At(i) + if !(loc.Contains(r.EndKey) || bytes.Equal(loc.EndKey, r.EndKey)) { + break + } + } + // All rest ranges belong to the same region. + if i == ranges.Len() { + res = append(res, &LocationKeyRanges{Location: loc, Ranges: ranges}) + break + } + + r := ranges.At(i) + if loc.Contains(r.StartKey) { + // Part of r is not in the region. We need to split it. + taskRanges := ranges.Slice(0, i) + taskRanges.last = &kv.KeyRange{ + StartKey: r.StartKey, + EndKey: loc.EndKey, + } + res = append(res, &LocationKeyRanges{Location: loc, Ranges: taskRanges}) + + ranges = ranges.Slice(i+1, ranges.Len()) + ranges.first = &kv.KeyRange{ + StartKey: loc.EndKey, + EndKey: r.EndKey, + } + } else { + // rs[i] is not in the region. + taskRanges := ranges.Slice(0, i) + res = append(res, &LocationKeyRanges{Location: loc, Ranges: taskRanges}) + ranges = ranges.Slice(i, ranges.Len()) + } + } + + return res, nil +} + +// OnSendFailForBatchRegions handles send request fail logic. +func (c *RegionCache) OnSendFailForBatchRegions(bo *Backoffer, store *tikv.Store, regionInfos []RegionInfo, scheduleReload bool, err error) { + metrics.RegionCacheCounterWithSendFail.Add(float64(len(regionInfos))) + if !store.IsTiFlash() { + logutil.Logger(bo.GetCtx()).Info("Should not reach here, OnSendFailForBatchRegions only support TiFlash") + return + } + for _, ri := range regionInfos { + if ri.Meta == nil { + continue + } + r := c.GetCachedRegionWithRLock(ri.Region) + if r == nil { + return + } + peersNum := len(r.GetMeta().Peers) + if len(ri.Meta.Peers) != peersNum { + logutil.Logger(bo.GetCtx()).Info("retry and refresh current region after send request fail and up/down stores length changed", + zap.Stringer("region", &ri.Region), + zap.Bool("needReload", scheduleReload), + zap.Reflect("oldPeers", ri.Meta.Peers), + zap.Reflect("newPeers", r.GetMeta().Peers), + zap.Error(err)) + continue + } + c.OnSendFailForRegion(bo.TiKVBackoffer(), store, ri.Region, r, scheduleReload, err) + } +} diff --git a/store/copr/store.go b/store/copr/store.go index 7fa4aeafb5135..5ef623adac0e3 100644 --- a/store/copr/store.go +++ b/store/copr/store.go @@ -33,8 +33,8 @@ type kvStore struct { } // GetRegionCache returns the region cache instance. -func (s *kvStore) GetRegionCache() *tikv.RegionCache { - return s.store.GetRegionCache() +func (s *kvStore) GetRegionCache() *RegionCache { + return &RegionCache{s.store.GetRegionCache()} } // CheckVisibility checks if it is safe to read using given ts. diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 287edeb4170f5..d3eead0444940 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -451,7 +451,7 @@ func WithMatchLabels(labels []*metapb.StoreLabel) StoreSelectorOption { func (c *RegionCache) GetTiKVRPCContext(bo *Backoffer, id RegionVerID, replicaRead kv.ReplicaReadType, followerStoreSeed uint32, opts ...StoreSelectorOption) (*RPCContext, error) { ts := time.Now().Unix() - cachedRegion := c.getCachedRegionWithRLock(id) + cachedRegion := c.GetCachedRegionWithRLock(id) if cachedRegion == nil { return nil, nil } @@ -564,7 +564,7 @@ func (c *RegionCache) GetAllValidTiFlashStores(id RegionVerID, currentStore *Sto // make sure currentStore id is always the first in allStores allStores = append(allStores, currentStore.storeID) ts := time.Now().Unix() - cachedRegion := c.getCachedRegionWithRLock(id) + cachedRegion := c.GetCachedRegionWithRLock(id) if cachedRegion == nil { return allStores } @@ -597,7 +597,7 @@ func (c *RegionCache) GetAllValidTiFlashStores(id RegionVerID, currentStore *Sto func (c *RegionCache) GetTiFlashRPCContext(bo *Backoffer, id RegionVerID, loadBalance bool) (*RPCContext, error) { ts := time.Now().Unix() - cachedRegion := c.getCachedRegionWithRLock(id) + cachedRegion := c.GetCachedRegionWithRLock(id) if cachedRegion == nil { return nil, nil } @@ -701,80 +701,6 @@ func (c *RegionCache) LocateEndKey(bo *Backoffer, key []byte) (*KeyLocation, err }, nil } -// SplitRegionRanges gets the split ranges from pd region. -func (c *RegionCache) SplitRegionRanges(bo *Backoffer, keyRanges []kv.KeyRange) ([]kv.KeyRange, error) { - ranges := NewKeyRanges(keyRanges) - - locations, err := c.SplitKeyRangesByLocations(bo, ranges) - if err != nil { - return nil, errors.Trace(err) - } - var ret []kv.KeyRange - for _, loc := range locations { - for i := 0; i < loc.Ranges.Len(); i++ { - ret = append(ret, loc.Ranges.At(i)) - } - } - return ret, nil -} - -// LocationKeyRanges wrapps a real Location in PD and its logical ranges info. -type LocationKeyRanges struct { - // Location is the real location in PD. - Location *KeyLocation - // Ranges is the logic ranges the current Location contains. - Ranges *KeyRanges -} - -// SplitKeyRangesByLocations splits the KeyRanges by logical info in the cache. -func (c *RegionCache) SplitKeyRangesByLocations(bo *Backoffer, ranges *KeyRanges) ([]*LocationKeyRanges, error) { - res := make([]*LocationKeyRanges, 0) - for ranges.Len() > 0 { - loc, err := c.LocateKey(bo, ranges.At(0).StartKey) - if err != nil { - return res, errors.Trace(err) - } - - // Iterate to the first range that is not complete in the region. - var i int - for ; i < ranges.Len(); i++ { - r := ranges.At(i) - if !(loc.Contains(r.EndKey) || bytes.Equal(loc.EndKey, r.EndKey)) { - break - } - } - // All rest ranges belong to the same region. - if i == ranges.Len() { - res = append(res, &LocationKeyRanges{Location: loc, Ranges: ranges}) - break - } - - r := ranges.At(i) - if loc.Contains(r.StartKey) { - // Part of r is not in the region. We need to split it. - taskRanges := ranges.Slice(0, i) - taskRanges.last = &kv.KeyRange{ - StartKey: r.StartKey, - EndKey: loc.EndKey, - } - res = append(res, &LocationKeyRanges{Location: loc, Ranges: taskRanges}) - - ranges = ranges.Slice(i+1, ranges.Len()) - ranges.first = &kv.KeyRange{ - StartKey: loc.EndKey, - EndKey: r.EndKey, - } - } else { - // rs[i] is not in the region. - taskRanges := ranges.Slice(0, i) - res = append(res, &LocationKeyRanges{Location: loc, Ranges: taskRanges}) - ranges = ranges.Slice(i, ranges.Len()) - } - } - - return res, nil -} - func (c *RegionCache) findRegionByKey(bo *Backoffer, key []byte, isEndKey bool) (r *Region, err error) { r = c.searchCachedRegion(key, isEndKey) if r == nil { @@ -807,68 +733,45 @@ func (c *RegionCache) findRegionByKey(bo *Backoffer, key []byte, isEndKey bool) return r, nil } -// OnSendFailForBatchRegions handles send request fail logic. -func (c *RegionCache) OnSendFailForBatchRegions(bo *Backoffer, store *Store, regionInfos []RegionInfo, scheduleReload bool, err error) { - metrics.RegionCacheCounterWithSendFail.Add(float64(len(regionInfos))) - if store.storeType != tikvrpc.TiFlash { - logutil.Logger(bo.GetCtx()).Info("Should not reach here, OnSendFailForBatchRegions only support TiFlash") +// OnSendFailForRegion handles send request fail logic on a region. +func (c *RegionCache) OnSendFailForRegion(bo *Backoffer, store *Store, rid RegionVerID, r *Region, scheduleReload bool, err error) { + + rs := r.getStore() + + accessMode := TiFlashOnly + accessIdx := rs.getAccessIndex(accessMode, store) + if accessIdx == -1 { + logutil.Logger(bo.GetCtx()).Warn("can not get access index for region " + rid.String()) return } - for _, ri := range regionInfos { - if ri.Meta == nil { - continue + if err != nil { + storeIdx, s := rs.accessStore(accessMode, accessIdx) + epoch := rs.storeEpochs[storeIdx] + if atomic.CompareAndSwapUint32(&s.epoch, epoch, epoch+1) { + logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) + metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() } - r := c.getCachedRegionWithRLock(ri.Region) - if r != nil { - peersNum := len(r.meta.Peers) - if len(ri.Meta.Peers) != peersNum { - logutil.Logger(bo.GetCtx()).Info("retry and refresh current region after send request fail and up/down stores length changed", - zap.Stringer("region", &ri.Region), - zap.Bool("needReload", scheduleReload), - zap.Reflect("oldPeers", ri.Meta.Peers), - zap.Reflect("newPeers", r.meta.Peers), - zap.Error(err)) - continue - } - - rs := r.getStore() - - accessMode := TiFlashOnly - accessIdx := rs.getAccessIndex(accessMode, store) - if accessIdx == -1 { - logutil.Logger(bo.GetCtx()).Warn("can not get access index for region " + ri.Region.String()) - continue - } - if err != nil { - storeIdx, s := rs.accessStore(accessMode, accessIdx) - epoch := rs.storeEpochs[storeIdx] - if atomic.CompareAndSwapUint32(&s.epoch, epoch, epoch+1) { - logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) - metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() - } - // schedule a store addr resolve. - s.markNeedCheck(c.notifyCheckCh) - } + // schedule a store addr resolve. + s.markNeedCheck(c.notifyCheckCh) + } - // try next peer - rs.switchNextFlashPeer(r, accessIdx) - logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", - zap.Stringer("region", &ri.Region), - zap.Bool("needReload", scheduleReload), - zap.Error(err)) + // try next peer + rs.switchNextFlashPeer(r, accessIdx) + logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", + zap.Stringer("region", &rid), + zap.Bool("needReload", scheduleReload), + zap.Error(err)) - // force reload region when retry all known peers in region. - if scheduleReload { - r.scheduleReload() - } - } + // force reload region when retry all known peers in region. + if scheduleReload { + r.scheduleReload() } } // OnSendFail handles send request fail logic. func (c *RegionCache) OnSendFail(bo *Backoffer, ctx *RPCContext, scheduleReload bool, err error) { metrics.RegionCacheCounterWithSendFail.Inc() - r := c.getCachedRegionWithRLock(ctx.Region) + r := c.GetCachedRegionWithRLock(ctx.Region) if r != nil { peersNum := len(r.meta.Peers) if len(ctx.Meta.Peers) != peersNum { @@ -1136,7 +1039,7 @@ func (c *RegionCache) InvalidateCachedRegion(id RegionVerID) { // InvalidateCachedRegionWithReason removes a cached Region with the reason why it's invalidated. func (c *RegionCache) InvalidateCachedRegionWithReason(id RegionVerID, reason InvalidReason) { - cachedRegion := c.getCachedRegionWithRLock(id) + cachedRegion := c.GetCachedRegionWithRLock(id) if cachedRegion == nil { return } @@ -1145,7 +1048,7 @@ func (c *RegionCache) InvalidateCachedRegionWithReason(id RegionVerID, reason In // UpdateLeader update some region cache with newer leader info. func (c *RegionCache) UpdateLeader(regionID RegionVerID, leaderStoreID uint64, currentPeerIdx AccessIndex) { - r := c.getCachedRegionWithRLock(regionID) + r := c.GetCachedRegionWithRLock(regionID) if r == nil { logutil.BgLogger().Debug("regionCache: cannot find region when updating leader", zap.Uint64("regionID", regionID.GetID()), @@ -1484,7 +1387,8 @@ func (c *RegionCache) scanRegions(bo *Backoffer, startKey, endKey []byte, limit } } -func (c *RegionCache) getCachedRegionWithRLock(regionID RegionVerID) (r *Region) { +// GetCachedRegionWithRLock returns region with lock. +func (c *RegionCache) GetCachedRegionWithRLock(regionID RegionVerID) (r *Region) { c.mu.RLock() r = c.mu.regions[regionID] c.mu.RUnlock() @@ -1973,6 +1877,11 @@ const ( tombstone ) +// IsTiFlash returns true if the storeType is TiFlash +func (s *Store) IsTiFlash() bool { + return s.storeType == tikvrpc.TiFlash +} + // initResolve resolves the address of the store that never resolved and returns an // empty string if it's a tombstone. func (s *Store) initResolve(bo *Backoffer, c *RegionCache) (addr string, err error) { diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index acd267ceed760..ed19243a40a6a 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -1528,7 +1528,7 @@ func (s *testRegionCacheSuite) TestSwitchPeerWhenNoLeader(c *C) { c.Assert(ctx.Peer, Not(DeepEquals), prevCtx.Peer) } s.cache.InvalidateCachedRegionWithReason(loc.Region, NoLeader) - c.Assert(s.cache.getCachedRegionWithRLock(loc.Region).invalidReason, Equals, NoLeader) + c.Assert(s.cache.GetCachedRegionWithRLock(loc.Region).invalidReason, Equals, NoLeader) prevCtx = ctx } } @@ -1563,7 +1563,7 @@ func BenchmarkOnRequestFail(b *testing.B) { Store: store, AccessMode: TiKVOnly, } - r := cache.getCachedRegionWithRLock(rpcCtx.Region) + r := cache.GetCachedRegionWithRLock(rpcCtx.Region) if r != nil { r.getStore().switchNextTiKVPeer(r, rpcCtx.AccessIdx) } diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index bbe7ff8d19479..d323f9f2f7caa 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -710,7 +710,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestForwarding(c *C) { c.Assert(ctx, IsNil) c.Assert(len(s.regionRequestSender.failStoreIDs), Equals, 0) c.Assert(len(s.regionRequestSender.failProxyStoreIDs), Equals, 0) - region := s.regionRequestSender.regionCache.getCachedRegionWithRLock(loc.Region) + region := s.regionRequestSender.regionCache.GetCachedRegionWithRLock(loc.Region) c.Assert(region, NotNil) c.Assert(region.checkNeedReload(), IsTrue) From ac04473227f3678bb1313b8ffb044956f8bd7fa7 Mon Sep 17 00:00:00 2001 From: ZhuoZhi <517770911@qq.com> Date: Fri, 28 May 2021 21:01:35 +0800 Subject: [PATCH 208/343] planner, executor: support batchget for range and list partition table (#24856) --- executor/batch_point_get.go | 59 +++++++++++++++--- executor/builder.go | 1 + executor/partition_table_test.go | 102 +++++++++++++++++++++++++++++++ planner/core/find_best_task.go | 3 +- planner/core/point_get_plan.go | 88 +++++++++++++++++++++++--- 5 files changed, 235 insertions(+), 18 deletions(-) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 5d6bb5c172751..48b33e7c1bef3 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -19,16 +19,19 @@ import ( "sort" "sync/atomic" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/ddl/placement" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" driver "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -46,6 +49,7 @@ type BatchPointGetExec struct { idxInfo *model.IndexInfo handles []kv.Handle physIDs []int64 + partExpr *tables.PartitionExpr partPos int singlePart bool partTblID int64 @@ -227,7 +231,11 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { continue } - physID := getPhysID(e.tblInfo, idxVals[e.partPos].GetInt64()) + physID, err := getPhysID(e.tblInfo, e.partExpr, idxVals[e.partPos].GetInt64()) + if err != nil { + continue + } + // If this BatchPointGetExec is built only for the specific table partition, skip those filters not matching this partition. if e.singlePart && e.partTblID != physID { continue @@ -355,13 +363,19 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { tID = e.physIDs[i] } else { if handle.IsInt() { - tID = getPhysID(e.tblInfo, handle.IntValue()) + tID, err = getPhysID(e.tblInfo, e.partExpr, handle.IntValue()) + if err != nil { + continue + } } else { _, d, err1 := codec.DecodeOne(handle.EncodedCol(e.partPos)) if err1 != nil { return err1 } - tID = getPhysID(e.tblInfo, d.GetInt64()) + tID, err = getPhysID(e.tblInfo, e.partExpr, d.GetInt64()) + if err != nil { + continue + } } } // If this BatchPointGetExec is built only for the specific table partition, skip those handles not matching this partition. @@ -466,13 +480,44 @@ func (getter *PessimisticLockCacheGetter) Get(_ context.Context, key kv.Key) ([] return nil, kv.ErrNotExist } -func getPhysID(tblInfo *model.TableInfo, intVal int64) int64 { +func getPhysID(tblInfo *model.TableInfo, partitionExpr *tables.PartitionExpr, intVal int64) (int64, error) { pi := tblInfo.GetPartitionInfo() if pi == nil { - return tblInfo.ID + return tblInfo.ID, nil + } + + if partitionExpr == nil { + return tblInfo.ID, nil } - partIdx := math.Abs(intVal % int64(pi.Num)) - return pi.Definitions[partIdx].ID + + switch pi.Type { + case model.PartitionTypeHash: + partIdx := math.Abs(intVal % int64(pi.Num)) + return pi.Definitions[partIdx].ID, nil + case model.PartitionTypeRange: + // we've check the type assertions in func TryFastPlan + col, ok := partitionExpr.Expr.(*expression.Column) + if !ok { + return 0, errors.Errorf("unsupported partition type in BatchGet") + } + unsigned := mysql.HasUnsignedFlag(col.GetType().Flag) + ranges := partitionExpr.ForRangePruning + length := len(ranges.LessThan) + partIdx := sort.Search(length, func(i int) bool { + return ranges.Compare(i, intVal, unsigned) > 0 + }) + if partIdx >= 0 && partIdx < length { + return pi.Definitions[partIdx].ID, nil + } + case model.PartitionTypeList: + isNull := false // we've guaranteed this in the build process of either TryFastPlan or buildBatchPointGet + partIdx := partitionExpr.ForListPruning.LocatePartition(intVal, isNull) + if partIdx >= 0 { + return pi.Definitions[partIdx].ID, nil + } + } + + return 0, errors.Errorf("dual partition") } type cacheBatchGetter struct { diff --git a/executor/builder.go b/executor/builder.go index 34c5e0c599f5a..037123cc1fa96 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3939,6 +3939,7 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan desc: plan.Desc, lock: plan.Lock, waitTime: plan.LockWaitTime, + partExpr: plan.PartitionExpr, partPos: plan.PartitionColPos, singlePart: plan.SinglePart, partTblID: plan.PartTblID, diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 0d70bbd3c8eb4..b04472d22da05 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -683,6 +683,108 @@ func (s *partitionTableSuite) TestDynamicPruningUnderIndexJoin(c *C) { tk.MustQuery(`select /*+ INL_JOIN(touter, tnormal) */ tnormal.* from touter join tnormal use index(idx_b) on touter.b = tnormal.b`).Sort().Rows()) } +func (s *partitionTableSuite) TestBatchGetforRangeandListPartitionTable(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_pointget") + tk.MustExec("use test_pointget") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec("set @@session.tidb_enable_list_partition = ON") + + // list partition table + tk.MustExec(`create table tlist(a int, b int, unique index idx_a(a), index idx_b(b)) partition by list(a)( + partition p0 values in (1, 2, 3, 4), + partition p1 values in (5, 6, 7, 8), + partition p2 values in (9, 10, 11, 12));`) + + // range partition table + tk.MustExec(`create table trange(a int, unique key(a)) partition by range(a) ( + partition p0 values less than (30), + partition p1 values less than (60), + partition p2 values less than (90), + partition p3 values less than (120));`) + + // hash partition table + tk.MustExec("create table thash(a int unsigned, unique key(a)) partition by hash(a) partitions 4;") + + // insert data into list partition table + tk.MustExec("insert into tlist values(1,1), (2,2), (3, 3), (4, 4), (5,5), (6, 6), (7,7), (8, 8), (9, 9), (10, 10), (11, 11), (12, 12);") + // regular partition table + tk.MustExec("create table tregular1(a int, unique key(a));") + tk.MustExec("create table tregular2(a int, unique key(a));") + + vals := make([]string, 0, 100) + // insert data into range partition table and hash partition table + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v)", i+1)) + } + tk.MustExec("insert into trange values " + strings.Join(vals, ",")) + tk.MustExec("insert into thash values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular1 values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular2 values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), (11), (12)") + + // test BatchGet + for i := 0; i < 100; i++ { + // explain select a from t where a in ({x1}, {x2}, ... {x10}); // BatchGet is used + // select a from t where where a in ({x1}, {x2}, ... {x10}); + points := make([]string, 0, 10) + for i := 0; i < 10; i++ { + x := rand.Intn(100) + 1 + points = append(points, fmt.Sprintf("%v", x)) + } + queryRegular1 := fmt.Sprintf("select a from tregular1 where a in (%v)", strings.Join(points, ",")) + + queryHash := fmt.Sprintf("select a from thash where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryHash, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + tk.MustQuery(queryHash).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + queryRange := fmt.Sprintf("select a from trange where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryRange, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + tk.MustQuery(queryRange).Sort().Check(tk.MustQuery(queryRegular1).Sort().Rows()) + + points = make([]string, 0, 10) + for i := 0; i < 10; i++ { + x := rand.Intn(12) + 1 + points = append(points, fmt.Sprintf("%v", x)) + } + queryRegular2 := fmt.Sprintf("select a from tregular2 where a in (%v)", strings.Join(points, ",")) + queryList := fmt.Sprintf("select a from tlist where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryList, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + tk.MustQuery(queryList).Sort().Check(tk.MustQuery(queryRegular2).Sort().Rows()) + } + + // test different data type + // unsigned flag + // partition table and reguar table pair + tk.MustExec(`create table trange3(a int unsigned, unique key(a)) partition by range(a) ( + partition p0 values less than (30), + partition p1 values less than (60), + partition p2 values less than (90), + partition p3 values less than (120));`) + tk.MustExec("create table tregular3(a int unsigned, unique key(a));") + vals = make([]string, 0, 100) + // insert data into range partition table and hash partition table + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v)", i+1)) + } + tk.MustExec("insert into trange3 values " + strings.Join(vals, ",")) + tk.MustExec("insert into tregular3 values " + strings.Join(vals, ",")) + // test BatchGet + // explain select a from t where a in ({x1}, {x2}, ... {x10}); // BatchGet is used + // select a from t where where a in ({x1}, {x2}, ... {x10}); + points := make([]string, 0, 10) + for i := 0; i < 10; i++ { + x := rand.Intn(100) + 1 + points = append(points, fmt.Sprintf("%v", x)) + } + queryRegular := fmt.Sprintf("select a from tregular3 where a in (%v)", strings.Join(points, ",")) + queryRange := fmt.Sprintf("select a from trange3 where a in (%v)", strings.Join(points, ",")) + c.Assert(tk.HasPlan(queryRange, "Batch_Point_Get"), IsTrue) // check if BatchGet is used + tk.MustQuery(queryRange).Sort().Check(tk.MustQuery(queryRegular).Sort().Rows()) +} + func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index eede1b4788059..2eb7c2b5a1a95 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1723,6 +1723,7 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca Columns: ds.Columns, SinglePart: ds.isPartition, PartTblID: ds.physicalTableID, + PartitionExpr: getPartitionExpr(ds.ctx, ds.TableInfo()), }.Init(ds.ctx, ds.tableStats.ScaleByExpectCnt(accessCnt), ds.schema.Clone(), ds.names, ds.blockOffset) if batchPointGetPlan.KeepOrder { batchPointGetPlan.Desc = prop.SortItems[0].Desc @@ -1748,7 +1749,7 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca batchPointGetPlan.IndexInfo = candidate.path.Index batchPointGetPlan.IdxCols = candidate.path.IdxCols batchPointGetPlan.IdxColLens = candidate.path.IdxColLens - batchPointGetPlan.PartitionColPos = getPartitionColumnPos(candidate.path.Index, hashPartColName) + batchPointGetPlan.PartitionColPos = getHashPartitionColumnPos(candidate.path.Index, hashPartColName) for _, ran := range candidate.path.Ranges { batchPointGetPlan.IndexValues = append(batchPointGetPlan.IndexValues, ran.LowVal) } diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index ba780bdebde0f..64642d4bef335 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -270,6 +270,7 @@ type BatchPointGetPlan struct { IdxCols []*expression.Column IdxColLens []int PartitionColPos int + PartitionExpr *tables.PartitionExpr KeepOrder bool Desc bool Lock bool @@ -532,10 +533,10 @@ func newBatchPointGetPlan( names []*types.FieldName, whereColNames []string, indexHints []*ast.IndexHint, ) *BatchPointGetPlan { statsInfo := &property.StatsInfo{RowCount: float64(len(patternInExpr.List))} - var partitionColName *ast.ColumnName + var partitionExpr *tables.PartitionExpr if tbl.GetPartitionInfo() != nil { - partitionColName = getHashPartitionColumnName(ctx, tbl) - if partitionColName == nil { + partitionExpr = getPartitionExpr(ctx, tbl) + if partitionExpr == nil { return nil } } @@ -577,9 +578,10 @@ func newBatchPointGetPlan( handleParams[i] = param } return BatchPointGetPlan{ - TblInfo: tbl, - Handles: handles, - HandleParams: handleParams, + TblInfo: tbl, + Handles: handles, + HandleParams: handleParams, + PartitionExpr: partitionExpr, }.Init(ctx, statsInfo, schema, names, 0) } @@ -626,6 +628,12 @@ func newBatchPointGetPlan( if matchIdxInfo == nil { return nil } + + pos, err := getPartitionColumnPos(matchIdxInfo, partitionExpr, tbl) + if err != nil { + return nil + } + indexValues := make([][]types.Datum, len(patternInExpr.List)) indexValueParams := make([][]*driver.ParamMarkerExpr, len(patternInExpr.List)) for i, item := range patternInExpr.List { @@ -691,7 +699,8 @@ func newBatchPointGetPlan( IndexInfo: matchIdxInfo, IndexValues: indexValues, IndexValueParams: indexValueParams, - PartitionColPos: getPartitionColumnPos(matchIdxInfo, partitionColName), + PartitionColPos: pos, + PartitionExpr: partitionExpr, }.Init(ctx, statsInfo, schema, names, 0) } @@ -1469,6 +1478,9 @@ func buildHandleCols(ctx sessionctx.Context, tbl *model.TableInfo, schema *expre func getPartitionInfo(ctx sessionctx.Context, tbl *model.TableInfo, pairs []nameValuePair) (*model.PartitionDefinition, int, bool) { partitionExpr := getPartitionExpr(ctx, tbl) + if partitionExpr == nil { + return nil, 0, false + } pi := tbl.GetPartitionInfo() if pi == nil { @@ -1549,8 +1561,58 @@ func findPartitionIdx(idxInfo *model.IndexInfo, pos int, pairs []nameValuePair) return 0 } -// getPartitionColumnPos gets the partition column's position in the index. -func getPartitionColumnPos(idx *model.IndexInfo, partitionColName *ast.ColumnName) int { +// getPartitionColumnPos gets the partition column's position in the unique index. +func getPartitionColumnPos(idx *model.IndexInfo, partitionExpr *tables.PartitionExpr, tbl *model.TableInfo) (int, error) { + // regular table + if partitionExpr == nil { + return 0, nil + } + pi := tbl.GetPartitionInfo() + if pi == nil { + return 0, nil + } + + var partitionName model.CIStr + switch pi.Type { + case model.PartitionTypeHash: + if col, ok := partitionExpr.OrigExpr.(*ast.ColumnNameExpr); ok { + partitionName = col.Name.Name + } else { + return 0, errors.Errorf("unsupported partition type in BatchGet") + } + case model.PartitionTypeRange: + // left range columns partition for future development + if len(pi.Columns) == 0 { + if col, ok := partitionExpr.Expr.(*expression.Column); ok { + colInfo := findColNameByColID(tbl.Columns, col) + partitionName = colInfo.Name + } + } else { + return 0, errors.Errorf("unsupported partition type in BatchGet") + } + case model.PartitionTypeList: + // left list columns partition for future development + if partitionExpr.ForListPruning.ColPrunes == nil { + locateExpr := partitionExpr.ForListPruning.LocateExpr + if locateExpr, ok := locateExpr.(*expression.Column); ok { + colInfo := findColNameByColID(tbl.Columns, locateExpr) + partitionName = colInfo.Name + } + } else { + return 0, errors.Errorf("unsupported partition type in BatchGet") + } + } + + for i, idxCol := range idx.Columns { + if partitionName.L == idxCol.Name.L { + return i, nil + } + } + panic("unique index must include all partition columns") +} + +// getHashPartitionColumnPos gets the hash partition column's position in the unique index. +func getHashPartitionColumnPos(idx *model.IndexInfo, partitionColName *ast.ColumnName) int { if partitionColName == nil { return 0 } @@ -1568,8 +1630,14 @@ func getPartitionExpr(ctx sessionctx.Context, tbl *model.TableInfo) *tables.Part if !ok { return nil } + + partTable, ok := table.(partitionTable) + if !ok { + return nil + } + // PartitionExpr don't need columns and names for hash partition. - partitionExpr, err := table.(partitionTable).PartitionExpr() + partitionExpr, err := partTable.PartitionExpr() if err != nil { return nil } From 7f4f5c02364b6578da561ec14f409a39ddf954a5 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Fri, 28 May 2021 09:29:36 -0600 Subject: [PATCH 209/343] *: Change dynamic privileges to non-experimental (#24914) --- executor/grant.go | 4 ---- executor/grant_test.go | 6 +----- executor/infoschema_reader_test.go | 1 - executor/revoke.go | 4 ---- executor/revoke_test.go | 8 +------- privilege/privileges/privileges_test.go | 12 ------------ sessionctx/variable/session.go | 3 --- sessionctx/variable/sysvar.go | 5 +---- 8 files changed, 3 insertions(+), 40 deletions(-) diff --git a/executor/grant.go b/executor/grant.go index 2b5efcd0803eb..49536dc79aa86 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -16,7 +16,6 @@ package executor import ( "context" "encoding/json" - "fmt" "strings" "github.com/pingcap/errors" @@ -429,9 +428,6 @@ func (e *GrantExec) grantLevelPriv(priv *ast.PrivElem, user *ast.UserSpec, inter func (e *GrantExec) grantDynamicPriv(privName string, user *ast.UserSpec, internalSession sessionctx.Context) error { privName = strings.ToUpper(privName) - if !e.ctx.GetSessionVars().EnableDynamicPrivileges { - return fmt.Errorf("dynamic privileges is an experimental feature. Run 'SET tidb_enable_dynamic_privileges=1'") - } if e.Level.Level != ast.GrantLevelGlobal { // DYNAMIC can only be *.* return ErrIllegalPrivilegeLevel.GenWithStackByArgs(privName) } diff --git a/executor/grant_test.go b/executor/grant_test.go index 0e41c75e44353..8ccfac3533fb6 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -403,12 +403,8 @@ func (s *testSuite3) TestIssue22721(c *C) { func (s *testSuite3) TestGrantDynamicPrivs(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create user dyn") - tk.MustExec("SET tidb_enable_dynamic_privileges=0") - _, err := tk.Exec("GRANT BACKUP_ADMIN ON *.* TO dyn") - c.Assert(err.Error(), Equals, "dynamic privileges is an experimental feature. Run 'SET tidb_enable_dynamic_privileges=1'") - tk.MustExec("SET tidb_enable_dynamic_privileges=1") - _, err = tk.Exec("GRANT BACKUP_ADMIN ON test.* TO dyn") + _, err := tk.Exec("GRANT BACKUP_ADMIN ON test.* TO dyn") c.Assert(terror.ErrorEqual(err, executor.ErrIllegalPrivilegeLevel), IsTrue) _, err = tk.Exec("GRANT BOGUS_GRANT ON *.* TO dyn") c.Assert(terror.ErrorEqual(err, executor.ErrDynamicPrivilegeNotRegistered), IsTrue) diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index e19eb9d9b3064..1170453d368b0 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -370,7 +370,6 @@ func (s *testInfoschemaTableSuite) TestUserPrivilegesTable(c *C) { tk.MustExec("GRANT SELECT ON *.* to usageuser WITH GRANT OPTION") tk.MustQuery(`SELECT * FROM information_schema.user_privileges WHERE grantee="'usageuser'@'%'"`).Check(testkit.Rows("'usageuser'@'%' def Select YES")) // test DYNAMIC privs - tk.MustExec("SET tidb_enable_dynamic_privileges=1") tk.MustExec("GRANT BACKUP_ADMIN ON *.* to usageuser") tk.MustQuery(`SELECT * FROM information_schema.user_privileges WHERE grantee="'usageuser'@'%'" ORDER BY privilege_type`).Check(testkit.Rows("'usageuser'@'%' def BACKUP_ADMIN NO", "'usageuser'@'%' def Select YES")) } diff --git a/executor/revoke.go b/executor/revoke.go index b090f048c62a7..f84bbac9676da 100644 --- a/executor/revoke.go +++ b/executor/revoke.go @@ -15,7 +15,6 @@ package executor import ( "context" - "fmt" "strings" "github.com/pingcap/errors" @@ -193,9 +192,6 @@ func (e *RevokeExec) revokePriv(internalSession sessionctx.Context, priv *ast.Pr func (e *RevokeExec) revokeDynamicPriv(internalSession sessionctx.Context, privName string, user, host string) error { privName = strings.ToUpper(privName) - if !e.ctx.GetSessionVars().EnableDynamicPrivileges { - return fmt.Errorf("dynamic privileges is an experimental feature. Run 'SET tidb_enable_dynamic_privileges=1'") - } if !privilege.GetPrivilegeManager(e.ctx).IsDynamicPrivilege(privName) { // for MySQL compatibility e.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrDynamicPrivilegeNotRegistered.GenWithStackByArgs(privName)) } diff --git a/executor/revoke_test.go b/executor/revoke_test.go index 20cfc8b0250f1..f4a9ea8df231b 100644 --- a/executor/revoke_test.go +++ b/executor/revoke_test.go @@ -159,17 +159,11 @@ func (s *testSuite1) TestRevokeDynamicPrivs(c *C) { tk.MustExec("DROP USER if exists dyn") tk.MustExec("create user dyn") - tk.MustExec("SET tidb_enable_dynamic_privileges=0") - _, err := tk.Exec("GRANT BACKUP_ADMIN ON *.* TO dyn") - c.Assert(err.Error(), Equals, "dynamic privileges is an experimental feature. Run 'SET tidb_enable_dynamic_privileges=1'") - - tk.MustExec("SET tidb_enable_dynamic_privileges=1") - tk.MustExec("GRANT BACKUP_Admin ON *.* TO dyn") // grant one priv tk.MustQuery("SELECT * FROM mysql.global_grants WHERE `Host` = '%' AND `User` = 'dyn' ORDER BY user,host,priv,with_grant_option").Check(testkit.Rows("dyn % BACKUP_ADMIN N")) // try revoking only on test.* - should fail: - _, err = tk.Exec("REVOKE BACKUP_Admin,system_variables_admin ON test.* FROM dyn") + _, err := tk.Exec("REVOKE BACKUP_Admin,system_variables_admin ON test.* FROM dyn") c.Assert(terror.ErrorEqual(err, executor.ErrIllegalPrivilegeLevel), IsTrue) // privs should still be intact: diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 1563f8d72cf7b..1bb69be14c826 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -1253,11 +1253,9 @@ func (s *testPrivilegeSuite) TestDynamicPrivs(c *C) { mustExec(c, rootSe, "CREATE USER notsuper") mustExec(c, rootSe, "CREATE USER otheruser") mustExec(c, rootSe, "CREATE ROLE anyrolename") - mustExec(c, rootSe, "SET tidb_enable_dynamic_privileges=1") se := newSession(c, s.store, s.dbName) c.Assert(se.Auth(&auth.UserIdentity{Username: "notsuper", Hostname: "%"}, nil, nil), IsTrue) - mustExec(c, se, "SET tidb_enable_dynamic_privileges=1") // test SYSTEM_VARIABLES_ADMIN _, err := se.ExecuteInternal(context.Background(), "SET GLOBAL wait_timeout = 86400") @@ -1298,20 +1296,17 @@ func (s *testPrivilegeSuite) TestDynamicGrantOption(c *C) { mustExec(c, rootSe, "CREATE USER varuser1") mustExec(c, rootSe, "CREATE USER varuser2") mustExec(c, rootSe, "CREATE USER varuser3") - mustExec(c, rootSe, "SET tidb_enable_dynamic_privileges=1") mustExec(c, rootSe, "GRANT SYSTEM_VARIABLES_ADMIN ON *.* TO varuser1") mustExec(c, rootSe, "GRANT SYSTEM_VARIABLES_ADMIN ON *.* TO varuser2 WITH GRANT OPTION") se1 := newSession(c, s.store, s.dbName) - mustExec(c, se1, "SET tidb_enable_dynamic_privileges=1") c.Assert(se1.Auth(&auth.UserIdentity{Username: "varuser1", Hostname: "%"}, nil, nil), IsTrue) _, err := se1.ExecuteInternal(context.Background(), "GRANT SYSTEM_VARIABLES_ADMIN ON *.* TO varuser3") c.Assert(err.Error(), Equals, "[planner:1227]Access denied; you need (at least one of) the GRANT OPTION privilege(s) for this operation") se2 := newSession(c, s.store, s.dbName) - mustExec(c, se2, "SET tidb_enable_dynamic_privileges=1") c.Assert(se2.Auth(&auth.UserIdentity{Username: "varuser2", Hostname: "%"}, nil, nil), IsTrue) mustExec(c, se2, "GRANT SYSTEM_VARIABLES_ADMIN ON *.* TO varuser3") @@ -1321,14 +1316,12 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeRestrictedTables(c *C) { // This provides an integration test of the tests in util/security/security_test.go cloudAdminSe := newSession(c, s.store, s.dbName) mustExec(c, cloudAdminSe, "CREATE USER cloudadmin") - mustExec(c, cloudAdminSe, "SET tidb_enable_dynamic_privileges=1") mustExec(c, cloudAdminSe, "GRANT RESTRICTED_TABLES_ADMIN, SELECT ON *.* to cloudadmin") mustExec(c, cloudAdminSe, "GRANT CREATE ON mysql.* to cloudadmin") mustExec(c, cloudAdminSe, "CREATE USER uroot") mustExec(c, cloudAdminSe, "GRANT ALL ON *.* to uroot WITH GRANT OPTION") // A "MySQL" all powerful user. c.Assert(cloudAdminSe.Auth(&auth.UserIdentity{Username: "cloudadmin", Hostname: "%"}, nil, nil), IsTrue) urootSe := newSession(c, s.store, s.dbName) - mustExec(c, urootSe, "SET tidb_enable_dynamic_privileges=1") c.Assert(urootSe.Auth(&auth.UserIdentity{Username: "uroot", Hostname: "%"}, nil, nil), IsTrue) sem.Enable() @@ -1352,7 +1345,6 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeInfoschema(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("CREATE USER uroot1, uroot2, uroot3") tk.MustExec("GRANT SUPER ON *.* to uroot1 WITH GRANT OPTION") // super not process - tk.MustExec("SET tidb_enable_dynamic_privileges=1") tk.MustExec("GRANT SUPER, PROCESS, RESTRICTED_TABLES_ADMIN ON *.* to uroot2 WITH GRANT OPTION") tk.Se.Auth(&auth.UserIdentity{ Username: "uroot1", @@ -1391,7 +1383,6 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeStatusVars(c *C) { // and verify if it appears. tk := testkit.NewTestKit(c, s.store) tk.MustExec("CREATE USER unostatus, ustatus") - tk.MustExec("SET tidb_enable_dynamic_privileges=1") tk.MustExec("GRANT RESTRICTED_STATUS_ADMIN ON *.* to ustatus") tk.Se.Auth(&auth.UserIdentity{ Username: "unostatus", @@ -1453,7 +1444,6 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeSysVars(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("CREATE USER svroot1, svroot2") tk.MustExec("GRANT SUPER ON *.* to svroot1 WITH GRANT OPTION") - tk.MustExec("SET tidb_enable_dynamic_privileges=1") tk.MustExec("GRANT SUPER, RESTRICTED_VARIABLES_ADMIN ON *.* to svroot2") sem.Enable() @@ -1531,7 +1521,6 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeRestrictedUsers(c *C) { tk.MustExec("CREATE USER ruroot1, ruroot2, ruroot3") tk.MustExec("CREATE ROLE notimportant") tk.MustExec("GRANT SUPER, CREATE USER ON *.* to ruroot1 WITH GRANT OPTION") - tk.MustExec("SET tidb_enable_dynamic_privileges=1") tk.MustExec("GRANT SUPER, RESTRICTED_USER_ADMIN, CREATE USER ON *.* to ruroot2 WITH GRANT OPTION") tk.MustExec("GRANT RESTRICTED_USER_ADMIN ON *.* to ruroot3") tk.MustExec("GRANT notimportant TO ruroot2, ruroot3") @@ -1594,7 +1583,6 @@ func (s *testPrivilegeSuite) TestDynamicPrivsRegistration(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("CREATE USER privassigntest") - tk.MustExec("SET tidb_enable_dynamic_privileges=1") // Check that all privileges registered are assignable to users, // including the recently registered ACDC_ADMIN diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index bac6acf9e7ec0..2a8be21adec07 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -823,9 +823,6 @@ type SessionVars struct { // Now we only support TiFlash. AllowFallbackToTiKV map[kv.StoreType]struct{} - // EnableDynamicPrivileges indicates whether to permit experimental support for MySQL 8.0 compatible dynamic privileges. - EnableDynamicPrivileges bool - // CTEMaxRecursionDepth indicates The common table expression (CTE) maximum recursion depth. // see https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_cte_max_recursion_depth CTEMaxRecursionDepth int diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 31ee28de2b5a8..33d09ac853e58 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1245,10 +1245,7 @@ var defaultSysVars = []*SysVar{ return nil }}, /* The following variable is defined as session scope but is actually server scope. */ - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableDynamicPrivileges, Value: Off, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { - s.EnableDynamicPrivileges = TiDBOptOn(val) - return nil - }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableDynamicPrivileges, Value: On, Type: TypeBool, Hidden: true}, {Scope: ScopeSession, Name: TiDBGeneralLog, Value: BoolToOnOff(DefTiDBGeneralLog), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { ProcessGeneralLog.Store(TiDBOptOn(val)) return nil From 04cb5e9f676208f0d9e3d8e229bee612d85a6c25 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Mon, 31 May 2021 13:17:37 +0800 Subject: [PATCH 210/343] variable: make tx_read_ts mutual exclude with tidb_snapshot (#24931) --- executor/executor_test.go | 2 ++ executor/stale_txn_test.go | 21 ++++++++++++++++++--- sessionctx/variable/varsutil.go | 5 +++++ 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index ea767d3d59f02..d04c202e1e221 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -146,6 +146,7 @@ var _ = SerialSuites(&tiflashTestSuite{}) var _ = SerialSuites(&globalIndexSuite{&baseTestSuite{}}) var _ = SerialSuites(&testSerialSuite{&baseTestSuite{}}) var _ = SerialSuites(&testStaleTxnSerialSuite{&baseTestSuite{}}) +var _ = Suite(&testStaleTxnSuite{&baseTestSuite{}}) var _ = SerialSuites(&testCoprCache{}) var _ = SerialSuites(&testPrepareSuite{}) var _ = SerialSuites(&testResourceTagSuite{&baseTestSuite{}}) @@ -163,6 +164,7 @@ type partitionTableSuite struct{ *baseTestSuite } type globalIndexSuite struct{ *baseTestSuite } type testSerialSuite struct{ *baseTestSuite } type testStaleTxnSerialSuite struct{ *baseTestSuite } +type testStaleTxnSuite struct{ *baseTestSuite } type testCoprCache struct { store kv.Storage dom *domain.Domain diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index f39cec6f9c220..7714f42e122ee 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -253,7 +253,7 @@ func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { failpoint.Disable("github.com/pingcap/tidb/store/tikv/assertStaleReadFlag") } -func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { +func (s *testStaleTxnSuite) TestStalenessAndHistoryRead(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. @@ -265,7 +265,6 @@ func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) tk.MustExec(updateSafePoint) // set @@tidb_snapshot before staleness txn - tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`) // 1599321600000 == 2020-09-06 00:00:00 c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) @@ -275,6 +274,22 @@ func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) tk.MustExec("commit") + + // test mutex + tk.MustExec(`set @@tidb_snapshot="2020-10-08 16:45:26";`) + c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(419993151340544000)) + c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, NotNil) + tk.MustExec("SET TRANSACTION READ ONLY AS OF TIMESTAMP '2020-10-08 16:46:26'") + c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(0)) + c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, IsNil) + c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, uint64(419993167069184000)) + + tk.MustExec("SET TRANSACTION READ ONLY AS OF TIMESTAMP '2020-10-08 16:46:26'") + c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, uint64(419993167069184000)) + tk.MustExec(`set @@tidb_snapshot="2020-10-08 16:45:26";`) + c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(419993151340544000)) + c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, uint64(0)) + c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, NotNil) } func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { @@ -548,7 +563,7 @@ func (s *testStaleTxnSerialSuite) TestValidateReadOnlyInStalenessTransaction(c * } } -func (s *testStaleTxnSerialSuite) TestSpecialSQLInStalenessTxn(c *C) { +func (s *testStaleTxnSuite) TestSpecialSQLInStalenessTxn(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") testcases := []struct { diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 0b0ca7d944eda..094f3cbfb2ed5 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -374,6 +374,8 @@ func setSnapshotTS(s *SessionVars, sVal string) error { t1, err := t.GoTime(s.TimeZone) s.SnapshotTS = oracle.GoTimeToTS(t1) + // tx_read_ts should be mutual exclusive with tidb_snapshot + s.TxnReadTS = 0 return err } @@ -391,6 +393,9 @@ func setTxnReadTS(s *SessionVars, sVal string) error { return err } s.TxnReadTS = oracle.GoTimeToTS(t1) + // tx_read_ts should be mutual exclusive with tidb_snapshot + s.SnapshotTS = 0 + s.SnapshotInfoschema = nil return err } From a14d09d5c527d13236859d91f09013b85f0937c8 Mon Sep 17 00:00:00 2001 From: Yifan Xu Date: Mon, 31 May 2021 13:25:36 +0800 Subject: [PATCH 211/343] Proposal: Unify Log Library for Golang Projects (#24181) --- docs/design/2021-04-21-unify-log-library.md | 165 ++++++++++++++++++++ docs/design/imgs/pd-log-dependency.png | Bin 0 -> 388834 bytes docs/design/imgs/tidb-log-dependency.png | Bin 0 -> 360548 bytes 3 files changed, 165 insertions(+) create mode 100644 docs/design/2021-04-21-unify-log-library.md create mode 100644 docs/design/imgs/pd-log-dependency.png create mode 100644 docs/design/imgs/tidb-log-dependency.png diff --git a/docs/design/2021-04-21-unify-log-library.md b/docs/design/2021-04-21-unify-log-library.md new file mode 100644 index 0000000000000..deb467a7b05fc --- /dev/null +++ b/docs/design/2021-04-21-unify-log-library.md @@ -0,0 +1,165 @@ +# Proposal: + +- Author(s): [Yifan Xu](https://github.com/SabaPing) +- Last updated: May 11, 2021 +- Discussion at: https://github.com/pingcap/tidb/pull/24181 +- Tracking issue: https://github.com/pingcap/tidb/issues/24190 + +## Abstract + +There are heterogeneous logging libraries in PingCAP's golang projects. These different logging libraries affect the development efficiency and even the user configuration experience. + +It is necessary to unify those heterogeneous logging libraries. + +## Background + +Except for slow query logs, all other logs must satisfy the [unified-log-format RFC standard](https://github.com/tikv/rfcs/blob/master/text/2018-12-19-unified-log-format.md). + +However, in practice, it was found that the format of logs is confusing: + +- There are few logging configuration instructions in the document. We need to enrich document especially the type and the format of logs each component would emit. +- The configured logging parameters do not match the runtime logging, e.g. `tidb_stderr` is configured with text format, but the log is in json format. +- The logs of some components do not meet the [unified-log-format RFC standard](https://github.com/tikv/rfcs/blob/23d4f9aed68a295b678e8bd909ee8479e3ba0bd1/text/2018-12-19-unified-log-format.md), e.g. `tiflash_cluster_manager`. +- Duplicate logs, e.g. `pd_stderr` will emit both text and json logs with duplicate content (but with a few subtle differences in timestamps). + +## Proposal + +There must be something wrong with the engineering of these codes above, and they must be changed. The cost to change them is not small. + +Rationale - for long-term consideration, we should maintain code quality. The speed of output can be sacrificed in time if necessary. + +Implementation plan: + +1. Unify log library for `pingcap/tidb` first. For dependent parts we write dummy code to satisfy. +2. Unify log library for `pingcap/br`, remove the dependency on `pingcap/tidb/util/logutil`, and clear dummy code of `pingcap/tidb`. +3. Unify log library for `tikv/pd`. + +After the implementation, we have `pingcap/tidb`, `pingcap/br`, `tikv/pd` all depend directly on `pingcap/log` and do not depend on any other log libraries (including `std/log`) or each other. + +## Rationale + +The following rationales are organized by GitHub repositories. + +### [pingcap/log](https://github.com/pingcap/log) + +As a common logging library in PingCAP, it does the following things: + +- Provides the standard config schema. +- Provides a factory method for creating log handlers. +- Hard code the log format according to [unified-log-format RFC standard](https://github.com/tikv/rfcs/blob/23d4f9aed68a295b678e8bd909ee8479e3ba0bd1/text/2018-12-19-unified-log-format.md). +- Encapsulates the logic of the rolling file. +- Provides global log handler and related methods for package dimension. + +### TiDB + +Log library dependencies: + +![tidb-log-dependency](./imgs/tidb-log-dependency.png) + +For historical reasons, TiDB has two third-party logging libraries, `logrus` and `pingcap/log`. `pingcap/log` is a wrapper of `zap`. + +Logs of TiDB can be divided into two types, slow query logs and the other logs. +As mentioned above, these two types of logs are emitted through two different logging libraries, which results in separate configurations for the two types of logs and requires writing additional configuration conversion code. + +TiDB-specific logging logic is written inside `util/logutil/log.go`, e.g., logger initialization, logger configuration, and so on. + +Note this file, which is one of the main culprits of circular dependencies. The following briefly describes the key logic in `util/logutil/log.go`, two init methods and four log handlers. + +#### logrus + +The init method of `logrus` may initialize two `logrus` handlers. + +First, it is necessary to initialize the standard log handler (package level handler). `InitLogger` first initializes the standard logger according to the configuration. + +Then, determine whether the configuration has enabled slow query log, and if so, create a log handler specific to slow query. + +[Here is the code](https://github.com/pingcap/tidb/blob/e79fa8c6b654e5b94e9ed0a1c0f997d6564e95be/util/logutil/log.go#L261). + +Regarding where these two handlers are used. + +- Some historical legacy code, such as `cmd/importer/parser.go`, which uses the standard logger by `logrus`. +- Slow query log all uses the slow query log handler created by `logrus`, code in `executor/adapter.go`. + +#### [pingcap/log](https://github.com/pingcap/log) + +`pingcap/log` is a wrapper around zap, and as mentioned below the two terms are equivalently interchangeable. + +Similar to `logrus`, the init method of zap `func InitZapLogger(cfg *LogConfig) error` may initialize two zap handlers. + +- The global zap handler, the default log handler for the entire repo, through which the vast majority of logs are emitted. +- Slow query zap handler, which is only initialized and not used. + +`InitZapLogger`'s logic is very similar to `logrus`' above. + +#### gRPC Logger + +In `main.go` there is a bunch of grpc logger initialization code, which is not in `util/logutil/log.go`. + +[Here is the code](https://github.com/pingcap/tidb/blob/e79fa8c6b654e5b94e9ed0a1c0f997d6564e95be/tidb-server/main.go#L591). + +The `NewLoggerV2` method creates a go native logger handler and is only used in grpc. + +### PD + +PD is similar to TiDB in that it also relies on `logrus` and `pingcap/log`, but with an additional layer of `capnslog` as a proxy. + +Log library dependencies: + +![pd-log-dependency](./imgs/pd-log-dependency.png) + +#### Logrus + +The standard logger is then passed down to the etcd (via the `capnslog` proxy), grpc and draft components as the log handler for these packages. + +There is only one `logrus` handler inside the entire PD codebase. + +Only the etcd, grpc, and draft components use the `logrus` handler. + +The initialization of `logrus` locates at `pkg/logutil/log.go`. [Here is the code](https://github.com/tikv/pd/blob/b07be86fb91aef07e8a68258ff6149256ab511f8/pkg/logutil/log.go#L260). + +#### [pingcap/log](https://github.com/pingcap/log) + +There is only one zap log handler inside the entire PD codebase, and its initialization is inline `cmd/pd-server/main.go`. + +[Here is the code](https://github.com/tikv/pd/blob/b07be86fb91aef07e8a68258ff6149256ab511f8/cmd/pd-server/main.go#L66). + +The logic is simple, create a new handler based on the configuration and replace the global handler at the `pingcap/log` package level. + +Most of the logging logic in PD will use the global zap handler. + +### [pingcap/br](https://github.com/pingcap/br) + +TiDB depends on BR, which in turn depends on tidb's `util/logutil/log.go`, constituting a circular dependency. + +Not only is it a circular dependency, it also happens to depend on the log component. This creates a considerable obstacle for the refactor. + +The following code is from `pkg/lightning/log/log.go`, which calls TiDB's `InitLogger` and then `pingcap/log`'s InitLogger. + +[Here is the code](https://github.com/pingcap/br/blob/b09611d526a754cee82e6d3b12edf67e4cc885ae/pkg/lightning/log/log.go#L77). + +BR also relies on TiDB's slow log, which he initializes in the main function as `SlowQueryLogger`. + +BR also calls TiDB's `InitLogger` twice in two places. + +BR also created two different zap handlers in two places, one of which is not used. + +These problem codes are not listed here. + +To refactor TiDB's logging functionality, you must first change BR to remove the dependency on TiDB log, then let TiDB depend on the new version of BR, and finally refactor TiDB's logging. + +## Compatibility and Migration Plan + +Must ensure that refactoring is compatible with historical logging logic. + +Guaranteed by unit testing. + +## Implementation + +See meta issue: https://github.com/pingcap/tidb/issues/24190. + +## Testing Plan + +Mainly unit testing. + +## Open issues (if applicable) +## Logging code for each component diff --git a/docs/design/imgs/pd-log-dependency.png b/docs/design/imgs/pd-log-dependency.png new file mode 100644 index 0000000000000000000000000000000000000000..27241f7ee693612ba98601d76ad43d0b017bb73a GIT binary patch literal 388834 zcmZU)1z40%*EkGPA_yW%m!P78q@;w1f=EdTNGu_pOXn_)G>C+>AR!&nv2=HL_fpFe z+xzGDJkR^S&-HzCU9)rUGv}Pyd*;kNbIzHG_@t(I_YTt?0s?}&N*~^75)cqa6A;|u zyiIz8DUR2oARr)oYa=iJNl9Lw0l~8ba}$&MA9$bjo0^)K^pEg7zvJSm z`R!Y@rU`hky_2J>y`#N5CqC8CaGl}aI+1oKL7qlOy&YvV2dbK{zclo{)(Ksk+pys% zo7YKP*p!0G<+A{uC$-{?AMk{mKemH%?~{`D5_1?dw)`gOb|kER9~lsPtJ*4H{@~sf z)4kW^oiYTUY42}SegzO#lYAGxuN82_{iaUs{&z3Q`^>id^LJ%V?y&L6-Fdo7%%c3_ znNry^!iiBNNz(hD2aHed#J1o2sB<=~JUgW!AS594)r25a?A@i6ptHD8Wais%I>uHe zmA8q7%z5@TfWAQ2F79#i;cQzR&GcB|{S!A8zZgL9cMdQC5KDmsS4K8ur-be0{D z2Pa#Yn5^}*pVjZ<@oTPlJia+3Fp$xDH}EaN$3kO|KXCzaH=P=3p{Hc2s!H(U2EI)| zOvpq)asv_GWEn!{|AiF@pAiuK3xA7%Al!z4_&+piH|0Mm{wDv!^Y4=Adl&)P&C~sx z?3Hutf2fJ0bBO*2zQuV{M`JGlk#D_Pwj$elmvyAlx4J^3dS zDrr7GyOBRY`p?&QE{`qjzIg3rsr`5!q1QeKibpo4{*DTkMXy`!t7mo(?U zC?s#-f6)A#9RDJ5vy3kkivp?K-)?dWFe_0rLm>)%5DN6tG7S92E| zXEz%sM~;8wnwmMeyGe6${?pL^rhorW3oo1hYst~|Kd*K3g8cu~@C)(@@c*yu8&;`* zu#%r_ye#bX-`P0a^w~`tG9s@;rT#_#|Eu}G7XOP=@BcUj1w~)|H|u{@{eM|?TrFJW zog8l3bd&kN@%j(*e^>s8QHuYcSO2fB_;)}53wzViGIyl-|98-2?j+3xz;DKp#pa#5 z_Dy-CW&b2Xnwy_z|1NJ}BDQAA#{DS*f;R+8@8q<-2#;Egs-Gyu{?`48eY?R-Kda1X z(ePxn6Y=VS=GPZds6TH_cR{0NrA!BmqWN7X?qb4keg$-Zmd7H#4X@!8kV636q_S$t zS|RQ($xgk=!!UB=K z+hPY=OPPX>ch_Pam^B2be;|~P>^Rk$=Q3v=oq~}@ZQy1X*QL+{bZf)z^DLRFtC!A) zx%jITSzP=lb1<4;{&PjIUDqqq6UTo4EsEh zTvU9`ZZQB*qyL(0P`-&<%Eo)cBxE0W&Ug+T6ZjO^U-JNgFVU|Ci~D-CPT!N35p<{5mf{Vui{uN_W@ylzT_c_AvLuc!t<1RJNb`gONN7U?RX)Wh|+ z=<}Z*^z^N1U1ShWVda^#j4y`NS47Y2WxqZju=5=mDQ|uOD#AZC!D^Y~N5qQiS7E9q zGQMB4%TG`{3!9kg46S2yY?)}chVkMLGf}18*BNvD!PnRi!YnH` z>)>xvB^lBlC#!lgCG+fUilv{ZTF!@IG@`O%B~>ZEutY9kY5RC7S@bcj4jvZpCDOVD>Yt-=l)lxK2! zT_4U-THn7u04U`ha0ns*q@%7;6X+=kUou|%Yp9TtPV9NW$&3_aPLa`|t6r}s;}X4h z>ckzJZj)cB<|lh_I&4kM7H!V}QK{y)X=jWTUE3n9%2qD_`Zt0Nv~=3?;VB{VITiNx z*qH0pInkbuV{B~CAxw^01voo00;;$w<6(=YfB`zQcdr8zOjU(=2HuOONL&QJJek|e zwgP25Ug6cz)Dkm*$l#A~8fJQ~t!X{kqZ{R8yO*VUJBUln0n3kc84sxZ@%gVjDX)?r zmFT_utSw)XvL(y5_>B|j%!3YK+ZCrAdGGaFbmPd~rf_vJS+dgUkzME6SWMoYPOK+9 z&gn%EOy?@XvDslbaKSN}l(O%IVR};l;snIA z1O=aUypi3e3Ac&{F^C+&(D&hO)L`&@25=^}Nwmyv4P`Qwwqxw>M;RwUOp$^q$$a)W z@zn8xDP1E+fI6Z?=z+JF3BAD`G8Cs=j4HF zzS#c>nOKA4_xQ{E0DV)ycGZqOqeX7YkI+E_gTh!qWX$l$);<%nFSU)p{F*M;euon{ z1)DmcfF=XeYHSs^1$G{vJK);RL~)Dcx%WNRO7LJv&-MBCt-@6=|fND(Leh%hP^yk9zqT;#m+t5p{$cNM}a0QQKV|^;yUg|P&7uOjR z6q>yRZ0)HGp3&2|x^F;r)IPNx+7H()DPW~2$tV?pw$0CHo>Wt-*lDo3Ta>34{_J87 z$2LpZE9$44u^M zK0RhXXa2-9)@h#Uju(=jpi4(>t-4FQU!P2U`OFDr>C!lvjVWW;ZhO(51|WY~eSX8` z0;J#Nw-P0n^fp%M9{pvpwyTalA3=|Sa7fj)_H=V@d(N{Ke{y?PwKwFk18jH0L&3s9 z%lW4S3ge;mC>X^M>Qj2SaDKzZD7Kf)=1aXQune|I&OiujJdnNFLC-F)yij9J!_r>A zA7pFutGfS3m2{2B8Cgyf=*ynkn;@KcTOiSY*M`dE!1s(9^%C@;0s~)nk`UCnU;nxg zQS|Zd6l;XR%xk(zE@zdCK76qxoAzxT0ev>&YW;=l$_%4TE{(^3l z$jaYpPE!x7pq`Xl>qk|6WOZwe|7DdYsykspJWTQHQXW7SNXNqdt!P2^3{y_!KBUi{ z6fNuCVd|T*MCX0U!!GBI&VA$2x(bCrhxmKXLIRsB=&8YQ5)Jp!a5N*nRptrZuklL@ z+qJhQ?QlBJ^Mo*$G(S}_Kl7mBy~JnDpdgtGUzvcamF1N2jZCs^K}AM4mPrs?(e)(u zz->m-kS6<*wJoAVfLtPPlTk&ch#ZNKHII^49ZX7w7rz z@1K+YKB2F#`ukF%evoHDT=5Odk4;n#QTQ;NC~RbFZO*H&`l?qB@nR;fIpxYtSC-V% zPfQW`nvFbn3?-IF+)bTa#CWT0cQv(<(L89LzRR2iIENgK``XX%WQkG@^VrIAJWQ?g zpZgwNi8d2l(Oai9Snq{se=f`Kf%*qE)Y8S=b(HGmk-1>s+u9!@>eVILRt(C@{>tp% z#&NGsJmkf}phC8JTc zcm)Up?kTBdi|nWB+fxPUas~V!AmY!kdaC8KM|}z~k{0hpCnG=TZWv7bS0c~(;kbKO zoi27-CM}O20J(QqDxA}Ye8wrhitYnO%lcJir3pj7DTc*_?+1}*(=%PcF)RlIdv-6~ zK%-t~J`W`hS(f&?t`}slo1E;Y*|0hSR#;KsTt4;Goc48PhZaqd{I9S)R3NOgO+UJ#{rla z!ikD8*+hof&@`*8YPP2E7MaY-ZHTZUuJx%U{uTZjC1|<)x4NR7)Lk;}RjxsV`|&Y~ znNTkUroL1htB5s==cjHzA>S_LdDm80>mbsEiww58tow=OGR1P0%gQ$xDeM@obictn zc>a8^8yN7)cV_k!9UQ^8no7UyC$-i?P(J2#MJat|oWE+P`@RI6>b|_v8&$^YV2uPS_HHTjeh&%)C8?=JCc?{Abe`OC#qqO+)N)&6iFG?DgXW3jQ4%M4G;UBOhPE4VT~AmL1% z8duPM-U&QkLMpK^3PWFLVWMxH?C7HZ^eoiqff=XUILA%=bc)oR9~P;fxYLvft2>Jh zu00DS`}@8uwEib@7gHHqHHHxGM-Zt5hI}N?+`X-N*Vg?SLn0{tnw`ArY>QyPA=Qa` zr(^S1CdVp>^>3x=Xt>8>9X)uCgHyRjp0zuvHrAXg-SDK3^VY zKhLleeqEcA3n&VPxgWC&E9n6y;@I~b%otV7Qn8L)yPPl#uoc|dE}84b`n#~%tcyC| z-N%(W)7xgZQQG+H5Sqlir|fvj?ZJ8il8KFqIc;Ifw%Zo?;aTr*#_njyga?*HpGe=E zN*k*zwG_uNiX+ylu(hS64Qe&rc}t*WU9GMa^vtoRCw4Oe)F4oe@|Brny9sHVLWaLSW?(%MW3+E~Br-+W>7zO1Do8c&l{vyO_TzDg4q3$U<}`ATXbg$DY7u$r_>5_ z1^2|vC2ajbqF4!G6QykQKic{q0{qcV!Rl=Ksi_J3R+x5G43_!U-!dyo6ih&*4nxaj z+C!7QM>wsw3S4A;o79V3mCcMh+;O2$ZJSpqo|)pe+Y+1QsGSQ!bz0b9IrFZ#|F z*bF2w&&A*cJp6p!&-r>!+^Jq7i>POuhw}#=pfd%I3&`SoYR}^AplNH}I-quclB1LNd6 zzbDCx3$0oiQUTiL#fXo9f})r_Z5X0)YxznT=Xq<*_J&&eWxCy~HlHV5zPA+(vX;A-*VTJU#U|T}4NY%yfPZ~!3UbcR!$)O`deUrJ-hU_FmXH8tSWfN}Q$gyvJ zl@X|tXl&5H+1$u2i0o4?AxyCNAr?=!ZkzFmog7&gzg70(XhW4k=l7{J5m}> z?MuiA=+#4)%8IAzx>~PKvx2Xi1;67L*tZdlWZ9$%?7#s8b=yD(8ZjVaS}Gt+<$ZbD zZTjbZUR=Pu;a5_~HMM!zt`F{B;m~#zAN^GuZ#xxTk{!O3tx>X8gfDl8&;>g3R4yAo z8@B2^>)uG=yFR2Ba}@&$ zDVs=x1>?VFRXpMTywSyMimm7S3)BY5>PG*`CDk>;Iqy3XdLQdOp8WNm_ZlT^5nbLQ z^um-eu^f;MPMs+&;n^9bK^`!Y3?efwCoXplV>iUI(i7`yPN(B6s5wTmr3JzXU^kP2 zj2IBzHP={$3%AH72Jt+YxW{rY`m=MSZH!{)AcR_Ylh zPbX0Kg(e7nHEGhL7>nPbu0w*o)y2a^yM6>|J)~-1v^sddm(sXkKl@tT3}TGnc=m@V zuY`S|2q{e=R(0Jo4!TPj`+MKB&%oA(-D3HJDD_7xl@(Mra7W-9ORRlSwC+k(r>E3C z>K^HpreC~oo=7+4Y>i5M_cxO4JKLwv80n=wPCnZot%#<#8t2d<(Jh(IOQL!y^CE(c zr>{mXhNU5=7w|NWXzBc!@0)Jr-dkF8Gyb<%7)pHuD`G|SuMU*hJ$p%B(?w=~=06)A zz;>_@!}t{p72`q&MVz`a5MQn4`2EfCblg|8<+su=b$y>VUFRicy(d%SPaI8|Q?M)r zN_uT-!^F4efmE`<>Aj6NPyYxsezkX`Iw{pcR5tU&m6OxZNfmBop>^heoPuW#WCDMF zKzGK1D_6y;>y0F;K5@E7MCLQYf?8oYlt)%it zyHbE1>Z7nBvTuI#N>aCf8lx5Y2^%JglO``$aO%)IfU48my=`Gy;2OnH|Hvy#11sr4D;t)GC+NLVdPc>)Cfb6>=`~_x8*4y+majG!Ti$+dGqM9 z5kbL4aTplgS!B5TA(ysmo<{RpelDr&$0G^b@{I-TmK0knoReb+{mArcFHvP@#k*(mGTie1z&p9Vlo0uDZIV0AfUIponiy$`Ko9SwgV9KmVRgGpQ z7$TWzUHzjVEbAxOV5yWEr_5|8-qJ%BmMZ+sK3gr7Z0{R(s@tQjD2irMFS_8>T!0?; zM7F!>yz`HyI+N$ZT3u}VC$wXvB2DD|6F-QvIv$-ib$9y4q{V(zZ7K&zuC-rwmS6pS zUaNk3z&&5}$?&gFpcVNQQmuu{tzI0c>Ze z4fEd42FRq&;Hb}#ihGp`W!2;#)1(5Y{Iu64A5E)eGSxb@-X~ub7cd^TU{h(HUifW$ zkIjd!oA(a1WB5I!CkpCjRsk~Bwu1d}G7NM4D|<0&I_t58`w z|5|*#*~?DVN~{_?P&87c+FDVjHc(cy5FN|GxpbGjKM}!tf(!k z&6In)KkL#|GVw8R^Vo6e@&UJ9@_^xulfx7Gx}+{O-V&V0lKtXzjdz>#JbSw1J=~ua zV7j}#2tPX=?Uxc~YGzFbcTfj4_qRi*tM%Apg-2~;)kqIR_^y4iVUC-R>K}C2p5o@s zGnL`Hy~EjikI`hnq4GGI(sGz}#sURDex02#BOE$!*uJ?0n`M1?KPXJQ_60XQnJ2l3^yv|asrQ4L|yw?Q>Q*3%$C}U5+7Qu zeYC(6P`(yv2VCl;ij~GDE6#}|va$Sh9Y_}=)An$7{_`R3w3T!c{s%S-&i0*O2&Z(& zP49~jOlL**(v&^!j0xlsNERindAFG}dHwV2QjXmeBFt<|G0yg=qt_T-5!=(>z(Cdu zlplH`SiFEC?-{ojVY#ri?~^r*{%6Jx!(dbVS`!= zT*@8q)!&uJLE+SJeE*k85p=lPxyrd3Xh7siGq1z7zV2_jJUaF0nEc56_xof{*1+0y zCgAn!-A~up#Qv@xapSQ0*7r9$ommAI_S09Re>}3pMjvZtor}MSL0xIHT)jlMNe2O& zw~Da4x9Y46MP;BHvyL1b{e_t<9rlxT1_h+sph%U{HB<_CrS=VuBw2xq z-r2g3sbbYa4=IUMZ$xmmV-7l$Rr*P~k|zac8Q(XAz!jCx54r&k;Ja`B{Euf=c4rcs zX#rJoyXwQNrr!Y9=e1#plTGL6zC_>CUr^Gebu2wgZd7*Lax`1iJ?K5@wTF-yBW=%5 zbYnBYQW3*S<%i(vp(gUk_}9iqR%Xf@lBvPr^>cSIqL|lV=MQvaN0tIgAdUEmtK%_> z1EFQVxM8w_J^HxnX#{*S!JWk3oP{|aeLq&^7A_=S@D=`N(j2`4L({hj^bfV%vbf&n za8*i~S)IO)D8c-zGGNaPz16nA*ONK&UWVuBb`aQfTD%QOvy)uupnKQi&D#o|iV*`b zs5WOsy}`^?R3(sOcQeFHcFDPGO7RV>BahPObb1*f%J zDvxL7STE8VtNi#g0Dt2*!b_OPX*qfJ2-;!qPxD$>vSG2ed>Fm)MOUB;2BF!wmH`n& z7SF}`uj5AWutbM|LUznl;owmTTiYnOjljfHy#w*$`14v*{NWOKZe}jF$)R%JUq_tu zIOx}pa1RN`VKVABgAPUI@N%RoP9z1_iDrVtv@IJ$$LxrGW<$$Ix!pmd;!y*uAQEle zYm*mRE6Cy^XdbVX|3g@5dWnJ-x_4>bR>v4s>ku<)-e8^-&xh_aP}s8aKNkx@6xJc9 z9CpG*PSzG=C%ft`B=bT0#I|ZN;mop!_aKhUB^Le4Ux_yI-+axAn83IGU5o>*14cLQ zg%RamWXisu^#1m$O_UD(($(!auu&Ol7j@G617FuDI8$GKT<>Rc&7~GL^HTO!Z`lx% z`}8L2WL@@EHL=jshRWd7kFwHb_DK(V66#8zU87KQud`C>$+MQotHKk|n*Z4sh*OOx z#O=(Dx=IMYiqv49L!|5dkz74lBb!BqE=7C!tjs%gLCoVSx^!Y!sEI)EWf@$lKN`wl zSA%5hxbfrpe12w%pUnn?&=p^PW*WMzC>?ay8LN<)H3gGf+DJD9L>e`(v~mlSMDD5Ps0>K!bkEosn`_{{az;rb&Ed9u#L;1u5NSY zB%Pvu`ZwWHb1GkIRd44l7oib^<@P(3s9Fn0a~Gw_<(Mxq(NV{S@qj@yf0Hj}$a-c$ zsgFwhB)KSrHsH<1;u1rcl1?3{P{hjIVjnmH5nUU=baAK=)1dSm}{YF68$-@DWlv@SutOHk=oz=KArGWYMCRpe9S{*(7jeTJe*k2)MKTS(DI5Y3ltrPy}A0bepi4?)~{H;_19k~NcNgee{Su3qnYJ+-o0W(v5Bl*4952DK? zzk%lQ;$eT37f`#_&0`91J%+p2X6p`(jinVi1mryD`>Gkjq~F{V*L;d9vJoG#MeiTT z4pglW2$1Wz451hjMo#I~>|af-%KB`%#8Gc&fiL4fRRXeF!Iw zVX6f$+bCS=h$K6M42$le;+s=7Rf{+9wtz6JUatjX%VZDZ7fv$Fct=>w%dfRW_?h~U zZj&8iBEGTrwd1Zbk20#k9FGAHPW2%3+W|tG+PsTZ2yDR>Z;mZM}jNUsL3%y?bg) z=gD16;rrQ-zHRqX|95|9ocLIErbpKr=#9{ZMzIIM9t9UvzMi?g1)0q+^ECcerB3hkO}_VSW_;h0X?S_Q9$ z?fPGH$Ami*hx1W*93PH0)XJDkhKU|ve&mB^{E!8}^je22Z-xsw56HgB;_WYUb(sFO z@CJ8ThG}&`$RIow5K_y(e(be?mu#tJhKA;_V!Z9Wgzs1+Dqul??9oCcEZ}`{C8bf(Rt$P@%6; z#?W~+i=emp@N71OhMumCVTSSS9c-VtMcOl}^V!$KP7GeVlF3sj4!=iy(&htcRE*}x z`4ml4m(e9Zh9znX(9g%=FxOLf?+2s$5kTL<-p5N0_177VYd=9dGyxgnqk?}<51QVt zQOF6FCHP;nmR{E24l9_%+mg=9;_lpG>w0(-Lz`;xh%x)jmtJmqI|@8;+Mc%J3kVo# z;py8aFL0*$A%KFt_-2udPl;V?=1z3 z=_+#YU5c4cLMl3&^QN@Re7rH1wahJ-=b1}9yo*Ze%cX*yT59ds6dcU=T#me=Ie2mD z>|Ch;9(N+%n_v~g)04ou-qP>od6(j`Yw#t^n(y>0SJ!7Q&gTm zbjJM;D2WSyiR4D}d+t=BtKf_ z(U;!}%}U8UX^X~up7&as?B=R_LuucAYXTepW%s!4cyk`e#R>QLcKNpsRnL6)XL1rj zEa*e1xpU8jlV&Y=kDa|4Hz<7N-LNhUhaWo$B_%TPeS}`zcU_v9Ny0M*?S2R0Y>JOW zG{jD_#I~ij{mzo#K?@izT2=0RCb1Tz8+NGbF+3NGl|i7w#w@PXo9h7=JtBrj^gLK! zqZLUw#Hkef2;MLaOu=$(Syo7Obccw#w7ojR#pmEOQWa6NI~*Z{*1Br`LlX}AJrPds zw(Ah7{@1d0q?UXa58QCvPQ@kY10ho!0tS3~ z$LBYq-M26JwVGa!>h$NNp0GQ6!$`9qeo}$D^yq0~7sY^gx%IxbR@ywLb_edae^!7r zr3ZT`XQwENmi?)O3~egN$*dUGypnO|xt-oDI_=PCDgEm!{u6R9cHO{0kW%q_Y$!8^ z^gPDI!b!y}{!8ROX~%BcN8ef}BX7Iy>-W?~*X*#lx{Er03PmI^t3#wIeC0Nj`F1Xr zXsT*s=+*;wdX=QC)r#7M-mFevD;w=)2Kb+N4>SG7PkAr-iQzj$moBng6(KS!Y(eKo-5%1~H)S`2N^FZ)rRRn15)D9utl1GmPZQsA@rOim~+iEy(tS$m5)D{aoF< zr4mI^cDhp}Lqx4{Jr;GzQJ_Ihf>Y>ptHEPWk)d zGO_|ZhY^eJ9ec_jve!i}(k`;&_jR*z}a!F}St3T|pIea=jLR<2AUOR}0=pr!Yk&+XXJy69$DmmZr>>L<-TDpW7A#_%5=u=$WTRIKgJ zpC_+oVa2b*jOhK>AC$cbozv$(@)H3lwxFd{S5JPqO3SfNdlx0Bu#QP~`+YI>DU1eQV>F zjr5PD!LrIz54X)RZ%-;N{AH7p`p-rJ%-OfWl!MOS>Ogxz%raVw) ztL(ikx(Y6E9sCZ9P*)B3EHDBWv+wM}#v+55#~C6$);KD@L>+kG=ducN--_;srTwa?Z*L;C8+@UV(*c=O5JF@pz&^m)kn!#O7J z3Wdn>xutQvvQUfx2rC9)>C08s0Jt4mT}wdw&*1N8iLAnCPeBjUTL>u9KyLXMyC^%X3VKTAvY-7PKR@r?dbVaLI(mPF!L4;X51RD# z7kk()dCo^~TiK#o$mM$3+}*ZIQ?|}!qs%MqyCn-I&gEbl)f;>oa|n5#ILSZCs#`5w@#K8K=o95DbI+w ziN4meSbGoY`tO`#(&ax^2ST&5uVfT<1qwJ%VkX8K<15ir(E@UGd#FlDa7x9exyzUPn6 z3o>0_FdDWAzHoq&pKiUkp6i-VD)c(?)gg0p&1qnX+B{PpVel{x7nNWs@4v4PjHy8^ zfi23)n1gE#Ai-bWqefC>#q+e5Q@y(Gzs`N*k4MlpMNc-RvipkLf?ZR=63l_bw|08u z?>^NUU0>&px|K&+gsz9?f|wlO;XOuXF9%r1XOdp06t9UK8q3MRhGYj9>Cv<3Qik{v z{jm&YCchEU<&T5PE21VBZH|u^$a$JBYd)57@A$#OR{@hn7Nw{ksNC`vxCsHn^T?9Y z`Kwc=%%O)Yf-Mk%&N%O<+@W)RTl?M|hUSD*42Knh$%W_V-PeH1l1of0eF1ouSmVj3ihd1Bm~Nq-;X5^mY=;dk5Xq%!fFY%Rx>Z0U;~^lKG} zUX8xm2j&gnajuN|Ajjw`z=ITs$kmk&p?OcTa#J79_v@V@lcEg)km9HO-^Axoc-;5j zya`1GPe=MPSyo9JRE_WK?%Bb9Tk(PXaT;MyozmL|mI|C64p?48Z`>yXitLT5Q_WA~ zTFoY;VzDo^o39?8N@=&FSpQKEyrT?$KXAL@ir}*+wK8Y@m%dh^z3|x-IpJBV3~Djq z)p?J0u#1UfTLGbBdF8Q~A&Ieg+9f!H%(?eFh}#oJ0~t~9f#Cws>!vN_-Y$xY&t%Su zLHzCRze)SkIOBS{oq5_@SZw(3(2~&n|U=EA{0WY0wUN z6=Sl=cmks{lCB#Im3tD|*|YD=?k0W$I!>t#3R2L3m7^yT`S;YsA-vBVjXnCax9F%0 z_=>Jx{OWs-o$m_wzg9P`VeO9B%o^7HI=TQb5A^*AV0q2%Y+!7L`8dw=a+A`*CE-apxsz_U*st0RV=XHcXDsuc(<6B%@IuN%Exor?_Ei zD@Oh!#BO$4>-E@ioHz?bXNg;u@n$``cHc{i;P?GXsv93jOY%GG>>7bVys7LvY&wtF z=#`5Gqx;hApBlgJ+(}qz3#mr1fZ}ZZ;&vQ+&p~Z-gVokqgaD-YHS~nOHV|??~k`Mi5g+|*PoNNqe|E|bcz97 zy`}d7P0iVugEj$LfE2hQB92`-II|%gjJCodoCuz;;UEu2zhHYWuQV2&^u9Xz9~Xhq<=Zl-eysn{l-RnXAX0(UjDYkkDd`avcT)1YR153!mle92I_ z1Q^X`94M!Wo{Q^U6rQgFz)z*W8uMo5Zy_&{iVMff;+}3l{#kQ=knNa_n$9xV?Kbk1 zPamIeVh`5S7KA`tj>psi^vl#*Vl&j*xLlob1Xy11=y3%E5EvBQ#{o?v98FKt6mouuc|ny_tsqD zKic=`iUy}H|M=dtc^%wJ-F`TyPIB*0MwcuW;nzdScUYF_1MYs`TAXi^YC7Q_Tt{um%cdfH~)D3B-XJ^<=LS4T5 zld>C^!SYoXIg3d4>iVVg=~Dwh)*-Vt&^7pwq0)o?f(FaXSRzPyU2$p~-P z3lFDr=+*Oee`yz!HEG!MHkCKm8~!uAmxbw+UUVt>nHxK_4J_h2&5lyS_g2ZpZoUUF zZPn-XGV88pKXCFBrItd$*6!T1p;l(A)3UyNORUb1dWk=3G=&J^ponm|C(;V1M<$ze zEy;6X?#CY_)CNover7h9{i>vE^{3QVOq8FXZ1av2w(hMlYjU&k3zp!M>}p8gv0nZK zlE3#Tss^V#=i|M2)-;yt?|9oQi*b)4E?U}-KnvkQ6S!CdmK=2&V?Xl2JqTYgl3b0P zb0Q&8Hib;_kG}ny!p7n=n8kbmT1n}l+SX`=pJtG6@E)1G8q8v)hVsNFcRuT@kwwg| z*w4=90wW9$!n1oOU(l6jMP*{o%K+r7hC9tr5+kpR2d>GFb1z@7gsqhCSA#Pk-e3;! z+A3uIbVj_g5P^JuZe(@}ECK(XDY!Bc1%Zk4>-so)eDU!R1lv2MP zOc+)+?HEs!Ro=gL#{N*9X_E&|QnXor6FRJs0jDg3JeIAFV2&a8X1&rMvvF(8Q=*r< z+2V)<8rMBM13%CY7*w=53GVutojw?DTs;~==qj8Y)K6wiInYz3t`Ogs1B~|WIhGtX z6F`2G?>uz)J^HEeHOg$b>ZrzX=9KKh&sm0_r4~xTUa+etJK)XsSx&eY9ZHgZ{#Qm& z$Zk>?Xeh3EbMuB3pT0YZ5)5d@kt|F$5*T1p8MLszET{Ce%tN=bnwJghF^bFd@86e= z45aE*$0kz5wMRWQZd^-&Eo%=Zt#_Z|_c2#z-0#mjIjmNC!ub|3K zxRHkz>mQ!-FGx-FdU$?PHH2WLegGub?TGld1o2PYv^te9=E42Q&H#~1D|DaU#+ReS zSh$+V?f%`+MB$68yUT)R3W}kI7DFF6V-LG3469<>V{#XG93!IdIC54!mKqp)qp1YD zTkn_BR+XmAdYIhnZeH9Bt$ptNL!YOcv*$AK&)BIi#C(&Xpa9S`dns?E4>i!uhX~wn z-&-rycUh)xsm*s}hMbDWW9xsMYo;W+$5Z_JjdI8eZmQaP7@cOUOh32bw|Ph}ZE+p9 zxL-$^)9`{hZcp7L8=NSc3x(%-lnoi-vpn4~>VID0^-t0WvzTkZm1X{Rn-qPrb>Mcb zrtHb)le_jTe|;Z`h8ZX}3pSQ(vZm?IY4?1-F<${X3@_1wMF`g{OU!vUONiBZ1>4o7 z;1Ai)m$G~@zDwbNDz!J6VDNe-c4*lbZ*M6S+bh>9HMRQg<}``fa>$=d3M~!0^ykzP zaXZ`;ws6E25_x{TZWoOx3#zkUWQB7*yXcH2pGhvqU*DXNR6iPe$+LJe1oTUnb0BVV zBQRNl)2cY9I7o~q@XSc#lh+tveggn3MfHw(=112poIPvOJ~OqX>%-qo=C(%uP?BlO z8$Fg#UhJ3udlQb^S9$dO!R&XJIqWKSirhwq2q&Z7BS5nAoGt6{rIuB}iq=d|uc5wcw$))8R}B}d zJuqUvIctdvhxoC2;WwKG=+afpmIXkX=klnLjo?94gz;CpG=lF_+pug(!C3U!JO_a!;O5q zBeDf-PyJ#>!dvZ=!x0yW{6_);19~zNysu8@w*dIZ4z9^zeF&JG!gzY4Hz!q}F80oM zF1%`*nYLAFI0eE;bnfA*+7sU`ijpwHPmy_!OTgIVUx}eo4PSSv=F-Hu6#`!DI9Kz@ zb)_?|z=sr?W~_jsF|O#bu8&{Z{t`&HqneNT>Y3Fq)9ddrwe#T0*NOwpU-w-QW1cQl z&N^-Ro{f7yS2L!nmf?x`8LwHHW)Q6v-t<>LrK`BAN|qh12)|u7mmVcE`}5YX^>sy$ zt6|Ac11l!izRJ;kvfe}nR9j(rqSLalUmUYp+&y85XXC|`d=p_~8F@0OvzP&t$n3jD zqrY^+-`=rolY#7kDX#2+xf_^N{^eCGb`}}gKY?1^Yw=v)%C~qG-Pf|#vSbMj$1uoH z^OjTq6)W_f1YNCIY=T&FlCA)%@`9>EAF%W4FL^$kDG$iMipIRt;k(-G<@DYokqX88 z{<2%k*QH1Ac0)g5(}}EC3THF^B-{iizX(Eg%k)mXH4P%P-r+i?a|x zmD?5Hy{xnc_4lOF{DK+J__{pdJyA8?bS4VFu7;K2h5dICb>*KPSj9UI1)nsr;i~PHG_58!R?$6s`x+h9$qhk0K*;+9LM5a1W0tu=pYEWDO4r3lHp>kng0l%y1}N{T#9S{{WvrV84gwx%<+Syh%Q1=daH{n+|0o#b>?p z)bq=$bw2-Sz2zkEVif)?MqBuS>HPMf;IQ*b5BYndkeUmH8{8Ds*t26Sw)6^C!i#o0 zu~xSY+1XxT*S<%HJs3T@t+PfH&qTx#kdDWj)@%V z*La54;gmo*YLkHMM+q?IXxv)AQEm`3t>-~6bmzd(aZo=se++p##n8RmwrO52L;RGF zI_)_yiDP)oRiHX;mg=D&o*o3n{4W1p8@7i>{{6xykn=5tUn1ZjwwEvb*jGta`#}dG z{MOH$$ewcV0s6JK33~rvp7`?82e3%2z7(m$A%^h}${)HhC4N;OFNzZ0%pD_Gn^j){ z98&mC!qkQoobx3gw1HAumnYJ>BfLtnm&nbtdY=a1Yh(U3j77dMV%X8v+C3Xip0FFG zu_YZo*zD`de;C){qtTbnOak0O`#SMrFfP~DKGduG`q(vk8Cd2?4&HNS{Q>p7XdJBH zZ>7?=f3WC!ZD2!^tekiOdiF(xO~=tMtownke64T4vC&<>eW77az=h-K>wZ&)kx~^VYv<)OG8eT(LJ- zVLkpg4*SC6DUGF)M)V9;TKReNHje6Uy~j|&kl*KL{T0_3>;E=2=48{kBIo(w#5iXi z5@vm=E=OfOe;)I0<=EGt9MU1G*cq@Ol3n*R2T+-=2E)_ibZJZg06+jqL_t)byno=E z^Qrb3LBO`J-PEOfuLpP?4xGA<|5&U}ibsy3UN2oY72GkDX{e@@wlWZ(hs7iLN^UcHL@%{eckT4w@IB1^da`v&G|LY*W$i=;%mrLr>wU)@% z>j!~K*H>|Hut(mdxU9an_6+?~6G{0$^plqjSgy}nyT=iR%*J6U?>%<3yMM~hm^`f1 zZXO!DhwJdEMTfto2jAqR^H)>y`IQR=_T#_yKb=2dA3W26f^yHBX{C4Vy8vVV>+_G( zwddBJU&?Ohz&V}#Hs_YbbnlfFgW_&m-}SXt!f373mOAb1pvT#UO@ShAvsCtV32FAZ zRLrl>Kbe#FK5U-I8F^fN1(N;x{FB^tE!}m`&z@hLTU@{C-^qDN-U-GqNtQ~#Ubw?% zoCSGwtj;Vkq?xYn`Ypy5>Y)4fpc@xot8$>kz7H_k5xC_CJ0cIzjk&!y_B;RzS^m&C z==dq3ijF6T!jVP}4_9m(B^)PT_C-8nTx?MZZ6Z>IeqSH}spJKsu|&pwD3f#f=&;Q; zT+CrDX}a53+-dyaq91VtzF%XLt$V6@v7%(Ol8@oH`L^nd-`Jbx?>;E8_dx&$W2Mp) zGix~!e12g>ti;tg&-f?QrsRl~Nw&S1q z!&AS=S^ePXJDjQM+kR^Ll;$D58Jjxy{|zBH0wfnQ&t+6Y2>swcS_~NB{YL^<$!O;2R#R`@4Cvfw3xSHY6 zyw9KFN^gJS^jd*8W-~TBiewCKeYrZ??!R(n9%>Pq)r}u2A~wCCL3dOK)OcH$wRMrz zhxgpAmvXbC`NOuJ%F`ED35#{dO z((HQR69;AD+c;)@IQLTbe6Szj<7W~*aY$%x>EUd#(Rr!9#ch|a^~2tvW<(wP2(tUX z@t5ze4IjbAU%)tQo~OIqT~~8^Pui zNB&KOU=m}PiJ#WSueM9n9^W)A#@+_GUE{OHnBK7K6jUg*6j)^jmvdsDr-q2JFFOyU$xW(1f4+xV0etzJGt5fA3r*$>!j;{GqYH zw2P>s*g4-;f5rhWZ{}`eF_4#M=H2EO2&Zm;&J*Ip*kH0?u z%z8?`SW}c=pMPe3BpumjYO{v2|3QImvfvH^Y(eaa5yFzy4Kws?&vo8 z{M)}sD*uRyq6dZ7v1G_GykF=nvVvAh^Kea4RTDhk(>#jdmjJp@=H3^z#b$y}_wsLB zVl+n-{wHmNj(knsAeux}yZR5Vdg2HpGuWcm{Pz;bSwCEoBQV92!;C|0b|!tD3RfRx zrLO(`fhoAl@HUuswfBppRS8DKiQntS?Q_obL!(!rh!Vp+-t&PC&c;}3WakB3()_B@ zkAqKi&Xuqm>%|2J#qc!OPr8#!J2$hJTGwxEJ%XNg|y;W6l?qyPNQEEj)yTfcdie-vGZ#jjohAb3xWdvcjP=Zher^@{`^-i3^o0X%u; zD!ax=pfYZ)2eaf@4z6`LW(8b^jR$jcP%AFDtV39Z8g8Pv%jgU1`P;cNkMQVwVP2Yj z?|bQsDXu|f4JMm}s{dGzxRM(pv=7O^X{_7bJApXzTyL=YRI3*!_5-D`>o5X={w~;_iHLp|_66+1NA%`ER*vdgSKiac;S(Ew zb(C*cvAqsWHVrRLi+kz8J3er<>p&fIN#-wW#DTtlGRUD!zi>5Okh#1Sl@Gv&q%iyi zuY2kK{(9;4hj$@@Q+(r>|5ieMIZkzMmuJEyA0PPf7e-hPPycU7%QigbRLuC`g534! z`vq3Hc`GLE`!Xza@V=ke(v1$@DfM@c0cfvJoJK=T%nRe=1Fr)3n*)$l$449h=L->I zOv)|At=qB0eeff`*N#8v+9392X$|9x3W9K;BuV8&0l5Oo+OF2ya4?|NJ*63}-CkpZ>S~gPeTAce}}05sE>V*SrTO58|cS{nJsu zdAbkFeO^eFOJY{e`LN+atl!pew7!>zy74(A1Qd1JjG&RFR~?WGe6K$*1!8^W-@X{%l+!x%hL6S);#-pOn_@uDM+tA+ONO7acumpwIJ%dV zH-VfX@`QnJ7s)gmTVrp($jZ+kMk57SpJ}RZ|IU|`_3vEEf7d^Tows1Z318>0e(!n1 zzR%l^XL&HDjA6Sgl7JZbaUmbZ)>FjqJ6a=3s1D6u)o+}m=NJLA`tqoz)ZQ&RJeTME zv)CN5kv9*0?QC2gL*!$7~Dwl25DaT^K4k=%xGgtm^ zc!-<-ix-r5pKf!^0G1S4TZZw~C3#r^vHqy6j!*%dWwco$ak_t?)}W8|~v!mRwVw8r{H z<@M=xigN6`<7>w}wT*rE2W{ke7eS$ms^E0@Ua>jAttLE2FAcvM8|O3fU!Q+|$d~d% zo_OZ{_QxOdq-$cC(5pC+V_@+Q3kF>$=`!ER=rLd2PZPfSX*8PFy8slEOCQwg)V~*- z;@nVCT*I_)DUFi~?vEER_@wZQnq9@;;vm0Dv}e&fm`lG+nQ?F>q_qCVlUt?n`k|-} z9@wCu_+V^|q?Xeer-?K15<&H@g|QdrP$vEpCDnd#s_qBgSfL2*P^**8w0c1tkLtX{ z3VFr9A2Nu802~eV6wBHchUdUZRN~Fz!Qp|LKi{%&W?BxhL_= zd%qCKRtbwARQiw7&H|ec8#07T7mUs8(AhuMT+ydKfdv^4qpR`QW#8g9zRfdx_})J; zLpwNC8_$e)se#>P0_gm=Zl3JZd@>qr1{9))sQU-u{f7c$TG!^3to?9gwIrKIC)gHn zy(*-C^Yw#P$vd9pFjfzY=Wuu(2xpZzx!|7s?Rrt)u<;8cIQx;J{&w9->v6t}Ufl!$ zS&NLE5jbc5XNh_b*BnvfK~djsSs5_PsEQIde~R zl-#N8dupWCtlx~a^ABG!T}!6x&B;xtJhq)YI{8+d^;Xln=8SY9ZW`isoN7|vF`!0p z>x)OUM_o+>!#oSlgiGyY4FuNh{JH-<0R*JeaRDk93AuY%s;j+#V zL;~7g532u=z+V`#e$h$!?F*xFQe*D|`Bi!*#Ag)oJ2u5jX#eU{pZD6qkxOqT#cO96 zF!m6TKZ_elIvZHsoY1h1FUxlRW%3D@u!Ls}UVG!;dP5#j7!2K;($9Fw8+BNPd%CH3 z>+%%Vx^fQ+^smo9&ZU;kKFf#jG`x=W-vImAz_D^vcw;AQSElI*W86G}ggLuf3Zg)_C5$5V63{q%HU0lF9{Os;M6I`zDDoJOg?A@6bor2>R}n}#6O2skv+7`yTFX7 z-SkcjmA7tet#=$61#K5zGKT2=vQfE%HbxTtmVeCFzViE}5RAt(WMJ%D4MZcJwvZN9 ziuK@W%(^oNeHD;r{aM85_MwD+|F^mgFfh;OG+nLN!QGp?rmc-t4yn7gAn_WJ5FF#Q zUU%aGP-YxFI4QxvbJnTtn`x5b-{E-#5sv#_^({*~>yP{pTVDXs5d-VK9}4i47#NFM z`rSC^MQin=*bgE?B5GkhP`C#g{hc=hq%=~owKfm=($#h4^(YK4=q}`KUX;CWV)U37 zaNWNrj>I$JO=n^d{*Diu{>k$FVV9}thixz3nX8gH6van*W+m}hF*jq??%t$M%+lyL zECx6>@EfB$!POHY7kJztj~CCVZ945#!8eYctA!liqNg53<1FvSI0VE%u@`~ey^XBv zIhoOb*?k@hC_En5>G|GU<7XFUT4voXpp75uiL+YbB~0_qoHlG@>?B5bB3_E0qpEF- zU~FkV%B-E78j}|TWSQR)o^jkEHkRhKe$MZp6~dcH!!V1AFZIPC6owy~OT0;-?tx{e&M zff@5OHywDx#wRBo#qGX$IlnAg3~{J)mBx5@!dC}p-^9q{pgBCN$Elq~;qgf3o@)S& z^^RlKcd=LAGaa$B@z}dP{wx2ehYxm+Y`$nA8piKSJu};hTIraEX7>+REc-k$pf}7C zBo>uU9YbuBqs;K0Cn%-_GkKW#%ln`>)kp9BqhR{@+5IEG`>xDt zj&B=GFlYai^H`|04n{iTxG}`o@_@UOHMsibVQQ~C;dV{|O84vY&+H%N=FTr2(&}B} z1~*}}cw!-qM1J-J6+P?Mn=v?Z+Q#ZL&*Lntql5nitG@Eq**A%A*Y@lkDEbbWwVsGg z-Svwn{g!{+e)!M$guC|Ly?-nh32`;HwB^%{O5>qCDDmA3Ey(vdM*caQhy(qv&p%3n zOV%IxA*cVNKmX_pqyM^V$4h-yMGv=CHr_v}4_++|h(RrHSDZL@IUR#AkD}tmjbG#s zojMPUI_SONS?uFKA0mT9&x4Sfg0vTSiZQO!#~u_9{6|5o@)k$>dG|zvmmhw%sCn`< z?Ny#e9{|Kzdp`T}H!S>NS8-0HXG~*`x67lYec7ppz95*LTyfAWE zlbb6x&z~fA{eyq|lo9pqe|bh|KSE@j@$WTQqYmp8{C0~QLA$NT+?63O1z#*MzU2Fi zzd?7MaHzMy;|~DCMU{1u`N%_e(8X-Pk+(Z)3-5Nb*137^;{b{XZi)z}{K`zIb7 z*8w+@XWj8s3ZMBmH`OrZV&zi7S87*#vo&$Ce?FW~j&1sr{jmbks}3BSneD`W9?u`* z9(~UiY{N4-awjjQ=FU&$xSyO&;N*+&uJ43+$VJ zOLvZ%SG;NbGVtg;2e7pihdz-tJ`nX;^j$ZG%9h_>tcz{F%q|XIUoXzo3eVH2$LMil^0YSz=50)Uc#i_i*XK8!_?O=PmGttF|9+{IH5vcS zccW=t!|IRsKWQ;|l{^>fJvNzPa;Uyu-f9!w! z_F3_aFuwlY{ovIi*ZAQF-fK)hjiUoB&*H`@@`p~H2Sy$Ao?pg3{{8$JB>L_{XtD1F zo??va^!@HB)b3NGbQ|YERrqs#=RY$sSP`5&P5TRt-`^ZunKV}Qe|`SB@~6CY{cZ|7 zL9;%PU!sLo?(AwU_zc#}^!+7rY)N|0)|h+#m_;w9Sr}y=XRaoO+rLffT($k^{_FG4 znWv;OS?k&M+{gd@KmYvp*FX3-5h+YK#r4d0l}zBJeUAbompO0LmgOwCev$F*!2Ia| z3R`H>_7c@WcU}Z>Ic~Qm-dw^Hj2HDRculJ?{gyq9 z@$2@rM$5Vnx|!R)Xb}T8<=BAbo1eT`o&$SvL)(A`p3EVm>dR?~gqxF9)Uve{ zwU0yf6GdaG>4y*Sy86KN-#idmaP|DQl3x>F{tFq0?bdJOsJ!*9H)a&9yR0c86dyYF zo@?$jbnE(b7g5JQI_Q4#g$*O@Lk+&ZFjp7jFqFqDO#nt;zpb0G36O|V#&~hlF#gK_ zdEt4O*^f;cL}^Pn{f?h;04=<*)>U}?>fA3p&O!NJ_1c$v^1XikWZm8NP}i!Pj&D&8{f?d!jk#`pn?u509V@63HF6xM%*}&mT~D9!A@7+f_5ULe>cD zr!~5}gzt&J<0}s~_G>}&P$pT}=y+)>=YVh%TluOFZ+YUQ`-gFL{boQNd0A>FHvitN z)fF4vnV0Ch--~i4SRX%pVT2V-Jv+h@;o*qhI=E!ecY_BeX6CE>#}B-42;U2C$1yd1 zly`=r=YC4ZdnC_R-ZgjM=YCbZDaTL zR5NpT{|FWCpwV?4<;Z#f7a!3#vdu{cuTt+l_b0BppTL#v-CYRZ+jiuUw7s&z$+}zs5E2*ZZ@`swnTp zN3;6}HPFIa8v~9X`6mV<_V_^$u(&r5eTwbV^c{}+xWvLe%gu}Cn@!lIB)($Qv$rFSp>=^?v@G+@OQBCI0U@LPIpY=pP@E zBYW})Gx4CKtUV!_A$7T;R|eI%XX=F47Rc{0STQ;yX8dLpae43YV-v>sbz9haE9bAz zKgnKmMP530%@Osjr+;IrBfDAp$#Qeu8R|S?4`KE0`zDEh-9wb_Mjedf4Q^Nb938ohm#<(70FM9py zNf*3Hg9G0(Vt2za_L7AEMts_r=lGB0whGUURD0ZP9-OBY7wj0?v5Cgj2%kRGy9IU8 z2{*|L0A@su-x0Wz2W4M44ldq_dt*0GK5Ui;+WW$Nb=0e)bCn*L4-ERjbzK#=JYuBX z0z6j?gL@umL;0|~#}8M>zq}fB?W~`UUkoImyLvTzU`odHX4I*Q*_bPkHB-imGl9MR z7r$_5RVv5q=uXRe1yKog+a{Qs5mrX~_FqS}UBAQE`6(x}0_KhT_!~+!nA#ys`(Z!p z`2`;@mvkI9&T8)X3=tswV^Co2@BNcWG@`t%5r!|4zNozO_gp=igLQk>Z+Iq<#xdPi zHN#u4>Zm{ds_*h7zV6-DEKqAoEDm%X2ZOs`h`<&~-K+VnT|P|B*x;L0*KxxWuID;| zFS=NS|L{otD2LBZWQ?`q(zv+76;9+If5QPGMUG-gN50j;cW>2MRx$cfJjO3j#}B0C z<|xnVUhA9K-=+4{)l%Q`wy`aS_Cck`9sOi>g|(+{8zq@E-w2!c+OMw~X*_P)$!f!~ z-i{BWWx)E03CzcC*e=UyUU{l(G_`E~HY~RFH{3%v^RxR#gwDaNvHC0zo) z*Hx<->Zx!24j7OW4&e*#JwNq>fB5c#uKOYKgLwG9jpUi3cmMSKFXTOc1RAd4o(kU- z&(<3gRqliLfNhZZT|cagYi$7cRR_a$?`Kn@pE-0>!e5_%l9Q&+d7Aur_1prW=(-P# zdjU|7$T>DfEq9*JpMx`N_I;YM(j50}K@9DE0W$5>tGhUik)pW8)Y!JOz6N(N&;3)5 zpc_k`KhRO?8je#WwQ;mPKdR^as2#GNAAL`;>4MRh$LMG#_Dgbt5j!ToKL4DQbdH_8 z$!8LI?$6nmd?xjtE6~hdWrnB|{xK+J?K^*e)rpE>UjW|@(U}~Se*NG!$uO009Jq^- zOM)`T9G0}f+!xTA{HOo)pXj;_^(be6*P;7?t3Y-6Dq5}z{-mz`Pj-J#UtQ~RwEpV) z4gLx`!on*tH^LvBID686J5b?x^OcwSrhMa8SH%eEa#i0NK1$)~9bEfw%-jCfmkZ1r zSKX{0kN+pkTkcp&@KtHV*IUh2*$vkS|3SVFo+124%pbP5|CjLtw*i0V@>2b2^Ue8k zY5QWRt8aZfDDInXasLn~=t5hUxAk&fjlST%_fI+gx7WF4uE{wxYgdDDMmN_{plU3w ztA&eFd3xnI`-=5$9vQl>#Tb^a5BmT7pZ^b2+3JJ5=;4<>>?e+zw2sBtxq3fY4i}E^ z{W*aoV*BE6>R4)r{_v)^vicfL%x>X3|3aJ>zGGDr*BkOQMp>%!_^H+5`DoS1io{$0 zh^`?G;)~FlOFsSblmqb{QS*r&`A4>DSfh#c11EiCg7MKawlBJO)DhI#!#D(lsADt|}(pe&}uZ ztqV(?dvp`=0J{Dg!rT5WPqHKx@af|@kG1m$mFd8Zcl)US0@rb4t>7|Nvp#F~QFs4! zf^L9x@A=!P{|3SyhpkD%R~Ww8=|)8Bm6GxD~vzw>IoD%Vj3 z@z!;7`vzK6Yh=mMDMWLE)FXBsvo|C)i7=xosaNMtPSU6ao6UXnstDA0^ zhHt9cOm`?F%~(Go7`C6PGD?3nLEFa_C$bl0Ml0zk>|KW6afjfZ7kFFn`54iTPq7$~#mx&s(ILO%QK=bwM*3!^pAf236m zNnRonLi=43x84WR%L&3kU7;Rniyzecb$D$QMTc&wu$ZA`Sqah0en$ zXSt+StafonQtyq3z%~MeTh8+%BP^=V56H=(|HPX(6dxh_uZb^98fN%^*?S+5{kH6` z@40tKT4!Q2p(TZ>U|X>wohnT+k=FXBSg@IUGn1gSwo+S_sI*cfM9IIolZ;NPL8WL> zgMXS5MW{2*I4Kn>+FAt@X{t6Lw$&nNz|fL}I2x18ov!ct?$2Ip?{m)czQ22K?7i@u z-}{`k_gdfYcdfn8Is18@^LyUs-HND}FAh|`B11_VBCf=Hp?W{-OCYS$Ic<%(V_f7f z-I{0eXKa(7^D&a3Q&pCZ3+9>E1m5sZS%~4t=2quOlcGSUZ#2+y9Mteex8l*su$;rI z&k>!T5uT$(f6=(#IVgwb)PowjOC{0iJn+0R6m7;exB7{Lm@msR;HYnY zHtfj9@trdgj;R+IYAL5ZD{9~(zoXg*qUGPuE9p&b)xvo(=5>iFezB2JVwrB}^yA;D zjP0n8XN;`6N#D9_?JdC^XB*$nZSc@>RD0E?>>fu)jbNC$&Kp&rF)(sfU~Ao*XT?$= zc5xgmIfTQYjU5pu*ybIKf{Bx4A|2uUy~d4vEiYQ=rQDpsz-NHE$!C=I%Ef#a%5hPd zTVujA^o%IIboas&fFq+&l0-z7t9#F%n3zLjb@hUxWpY@}yZz zON37Rs6O*!A3vy{2-tNpu8y6x458z3sj*CK2#^;mP2#@pKX?25_q^x!$}6ur5OsIV z6iBajR^lu`3{7C$9IN%4o3XMwG$4_ z=U8&w%P$OZ?5-`$T8B7BlLO9`2Jwzp8$1>qwqx*h+x2!0VqEsM&~$|{w&B_}#Z`%pEb<$V9dXqEkAXVw}o)cv7E&wC>m(l#(p^9@#Nh(&uya(9!>hP z&%uBPyXfFdEq9(PjwjqXq#!r%0Dzx}2gMts6(-UuJ!oWENC z?rpu6`$(%yOe4BEG6o>~^SqBM=wuD1ra%4ZPv3qlZWw*l2R}HE-^?LT{=}}0!|}ei zTINN#sg>Rg&S-h&)J~U1lwWMAvFNW*MFg$0wT%3U8RPGDD-K3U&{BJixW{u?=56B_ z`YgZvZ27FGWQ|AebPr&!fjAo2GYr^@#Th*&##fJgD`n15Rj~X#-Vb-jZkO0?E*s23vESR)0c*g&F{uL;) zj!$f>+lVxcqQ^`Xv{c+8=kC|_Lw@F87~}i?OxyV@`x&NZG+uiuAIUS{aUJi<)|tHQ z(p&578G9Ra_g%Gx?Q`a~X3Lu_7{!j2wlNM)*OLpa5uRqp5jey5Yi@r10= zm?xYb>Y;=U*Q^&kVrd7F74*xDaV#h^hg!|0U^>3Q?`+*$eHlb1eueG|&w#6&}Wo zd!%6OHOT*_F$jcsj$M6f-V%-w}Etq=2fzP)~un+Yx&X!h_`jU98< zMsL@!U+0~fdBL2BT=9VtN62|vw$_Bx?c_J-6O@Lm^NH&~=kR|1zS$g0!swuviyfWt zXWb|Hcm7qsNlnPrT!6ibPl2<>AkX?$^R|9KBM<#Ed#?ugXBc9#I)1|V5&J>lhS6)W zZfOIG*l`|!v84yLlpWzEN#X9-P|<9~{Na(pejZW{n9A-}A3w1( zY*p8aThAiIF&7A|QH9(6_cY*&?f=nI)vf{^YTCy` zyw85e5Xbg9DzE3Q_t9`A6)By{r~|LBc+6v-uhYH3MrI0Y}SAJ|x{s7K4AWUSzJ3;M|1k!yeI`l6XxeKy3{1JGxn=UJ8Nc=Q_XVS23V z(&qSSF80?!)C3kS#^d_swWs62V1?>&^|j5og3&p{6s|G~=jTn&pU6{7gxIdM93^+4 zc>WnD5;vrdimN8ofb03sa`$-E7b*!JYXQY&{bnc(w_Ea8)MPM7Q44tbBXN#jf42SV z`3IfdqDOvo$&VF}7?v3JSr}dK0*Vxkxqe2U+R+#6Y6Bygd-=sDj+GC-Y*v(xx?v%V zHm$Cm*V~SxR1rLimv?Q0>2cxNh|`xfwvL!E_4>_-u6L~=Y@dTbCfK9x8o96;e%M=| zEDP&^J&VHBi2C)jV%-BPVdAni8+g~$yBEClQgf{5VhPJRNWZelbKKAacU33M9>*tp ztg}aZJQU*R<8i}CpD_Bsmw&+jNgC^3ubn*DuGW1yX65hB=(B#}rar9vJo1K7z7C4> zCpvHrgHjPm(jt&BFPOT-Dj_L^_P|$9`Li{%+24~YG(TbVv9CXehj?D%=$s`R(z}6z z|BTu(XK`_#g<)C+rsgKtHK^yxHrfI~<19`Lt1C8fvCHGPDEYW;uDji-J5Xme+!c5t zs0)=T6jT?>8_kim4l!%wBs@`p!{YX_QUmrmBB{Y(GPT)gbbaaJe~$JU#(L}gP2Ch< z^Pg#)`QGX}&vM{nSyE0_+ZRh=_FCk-x4r0`qCf4pe)cyGx?%L5_=M5CVFY0m!#a6Y z+pKHjU7ZAXVHxysXT8qTF5aZ(|E#I_i|y0x6Gj~(MrsZ1I1uv$wc4uL>!ZCQx_Y&* zFxKE)j^1GmwJwkk)vnpwil1T0HRD{fzT1)Ce7m2AmHnuBRtLVG|ExB}!6*&}yX#E< zsM*Hz=0AxpjfqM+`dBnq6kgk_*sjr8QALhLd(|Kfbf{_P6PHep(I$@Faq9P&`r6>j z6304jMD8_-aqRP$3v@X5#8@}8H|@}wpODZe1bV{g^e%i^Ju>Z@;xGpISi#YeFK0i` z;qb;7v?dW(24UbDFNMPNm?PeohWo}vwg@Ped1a%+vT^VFu}bZC{fbnJ+R9UPcOUIO zzV4&Viw@PCBan?_=0ks$t@52mr*bx)gX!Z|JLhrKqe7&C&U*H&x%OVyC58_D9ka)u z!g0)fx>-#%`uLaHo(J7?8)yl}Mf6hMH5sUsD@}R3uG+oNr+)|gU{)oE&G$Jiwf+1Z zud1iknTCQ_{z;4mx#XMYUk5~+Psq$SfIcyoaf-6%V;-=?vEmSu#gA27TgRxcc*k)3 zslTtEn!BAZ4Q;E2ecgW*2YD7(F6WDV{3!d`@q>#+^UPZBEMnhtG+>`=*{eOTfmOMm zHje6D;~1^+=NgGf6B|1hwKCUUDNVmgCjq8rRvPhp;llky%;u{`htq^3Kx5?Ugdi#F3i56J^dWNXFkD9 z-V=?U=Z(~PC9m_XYx6{Tg}>)d9mcQO9EEm$@%Q?PV>@QY-pj98eTinBzg>eI-S?E{ zy*StFf=Tf;!`fTWr=7RD&a))Saqs#OhdN~Y7-$n0RvUizp5Ax9+p&)yUB_LE8%FcX z`a3bwEbG@v+NrhimL!e0!&)et{P|K}KU%lLoo{%e?s4t!9QF4bMzbyjbuE&3yy~vF zTZ9FFY6Dvu$KwOWTG7KXNOZ;QI#1<(3d^dK`Wke;^RX&7UV4r_d)3uZ*W){@2Mw&0 zz+qSQ-W$g%$^H0u73-moMdev+{a$8vJR z=!}o{p0oF#gAu;#>x}GuX`sXP({tQwcOBu-fpZn#{op&_7}uKZJP~|`>#3*Ly6bfv zX|M6<$)QK;2oYPfxnTpp^Pm`<@WXg}jCQxytNABSVUC`PRqJ^ujc1G?jNYVuu3bN| z0L{PFS9)#@Rway9VLWw9aCDx+^tdQb=Q$q#{1jsd>G;E!rA-bje&xf5l}}s8sNdtn zutL`jHlb_N`2!cLXmO40u{|y}V^Vd(X11zUBH^Dpz8wuG)3{SEaSr=b%Tr)*0qS;?kx5 zkGC5}x_<8|LICCILRUV9}682j3Zn*iWWa9X!<1LZw>rMdwIN zt+LdK9gt-_dkoAyOAcX@kE4mS@vc#}pWWk-qdK}Ibj`*9lSrAw4P}{COd4JODXaix6n(L4sjI0+b*SOy>1c%nu zIKW-S;a^$!#CM&`8%6>27|diHV8H0^dgQ@^FK7C{P~70TYj>EE9{pYd<2+4er!$1SzgcsAYjLFjHWk&Gd4n-Yvyrg1 zS7Q);k0oK*IU_D!zN}tF?I8~S%yQL>;WbppU$s14&6!^Jp>V9vDB^IyQIq3qC52fv zb}dE0?lJu2#eyUMeng0)EuF#!`m=G~c=G0R_?~OmAopVD>mPaRjZ??$x<(p74lr2! zSp2*|mp@~XKW*%|5~K`%;+$^!6;IyqtVeF;4{A7T2KkQ?&7pOq*A*vgX@JE^o%GAD z5wt7r78UjQ%Iu6031ePVlLhTnF*lui?pqNxZ@YibP&HPoXV~%Rfxe1^!-A*Zd5+Z# z3PT(#9~wf^*16Dc7<8Q4Y+Xk__*gu`=@^zDg>wsT*Wdf+Y_ckflG@j7A{{LT{C@tG zcUO%08v=7K$7?s=KU!7Dg{21R_cn3)Yumk@QT#fh=W*$wq-&nGUW=$&o`3Mm>VX2AyKK#r&UDj~*5cELwXUzbr8`d#-7*W;YF^7+)gfBgt|mPdZ)qQW|Yc!bf` z`C@6wN=yWZ6}x7n>lDtK>^laZ?J?4GfIU0^U}A+OKh@>3JDs!9aAdG8#OmLBJ!g+y z@u|O8DBc1UPr=D=JPdrydZ7_WFe z>Uj+{Re%Pz#}sEHPG1%*G5Q_jSVR=uUO%L3fOn034b3qj~lE18i}J zX9*DPqjP(YKr5s@I)8}C(iR^LEV0sXi@17Vuszn>jK8w<)}CSQTl*}-itD``Wl+Y; zW}XBCQseLSb?)B2!mV|67FVf#9mj1$MbO1ELfMah>XY+JuN8k+pi$L&APjiL`#Kxl z+`!Q%5B+XutcwPKE)#jUVKf0T=R1r(nqPWtWM|HN z4lVnRQ{WzUAx7t^8xvjoN*4^a*N_|q1{uuM!qG_G)!b^n>Fp_P#d)Yk9`g*^#z8@I zJi|dA?9_qq3=0?No&*1jTE-{dv+L@`v_jCrg?-VQuJ9LW=i%6SGe5-Xs70W;CLK7A zGYm1=!qZ||>!^-%8(nNiOR3ZxN!;e#x(1#&arGQ~`>c)~)B9+Cc>I+#ekDyhit&Ks z|7oKacD!(Do~4QYtZO{;(dVyIf)iOW;ufo?Y4;;O2=gcn!qp909(RpZX$3NJbK7i+~(t8nh2)&fJg^l0&1GiM+rDr!a3!%H9>JlW&gbN8|A7_enm zMD}9GX|YzP+$+9~xFlQ4=whk1ULU>H*95j^DG{jWZ#1wVo@*Hy_FRXkRgd#yKd4=U z`W079{3FI1b?)iY5wQg+v9Uv6=K_4J*&Tz&cE|N7y7<6idBIlKkRufjW^~Z0F)-dS zhLHf=o-YN4C>Yr~+6M-Xw6mV*G3N!SIYV2q-iD7Y?%LY5QzLf!^;6nz<@%T6H6lg8 z^}K~xbGv6`$Dg6_?e(rl1Jsqb;*_w6L1hdcFx} zh!SHhl^(gU@)gc)V%UyR8$VV)ZRq5&tWHFBWWp!jy>;Iii_{9s9`Oq8p7rcP8*AQv z)>M?%u1cRcw(2-no&%qR*%(xmQL1ZM9-^|e)shNIIv4i!OI&T7C+>8O^u(@I?Lqa} z>&O*lEO9Itto;2Uzm8VgNvh*FU6!BY#cwV5tR+3Ntu=M;e)Na@mSy$c>#f>G@;t%& z?~W0IG0>{x6vzSg_>6$R+f{2)UBt=0@)F!VWS}x|rM+MCea$KKu|=lJP@16kW3zMm z@pY{BYAKFBR@&&~Cye4dj9z{9)p;{$&V3Mg{V$IAGlswr)%7M9^$z!7g&7-u>?{NT zIDLoF1cj4)8*UB`|Bjml4*;SQ$F07ZtS&fF&8?e?&O5TVBG8e=jcpQdsEKwbqkR?=j(F!hr@ial(xUZ6sfwrs4;F2#aU%+-6>>a-V6N8G*dH{S*r4D!Kv&5rwYirr8g`DW z6=xQo>h=imxQUmF31_+EET>}P;8a-g}?E8D3=fC}PUOTUI z&ijt*zOL&YUF9va3WVp@(P9fXHK$_d;dxSR>l-s+SJ5Be#|bz8Djtn6?J&XFQETZ# zW4VYl(uShp(R^ute(Ar1bv1cp+dG5r8sNp8RzBId-&(7>HQMJ|YwY2>h|c8&}DoI=scQScCW+!zdl7p_fxmt>UGjFHjC8-J{N_0wIUS7wr%y#t*zkNDZd(J>W zAzCu-IjlU`R@>eOPEU(ee_OMt_4*kZ`NoR_f9zc)5#y_>USnv!LlEUrZHA8bd9V(d zTBfV7r{KEilNk24B)KSZ-+3f_9<&V(m@S`0N)4ZMXKg!Fel!+0n9MdX5KL_oWj`@M zxOehtuDu^=h@*2N0I_GUU4Q>-VNfQ>xlwwZQPwCYPvPq$pImFNfw2S9Rvb(#eRFUC zUh4W$`}5OMRg^mDRxh6HEYk??wrUil9_RRLAKGPYOZL%4Sju-2xP8_@zX_HE6?2)h zH95c6B;YiqVif;g>IF)X{^>X;TN%%~E}w%1Y>3vuFP6!a#x6Jb)vfL)!_Z);VSD#@ z@w5i5hdI;yyVq2d;FU$~C1xuv`B~xr-wQxn-^JYXsqU&7+~K7E&*_ZkXStlB_jEPC z0N;VYy+0X~WZ!STX?$aOx*>Y4t? za024e1_VQD(F=tI@8YpgKX!cf7nuwfvCw;IH?DX_||AqBG zPZe$W74BKR&siadtaCc;>I+OJ6Fbf^($sIz;Tb?&^*fs%=gMV)h0$bP|FG`MIKO4o?{5UWWYQ7l7(IW#7dFN{MCk64jCWVid?uCqa4D{x| zaU`uDRyjIU#8K{gO$G^AY#x1?!*kSDl5H;I3yjS9??pc2%;6u#^9jZ3OB-TYuQSiQ zf26rOL$#*+Zu>fZ#xU1CW%qsB3FOy{e>w8dnZnGQr~v!yI;Flq5O5oHg$UBSnR;H= zsp;#-dEjT*VqB$zNaJcfnp|>oFY^;w@oX8^VgribL%KX(gac|q6`6n^CYhRE5ka0K z58@O-r;BCBd(WHTk>w!N!p^gF%HD~GapLn*@NU#=1$Mvd?CwLgmKlA|kHX@mb-q=O z$#j^kKUHjEdS=@F#ZQG*W!s-iO3>HY`R4tCu&dFeIH|bG;!w1iWrjIFyXz2rP5Gcq zOW&=)oY@?nrC%GLAVOjtJw;QWv#7>G34jYmO+`Rls0b>6`cQ?=k1Q&9@Ee8c&-|k7w<;0zcD%IC72rf4E6k zsZC~4V;cMnE-zT}*f09e{d(4rKAMGa!07t9GyQ5)rcYN<;{5o~E1yDn(UmOs=dIK& z8v#NK$DD3G>f*$pg0uN^$?jEBrv2g^CMFh6;WX!N4Aq-^eQ{<$8rRsRe@hIW za-W9iLd`F3tLV$Rp(rBuPj-AxB?+rdH3qF*f$NMVhfF%qvxrk>2iwk7ra66Y=iAck zw<^AP4dgMd@iz5%Qin!JsE<7D{a2Qu~#SDIfjt zQ!X$jF?^I%oT7e`oSVgoL+gkNQ0U2mW`2CyjU3 z8INi*>S?m9>F{C)=H~DqoDcnVFk}$JTUn)%F?F&EF}wCPO44MLT_V9%BEn}w(b;^4 zI*K7jZ=T^=sSK{Q`qqbvbUnHv*vh3{H#_92(q+PNfIc#%0S)Zhzq#LHXSOMORjF%f zjU*LV_v`MkgtPGC#~7yrBvf(Sry0(s%>7Zwr$Q_Iq0wkI2vs5JQr^UoCJ6WW@T#8yPz`jv7kCU#Ay*t4YK$ z7#^38f_L-8xaSC<{#1MX_b%n9R^>L69;h~9?EG}7&uHMIItzjB`$eLd1}{I8_7@KL z=c^~L?&A_1LwiF$|EGS!oR?F_(gz~+HRxghV$Bpe6$yzF$>*6fa*z-uit$dubQxo?@)RaVJ(JwLo@jJdmL*ZDn-i^6WRXjl>R4u zBC8UuXNFXu+i4u)uiKKnA3z5QZ;a67I^~B~_DFU;2n`H-dr-E-zpUiv;zSv(wq|fG%Ey zpmPZd(wjSuo`@HDK>KI`Qu<^WU|stk57J5yD=1{mZ8Jf(+XN(Rt42B6D}JZN$q)7V(4+wCj|_xT^N}C`ls!p{GcjM zL8gH>frjeZz9@DeON$T<7`3mVGo^Bx=H7%$oW7XFKQ8?Tg1_dX)N}g zPwBh}M8=iyQL@{={BwY-C}&*_Pl=oz{9`pUa+phSPQ1Njcu%OgMaM-lCINWG*H-Qp zM;A^LCpNMo`;jWkg;z;kRqpn0sonIj*>i2Fy`gEizPm5=d0zG_ksXc8%{L@`-|#E( zq9mCmaSg<`%%Q&W8RTJ-A25G=DU^0O#Qa0bRR09<@o15BBU4f-r(E(PIEBN8ap9e!8LQ(XijD2 z_jIY+`bLw9J;~S-L%rq-hd8&qQW=mm$67PSV#7G^xp&|aqt&rcggWG+O9zPndB@piWNU2?QzwV#b z@qExYXp_!6gppA9mPuf4npVlSgd-NHjui0eOCMavUA%Dv(6UD9$UtyhJJC~GtQOU3 zTl$w>M`5GkT}BC z2JUCjL$e|24sFUaXNT`=5s^N4#~s%}Kx$}-2wzy>l+-XFYs!j6V`u*G&s}Bh#(@=t zw*wsqz$qj5{$LKL*ppN?=!Fh$K}go1)W6e%AC2N6IckpmZI6lYa>Dw`IHI20XJr*@ z;ks0p4zkDb10Eg(-cxYxSRiLwqjaO)4vv#PTv8io9%f|#^39+CW7(vuBRqyW`}}Hp zT*&{zz(vpQS-LbreI{MI;aRY&+#fTWrcDJ>@)DY*yuJR~*r=UiM`C_Dm&@ETN4fBE zKf!Z#kE4~yq8X>O%JFMh7dPyM#{5jIIdpAF!hNv3XgD~ghXyE4p~wamS!g`mm%P~g z1Apbe!=lzICkTKTk~w9}z4aYG&l^Vq#5@D8oGa;>?tR0-N5dRREyY$0bL&FQ@(J@+ z<^%>k5Tlgyx^5!Nmzhq+6uOgH~$QtJrRzPyn+SZ4wOt4oXcwc=2)56HSPt#u99QvR` z;0T@2SJQhmdQsA=l%HGJtHvoj|9@lTM}dSvR1loPw7o+!Kr6R-^j~yy%k_S#p$WMM z^(;las1x(EI>pJWdH){M{A&w-^$<4UQzj3U!C@}yAk3n$^yZ_AnYcMSEZI_}bhgB= zaYq#Ch#S8|{%VwFDWYeK)m8}S@|Gfb=Na$*xIW5wlO=*hH+_%4yiha!iLcSEaxodl zj*ozC^Xtt$#tg;W;0UyI*KSI8m}q6le_-u$Q4ZI>uWjLeGzE68i4Ly0K5#Wt?W@gD z5uwwx|IUY08&#e1_ffESnZPJWrj}3KPTue+KjKL5JW;IW z#Rrw?!Uns&KLKy6m1n9|*T%3$E3OXFCm|C2IubFTo_b#J*?_KZFPXZ3;{K#(wUNas zzFMG#Ein?ShC(*yO<`bsz+#fwK@IU$$a|WW4B%;8^f)M8P)Vsls{L5`MQ<^9txU0wBrB9Q@~-j zQZtm&;P$)H_8 z)DFU!#F^UeSqTL*-x?4m|M}-tAw0`$Z=eOK+X$b%Jm>q1G=QUu?7k?Ckn)s-TZaC6 zUxLYu>G!>Q%byZa3F~+q^lzuMcBwBn76zs@3+S}Y~Tg)vDi&E(M{8;)6G1NA40xe?;uo`@F>~CGj@)% zCn<&`g5o9wwQb9PROU#ieFy2`SCuqm}^4;e?; zGl~DC{8$FZ1CsB&A{0u~X$c+%cc{5`^(EtC6_|#9kyggFgp=GnVceC-jRqqx!%m*o z#(NP2(rrR-6?!8i!JiML$)wfjYU9+FbQdq8sVk6`n}+L`1@Oz=y0fcsWA{tpCDWgj zZn2R>Kl%mv=DUA?b=OmbElGGQs|s*hvut~Wugf_x*UUKB{HP7=e5WRySmwXB%{c!f z@V|>AL=>OuRc-c53X?%Ux#d5>lG;$ya_58glh)a>Hzjt*uh^ZBvuZ7WYx&D+e?(Tk z70zL}H!RCR;0h@ooX0DCF-6{t+^dK)MLl#xO`~;E{s^^Cztk}RAt!@)=CJnx)R(yu z?Au5AO(xO+j&1#=kCPW%G;GqkVi9LRW<9xwTyY*T6s4VPM#6`l0=bPw3S>c8vKV{R z7E}zkTQ7HQP*xs7n#8P~+Fe9KX_>*1^A}-mQC1%D0n<=M?maX7;4RLaRPoe7n2vmP z=7hu!PCo15uTqf1c-|5yDekks|ol}QC$o+F3 z5W^B$GH>OxhMmJ*LGT=x^RLXOaWNz}Eam1A|F%1r;lg9j!oK)huk7bO{R)kZyXUcq0 zIBH+`t-Kd?1GJrPEd)Sc@;YiR_Y8DFriQO>n6-Y*PSdFCjN@}em-R7fFzGOb@ftgF zd=uiQ1Z!Drl>fVtb~9@oWdna`K}{*onU_tlW3-yZau`59vl11nzRR5-vY4x~MNaS9 z{J7IP+j!2fp?XoGMrlJ8zS z%&FPmnIEbVjzl+i(go5r*+pN1kC$F#l+5Z7HhYjzv}F%FBKzDx*)<)bE~_IxNbjXG zqKOyu5WG#K4mmCeU-a2U>Xz^Kng*a>?|O)BTGwh|p4KC8#10-9bu~(#sy-IRHpc_@ zh`NyHEo(CWepG%>FcZk~>(P4u1zclh8~HxdFu(q7TwbS=nVD|Au1K{M z&5QaO_99^gi|A++r|%d^NFp2|6P5yA6e)g;C8ryDjG|?(uAHh26HXN}(>0wQ$qt0S zHSJKQlp~InT=~#<$WqGr{Y!Pb%RXMlJ$FLv5Auc0__=2UAhW(3CY}!0JkdD&up$c? z@JKhY6zChW@BpeaeKYL|;ZAlAEt?7|)NAPg+?L(~n5owTdion9kXU5V9g75v>V>t*kmto0R(1@~ePo z2&(-EA!937tYC4n=@vwT?;uVSq^jw#a^;+6wTnlry_nY=w2Eh}%Pw5pB9%vMMqj8% zd%#F8!X#zQ42ICh_Q`W zsj;)mpT+L;KdqEYQsTKi|7aOc<4{2i3;&4v$?`|G*A%UwQe0|G!=mwNOgUCDU(XCSyHMA~N3oR}hi|8F+LO`B0J8W(YAN z)nxi~OKv~&^u^-QJB;Eh*SuIOsO^v~_^t5fsdU7;N=)36uJ~uWoKXSR^UNR;^dpNb zMKq_5&+&gDiU$oIJCaQIjk-1KCrT6xCUd+(xfb>4WxmGTJRHx^+0C?wG7e7ull9wb zggdFXrMlZ)6?*DhV$~J6R za!1}Bd3T?mr?PN|IQcf08Atapc9cK{mZsFHfq}`O;{G<19I9d0q!JWAL$1zKpMz62 zaGVy@{Pjgj2M-F`MGmU(Uu1XhFy{fXTcvP@bLmgpO0~X>SnWv)qg$S;nu0_!&RC=%b$$*pxnCk`_Js0MUhBKkX%C*^?|3??~g`k1cUii4~~Di=ra zpYbX6icKBroLEs4V(ZdnH~(5br^vFbsRX5a_t~s9(TCz?rqq-|3lEit<77UgRE2Zp z8n+`hz$|BL-!*Es=Gp@I@>h!;i>4oV!Nm@Ef9l__Ti^4B?`X3BR$z3ejeG6Yu~my0 z?ev`2o133atNS~eto5zPUPL({!$Ft^GUiG7d_s{x>q^C^;TC#MK{ObM2=XhwDVKXQ|)DOVwpo z+*ZCc&aNky|4?+~Y%!?;9mb!)^E9p}@4OG6Zy8X@@EKkb4!-MUEdLfPDB(}KRZO}r z98Dhv(XXA#D{$<0w1@rk&PWhv?L{v=I-ezkoI=yHuhvR+QTXTGr!y1d7Do!%z7Y6y^( zyhu)QA-r2NxEGo`#QKw$6Bw(xM&68`!S0MyrAAhLG|m7KF=ejz@r<-$jV(Ii;kps_ zJ#dCn>+#zRirObtJXXudIls zW!j|Q{AEXU%6W*hG-M$X>mUP=#`55{zu5_}lu}N!&fdR9K@7s?mARkHta&wYlT|<$ zv*}?b#McKKX;f&mje^kt!Hx&tf59+{Lm3K2Y ze*kn*kHZ+kB(+;Pm;6Liz7dN><05AL*mSLR1jr zD=}J$81|!*u2d%GFG_Owb=*oU)?{O_cZcB_M4STBYXeidGn@ZaX>@v%`}aqhM#o3z z73Zd=3_PUTQ!YvoezWtM=_iHWmL3jfC7&Koyam|L#Pg}tR?|P;%D;8J`PkJVV@3KJ z{A8kIvJK(sWp-0@sN;(;rnU6b1W z{XGTSO%IhjpS!-!7`3%0sxB`3d0t5?h=`@jbH4K1IC82Vr~f8?Bu8!cTnSaE>bR52 zGI3DMW{yPBRfc64f|}w|^56a|;!t+C&`){(BJTDSp zTglwUeq&lnv>5wW4;=|oUvUG@VI?a5cB-q9f+tgCT}W~BPGeG#L3RLuU3d62bWxZHgS{j8P9Nbfr0`c zi-^boJe)DOV$2`E?%mIbI=ue&rnNsYrB)=&=q8Si*mYmKK(|hGEUqKpw@&S!)DL-c zXj2oYod>h_>N9XF z8YE&DwJSqez8xC%C$`etzp$OTWtiYwe{rqHQ-dFZHr%v?bYTYkrXH(ID!d@v{(y1S9oy;&yNO#g~a;DM^;xct7J zO8dAj*)w6?TkZc=)zh$AFtVH$a>JVk-+vB*NemU6SELYh3{w#Y%PJ*?3WpqQdhOC7 zrymaP%0DQh3u~e@!)S|YYj!nMJ5Ygn>aOKVU{u5 z{HL16T#KL(Cl@)uUn{|H`9F?+S?i(olx%d&tu|9L#qNrHqnpgG`I>j(vMje8cXrq{ zBVy(Ws5c5;J-v*%m5Tuk&A1r_LTC$47Nf9AlYDJEGm?GpsQ7x>HDAdx;%BrKn~9u$ zqS!${QyxNjcoj*7OZc(t|HNrlW1{(*wft{7NAmm6V}04sg|>cm=J?Ie)(4R|1>!^= zH~L8N+JlZ-*qm!c;Vx6>{Q?;$=71fn=u5Zmpbn~md=Z`(ENi8vX~OPwK4<@KWQW27 z2HVX^7qdkVU`98WPDO_Sa($Bm>4%s`x613yu=v(v-!@Y*CWOXN>!L=U(@rCX^{V7Q z{9BvStTRVu4&xwqfxpKmKCx8=+sbfatn9sAoT&2;Gea@hoGlN&I?np5j`L%8(3>4& z;x-nJYd5~;TMD>zyBWXnzcjwtHf|vp2V$KTt{0aGO?80l6~Hj;oZ@DhQ}iGuxX|h~ zakEjV@p))n;7TE(dJcKT0z#iU`S&N!UuPH}M>~RUl-t~YG|>1E3fr$N);zmjnJ4H` z+f}zrWi`(*>706Z$!D738}|cDy}cJAjIJ!?%sJpE)8zY2A8L*i-I`W@QVvmNi6*u= z{jTWtWamr0g62NYLm7i-#eFX!^xWisr#-2G8k_k`ld*-tPqP+;AKXaB4NRV_-;GX$ z3$JKB3PFqer-O8mpjA(fldP(aKh1pD0a|tIS)%P!l13|?@8Uc*TzP7Y*<3Ba&c<_i zv7&RAG3;0>@bqe}_Xk_Fk?lLB-1pA&!eg(Kh6oFDk5}D*e;8?MPoiO@wTJquP|gx4 zh*58|zF+FFcMb1b=ZdN*Ea8}wce&Rt(;KD)4%t|Z^wFe7pzNf${sc@wKD~6HOp;y` z3)~t1?JaxrQSc6@yS-mCX4^qH|6^HP!Wji$y17s77nz%?Cs$SyF?spaa)&8$wpM%Q z8<~EDP4D;PuumW~ea|P7SClGcE}UW8&@1d_NG;*K@QTmIm~P2559{4v`VjLYGwwo4 zAnDL(;iHzUL==ai;jk;tSvl??rr~33u?)b)+$y1BjQyB#J+k(}4pSwMfke#rDFpLb z+gL&_Tj&ce{9+UvcubQaMrHOgcs9(#kN6Zy+Y7wqhTHcii%7scwFn-g*B2^_Y&?#UyVvFX0`Q4 z=0o{AyGMTpBHWQ8micR3uU>D@uG>NOuJ6-RXaj|9s}hq|Np?xTf4{!RrPvfUi=6SL z?(FJ<86E}-Mw2^x-(|)=GQfVOrM6dmktNwZglc^imnf`*Fy1V`mFD05Da1s#r^Do4 zS9Ti$M;l&(qtl##S$HrpFtVefd zN^&l(Fp`s<+LI(lKtdv>^$gJ#ATQSlWAPqrp9>&uT{~)(R-SiaKC$y}xV}flv+yMy zB0Jstw#8Z@MxA+ABF5yaRR^dP{+|`6z@ibTXH=dQ(z2ntUrU+`Q?yJT3jNq2LgTFa z{|SNr!g%5b;q&FUWTz6x5*GNjMC=A~FTSnK-6T-#Oa*+T1P;q}B-P3tFU_5X!OKhc8= zEPM$n?AU$1yMJ>%cXKLnv#~2gsA+>*v6cYP*Mge6?oUx6Srp5{uAyO7qc)vI4mIB+jYUSL>Ea{tsK@?WhA|&g%5R5L)#khu zvJB(Ns7h97s1DZBM<*}Q%H8ywGxTY{aVb^LGqWXj=Gf7ODb!1JKK1 zyPsz(F#_3+YV(^(maEeLA{KHX7;KKp-ck5$|1-YQbL*XyKcd2+ZjKv0a$7;|RcQ7? z(h=S4(ZLG{AzWntcJy3e}FiO z(9(m9x}r(cMOLRdnO=&_JKJ=3D}J1sx6ao*5^jzkmxuw*IWp(x42Y@vl zuCk;Zwu;u_BNYOD`S~sNcw#z&5Ej3T2lI)LIv#w^OOmq;R6O(EF6=~yji!W!`wX_r zxRPeNb-g2z(I;7hYrSombtPzc@sG`v9;}4XNmH+KW39&Dmu1|)J1d@9-5s4}P?}KD zIz8s?Eg65-RBBv9?hhm({>Q`g3`lxNi)aO0Wzse7NG1PyDE@((c{P`7+Vhq%awq9V zrEhb)LV?J$`PPlP>!~^8>%INqR>ubcCpys0Q$FaH(9pH{SxPW5il2xcED>fvewU}9 zlm2F(rzcrAUyMC37uwKX^R4_)Ede;T-bnAX-n6^MP{*aM*UrI&kZp@@LB0GkM?gJMtX9pU`!CWZP+C#`)J3|FTd~H?fSOGJVT>5$D3Zc8}S174gPNt z?%Ou##s`O{`P|hkB_9#wg|Uqf$Il%^e-^?PrGnz?&@oSpm4@P zP*@W2l@oH9X@OP_mWu4KuEyi9&Y9lUi?-fhWWOu#*t6%-4^k+|<}6lX^6N>voeROO z^!<qIfsTmye3mCR7J>ho~lc`x%H2I;o#6?h(0f2W?g|8k{B{H~<>j_l7et zam?&x?LJP*O0xCt;o+X0eQCTi`cu7&M-;k5VJc5U`CG59KQy)Cl-@mNA<*YZs~iRev{Bl;z z>**4pjY#<)R0br?KP^?tnD$j6FvUja?iR-E3;X(a_8sA-L6*xIVkAz-VWXPZ@Qr28 zFm+}kNrq!?4suZQYO=E#{@JREgW?+3$Y@t3#- zNA3=EH!V#ky1=i;B3Qm@5C3dM>^GFeewRC#)aUBPvTMyVSMlDR6aUr)Q|)1XXZvm3 z_pB!7J$Us&l9#x7tgDk(#f^Y+wbQIKL~}H>XeD zKR1Az7Js?z3G-w2qDahfq)VLFZrEc21Srw%7n|jvIRZI{?uGb$8NZ zJG+}LrNFse^{Zc?1fQFdrIV(^rq~OL9KMtiUA`^@nLTX^G?jaqGebk+lxPrEx; z4Yq>y)Z2ejQSMM`ld*ET7W++9T`UimE**p8yarx2scF5vW2rbHeKzGHC?@T#ZMh%( z^UD?DsE4k?_xgLUGEUH1wnnk=bB8J|QYSb4gt+MZ?KzI*G0h z;)0jn8I92Az8E-HsV!;XJFhT%xHHyG0#QHA5A^=9%J=bvv{ zYyu})O+=fCwyvW10&VrN)t2Yv809!F_M?JJUA9Z9M-H4+UvY)nlExP}B-B$+8o#Nv zczWnvlydDd3ITIgJq zkF9dnJrd0sF6LnbHP_IDxgRm5xsjGTGTVq9T_Cdsbgp0OPKEVINgI(YjE;pzIWo7% zTz{AiH(&S)V~}QAGmt^hpXq{pM6k2qAEw<(B$PSu#6vj-^~@=w$wybZb(Na|(!>k? zN3oU&2-jG3-REFFvBvna5)AeU`a`640ONX*u|c2W-}(oJ+C{Zib+Yl+uj|35?~ON0wyiob0-NcdixPHdGpTJl94zEkP%M z8-4X89v-CDzzqbC2h(=$zkY^VBso}?ER7u)!8-Sf6FCIs({mYT>907cN?US) zjWNnmRbe7R_%_T;oKXA7Ba%T`z{!xKabNuNm=cUoavn6yggFw?|U)p0hrwv2p~%YeU6wvp0E09Ev$Y}K_6mXr6X=%F%zI6w<72FH-InS zc5+&V5xV%Wzg@ER&7C?}n;k`u2X`BQFGpu;ZqqAkR>lu;B8f0&ZXqtY!jsWptz)CF zMfPHjPX34?9$jLE4u6Mesp`Sl(HC63cFJdoJ~4tZzIy+>gC(DQEZjX2Y#{yGDDC(| zUmjOl&GEX>2oBhOP0RQE5a|;38b%pH#&J{)Ks&t3FeiheF1vXMI%7rzioo#ETVBrY zleUz-1STIX9<27Q&GK!fHp%^n(3I z;=Ry^sV5j-(&y|jHn11> z0i7?gU+^c*^y`WU4FC)`vX*HGvDHAWx4y?+0pi%epl6QETRLOSExdz^2HpVY|`K<*r)BVS#0Z*RS6$!p%>5hw0daeGr+2x@c zZ6<1@{bgXYF@^Q;9hVc-uIyjGl2)^QX2qD?DCmUF@ob1!dF%$ererr#LdI5!2V+I| zB`h!Y>r(RZ8OV3M8FWML6t;NdEl(Y&wGG+q+Ld&i4wkC=2^8fLzO92_$-W1|*ji68 zlfHOI@c(9K_W(aNJuY_$uljrDTVR+F499%i{!-eMc$*pTR(83pe~B z-Hr?*U#zF);TfFtbERPTZT;P0-1MbYQUm4|jHa=vZIwjCy+f6ghTOq|T{BB_l=73G zp55B=W%FTX19IficiOq}VpYM!Tj1S*tHTVZkZ1HT73}Q?i{dg_(tTO=)BP{Cg;zDy zQEq8`lnThdus>$=!oKU1I!`lMA}qb0uKEN`cWl=LN?sC}%zaNB4DCpMLuWom_9}yl z*}%5l@@J8YiSjV;_Ke!w`wgafd8Zh=01r1msb(KjuzKT%r|Fh@^h4DyhrH^&2?1A$ z)g4Je%B=NPzlc^c@diGwx^u-WeTYVwtVfxPuUxMK3vKw<5alJh_v=7FQ*cZrc-G64>pdAF#WiIiv< zf+r@zgWacN968YB_I-VO&o|!QsrQdfL}Y3%C+pyAkv#m^E>ri=lYovIDD|$4(WP_Z zq4od4AWqRN=aWlU0(h*=#gmvM09_(kZQbk<6(APgZuA^jnAaCLD5kRenYmQ@X zaFzLot7p5t{3fUr!+y`nD_%deHUUs5hr0&Hp(OBDk;^#jN&2bcYf$>rSqf z_MKdnMQ1y8#dACuBT5{G%;716fxexlHLdCwA3d8+i})lA6!`*8V_hCPaReUb%HORe znoo?7D?cI%7qq(shaPP!`3IvloH8gQXlw==#f{Z(u~(7PNN5kll@T=uoe#?)QM>w3 zm9I}J91o~!&Z%I%cnwIE*aekJ#=)^S=!RR8by-H+<%+{^3S5WCtHE-V1L#(lH)?V% zT}P zgXyuW%AkvetUiMVliuAIOTrXklU37$d^&y#L>$EoBCPpzr`c4Dy_M}*gGtF|`x>dK zY{C{&z zmffknO(HzFMb52Wd3@d3uZ^7KeSh|A?Jq}*F4~PiX=^;*{-cSHui$EWEu8i`PEp>o z_Ylv+?i;6pV;LF#>f|L&}@&L5x3YY`${xa!!17oGPHHBAQs29VkD&hvZ9)P z6zy1E$WdBFV^L>-_3?@d;4+B=V~zQrHxPT@t~8gge_ituxX2s%4;IjvzGdpx)fs-1 z+GQR7Uhcx-ko9w>wu-^1ypNEy388Pb!&+^X6R~X0=8jbnDT&SK<+)p{o~^S7RTlTC z)UPY=2gE!%PnC1N$I?`OS97LJyu6KE_)-oa-4tfrrug|p=m9{+{qyLGTz^ zigo*TrR$UNhhM}Vbtl>hAjRGpbxVyNQEvtRF+uU62*KrW2>MSGM@0P6n=lY$&Iklq6^OeV1#npfLcW)V7X8jo8|9mJ#n_*Yb6s+^$Z*c({}C z@#5Ly_2~^x!s%v^$-iTvMwP^b`k;{x&gYg*<#atkaXan}KN zJqcOi7qg|gXe0BlD%AJvWe$JK!QzehigM_JXld@9AB(y%GWy3Sm}q0h_WUouhKsU` z*)iGyUue(xT`TnFkz@3p3*t?alJtevlU4410Npb}4rK^&h{}aDz$E@$+(;PIsN_H-^AkZ>`YXBZne`5fvhTOn?P z&DLW@L_Os4gN^O6W}ScbC;A%gH~g;&xs>{EHH02cL}3G#-4_q<^Tgdl!RY2VN1c^5 zv4z*TtIP67c7}diJcleAp}djy)!odukqxG6cREbN2l~*Nn(PZ3%1?}Gl=Vo+wdc|B23%%4(#A!;f*9O&&M9kqu+}#fK1cUvR1);0x z6iN6x;!@ErIaU22^RAij*Hp5#=46GGD{-7L@@nYX)IO=rMUnYP{2Sb)#&DP`N5vf~Xp zkZ|d_ntStqNZOF)@)7}geKco$zPR6ld(M@Jc=4MdNGk}-{;zRiX(C#Q-A~-$ynD1? z;z(7=#r8kG-F}$KRKrtuj%domiUc^6JgDU|r^TlIEcqoF=+4M*EdtoRaW+8)-}Bno zd|aCjM%|WMLxw_EN1Ecgdex1IR&jJ8z|>XX9o*S2DhLm)sI3OWS>6&afT zgT!)m@UzfT&O_v!^!;6Sk4}D~)M4&pC}6bN^M~}L6?KOF#&r0vnVjrb5@pg>W?`a8 z*z$*G=KTT>G3qGiCSku)C4-EUP_D2C)K`AO{B|L({{u2Y`6kvU4-Oi>vgvesBOOppEaM&K1#s-2C)v zU1ag;KsS-*pSo}2xcmbRfg7!F+;ESJPeLFLSUhti0@v-)2@e5Es1WVP|ynsK?e zgX50k4$q6k_n$EqB!1m1_y*&P{JI+gapZ;D%1M~R>y*Ws2e{pua++5xb z+Iq=z{GVSKTcQ9GihpD_pEo1HqyH-BJ*NTG+HJ6y52Rs44=|}psSlbImu2` z$rRU{TClEltl&*!TH03r`DfSY|I^3(VR2HosRH9hulDxwnR9LU=s?_VfbmsN-Hi;e z{PO=D-+T2^T`2>uMs_a)m&qIhpSNCJU;?+2W20wvJj-5sl(`L6(PWf8>U0+ry_F1P zz0OlBh2SqVM;Rg+jrfI9eSVedV4lAbW$F-!<7Qi=t4D1{=$N@;O-cXyW(v{=#N?(PJ4 ziU)Uh3zopm`+fJ`|5@wF%5!q|?AbH3X}GhGTv>;z-wo4O&;%apj z)%qXj1y{Qkx^JC1#0S3$-3|zFeHiwD1O54JK(+ga4`tNc!24Q`4s7(sIP;3#-z2#Q zxAL%y$1hZgCP`nl3#r&1o_vbDPr7?evPREkC+V$X^Dy2&wXTI?7uihW<3;x|7Dk(1 zh^kp?;8P#1G#r@YF758f!`?8S93ivjJ$x+A zbut*7uPs60?zi-Pkke8OMF4*Y!!~7jk@6ul`dIGkVBlzE#MTl+7#$yqF9!1~i;Q(( zP@ig2147yHonurdS6s5F9IkHDoLA%FO6zjqs#nc--)?1@cdNo_`EC1Qt;i2p3 zfoBFHhI+(MIEfa9B{!~Ug(x}o$GF%n6#;mSsM4WKElg%S=v;&Od|I{Z6#8!P=u6Xu z*e+1xdd?O9`4U(BU}QbXOq*jK?RNCWgX!$`_MYY!J)hco%hPqRj83u+ z#9_DbNN2lyluOTNrv{pZ^uyA;BbXs?e^Btwk@WNS=|}6To6-igZ_f>ArQYuAX-$C1 z<%?@v?awahJ;W*V*D+mtKfS!3*(&!Xr+0ct2C|Zp`gibrJm-^LxZGcvdi(8P&`_8E zFq2Z6Sl9DM1`&4J7i^O(C{Nn^Ty7>ZS$)%k`>;jpHQ6b z$Wo`_F+oHv=-4)jP@=26PnY0|wTh;x{v)GzxnER*gB?4W32J&sDV!McmB-%-98$b? zt3~QXKD21KyC=ufZ`yKvHf^Zz+bq@>*rn%C+t=MoR_qoEzo|Sb80KUOM(3{Ddd*z@r{p3={V+0^~WGI^rBg87nKPpSN4Zs(W?*;AdJTj&_b^ zJtXblgc0npnk>^dx9kT}Xx363-f!m4&D1~XqpI5c6hW9wbKf{FN|0wHF~;#*b^Qun zK1ED5#Q?F-h=x#*+um8VlpT2N*IH28i`H~7%xahQcCO3*^!z08n5)2+Gf47=+$2B z==?YkFNzoj_2zyY<9JExx`GyM$=%#N&#yZ@9pU>%{Od)u$DAnk*W~xmdm?bxp*w71 zIed!)=m(2^xzv$8qA=%$(mi_7DOZ{ss{g3?I6Z~H3lN9sTlODZhUNWC$D|D|G`3eo zaZ-H?lXSbJSS(YGr#{c&nm+vzck}N#fq77s7iT*%xp2-BzkO})6_Y{N zXhD6auz;s{VY!7da?45nk@FX7_pppdPJd`I33rL>(JL};aNUKAOrG)P-F;Mj8;J-H zFNU4A0-u|~f-|^&z7kr0z110_TbScW} zc~T<+r`^a-h^S>)etpq+{VKTTd8w`n>ec^y1?)d@3$U21!b$5ARW*DoZifiUHkJs^ z$^MhB>YTm!yLtYfLG&4SCglFt7L3I;;ZvZHMpri_(I+aa5a%wx_x>g%R9H23k+}Cf z2GIMXDoS>9ei-9>F{<0L(`|eBEsS?!NVLmnErLwF)!OCDPrs|$=gDJ!s}#4-L1bk3 z#lnSkN_Z{xoT%j;1I(81LHV7k!rwF(M-KcMad9rd2-6ird|AyOjSq49m=p>GDsvtO18Id?jbXAorHC1&313lj<4$|}Z*!Y7ZZW2z-@ zoURjMeX9N&_?LCkb^ROCmm$OuvppML_r=rZf3zsV2I$Wt>-UsR`WM!Ir>F3jqbGa+ zQ}50;c4)b;t~l95r&_hxn-t=(aD*q>z==^_qT{q=Zr$2ICQ3!<%#+$B_e~`451e`B zf6QHuL@DoX!@^0AEqbe-HCfb*j_$BG(4F9*AR!6^KJbuZ=y59dit1$v6{bgBDgPn6 zx2Y=1T0PAH)%h{%;D=;oFDV;;01HI=&;IL&kWKZ%ZfNS3y^AD%r|b6pD1osm+b|tpQ*TYf5I#0XTbayF%sPGXj%mQ8*Q35(+H~-tFkbG5_mG!Sely9^5?jZDEiLmFo*?=fvj7 zKKbE;SEkU(R%Dg4)=v!PAyg6W)BgkZ2U7Izj?=CE zD#dFI%ya_LQHwbF?=pgXrV>PD8W!@_f!|a4303p3vAaoLg}x3RL>7i*RwMB!9ambP z&EW#F%5Coi$yT~)hr@Y*SRBsL`_Ie)=n!O_z2pa!gq-yx(! zeiz{gaR%If&%YU7htN}Y(?I_lU3sKWcE3FJiu-u4b3EPI{V&Opeg5Jt-*7s&adW=V ziTw0Pm+e4}jjqjE6^YCut%KWvcDTGGsc&t0S-+FRCT4pW5gS%+yd5TSv+P76Y8ZV+ zL9XGJ_uR)3cu2ob^Tj=Od;L;4M&O=i5M#j2qxZz#!`e&@UDP^?Oq`i$8?Pws7pHn% z7^o%XBJejN316GD%kg_3XAFwKz4UCB0F8qNjCx^tpudyAOoZ~R!Qrct!i~#KqYD%@g}eP913vW3$-U$*Ti`kEu-v$8Z_%<XWzk&F4qb`WrwFFU?@+h%hwhLP$b%2Ffs~a$c8H|E z|A1%Teh5mTg~tamh4ozfZ%5CIZF8}1Y%mleGbbRuE@OiuRr1@+q+hmBBGX19R!)z_ zK5v}nA=YYe=c7N&zE4@2TouyK&_%?gAGB~kbR)?8Q~v)nJ*CT4=PdzL$Taf$v=el{ z&dD~{#&7|qYrMNN^Kdsv-b;=^)LmM``+e*kkA#1%WxSqZDnM?DxOW`+FHeY#1;K%D zIA)5Injgdgn`;XmfU>;~s9riErCJ3fphjkEF#Fu@aW$(tC;lk!e+j)BeWCTXN3L4^ znfP$Vpdj~cfJLC5>jU5G*XoI2BuXh2xlherUrY!ctvc`mKL29~z`}*sX`@x;e&G8Z z@QJQCIy$!-|4ot{>JQX@Omd;W?{VmdD~hb(>T_D>ev^XwWU=n&=icrWKW}GM8|K^G zPdgSoC%v01;!T$u$suMNs-EcV9XHdvvm7s2RT8B7_x8zF8Hd*om7ER}fC7~8L?vy5 zk6hNOb3wgb^BfM4AVDbPB1KIARNehJgX#X_+uFxU=-9a?9z$`LS^i zb5T9W;owgLD(|J-B(dnZ5Dto4WwMH=UH~~2TN+GzPv@=&@1qED#TT1%<%7`Aldei6 zhqp9nQ9Lz)1?O&wxC;Ec_b~1{!tH6H_K@7Xe5e$--p$ z)ecTCVaWc~8xf?Q{TiXeo`b4_VWd=n*kQLzU!ZNN|3Gesd+iZuG~Ol@&pmOJZaXTI zw3kHKos__9YG!JU ze3vC;)p!F>96X&96q#F!4&<5vd?kKZ?IUayJ7lS*_xE_t;ZkmA2Unxw75y zS_vBe6H`zNgL7OwlIW&oFFBQ>FggCH_;&dAj@_O~_j@_-b=bn4ET`BK8o*I%3=(TR zsn=)~cE3jzrA%3TFejR9Qm)pp!0Iuq&@lD3LHwx*zXB9XOY%scPGZb-h8rH;aWm_&TbcrF zpY6f0V(_Xdj@C9&2>GMKZovSJMV*x+`Y5EjO z2Urw0%n!$~6)0`Ht~UO9;@e*;$VvAi7qID}O;Pc!b2GLtMq~;t?@zdZHK9#|rw+Fb zwzR9-e=mQ~8eIeU7y$pew7Y8M`uRGalZ)a*U?)i>`{TRkqpI5tV7FM&! z7~F(!p|;=%^2Hs*)9lgr&;Zia=RW)G*{AoWyB2AU?>0=wGdx6Oeddjt%cA)%yJe3U zMfvg(>zj0&d{axS?i-*KcPIWX3T>PAfz9ITqSATi1t8D_jn+PQ?4#k4;} zv^cfcy3PZX`V3zA%{_cE{w62@7rP`BHb?Z>!o;EM{gAt{_-A042v((0%m@JLYBQB= z)>(1<(94vv7!8YqFis6;^SK=!g>@Y5K_l-g4I^vJR9R2@Rz;J|QHOk;V%|PikIfrs zE`-_MRH13gJowZb4LFhzo1=rsT&BK%75&OpM_rP0Drx%F-LAU zf5>(laqVdF?J1s^o?kQDK**_8NI?SUCH&V1)$(2Vs=kP`lcDxmQ39TNMi6&`_s*YPWhSVhrY34H8i!(1lx{ktn5naeK=gcAhO ziEITln)9|`i;TRArtxmav_K+VppcKvETd$pffyR`$ETBxYQQejvIDh+$x7ySRCy=vl16P)x@AKs@4bPd_l5b} zH$(M9;WD>V*f?UtG;Al}IPbM;4G89s-EWlo!&XG{c4jp%F^+M&%q9D@%aCxFq;o1O zHU>8a*^?Xmy7S(g><2Cs$y84B#do&`QwOWk=nG3lji^ZYl`SpZVRi#!Q1WnJC<+s1 z;ZgS_-;L~vtmDuCtT;CMT1E&qr?+k2pf~Pj@;%n04()ux z3Ttrbton)HaWj@d=>^H6_r|fY0p(#HbSX1ZiQCyCx=kgr38rI_*{y(1%vAD261$de z%P5DGbT%%;70$GQN1o#$;{jd+j@jbN58u8uU<3U1co%-3+>K^R>ihi|ejS0wY|b=| z0%E=u9!b%iXP4QkW}7;( zQ6`^7+I7mJRLhz={~3?Sk7p~g>4{>hMgXjAubagD>A`vYAP>YMl+|S>(2h4mjZ@Um zxs-2G3hRlTh)Z<1iikIe%#GipvKn!nobA}8*`mh`^G?jY&&M!LR~S$Q|0tOU43_x6a^wCvgX-l_`L(6Prv-I-9o8(w8{ zuno&`pC*{3^?;plzT5w7sbcYtZDm3l7WYTn_Edm6H?;-t7stn|SYj2+F-avTVTv9I zon3A#fRotH4D>5cTbo`|Yep$;V%Nk|PfcuV$Xj2V&*!7G7vG7j!w`8<=kk(Z-mB+8 za;)?dx91}N#*HJYFA_dKwo~+CwmEUI6Q~+^rxvq3=u+-p}-&iJ~PCn zd^X+-nhrj|bsvP9rGsgeLK;Ya&->%{+@r-}q{=GES!HF;fD&lS&;b3@?4l12eJ|?Bj-2qgdUb@ykua6k?4i@r(#Dm-zf+4u5OJf94 z47NF+zJ8pv;!k?9s$~%1E$*5=w-eOzA=_u?<0attHI4Uo0O9P!@;e%#ZepTS64Ej> zA80H8Qqd%G)S&j?W%NF&4PE7lRu8ur%L$eHL~FgVeV?hyd@*o*SYAk(?CRYReM)vO zuq;@X6?^(_@Z!cz{bnG5fGc&-;%Jk=?VB~o`cXq^@PR~ z1BTsL-S0x8Ggc@?=z|!kR_^HB&DxH+kuKAW`0Zk19p2}1)YIg|HEG$L70*QVbtVi1%M z+n!r-2Ofx_f7NMhU8Oi)k~$!gR0Y>87knSOR$;t471UPF zf3(VIa}B`}2zKe+0c`K|C0GSpbZIs!%HYNtJfLSfsq#~6;v%H&Z|ALD2zT*%VSUGd z;p-RLA@tuPO|1rR`TR;rphHA1`R}5N-o|ZwR3}=5?K+CK1A;^UD6*5Q9)Ej53f5W< zQW+a;TsPYgnCM;#QknfD@{-3MGJj6Z`_0wVVOxj#mFxg|5miuC3-Sl9nw+;{rF&RI zAfppE=VWoJt6PuG-!G{wR&qWeTj+YWi#z?0P-kyvHP{G%wmHtSNg@6F=LIT+lU|;6 znA>V*pn2E;m(;@NUGpG&SOe$K%JfS zGC{pc;(sKAMPc>!X)36Yv0?EnabyF@9yZET-Ky$P<<(IZ__X-V)}KjQ&AvBP8(23( z!D!4l*JGRkxS04{mpO7Pc2O&rNjIKxNO>!TUjV6bSLxeZ6Gn}|ntlc2lnKe3fWRIa zPA)(kmxr&yf9YDi^Y5v1vn&HMKEdAq^ewXnL!ij`9iKvd`f`{t)Zwdf{Wt3%|1 zG>&wbVH|2C4y7mxgro~-i!BXhl5Vig23}c0+z?Gv8TtJ^8`-H63UU5?n(hw^v+|?0 z>M35XpByLOW5um(Z!6Gt!QTW*_H_P!`DkV^R>#bUrwvTd-~Fba{z6et(TZ1c_T*nn zH(zQ0Zik8zmH>(84(sGjmmEvUS*35k_%aesc0YI>j6*QzKUX&S&x;03K){l{1&UwT zA`leue2$;zRRY#Z+3&oo+~?Sb!nQH%kE>=~5mH^M8%A@5hB>H%4_~m)b`on}oPup* znJ$k(-1HnSOn`C`Tl^G-yb05UQo%2FR_+OhM6uTu;V|T+mir}bx8gs`X;6k^x8S)g z{TtfZFFNM&^4)hMK7Hp8829x~3UV!KKm3CuyM}_o51^n&aKYT;Qr;g*v?8xSE2~A1iKAYfS*_8RGmSYZOCN>K1#KMbu&OD8htqVTdmYa@i(0#41@P zn>LQF0#k{doB42|ah*Y2fke{{^SHa!S>ANnfL0Pb&>_}JA>Z5(9|sTsVfT0|ov8g5@r-C~ohujVBB z(CuO4_wj1BTw&(Ph&^sj@GMipb8`1o2i2*Ebz5!0#S>%ygQLg0RY*WFLb+O`7mMoQ z8o=q4+o!NPeddb$MwUIf&aDL2?(f{KGF2y;OvfA~9tVJqlXT_Sb#{i<>p10GRXiwOP4DMITGSq?SU1NKa1SP-fvT!y&o{6!`u1ulX&n7rpawOdeto+FJ0Kg^7+?YJG< z%?xb8vVh|Kdx6|VLgL){$Y0r0knTyhn~k{0(PW!bXxpx^-y@;EBl`JSu;vmDgOw_X zAczqbYI*D4v+Mp3ElX6V&VOYvT}|1iU<4~1pK54g>TQoHui5(q+swiGvtsJYf~8ya zRNa`1SXzT{@mQH?6+2U@VS4X9%2bfh7SFuU+M3x59G6cDPhTD z|2Ak)do(uDVJ7^eUE}b6%YNoQwzPSMqx8;OqVDpL`j)^! zL5cxVcTlfws0h&HZEe*Z4==WS8m7*1Zo|~~v zd4gppU_76zpNZZhTPcj|CwNLYwaCR(Ti_SA;O8!XU2BcVtVJxQ04i&nBcKxxO7<8U z;OIwYm7e)8r@m;>kL+4={7bn3_&PbKioi37P*UDiajVhq{L{$_k_u;6Xp7;q0M-fK zVF5{oY`H8Md6`2k0X3Cjcla8O+1S5P4~8>NbN`2Ulv?qnBTvyFim(VNt6nQzAI3i& zdsb;vq&9Q&jfiIXA?O{jhQo||Lc0!`i*!H~aEMZ%&)8C7SRIJ3Fcz>>%i`8goyoLL zdD$DdGS~|nWomqmX?#Awq}7a36X5c*g4UmOZ}(z);!n>j7PV-Y{tKjh`SU3_-qdy% z7RUWzXGzBZz}Gn>rd{$1;jRJ(&g$J|=j{nxD3h~!hAdQ88C6&(p(uP_oNtYZP1pFL z8>6%NN**F0l%R|mAkh;n#Zi)S1BolV!nLUl%Nnw@$NQc70G#wA4##EA-cLdU>_h(4 zed-P7@ceqthocmf5HmbHAqeO-u%dpNKK^isYW9BBuI*pC^V909mymd7IFiM$1wk!6 ztYNfX_}467(=x&M48QOGQy3f?P&Tr_;L-u9@N!77Lr-?AZ#cnB<1nH%T17zXo4`hc zn;M!=$&S1m}EOTXTj{?L|ucPZYU!;dQ zoJBq`Wro;BNw3xGB4)pb?>cmNwwM(9ZDSLj}KqEfB7DQQ0 zMI7TZRXvDQu?L+V4(JK8OSC4OOO+p2rps@v=Z@O|VKJ(;7`P1-Qm zIoLkRbJKP|yrqrdeQ7ttBpyqmr8Q}AJl%NglWYP4<+fPlC>36EY8LB<*Rs54twd_` zIjE+0g_94gNcW1hcQPE!#7_<)iM-N>`H-@@n6F1-20!dJ@xA^%{ zj>iRvtwHtN=T6edCRM{j(?nL$tkq`=o7DS9V#S)P2=nfB_&&b&YYHFP*990nvaDo_ z%Ql#Yv|S6B+>!JJq!{)>_WaOR11Z*FnM5=|^_CHeygzQRA|cH1e=dln|65{Y>I|f! z|3%!*Vfudm-dsHRqYj1#U~?Irhz>Lkwcq`D`+cSTgF`a>VG-N~r3~IvUN}AAvhbIaYZcjo~S&2LH;= z@BZOMJe_y{K-IG*-0N27V71m0qd^*zFoG!}l>}qPd^FuGIF4Cn(2Zo`%2oGo2-f9d z{R<9V*!1?mlR-DiT(9>tbFolnWovnX9p&Cu>1&B z+#caQk5^@PgqqYZ%KIA8qev=uE}2cLaOd@!_sQ~ao6XmovoxDA5PVmQ^~IJJ?(LQ% zDt~j@NyB^M+fl&c#7ZomEDQem1b^t2_W!OAZ9hIVBdy%;omCvY%gU~U;%&f#jBWj#zKt1%uI_IYTp8|4^D1!uBWWmRYX1L4 z6c5D&9HBVT{Oj}rZo50?M8X~(1hd6%|>58F>EEnksK|6gU zh#*$-4^R8vqpU5g*?kG;n&2=M^zV*~A5FTJ=?wRs{+0?coBv0MGOKqe7^6U}?HLiG zkYTCQ;{VKo?+5N(>90Y^wfT16W>yO4t2c%L#4O8Q-xMx6P+Z$;sS3@z2?d3@IPgRw zi12J$eJ|fa-|%?*S9{$Djy!EY+VhjK?POOzRvLNx@)xWHc3?{jb+bfbT$hD)EWUrh zrrmV>4SO||%^ZPCT(xvTBL zvJIh2A?29EdQH5b6B$h)Va#KRn~fTGM?62M@9)_hRr>(N8iIi`qbG^FQ8)VG{NOS* zfh!59(7`$;o=qIl7bSt2URTV`T_aG6xDDbf<0EQWt4bg*7Uu_?2lbT|F|5MJF;8P` zc%wm|kBt)oKnS2PwHox1?YDkA{@O5g*0l4zdHQ3jRmMxznlW{6cJtHDP+ak1*H*RM zbaIDL0y7Z8(BamWce`x`vgQw)kE3w#!0|6!e~Be};X);CX^b zXGH&zinfB@Y5$Y_bN_~-lYBBcfvk1F65biU!E^-#}NpI-c z4k|SZbc0ua;KCe4fm*km1QV%4Ee6*FXK$Xu@&?7@A-qLt;WtS|OIDO)X_Zd*!UlBk zWChq|JOw$)Kz3s_>Qj8Icy`rBT#@)o>69Hr19*hL<0d8*0#cS{k5-0yEb8rd*)Gjs z9UgTCJE!o{)hTZ=wc+5FzZZbTAH;)u*6Qu5b%kGQBB)|!zpykzs`A_#0^QNA=kI*< z8ARe7-t~vFaUH9EUMK*1HGSke+m9Z8+4t6MVZrPSM?it&K>`M8D$o2Mta?Eu(c&71 z4Y&JPuL@@(5bbgngAf?j4Y(@4cdK0lx4DChQR5K1e@4_bbP0qRcQO=ZdYboxrZh#F zB=EI?JzY*sZ)6A&=On(+5(UBbG7F+$u#qtv9wFD_ZGC9iu0{hOHr=&f%#m5KxUpUI zSP~R(bqu6^PKb&o3VS*bc= zIX8XdQPt2o8(8Y*R3*fD$s#eLFBmrnd(*Us@3sS?x%VYIvF@DS#866bTT(QBaaCO7 zf4LY6R}}G@Q*1^AC?G`ScXQxlrE8nhE26T2IJ{?GVLrKzec!svER(@ZaqkASGCN_T zyy}+0jFd!fGg{aFip%l`%zb07{Z@}xYTR0$Q( zEAYW*#NSv3?t-8az%s2+w{i>-)CpGOX9GT^~CJYC7= zw1&6iprVM}%2(~mL~CxxRdUSo*ysWYi`EmrF7+$Dz|s~{;LAn?B6~pp6l72GKK@-m zmb?~x9@oFmGQbVVAB+gHAE~ofAOyN3A?26b@QQQ*K&6Uslca? zgRM{+eNjqpL)`A`vqI)XIJ`~47SW{s{aSVM`kdSolEfeZ56mbIfy2_wRAP+xCY0f? zbg)e>aL;Qn)KOeHbK6?^e^~o+=kSSanlHpe8m#LI`o+!lbEz=kjs=QL`9}ZFkWy^? z_1kUWJ(t!7ig|~M%Er_6U_n5Krt!Iv7At@K9Xw#0)SqER*2uU>{}jS-O|3r2GlRnM zER*Cgc`x`3DuXUILBZqC#M|~Xuc#;xpT0gw=@_n{G!tig@fdfMtbZ`fjuEP$wvg}C z0bS)dr@WZUtayC(U)@ z7m@k0(0N$B8aIY@FFJMfC2vxN%qV&Na|`fv)dGdfi%0PYA|m#c-y8^ojKBw*UTpw- zcg)Et;r2xGMMIgMsLW@ZbrMkOA4LR@05dbt0}AC3e52aS{U#eA`_mB7dU8nB*=sWk zS$_0DfP`!9iXZLwKUAY^c=?}Aaa|P@3wQTQ_t3k*HZ5Nt3$dqRV@ux>N!z(s zl~+}yJO4ZK!YlI%R9c*N`a)M`My2h=234O{x;_ind-zqlwzQ06!|j0c-;;B{)(f{6 zyYt~$WrU$h+dAZ_B{#5He}#$6YD6onT%%Ws^t46=;bKNH2+It$M_FxB9!y3ek*XcA8?t+ox&BLpsm!_lqCEDX^R=F|9w;sYhMQ zVFdDCx5*Nz;%fjKS&cZO-o4s6dPSrDA5`QYl7TVK;v`2-HpKG6I~39+(!ix*f|w!P z@@TOCmh}fD!B?-^aAJ2NFA;wBN{`1w#LKXZb>!?Uy?cXYTxCFw>{+xe+)Yux3VEYw zU?kOPu%lo`@ZJGecWbj+>;PHE`d3@~<|mOq%s=#j2%&tkt|Q|upQ#@(1V`s+-hsHz z3~^eVo*U;8Q+iu;_Pd~XEXmbEF9xDG;+b5uyqQX;=r3Ifg%y55L&S~OOR6NGkIL-z zs1(|)BuS6vq?C*G8?&5Sjd%@;j~Z_hzry3I^$?L9<7Gg-`41D zT+*;SCRMQ(d)5d~KOCAXQ!f0+cUk-|-}RTB%+YP|5AAK9THphiOb0}E-aF8B%)3DR zCSj`3FNU*P9h8e58iV0>TaZl(QrM-ur^#6*&5NC0s5m2XLDzBb8QXVFUHd#aR~ zyAdHH|AqH!p!{qv840TJdORYdA&<8m%%E;tp(&I6i;QinM9s_`7tL-#t!{B)?Fq&$ z$l%>>EL8bc(~G)`ekSrRX1VTKB>H$v^CLJv1o#uaI!V5RSM?&|v1HedJ$XO?c(IRAvXh=52GM)XJE0))ZL z+IU2IiePTS$maZ^y#p$|ONJlv2WakcHzUOpAUpJ;d?_=JNFG35KL^mCFQV+8x3dOc z!w8746KwKaW3p{3=9I>Mvm#l!8@;WwO#g2Eb-J9LZ1W&>h^4MjdZ+#8vJz$G;Ukf? zmfLe;AR5or3AVJ?rz!5}LO}I+UgX!m8=;fVhnL!B5Z#d>O7=Ep(e_b?SNaGFHOHWo z4Ts}4I^VHG4n1DnC&v1p6e>R7@PAa+!2j)3clChNzzX0u@yDtarQYmgIcMz&I|QA! z92CFsew(6tUS-t~Jis|$G@@kar7kwYY~tW$8`gD=-sP|*Rv_nI-A(3f8h#>Q)m4JvMw)-+P}jam?HTosBJNy@UEYbmWjI`Gdvh?8r32ITv_SKlBEL}TC0 z<4VTim0Sw5i(#(^AccqUQ{n!S-yhJ^up}V~#X|QOvV+oQc-Q~4>bnP+2BI+QHk4IW zEX|P?ZdC)ovS)U4+@90?+HIZFCy&>?Sr<0XilBqrn)3cb&2rI~asiiibF!{A!kTRl z>y$wcrm=3pzFXbItv3|x565tGoH|`G8-rYZ@9I+tEUP%F=t}^K4?z63dqMVZ^;C+2 zPCZPt&TOFx^Vi_=P8F5)X@dqYOSJgPM`OECP!FS$DZx6f+V)1F2omN=nXzQe^2&`FHtKK|fSC+j6HyH`uZS25GR zO2Ur8L(E%gXSvF6A#j7l2bZ-PHz$$EQ9&VgNcIzqlvVv>HgFo3?^X`ahxp6*R}s-{ ztJfnC{!czvjpR7;V(<&l*NdMAkc_FX`p&*% z^7;+RTVi5rUw)s&mL>-1yZ@9-2NnJ^_;Rj$Z2lZ`rMpWmKw+W{9BEn7LVyzlmYhNZ z%msJOEivr=VAY)iFTM#U0Vy>olV#3`8M@Rm|I~;4;1AW-29^q3$UgM3d==znBe@-i zcm%i2iK+V6BgoYx9@~+My&{37IL`-c(XQ?C7Ws9HZ6aByBFpA@@#X_sLr-WIVhk8> zW{6NH3S@CtBM7$~jhYBrENRgL*9k?Hz+5RQY8!Svax32iJ?&~KVW8)X5p;%As=?zS@nkejYyGt#3Uy6O8jJr~7*hQr1)H(aSpj`J$(T~&;;W>~aM?lgN8W$#Yc zBiK1Et8qoS9sCaNO>ydbI8tYmMS*k%(KW>wGFsClt;vc+6z!9P%!itbQ^-~A)~DoE zC;KZ4FgI6%n!V3>7sKHm_v#7_lLCt&X!-~V;?Fi0IQ)2m)89wJ<^ELqm?HZG2W`k4X3*!CT8*x3&?zTVybae62>SH92i zCtkQ$9p2v_^c>=Q9>5WKizj#VUts~!5N7RBv?aP?*occ6%@hXeX<^=^zPn{~un`OX zi7afGAd*^e`#_zU^;!1h`B~1gYP4M-gpq`GO!Y-H_1r;zaU%O7_O5|)^l9>SBnPv! z-f9tVcMfSKt+-W_i@-pr^+(<(j}BEggN<1CdO?U|NzD=LGFLOEU%+dOE?w`P6Bk)U z?8mfD(t5kUs@a1-y6#gYH29 zRp<*X7esa6O7ALf5~b-eNj;Um6Df!SU*^>@wZ|VO@*$NhQE7QCk1O}Zw<1C(5bt~{ zNflT2!yz=mK4v9I)JXb9FW$#`OKUG8KyD5128=4M#z^VN$ME{t`8Y!OJ~;AT)@LYC z%q^yC2NM-aUgd7uh>0TJ7)YA8X@UXJRPc*d)UGdG-S9;zLSZ`Nyt=0piT*@|5{o3@ z-#oc-@62A;8=LtQ5;*7q&Y9G(;1bXfyln)xi)Lo`x5c&`wRUOu=E937A(mtW#`bAN zG@dCT$_AbVf} zQ5n&3?^}Pt%RguvEey`2XY2^FtG1!8_ zsZdR69R6a-q(4Yp)p8%%-^$zcpVD4nth?Qy&l5hizjidJp((`X&sf1u=4xz>8hmQ~ zUF~J|%iKuzml}V)zTAZVaV;+BV;F#H{NL%}^!CO(91a7-CQI{NEX_={{a4)we0nr8 z%aBpeH$*m|V38)YJjeOQHN;7>M2Ll3Z*rqVY$vWownr_-im$~u=X2;Z^7xrR7b5dQ z18@D(OJZ0`nK;8fh?>b5ondWXvAq?x8hh{fg+!HBvd#8f!?5qL-D!4INQY=6Z9BIL zn@;2eHCcART)Pc^hA$seL%l#L`eu%-;q-Z8`pU)@wS}&X!`9}W&hrJK7UehVg9Qc9 zGe8!<`sDN^FVK>`Q-hRs3Dg9BdC#uZ!#Ow&tlIFD&CK>6iiLqEonrWxJF@>?Ni%Yr znK0n3<;5EaWCix>xu;DOm-N7cokD^waa{Mvi!!-&HVKy#_%r|5@8y6vWZ%TwBItt zViUeqzx*KrsTSjgb*b@gcz`Td;$ii|FB}$tK^91r8Jae=co1V5|?dmQq9C7N1`J4!sgzdT2=2VH^m(L_{08SRsJfx zQU<|gz}>H`FU@>axrNSy$ggd*(wW9##3x%Nwof7P%Ds&UAg?k0Nf)@~Q-zXx9C=3b z^CO%_iMSi7jhBpFuH)v_#F)le)I=9%wu>5ts=;}{ zfI{j$SX`1K1TDQZkGbH=ux*y&Sh3_5%T9rok=3GQN5}pwECk!fg!?JZ>&%~6x9;%U z!|kz}XIFjijLVZaJU>VI_ZQ<r&Eo0P_)nOY2I1(&x_+ShOYKCct8*&72+(&{AC1LO_UxmMDVq4UIrw)aF$09&+~ zaMNFIW?rl(vQr?>lL5MUaDoC>PLRi6PxPP?kCSf;FcT9W6?Ge_;JTcmYx4zv9LH2& zDf8Isdw?%VIswRA(7+vr^8Rq9;INy%U?LTl#{h-A>+J2I{%v>DRj9Yc(ZhD&n8YCd zHSM+7xIfhnTU791>4%+wM8LyErb25#gr%5&_qfOo25B>5det+B`;v*4=SE1NEO<=; z&tsyhIli}Qd%Sg{f^;0;PGm?+qn31x3z*$7f+7NBXoB6|6NJ> zVHVNdO@K)$!IyoJ7(&D`Oo^qyPz0F)pL3cZa8^kX|54Y|F}jCD!r3d&%T6 zgkWDhp+?1j3p#5RhBPn|@2Zd#^hi&$U)zdQv@G?{;UM15Lf9jL6Q^$=rq4YZdpQkN z{FQ%I9!(LU`9E;hG}0uxQ}(L)i&q=2X>!ClG!%L0(u zj`9wuWIk9s!QEvOo1lIA({4MA9hVy8Dy8u_U5qnMN;@nVHA3XZ<57S?!jo{oaGOxM zRB=ox@F-h-gkdJFCgrp(o>xFCij2vB9}}PJ?ghyBWjRMEx$`3oQvHjS2Q7Han8Gq{&ZX#tubPwJkUoV@ zAi$cYM|fgFxN`Jk0Ms3`!eHt;@F=t`dUwCnA8~M|1kBHVNJea z-zowk3ZkUM6i`HBgp|Zoqye;2F;A{BRR%hKCEzI>PJ^3Yelx z9<7C_Y05=Dr!siv{WA+$Wa`$nr3LeBQLjflmw6OcY^|9cUYjy9nx@#mdQ$1UWclP( z*W}E9=K@8cIie3Y=88@?Asaj&%)OvG9(A298V=EkZIFULNWV)KE;M})lbHZbn` zRps0q+IL?Oe0oUB7G--;8b9`*RqFADjfiX8fq#PTP<2CQC8YIHzMYecq+7q;J+>Ip z>i)B(YWv)%>xD`+!o{jLc!a+TMyFcEOS8Xf31bLqeG_ zssk?QAXY51=DbH_cRKN*tTob!<;tsoL|Q*whz=LgaL{iijK1^Gf>aE;#&4p&z1A@y zjafQ7G-yC=>#FkJ`UBtKiu(Ek8fMg2J@6WE-aRmM2|b`^8aEaUNgj~q8C(7?+A!WH z;&4nzcyPYKbi%w~80;3u`fF@;Sz|0Bv}e6!=9E^L-XX`??_LW)Wka6FVS`0Q5$x3xQ;d>SefPS?8;z;EHAe((O#rQm$Al3e8`w~ zpcnW;YSnv^_ty8{+a_yX15+^xZ^mEL`b=*Cir|~_TM6q@pET`_#U3YCP^KAKyZ<)x zx4NWw4~5qnZ#?-osrf`r-0K{t_bnW_j75LyPXK?rJ2H*0+9 zj*TH|Ekv0_SNlF?^hv*Kdz`MBWk;_(W#jPSMofaI*xnj_$$$nQg%nAF|1U01(Idy3 z=Uh3hoPLKWN%vllA>5U;Zv18$&H%=u%@U{xjPuhZa1?lI`=>^S+hw5YKK| ziMUT`VWGQzv4I%BF0VctkL7bt_)A(ErxS)HMn*5yPO2fa|F|se$`4K-8wjn90Ca}R z^Rur_FiEhdO@M{aqaEXRNk}7@vJfTv7n$(1q$}!pmTW4cG!Yj3(qC;7^fsOYxQM;c zHw|5?D^{RyqmhSvYA2(arD64IaW4%NUvvvs%7SBfaiS8)@dzndeeDyvI-Rz<8lcji{yurS&l16YH0iHDN9SK?V5C z_vs)Be%i_9Xjx0`SFf4u_Yg55#Uf=ipBT&C~hF_A)4jXKrI@V&42q{h(S z=uRViDQoxztzS4e_^19558+|EhA=xSOh+tN`RlWPtWovft8Kq$AzvHW-_%d^C{Q#M zEDU3fxx%K`%_!)ya4p^yz<{T)&9DB4ZI+>;6=ipnjbynL_F8iC_w-KxH85Mu=boo` zc`o!_mD#LP4|l##Dg@koCrsE4O>o1&fR?#KV3VgPfGqFzU;fu&|5dK*nJWo48nfWtl@{iynMnc2s6rYj_r zvt}l(VeGaDdZ#74J)#6*QKaKi{?-%iiLC+!FWbxG+H5v0E3$ILo>y`JhW}MSEKZvAFgl3E75O*wJt5Lk*%0VMKh4p7Z>dEt@I>MD$AYH;kTS2XRW0*o1-OPS`&kwm#7*P+emP>dm_vMe6sAj7n1IO zxD?qXN|)#Hp9fT$?>(WI@oBR7^5DW!o$4^Lo`B<*% zt$?e~|EjfWJi}F)23Yb~LNfy>I{yp^SORXVT&ZRqqkENDV1SeduPet zIe&@TH>M-swKw@{R9EQ80^XNhuGhEq+s6-QH5N~rQn9OJe=x%US@s^)jS9A7LF67~ z*`{P;Qe+KsRNgHGnQJu<_2jAM%&Y z=mee2Fy5ScCx;EZRDt&{4|mURTTGODac1pkE69V)lZ;CO5sm{JIR&23 zatFFiq%XIqBCR&0rwyT2kgwyR4Z}L^I+YAYodPkOm!#HJ&yRU9O%hbJ$?m8-H- z5)P^6?Ker#q~-m7o3F3b8}o95cO5=J@ntyKS?JhAa4OY)*# z^Cgm5>ry&?N1u&97bZP(=0d^vNa~kN5bxLI*o}pzr_amkf~}n{PqM~-5XE$41r?b- zRiVvKzHm?`Xeyn|gGS_G-Zj^U7k=yIG&5;nHNlw!B2k2A^&}w(+xytg!-)m~c z(q%hh>-H%Fq&vlNr#QBJXstU_k2rgUA?--^`Gto=5_2VC#+ zrQPl1aU4tbOV(+pm43_^;aRDAd;p()N#_r;_5XOkM0eL^WF`gYn+b&S6}GKeY#C+R?k1AXwwpPr>@KO@^V@FVyDkB-5hjWi5{GfZ!QPt@I=!srwwN-Ai>%B+e?Yg_Ieso>I6!*V(=4FUq@ z6se~lbF1Ck9sWVFFPfc)`~hw&sJJuVa4VPWa?U^a)zic6(*6)AL~-(MH0Nen?|n6_ zILwV%6KJKiR)jLHmNZMo%}eN1`?!aU2RB3S1gjo=*pQu|qCm@|k{_gmk_}Bc&nm8z zrGq5R{n~lb0;F^((Nr#kZ%%&BKLfehx)S2PHPB$Dkjt$AL09K>gSB^o9TX>eot0f? zZ%62eDFxd2H(4UJmeGm|B@P9C$?W+AUZ7XT7Y_(h3UXpB6Gf&G#jg zE#(n&`q^CzA?~*K?lbm)f0Mu1V!WE}Gnk?&v&>^JpSEmHMbq^0ck_Ks7B5HFXS#-& znM}`mGgX8jOiRGKA_%&r!=*Zl&)m)Y6$utEJlt8!Y^}qRL|msI2+=Nv7_KF|8jFnz zJI_%L5l*BeO6T@Tts5 zWlJQhSkCD&!c#hZ!=D;+SDm>1zFbcX%cTtiRaSThcK%58Ki$px zw*9lz;u9!<#s5ju>je9rf2(J@s@HzYSTdL}N1~qHc=T;&F#Jv@1cc> zrlWOh1eX}Ey9pw4NBUNz~WaXSNI<2d_9pw+NqJ)%8=U+JLQ&52$$5>GqUFC>B(bfJ{__3#+?H{>C^ zWL_^DQ5A@}(go@PW|^0sb2bdJUsx(%rvltY{;M{ui9>T+-^4`Hb5b@FibKR+o*mu& zMZalh!k%x^b5b1o z_+{L8_`_}oZaT9{C;5hV-_NVV(JRyduI|IX(i$|k18rzNu&HrtwYzGiw}Tg0^Lz7b zrhMzhUouqaiu@Z!_c6XzZ6ze)a>g zMg52#m_%4Vy9Q^D2TPp#<~HlLKKN*%cC&%c_d*rw!C8ZOQPuaEdOtISo-I11Y%DUE z1-x3sg))`3q$<@=(<~^VmF6j1OTWv18d({NKN<2nI5HvP&)>pQ(>+;oAmka-BjC$- z6Yc{_bCkRVIYrC6oqil-C=7xx!UUpLpUw;aYI(@DD!%I@+z_Rmh=t z*ek#mb0XwOad)YJuZ$`5f++$r*&D*=fxvo;h7GB*6(7s))-K*I$FTg_2}CPa$Z_nI z%L2ttGDNYHc=f2To#vcn>~1CK`Ti~I{-t75V<<+^8Sej(fxF$K=5W$FDS}UGl$YXe zKYOA);Hfef&X^f(ytX&P-n{wRbv7|vExLCUK9>Bl-t4{k)Q{zTCYqqme;h#=K&?e} zAJbhV!DxC*W#4Ayy8`D!XD90z-)B|#Zov9wtsd!oML#TqnMnUq;ovjS+-~(9*MQwL zKABFdW%qlO>>Z2r8}XuJwf*D#_{rLQcNDeY0^*|ZJZh$y-{^KKU*mfC50J5+H1vn) zS$5#J%URano;Gs=Ps--G<b}a+5Uvfk>2H)gRm0w){oBWJnUPX(lN-B_(zjBiC zaQYi&iWS>vuVZKPccJ3oip+2C%m!Yk>}88@>v=(r#Z2^rt+bVM0sZTbi!wIK%*QMo zUO623IQ}f3mcDD)H^N0cAqa``MhQuU_ zN^^x?*>6Ij)qNJ<;?%gUullpYDcenZyDO6EIBf5}>e?|SFXQu31KA!?C9>C|10r&Ztxd z@L0AeRjDsIRr>YThYDaD9SSH{v*N5TI}-;<4(ALz3NvK%Nf!ACnVU;GAT1sk^I)w( zjq}?KzUZ|=zgZsoP1cls!MRYTvFaX0$-_Z1aS;VuJo4?60ujfOfsHLUvA-#eI@UB4 z4s=67cB)XWa;g?*QJ}UE_lL4YFJFoPK_;G`ZMnk#l-GvDEV|ll6~D>~`uxwe*3M^S z)+|rt2|cH_i_QqI(0O@wq#}QIyL@$s>DjHnM$2r8v!r(~fq311Fc-pKurNi1U1cabw!E}uv+&{Or9aj_sxBSHrlo#QZroAwm3yW| zu!x4}4xwjU`K!v=6~k!ui7e#mnzn)SD7_RTW?|3aSpWwrA=SCZ8ZGF0cn9_S+``9S z)j-zQT~23)VXoci_~?-cGrhiKc2+~q8r^1t$(lIp@9zN`@izrQZXcYAYlr!1D1xCb zVpTw6U9t#T&iMR(M^hr4!*)}e*f0cb@yH$1H6L`KTsji;PjIv#skiB~n2Q$~s*l5% znxOn_(uKKR@7Kw9>_LtFPU-)B3673ji$EC@{Ky6GCG zo)4ja+`_ZQDCo3gs8_vQ@b+A@t+ntnt)m#7oAghPMuV1H8AXy!f_gl5w!vmP4Xc6x zm{?zN@R@es8DtG=?&{IE3)F?2wlTyWbArgn2j$iu(fXAkM)o>V?cOm%)IdcLAzT)QsFb=PLgvfC+R&V{i2d6<#y z$%7d43vv5OA*ENViViL6KGG!36!N{fA8xle>3HOc5FgC8kJch&&f6dW15N`h{wr>= zK6tufL_2dKR8U+jvXvm%FPUw{bn}(NcfP&Ks6Ub4t!&$L2%jx91_E|J9m&wD{aDlO zZ!xnLyZ#r=Yx7IbVIv{5m}!#8XV%dJKE52uH;eCr6nQDXS~RC^O5nbm+#bRy$S0lS zHDM~Y`6Hf?6}WhuER+>Ifx3Yob-EBYZ+OJ= zA1(FePb{u$%W!G?v^%jfBMwlbt9M)ciU(8tp~Plrf@RKdv8#vonOzE740EnPF5u4L zzuag^H9(TU9c6}8tbL}+yr0y_2Z{{-FQZxcAq8%7!TCG6Cd@etL);4NlpgL^Qy?cn z+~=M^$aZ$J403<3PfoXa$zuw?w7hfsux@=5A zGDV5$9cuMOjUt$B%yn6=Mc|EZkkye>nu6M#9TjWFxn4o%Q3K~C*}WX*Rw8vie;j@e zRkX@IeSo7FU}ZWNav{#A;lQ93j3u64^$0xr*M~MYWz}lrMSYa;8)bzj>)9mpx%?Pc zudy^MOn7$HY%5d!3^L5vnnIr1+w2ZWmW!|tNMZ`d>|3@*F+z?~@;lEaN^7cg-fMr* zc`G3MHtlneMM5n+`w&UzUgXxA43sZ+&dLMcvM}iyZr2ek%uqf57_^wzx8y#|= zdhnQb>oQfTpJH>};q=dig;yBt$*=O1l-392!-C-QMkuGV=boQsYR}5AEMEwC&%xjB zwI)f->UeRiN=4FdeCI-Dvz(en$tD(M{oImUN^8oLYb?E@CETR#TcTJ@NBkcWLEnBn zg$QkCtSlEz5y$xP#^0^0t;h$K{$Nq&WYCa^)~MjQS5@JVFB4+e8H1MGzCKrT+?7S{ zJz}Y{@L4%|bJ*$xpU@s|?9n#fvA&w%>u^fFIlJ?h(xr%hZ|elCS^lAmDN0Tg!z4ox zw68>0gwk*oD@#;f(<5tb?`#2y4>tC&=)JY_iv!E%303>QnaF63jR;|qqdvX-Y%c{o zDbDbjo=tx$xMg8G^5}4ubMpSObI`Yh?!_4Sf1L8J+5R8nyXmgo^tWUKRRQGUHoZG_ zp#xXxMOv)Y%Cp4G6^-?5!&fyCEqL_cA6afYF) zJ%ypkyh{#)I>`U9si(SGQCNyUI64JIhPD^btEQi&@2cCq@DW@5kP&yRD%3p?Z6t19 z_+ISkX1XU=11YipQT_Fu4r4+Q4huT|{Y>VD`%NY(xo>b+)ee_}bq%F_8D$5YLy+^E zNSFC%J#3rhplCNAy7PKB8@t><=l-68!h3$|!e%P6>a$eaE5mt1H);H(zj z{Y%=l|K?TE=(eu2FAYs9CDRAIqq7lXbhEjC!~}dqbbL=bqh9NJ7HGRS+iJcRbd-B? zXuI@iv_kU9;KswG-j`^-Vl)1>X<(_&WTyX1K=!9-zQ5G2R_!r5)BJRiKblUO#|0ue z_>ANF{m7GOeNZb4|JeRUsz$?T_egE` zsp{+(J_Gsm))oO*5Kq37cQcq0qO{{+?IW6z|5U9f+9@6SS3kNs z%*b7_)@!l=lrqWAkuMV0HTU~oKu242bLb~(zHW7z?eH*$(d zhJPuIEN;)+^PjS)ES}$}_W5(mphW?JSGQCYxK?U@$*AA%r`W6M3kE*&XT5^f68Lu~ z-H#B6zJlQ1g3ubbli_XfgdLeeE9}I$?A35gv&Kx?q4bqyfb4oQ8^}a4;#~++lM%8t#(5jkH>)#%d76V184g%n@8xhrZ zPW{fGgf`-M6~@eee^!AZkPf%`G%1Czg$KzBh88LAO(l6D?JaWpZ-x$z38a6tDgu(> zz}~1TuE6DPdJcgXerod;aqf?bY)8{K(?5?BRq(J+E6#zd_kBgUXYYm-)E{c!?c_|TtmmfRs%&rn_vpzK z%adwdlJiPp!Yh5c8%6s$82?{}k7*Vm?B&x57(SkqUTOxq;Qm{TJztm~zpqJtu#~~{ zt0H@X?Ma5HTuZL1c$e%S4N{cjuaqi6PvI&TrGR!9e6!zlGR5tx`7;~nqCYT7mwg77 z)4|%UWqk8ir}>R`+y}D{RCYH7(8(7K+0Ey#87f+(%JN$)RifBk(y&UCeQ=BM9>*W; zrtc)!@3h~iMGr>+*kWb}E`Pow@Jc22i~Gj2-TQWQjyMfNOwoVfNK@b3@a)fkh#Sov2)3P#Y$46B+D(CJ%g%Ko@_k z4BBT4>h$P@xK_=hrDeA2&j@4qWC0M?mvaAOx%=dojuufyQx!l-_{3=vtpa+muO>Ed zk#w|oU}$zy0y14Z->Cy8n|aK56wfU=Tg&Y1?j=0CX**H zQ&3WT-#Z=ns0WeL+*j@xQD^Lm_m=skzeAKkZgBT~4PJ{+M*1x*iDY@U4ZTOKav~fW z1~ObLn>$XRxKX4LV!aq)aj`nLP`*~z8U3H`A%XpEwCHgF zBjZqYW=CI8zUb2(8PWwtZmMArlZ99(n;^((NPoGD^=6f%wL8JbGRZP<;$ENc!k%ki zEDTCSv6YaEK=>-uN78miA6xL^reQj;!f#2o{f*iwHhMFouK5rJG$&1#gB4H5QS>6% z1q2S;x8VGz!j>bWEZ)CvL0sU_~{wl}!+0Wz?snw0SK0`c1dRMHx=;tb-y;DY#kFC+6T;ohT%YbnVA6|qq<;ii>V-PF;L_RsI|U-+OG{nFbK zA5XC#AwKE+yoZHf1E4wmJ@gG(HAVrS?+>Mtp9jfpBpEkB=X?nVZhCi8TR;#;7jKRc zwDPN-581J%**UQk0YmFslaBG)AcnbRCI&P2e_@{mjwUO~MoD4%OeqJrT$z&SY)*XWd z6}W&pzB%c8&p7b1W`ruUSRSt{5!d&?qbX^9d`oAUP}9N2DVQh>M+7aoBXLq77yt3Z z17y&}RM%OG!iqp9Qej&hSraCYPu_2u-;+nopF&SoWq_OW-Fy4liat>TztZ^qOa_h^Cvv!EDGA7QxT zaXqMvgJ9qOjJOW0SupM$tdRgS_iOd|3Oo`4FWBU<_{BVq`ioP}FF$ZZOm_Gk3!vBPe5JV_JUC)n!`aS<@!U!eY ziMY7cGT`c!jjEMtn!&VhD%i{IUS6t5x2FFy=X3Z!aBSv**ms#z-W1HD@Q&J~D0ItZ z#U>k}-VtVqYK?1;aDFTL?`?*g6Yk55OQaf3rTXsiy~QuZ^HfgX<#Q%K5E6b`n{zha zj|dS`L(uclBOLtC-UhB)=g+h=clTx>K1iw3QE@*Il`CM(^bn92@ zS5sw(Fy-oxEmSP60 zs03ODR(;9AmQCGNX2_th#MAB)(+g@R&i6@==+XDFVMx4AC@P1yYoLfaJAldIyrHpYzSq>9bQOsIsf}v4BV}&&z z4Wy?!qSnfeGuVP(7j*HoY>}jSx60EBqf`g!$8>`H=RCs5=W>1V3i~{L+8-fs6gTvu z40VCTenH9U;@VWc3X%8Z$o>rr8AykbtWxNZJlWR&91eBypED)}U+e=wV@h-<9ewt} zi&8}08S}$3sOf(2m}k9}y6+iTe7{lZbNp7`%zuA`@azH(|lMlz9O% ziAe{6rVUzaVhVck&tuGZ!NPwz9=hg1g&!YEBdvc-&XV74Mgd4z(>S~X zS#y?%T&?KaB|oytIwc3sc^n&?L;cBtKoin3^~d+Y8(o<-L5F`C@$V7%^rn-Zz?qN4 z8BCb(SLF1u_8GYd#jxb{l`;UqQ=zRpJ@NQGR2u*nMPw#cQY@ZB(46Kqmn!wJ;oN=$SbzoE$O}&k?l?=@Cry8NNA*YNiKEuSotcC zt#`gvY+c$Cnt*6~gl76gC@6PASbU>o4IVdUDVl&>aH83%YA$$RF2!}VWEgyno3dCw z6%U*}sMV!iI59qGXg(kwm?Ji%>&${FRi9?Ao>gh0A4zFQ)8OTb_+yn+PSb;czAQyP zgw{?gW3a_Vb5nW)p%~&jaR%M36J=~#f+Rc5WaUf=#Aaea=G6q}z$`OTv91x~q*Df4 zK#7w*x?cOh3=eBgdoY z(RbA2$s>`%bxjuoA{!l)&mQLXg#}|blg`ZCS};zVnN>k^Q;-ZiB@G%^bb#AO#iPew z-NoOD*_@QAar%bxBWan}Dx7Qv50bh&DZx(AO{%0Sz~QG%n3A0L8Li?p{;W=SL2zo< zY}csowzmF+(1 zHMsOW%=giTpqu2=pvmg(n|3#AEs_RW7Q( z^Hk5Ad|@g2w04^pMU!T;Q^msmdWVk9hvI~-@F86_!^g0|=i5R`MjWh_4G^ZUeh(+I zF;QZ(4XfJ3XYC@&Z`1Cg=_C15itZ?#c!P%KM0ZOH4;TsSs!;8RL+sE&i-9xDmc-&n z^KSaPrJ_hWdC&WBw|*+0L$}Z462`KAvDKXRj*nG+%p_^g282(Ts`Q?Un~d6u&a`Hx zwwKNfYhdX3h(D_vLlWZVg+M+VF}un`$WWEgu-VyuQI-$O8Tml=5I5l7s}LPCmzKPu zPnuoy8Bg}@Xc#AXEJSzB3sfI(|Cu}&a&$}^aq_noY& zqfKGTca@Wi(|vOHkFj-xG0Z@%Y>rCOQV@|ZiF1Bk62i1SqZu@w1NJcpdXjg;kw_U*VJ{vK;RqwdWO*JJg@SD^I^@;G*5Khxb z!3}dngooUm669^&5@aJwn30GTXxs_0&!`AG;Oks{U15x0=CEklvKY5hjFbEwo(&4x z)>1rWgkxB`+^=PQOGh45#G7WGG#@8%E-lma2?bAlNFdI#P>$D)k4eUhyK|6)HE&&L zD%ogSi*%x!EDt3!3d<1Y$>q)4zlN^vy!m=xrfIW|LQr+dCy{Y&plZUux}dMShGK6v z5%%-8{;P)e4Ver2&`n>;rianPI%Xs$gOx6N`UXJc(c2;U1J=$tGJviS<0MXZJ_p|V7A2aLhP82DLz?`W99@gOhqK4Qn>GF!9JY6KC&p;@d<0M5 zaH7E4Szp+#J2;ox96Qz5fPz4!}yCAls9_z6w4KxSFz^6bipkhn!@@<%jbS>dAo@ zwE!G>I`Gb?xwyY3O^nL-=-69tX|&#YNOjo;vH0o6<(ulaZ!cd*UcVv{#;*5EQi3t! z`Lm5(>Ytx(UA@hGS&xb77O&-ZLAvf+dAB;h$wA@oh+b1WaO@dG;LuzS>6B3g_6EaG z4ZIKF0Wj00S*Io2@B`|2`9DENRpvmX_qi!y5#)=&0c1(_GKxi9Zqozc!y$6E3KpnTtJqALkNTJAzkPbqZocU412rq+4rh8yU z;iWnupG-VHGP#YisOeaO&$(@;FjdLhBZ77^So8S=;YL#d z@bBfYOpHL?pnp{J>GZ1R(5a=1pDNi|WgfWZ4u9-{Vx9t4Cb(^YIcDD${FR%0k{>UM z5{(b`@+bF@%}9r&GohiN@$7j7`9PsQod6)~?SDAC8F1{GY3fs1oI3s11{VT>JLdP9 z<)H&EGE5c0V!ZopCCevPLJF~Ilawm)=3;Z((;~k3M@Yz7Q!6Q^FSEEx<66FBt*n{6x8syJ2nxV!yWdz)Qlm8@o!>|0i_}Non)H9w zWgqC;cDwsDFetC%@k(2$qGZ3j?tX`$K<=^BTlNNR6Inf*o8*bYXhV-q}$7WASHWLD-uYC=&UBAH87}kRf{6@cS zS~c;$1f&bHc5JN+=OMTQU(L5|ISvJUIqu;#d)>V+?n2im9oeBYn-t1aRZGMS1`__5 zw;UH(GYCx~d(4Ec7AfxRa6)W%e;!=$AvtQEc~bK&2yB(%6&B?re8Q_Azm;vA(q4tZ z$w3=gB_<2}DX!AQ96WMI;4wRQ?SUQ&q>B&mZ!zXE-S_Jrup4V2lY@LN#?^F%7z5@P z`m8hk-}^T$Tq2}I5)5Z{jlevfHeL)p1rAIO7&>nggn zlpuhej6ESo@bpBRC^erQ^s#qV=I-L!YcOIr7bxi4j1tNJq#dwS^I|L#ZT37La$1Z! zfo13|LVLS<)j+3?XQ@K$ro!vai`W)MK>+b2pZ@GMMPI+D$Adkw|KYUPV1Gd{)=9D0o*%+M5>ZY6 zodU5BsIv(ni4R_6Rv)K+pDQ0)DId@{oR$!fCJTcv2fbeTY@`K~Miy3!*vx?HR$lZb z?G64}D7Sre&H$KrwVl!!X8z=yH@6QP^Bh?c!T-yX}Lxg!8?uwj6DgXI<^s5C#o}NdY zxuVAsd__cJJJ<5jo+fVJ*KeJd+X!RPiF;cu_rq7z1Z0CNR>n{kf1LQ&2)j$Ua3zhhrMbWa;DyLkHZiK(U6a0_iFopWF_2Wo4gS=Cd#yy`n?S zrlt0hdgiAV%a(Rsev9RRrnxj}>WHH)KfYW$J$qIbfysrIGRA2^7@&fYJ7NZe?Bakm% z$LNO85f}6Z>HzvKD1#NSzE;wGEMS0ch$McWNkeZ;%|2pB703P4dh@A=nr}NE--i?i z`Vo&%#!Jxv&^H}(-=-Oaj4)$uAid%_&)K)$I!a*MA8ue1z-`lG-+%Kb6n{Kb1M3P= zdsDKC&{Bi1vY_P1n>8q`fmdKW%hHB)XFYSwhMoO88L-&xA{ZxppfR-Hd3n>BL&n3p zFW9Xn(VA-8I^5dE>yyFjv2#PX&M#|aiBCuECfJCOBFzyy`TA=!2F}VHgTZ?fOMxGP zzzj`jP>K?swIqo2V|MMCe0bFJ91=nyWx!4 zR8+$fEJ%SZTlcEEU|e|ibrWtofAv9&#U~{;ZyyA z^Che-k6*#x(PpQ*UZU0^t#FIcAgo}oh#) zqGY#Dchq(8tZh!pNkicYOIf|A|B|oSlZugxfAXMr!Drvth;Of}!pMgSsEE2PW(&UCx13gWz48pe$j z^u@l%ng_o&X5NbpSPq*KzakKxO1XJ|R_}<~ggs=Na-wD6+&b@jJ%LO8c-CAI2q$#V z$TS=L+0xMU3%X!w`unKUGXL&hF|yXwm2Qu4y)>N0nj)6ohIR&+P?&<5@I_Ra^aL}x zRP7B=_#sqXTokANfxS$s((+`Ts0=xr^9QlalRFXldY|}NAjy~1hy0PRgkeI_A&GN3 zUEtB+i=?_oQ1WQR{cF)#I$QK=Ga=WNWP+<==o2K~?5Wce==a%VE8O*okIP{Spx!>=*x7H*mnIcg&oa z>apx{1{C1&O}{5JM?U_M{{tC(01^hXm-hEJTbZ_~Zy|h2aFqxPj*4suw+>EDIH=pl zZpZ%Q&&1v0&|8~DvGfA^&Tqx8JnRWGA$b0{wwotuclgw4>gmpb;uo00C0D)MYKL-5 z&O<5D2n1Vz9X3QH{{D~#mW_rX2R++u%}o>dTb_|P61TXv1+RRZqBS%UbELr+!+q!C zgsbiE{j*mK{-+l(_gfQVP3L#iG&`0Rt|-c#p6@UEEWdVU5eVvR4eQ6oB=W@Y*P4j+ zVp%dRNw#cXbuOD|_B*>-@w!eHXW&~uCEhSrATQRMbgropekns-4sAGHoVyRBh*3S# zHEphelXtY+KRb+aEP!#l!|UQtOWNXPaa@WQr+r6~O3?NnIoe;3>y~ST6%sOIjTYNR zp0+5{@WyTeg=xw=B(e^^1XjU5_}{oJGox<(*%m6M?yP>Iej=J%O(PBGDtb&}WwO)n zJl8!Git#A~v(H?>;I8k5eq@td>OBfz>iIWP93PBw+uS2mUA-d@ziNCsAgi{h3*KRDV2;_j z6G+#d0VAoQn(Tv-3LD#JqKpAA2v!3I4jb21DRk0MU>jer={_7c@x_;iyn>k+8xpPq zZRPnSZfEo=y83;N3_MHD+C)B+?hHQeTLNcZq#*0tsrin$eU$NIuLpO8>yj7^Y>G$i4*+-e9+5Wje0ggQ^;>l!&)lsOjcd-Z>_ z0F?65NewU&O0tSoZ50SCb~ovv{kmUP72Pn}du9AeSrxZ=zF6M;TQQ2r`@7FL4K-g& zm*3wH9o&~+iCY!TvV8>p3w)fU(K&hg_PtAN)y88T9v*%6%X9_}i>{Oy)}b=73rCh1 z-U8~Qi>9}7rzc~*z-irtAs4+z0ikSP0<7+Sz1Yo|SS%5(qTEV(W!o6=XdI=(xEIbK z`+LTrdBybLaekJ&YD3e)9d(%u#4pOBVy`0)pC?mZq~g*`60Y-Y4rLbAn>>L(fU~h2 zo3&v)_-PD}(0=pu7bv4VS}c;lPn|8^DKmuA@jtK1)F*ZBJJrmwz+NL#`snd*ft+hd)s=(`?@*yxU!Vv)Qwc2N?iHUf+J88}!1zfT+2=yQezMSjlO*H2%d7 zQ1N1HK94gLI_S!eFtL0hBVYXVIz`C@$*;b+`@=$5O#?MB;FirzmP5#)AD4aDp;Zuh zZUm*=LVCh)oX?rR`fa6GT9xOHSvRDC9K1iYn7qPeP2xU#fPA)A)XJ)|liOu>TbG2T z{0BF~Z*tlnu28Ghg{93-%w1Ql_qM*r>f>rl<@zBcM?H41Sm|0=hQg-X^gqowS+VZ{-3&C2kLxtz<{pT!tIVfBd0ijyuwx9A z)^%K4KPh9-4NE>;palT-9i82T1Z=y(*q|nnVP&TpUNZLH@9h0Cw$P_$ikO`XF**4o zR}1Bdau7dQ$Va-SK;!@YE`}-v+xSYWmnhr3ErIqbI|0H2H_J$woL{sa87my;52z>n z?WFj3UJgA4D#mX(Utdv;IS&&s6MD>U+8G?xhWu{#e`vbus3!ZjuOc8VB@N;T0qM@E zl!&wnN^Ep@GeST>TDn1{q`PDE=t*}sqhl~O;J4?T_x*P}J7;J6-q(HIpET9&eAg=| zMyRyTilYc8%ztkr&g03+u&1wBke4_57y)JF{cB;_u zpk7d=PM=!Ps?BRX=b?QEv!ZL8(~W*)E3|# zla})rd;tk-t!=%}Dog}5?UW6<-tQap2+Eyl|bXcXpp(RC=y2wTS<>7h90;3@Sh(-KH_FCK9A z$0cQ9(c&%S2(^>ccI0t&CXfCGyh;)Y1YXTJo`R@{Vk|RJhb|i+s3QtnWp1#1UJc}J zfO`AhdO?Vk3g_Dr-5h`O9JIQk4l-8Ebkz-|!nh(_v*bxHH`aLq(BfM*y3ONuoxA^l znB22bGT&c|=cD+E;xeX}axV97X9HUu%J=1W)IY^rJRlf`q#>7#b(x7 z=*G|`av4H_*72x8fP6Y|hEOyHl~hq2!Zq!}&!S%6TylGy#3Dcq6NHk6oCWpKkhQeW zqXdjbWzzO#I!1RIJLN&J;-2}$s%xTz!!ttt%K5#9kL@|If%&=hW%tm@`*&}@g?_^! z!xqCzj~orl-!ll5Kc}Ed8oGJ&_I=+^K}|{!MS#=nWTBc!0MHFz2p)hp!DmPrh_-xOpP%GqHs#VGiHOC z=ybs@sO00< zNql?XPwr?055P38GV6Z`O{t@5qN}!NDh|>(=c{55=*%tB9WAq?pZb0F;n^5)y?rOu z+-8wqjnb+2+e_OyUOw71+$!G*7!6=j&F0BbAkFi6AYGXKVWDF?NWxPZ7=nkggxr@+!ZUWqDYRo+O%tO1!tQ6LxJI-ex_c3@ZdE=zj zXfUzbrhLhPNky!e%#n5YS=Lln(m8EO!pVC97X1fp{@5VJH&KuGv$sD~lb7SO712ZQ zLl(wAY2K^ADB{jZMr`EoG*)>}W(E9H!6D9b-PvgX_KK`s`IuXm3YmIh<;)|`MMy^p#1 zeai&Nkcj9=O->p zw@5pKtdIVNQ}`>Id*8XGvjyZ_=gVjdc>>bk`jwp=W8JVFNh$ZXSF9Q(CuKZSI*Lgv1L z<%`GirHcdls3Aa;*{NkGiGLxwwvP7-TNu~a(=tM+#JJNQQLhgAm{7a}jtrT8vtoCf z-fidD*)Pd${#5z#ePl45xo#e#>PfP4ov5@y!7sYj3EBE>-I4VqHQe0u1!|?h!C6L7 z`CNe^R#UA7t>1 zAbJScdSHGyGc;QBXwXq+>j{))ybQSjKX%UJ38=NlmQ=_=Sll#GNY7F30FU9ZE%9R? z-J*G<1DbHT6G`3bdw$r&6D4;N9-AT!#Ka+OZBlq)Kt$Ts^OoiaHC>w9`L)}S+ zH|Zp3wWP_0?!u8-NNt16)!bpqZ2N87aT3VCUw&v0go=<);fYGE`VZtH`dlthjguJ+ z<>f9D{9^gdp3yps`WS0n`+OQWjPCfLYmRv&ej9meLp%+?zW#o!gs>XXeM<(6`dp)= zGnUm@Us*w$-X+vM*D)3~rQ3epaM@@RuWG7|bE{y@bPAToYq-?xA4pP2`cwRFd^s!- zP`Gw(`DS2!d{&c z5;zoR1vq>U)Nm;WCwh0hr*VG4Z!##ClBB5dOOoP~%dDMfDZne$Xb+E;*SK~-*H8*^7&ZsrVnK+2KRd`^E}jC}MH zr%QS_HyJd(Pp#rycYITocqAbrDZ3$0lu8Ac)#N*`&w|;{u6lBSAzHrP8_~3pt!?y5 z=*-?@fV{Phs$`)z&IkP*Pu=bZXOk+L z_2=$8ZU^fePR_S#SkNvg7^MbZLwk4hVA%cX1aOR6iLi%1-O!_OC=ut#SrwWxmIk`_ zNBx5Ub=psg_)|&@{jSc=Jn#LRMoA!@?{&##FGgXgivC;7vM!f+)1};&zDMo}Jo7BS zy2jAC?#dncGDi+oi^cX%hir9hu>kTl3FvR1d)LhO(#7=`tTc;GZ@9S#m2S8X7eo2$ z>z~hfYR}bO*9epdREX1oX1B~bJN}laoqGQ?D4h{*ntog<9eB7ucwqQO-50`3%k&Id zOHDqjaq2SYSgdZ+LeD+$uZKUE3G_$vqoai(-ASM+_sG|W*p*C5*j1L=&cRZ5%NOIC z!vitc;@%qld@wuYKXMR^L|C#^Lhg4trX3-7&WARCfL?2ngQ_dneJ)K|9%pEp!}~~a z-on}2K)LI1jhm!*;%#@ALuk31v9n0zROw0+jCCsxIdOrdBDg&i;aC1Aw=N^HSKOYB zy(2>ox{W5@S7CCSK+yn2^w2C6bF7P=lG^lEAylI3NWAhR#Z0+iRi%}?O4H4;W56#) zE9g!#)ppKtEG2T<1dNYR3R^sl*?NY)dyBeBTj4scL%;C5nbc|9z3}9}OK|D-L$WUY zDfw}9_E$Njru22~75v_}>C#`E+tL@Yutk-zkabvGi1@m8Xx3tUtu0`qQ*Rs`G={1J z=ixvMx-(x7PX5Z6DY2_qr4Gy?uKIrAH+$CJGPBi3-IFPAg1~gLzPkST8}4O2f1}bt zSP-e(-wunp7g2(Z4SMXIz1&q>58FSp1w zw|=kmgLUbxg5r&$jQ7dgiuk;0$>2x)xU>DnLXSLqVi>z%Kmq3N!}W&n;$e-Ln7)^#oQaoRZL5w72!rT-cFwj#X9V#>2tyf~?kYOeTs@`ufYE5CN> zk;xe5&v_St*yhZ#c`2i z#)W?f`Kb1^DhK*p>zw#2>nru!f;1^53K<6ML*gg85B+cS)t~rt3hVRGro9z2ntkzqTz-kzV`XY$R_uszldEs=jLYhWo)sOx z3_!L*d3W?X!w%0{4nC4LtsD*lV&obR+F{gMEf@?8s?B_9%g<-&fBx#21y#Mmcjep9 z57AfrWy*|cK0Bhk8*y-=hi4O`7r`3nfg4CN4ryeCl*FvZ3VM`mi1G++!%zG>LFOQ^ z;R?;os(GS%@`rdp&Ic-60l+IqjIG&t)zaql3@@w=MZRSN0TFyk zxS?OF3AiHJ+Pn&=$E-puX_kILa{R7d#X>s$1{sWQVz(SuPCCyZ@J3JX)s4acMaRdg zrW16#xB$Z{>PQQZbbZfiKf{do{BfZ>HVDyDyjd z3n^9#s!8p4cn}Dao2opvmog-GJNLWdz28aA9Sor{`gnAn$C&~T*i%)!%c-rf)DoD# zw-7&nF;00HQ6LPDxYTF6;kp6d-F9<2=Rw(U0bBQpDQ&H-i&#JeDY+9-VBhRHW_oa^ zpe%RYsn$>q&T~f>c>oYQSb_GPX!SNYT7St4c+cL@R^d@simY$%yGjMhm1BlrcT15v z+~04zN>zR*+4f?1O;tvZ4=u#P;|4eZo7xTr!;Soaa3u zrN;_R{f}ovWw#Th&#rtE>38&Qo{uHC(Kq)G%hnePD%l?C=6x5pU0I(=rW?P$W*m>V zBH_%R@dG@AQy~A@u%v&I^?%NjOHk=k@k!&^x=IKWU47Iptzc!~>-S{J4yI&jSE1D? zXBLQ|`^vIS+Th3}HYBArLexRPE+4yTSe@H#mv&8GEf=GtV}4n)`H~(MOrIq}TIC+S zcxRIGDf=|lr612>YTUjPU)!i}#wf=-&-E2Gi%E5k&>5rq6C@+B&zLp_G)+yGwXs10 zs{f6s)*s=&gS@R27z{<=aqbyr21~~+en0?|^#$L@Izl zL|}zw7xS#Lhyu=UT$8Q;kbixnv0N_WV(r7taFPtVb0TfC_peAa8!_KV_hh<$Rn|MM zSL2Zx8ef>UbiKm{`dxc-9@0l!6uEPEP}0dt^S!Fq_=){NIIcLYik;2}$o#5&PlZ=r z>Nd|)5Aq&61y}FXP2anumvh{V@3ilTE`$PVU*LOx-3(L0G6zPlG^!N=M5x<>|W z23LKM8xr-Ie6RUJ_)8YZJudckn_su=X%^I-kZ|t&!YdPy&W-2{h=<#Dl z{QDk(3P_5cPmN>feSfXLgoI2?v3Q$b;i!(;>1;-s%{6 z#|KW5BWR_dCzAf4&O+TFsQsgM9C*+tOpKl0v=V1=JC~6$ZR4I$L7F{5oiX%Wd_|G^y*m zD&5BhtP%OIWG@9A?F=$GI!9KcsUgN7+iCRO?aHC8gD#JFW1lD|Zf<}tYRFd&E#5M| zZR)W#a#`UV?V{iEe(y}Wcg6uGky2aH6@A3xR_sUefm2}yx-B|%-#Va%X;)n6_AW5w zWy{vc@%Nd7VL9?7>b`PoT{8Q6YtqEvi1(!niwt|z$};&f|JtP*>=@t=Z~+cJOP{HD zX*2aBOkUDkCpncSbD?)K+VJ|^Ee*n}+t3Wk!aYfu>+-$xAqufktAhZShcuky#+Ugz zE!%matPG9mj88Ut7L&?)W!+8iM0~2woQ?)lxc-}(XL zYrW5Q)TIrvT|M2G$)aS?Hkf~;e?^h8dCc6kXKXgA3wqLNa$`)1$G}pahCec36KImY zRXfu}#i1Q&7%^FYWH6@ZKVQ)Q&%PFvgP%)YRCRW95tydyzIkmO$Y8AZL(OO;>dpAI z-w&aAK;;q;-I1`g^&3<&v5ly3?K!2K*F#Q)+Q#5B$i=%h+JRQevnl8^$^Omz7@}}A z6q5|^ai=XzS>pF<$e;YY@bc+epj&)ALIonk&6OvO;fV@@A-A8Gx|}B3(;z%cm!x#7 z5~$Esf*GK0A0zNE6{2N!F;!_Z{W(oQWNu2bWQ48^&3w+Y0>#Ycr|H3BG}H%vq#qNM z#_9<@Tj#5L0h)*WQgbTHcz7jz*g;YXW-ArT`DLYk#e@2iNRRE5^b=Wh;sqadwbE6Fx%UaE~<933q!3=D;ji?2%(Zaujq$s*uwAyN$;y0M@jF_ zm*29}1B|L)p2FYGxJTg{47`*fi|;WiT6az9(B55>)SoynemN4JRk~L(#M!oy_opIq zaiTq54ARBNx4=D;o-FUH3(%APDtEiRcq^SO_}w`|Q$x6)0D&df(i`rBEZVVSE%zSi z&>S`qxqPC;GH-fxj_}wxY+U0popOdObsz0r1&ovQI66Qj{<{`B^aM-{tzhj<4WgzF zSpplONK{g9W_k+R_BN9DvzfbApu!)|w8XEmlkQ2UurSsKeGMa@Pm$ptmn{(6%mNg2 zNQDs_LM1~^s#vUL&-mxODR^k>f+1#g=!~Fq%A9qAKJtWANz!8|Fe^yQY#WWlgsHPw-C}aLpPgHXp;_qq zW)m6oy_ERj!Z!nmHmOtXU_g~;QUlm}j5p~Y$_Pcapz&3J0xQW`4-&qAR9j1ToF3e^FlEkXgxe(P`us8 z!h3TpEGdlH`HJ9JnE5ni>-!8wr)iY`;FJ3Gq#)(O_*^?(Dpi!;_i4?Z`3^$k=eG6k z4wNf#+E)VFoj*V-l(Ee19qF4D`py(<$=AfDL;@fjO^%x;lP|aXFNWS6W+%<5z3HoV zUf&T|FsA7{{3KQwTEz3h@x}H}BBiV59pixznc*gDfUxyheQ}+HTxGvlT5mhV$oS2ds6m zbS`3`UuvHc;y(4h2q1C`I}y4Z6tlnqw<$mO^1tx-7A15^2)fl$$(HFV^!AbW?_6eY9BAFWEHqUH8%17Tho66w1J$nD!^nE#Ee=A$ zUJHea{SD1&kC7sMJq31>I&9897nhOolJ?Y*ij%MLJTA3)`IhxmDKudOtlCL0N1ptN z!RyzY>V9Xdazz-K!7(L6?0~lgF40CKX^dcMTgjtRzgnBI{_R54OrH?y2gTeYsY>4*o(YPMmJIaZR3x>5vBqagSh!agu4J*SVoiT7hQpIbnNv7DhN(% z&_<1@#~o*cv{IF|Tjjx^iXtJ~x_md8>kL9ZgtYtEGu|O=wTf37Gu3%0!8xbdxuYe^ znm>hSp7MacBK~1$4@(hr($TZ?s8X@vZ&X#|T z;Vz#k1u`j}o#^4&I8sF|nxI)8CMvab?ZUQS|=>5C8`P=4qr{MM=bpC~4y5hax zEg9aSs4^uBLNI3U6;(^u%knclLZ$S{`{qNaYUeJhBEZjkJQB2CF^6-t0$segnc-Qf zuF%cK%)yg1W@nBsRNan!Pa`?Rk#`l4&;CdNoI1*7+S`!8&ybU^R`icwuZ01xkbJ4K z|EQ(S&-QYLBwpG2`d?W$zuEEgG_mo0EdH>`OMMU^#(gw`*JGQ>R_`7xvh~B0+0)qB zLGVV`T<@MrM}L${qzc|-&gjj}?`*vrPrf8)+4mr`=lr}qm{FR^J%;wMY+Kle0)+F2 zBxRX}R>M(kL)4BPuY#}MN?AiZ__E5%st?}!uB0BVt2~EJUO-ryMi3tVDnZ}e&Fnyx z`^(}A3nriEGT}6iE%$xdKs+6PDMSit?6R1*`FRpX$EW{aE0?^)&ud-rWN`^*0DHOqC_n{S+M)Z_ArCg}{?stvx;hS}@+GKr82<%!`O(4nz z6Gl!Xc#CG=Xx-#LuHcCTe^md9&(6aBC*U?%)$0e0C21$sw4nm5aGk87nx`J~FcTWh z`>-_po-ZW=LYL`#$5Z$%zhSsTLjuRq>lBy>4Crhbq66J|m^QCFv?L-9J)>PVc>-+D zN^D3fMe1$w`hvz5eV)H%I%Yh#kt{DKZ`Zcld>PcK-uVuzllboaina)Bq3TB3EgAvY(?%3v4`AnNdX-KgZ}nK5x4MvU2?y+|Td5YV*ykMO74zR8QtfeRv zPDR#V#0Z9{F6}~xC2`g?b4wny+FVAc&e^#s?8@-65$$2qGG4=JC`MQYX3w$MWaLLnto$*Fq+)EF?d{-SAWf7HgyY=hIcrWp+zZ&NCV=>pCzo(gY z-mDF)XMPP)0r*>r{u6bL8k9K_5D2tmbBWphb7Azuk z&rl-`W+Ek2I%HOw`sJ?77oSCja{a!G*uV-p ziTr|2YUKYqhDDp(_b$sDN*7YV%Ura*a2MB)SE9-0e{^(7at?Gl-#w%Kc@_*X_|s7S z1V;>QNm@5!e`bCg1e#rqAX7c@ltuRI+<`ZhnKSiG6QGXnx9*wCns)k~jmd^}kA!F% zyJ>K|*wX$TyNp)H%05 z!%#8T5WIi`R^tpC<@#+~oH%7)x!9TCb}~JWayLqqL8;iX;#Z$9Z<0Y9h9#e2*n|X= zKQ#U6j<}_L$9jD~67!WakdJCm&i%XR3-DBNLAJ}M;*u{Pd6<90h3ww=M;zkj28RV{ zS*H8^EevyEBa6iymIC+uK48>6^d(JO27Y+AAwmp;i=hY3_PbPM=dHgI4-sRA;jj5< z?cwzQ@IKSv5H9bonRMD&f~Y$x@|;7+{nd3>@=T*6#fww~`(GOiS0F25_y+j zqHp?d2IkqL#Pa5iGXlEfv()5{EJ+=Kr-#lVW50#5kfBlsl7LH@nEClTAuXBZ{^sac zKlKV)*KBp)@?r@u1zwl5Kqd#gm$ZgpT|t`-jPufK6Y5IQ-@HUs{)8(Zrz9(AXZ3(p ztl{_$lb-%hz1+;|OKs$F@T9BHyCV;FR<2D>$yJNO$oHsd-Czo~^qj(Fqh9q$&&W>( zF>$CGxrSizWAu5}tt;mAOV+v)pJK)~0gcdExRKMAU+na`g_5h8UKHnDVZ%LF&QHnu z`CDq~kC$;>@0$S>gxy6r?D1%Mw@^4oj?X%?6nmirFk^EtW_|(qs0_ z?!w>{MSb*`3~ap2*Q9Vwpjm|H)r|t? zhNGU0@10gf%`c3k0w&sG9?=Pt&=Yw2$`ux#Yi6hQ7wh*Aiw zMETY)h*OAGKzpXC(<;yRw*#YkYs&R-@hRw5?#z$Cj{-Kd2=rb%)W6F(!w~$@u*7OY z;|ZXM)Y;QcwLemfd@!LzQ&96s*8)F5ojP$hLtLt82rTEJw!pgX_zRweg#bJlXn(}d z9H+)aFZaovc5={II>FaTtnKH4@ChYN)IRMOGB!?I6+=zGJ=JG~N(RO~A3Npq6%jdZ z2|;ltspA_zR_MQ0)`e087SG`p1xt`iXb@XRCH2GD&=$UYpTWA z0Yhzi-p}V@?62nd#7l3x$9`CGK@vZelyJ7J4nv&n7@?LzG@aG1QV{-dad0HQ>30jp z1claMb2)#QhO?n#8du{TIz!YMl3m;Lfaoq!}5N+>wh_!rN%JG_Z$5~+b><0;H zUad@pr2lTWWeK_pJb6a{2dkSzC(J`}e(vh}WM3ftUf2Tw>JD+1RY|T34~~@$i!|!7<9%-%G!R z*x!zUbsr!}ADupwt zvq>CWR5b*}85-xoaiemeGur%GfX72N0j6sJJqvTwKT_0gj=u1JyDdsdl{|s2yppy|Ax9G8CeE)yAXURnlG_> zn&6Aj*({)CtHijI{RWr#53C$gf!ddc74K=0f|5c8HQ_N)FsAD^8GezcD9^b`R`IxAXT6`qE4(>>~v+!6G z>+T6lT!X><^?d9TYduqw$|$?0X5;y%Zx9 zzPZ-#e5nc8;Kon4-kvaul?Y=dQlb;40eM_|_YSZ5Ovdb>)={1xWUp5HYRdtyBdP4v z3QS`W>OkoS$d9OcFDUceHYg^?RJJ&nCd z9<`G*WVGAt=Xp-a0nbCa@g@7<{2SgU!_>T=a_wBSNGjqV#T zQi{bJTG<>%X|MHb>KTdqvq4lWX4psRa?vNR@ArDBy9As=Zf9=&A;Zr2n0(lQ_fuZ| z&oHS!yMR-rKgWO{&tP!xzhHi;L{#GAEO>)Mx4c+=jV+}RPl5S zv@JUs+n?YpxVd+D3hSTJ8{|uZIi}wb)6) zwDtX1JM+MLNrZX1pGy8!Yh2N>^9L2?>hsqF48f)eZPZcw3>LHYTlMYBRv2CGEVZ~B zrFMNT+C9J5h=8F1D=*66xWy@5$Po!kMqZ=Xd=M7zv=>p1v`-7nxup9A#wxh_brH%& zIA=HHssFxxt%Fyvt_n3=|5M!;G#tvqA%AcwTmNvN^ObGG=3#2|I?>M!dG44!^mpq6 z==6B9Ji9vmPLRB|%pfTKU;v*^4ByT=ET<#4!cM_e2uES1+5?u{ma*fJR?`sODDo!LXE$0xK!H-sEzY?Hal2?l?4MtS zPt3900E<)j(AYydk@_&MkU}K;q>P7_;Mn>Tz1(ppB`9faW=Na#!EhQ!eCj)wB32F) zQXCEZ^uGT)LPCQw6JCK-q%3ThIq`-n_&{8)X}qFZ+lgrRO^y+SiRD~_#nEijR2Il8 zT6DT_??9$-V7w<}6z#J2#5eA(siH;3AC$m{m~)`{ku!uRb-I_6SvzpkIgX2MAm`lf z^qc$kjn8vtv72k#tf}O~CK6`4FB1p~mV zDhbWvk>Tb-SftvzULN>#|Fvp!{!-JNk;e{MSxvGajXQKrwVU~Ios_&qm@5ZUwZ-m6 z|4l+eD-a}$@5u!9_YLNO`M^o9s;w$Z$TYleD@U@R-S{-YM82qy__o$F{hIOeyY7QV z*H+)N<#rRzxeoKF>dO+-!@CuD+bge)kI(s`ai*K}hq6l7 z+)CX&k^gca)~NhXsFR1z^s;|M@6$D61h?gZsu=cDLtTL`4GGxS(IZD`pk;BuUaaHW z5%YzesK%b}4B=UF5_uN@j>>|3@OZoS$``TooEl2b1s4Y72%AioqO`Rg-$v_b>Ls)t zN%2P(*wY!BxU6azl{4e0Dn?&MG=@jvpdx#s7*l?iF7}HB`c~sx!sxD4$h6lt3)7hy z(56M)RHa(?=dpLujB$}5MmFJS?h^E-dy-(2Y@@ob+0bkp4Fhg20OIE1?Qa`6u-5X4Zx})p7(tT{!tx28`VWeU?BadkeBo4 zB+tkJZ;pS67S_D;e9{zo&10q_*avr}C>2c!(#nX)uNv+k701Djj;%Lam(MGxzpR|# z{rj_*F%4}l`j3XaJZniYJ>ao^yBQR7;EJ2Oe`Hb)7MXVq*wiJ4|1Bt#nQGO9WH~kZ z)q#5mI2SXXMLm!LL5TjgEoyX#j->C?BZ|KQz7i^xoL5_6^(I;tzp%4QHp*wYo4{AJ zQt5i$LMm@~mb*4Q%6c%ZPwM_#=qJdicam`?(t+7{Gg*0B(s9rlYqWW44ER%LgFCR| z|6xuNqTWXRP>SQm6+LE2DqN@!H{|8OWc8md7m)4UK|%R(k~=Q_n9~>w=7JB7&?J+6 z*+bmNd~CFAH?OKfM2M7#d2fFG^;xZJ`D}K45TfDV23iuj4+&RNKu+J#@k7%n_d3(y3Qt*aQeKeQk_FWORHuh#E z#zk9p&Vs@wh7Pjt;BX+k%X@Z0KwIoUganKFGI_B>^!HC407Z$FSW=14MV#!75Br9U z+9?B$FCeriwTf^8&I*qkr#v~eKSiJ-!6wE->^(_HR@D-FN!#FIol_z|PW4#u>wn8K z^qNPi7-@X%>~DY1^LJF$iM*UTs<;}el|EKKoG;gIV{1#)F#Q(ioQ8aDbK&68!ZSqK zt>SC@bFC^6Pk8|Q$h5{ayLIFa8}<;R{3j4tedl))w@n6!tA0zF{8OEn+8JYENzT$C zY9;T_$w(m_=|18QL`o9?|M^JwH@3md)yB#;J*7+&?kcmdjcLT{|CQEgG~tx^WinUz z=UXHJ|Cljb3pv;k5i!%f(gDt(q>Rd}ned@Sz;cO?D(bmFfdjKzhr5oj4@Q@_$~Z}m z4IgCd=|M&eryYbVpEm;KV`QR$7oF~Xa?kQ399)iz?dZXVXztDoipA19U+tI4 zQA$4ZV8hS>4C`a-PY8I=TVn7_{~c+?<~ZN=(IVD^&jE0$byj^la#vf(y4|+)yyRIo zOY(@R2lH13bm){(Aob{{m^Q1jRI?gy-#_Gj7prxo4!-`Ko{ar>J~#lbl1eQBEW}d) zQo)hl#^KSH5O}v&2{7MaWr`hZ+H%CNJ%1BH)g<=A8?W&dUuiuHQ#vbs6PWEUM`B(k z!#(|viac30@iq^A+*ux`zii&DVU;8;X-v>9tR6UAkx#gCW28d~H@7S*w9ee4ZMH)K z8@Q0&=v9lW(l__~T-BQY$rQWv@+m=80hkJ;;`%ek>93Rdo05vq?qG}<5AEf_=LDLW zzf9=C5bKW+2ZqD6V9M!yu!y*8qyu&bIm8N+a%*I@V%%e#^P>I(tQoYW4!%-PC6ly_ z3RDp>=vz%^{R@8EMWkUspzv6;#6@8h8j)$E1TFg11|DSb5XQTAKdrRPk$lzoi6K4p zjZd26Jn45iLy5w}?VOa87e|S?SNknFuJ1B7c90e_0x^xl4=uQHs-|Yh+E31W;(^$@ zo|=o(LAoFxuJr7HqLZY&XdvGOdnZ_)`+?YrE1dEeN|Vh5bA*lW3pH-g0n|4omGJ+@ci!b212>Bt{@bs`lLGrs&0&r)!VBWcNUKis;Tw%_KA6&&O7uvb*iht-ibUnP zDO+3-HE&ff>k@V`oyO7^t71^MBYxuciCKY=#~ZQ;ubB3X`A$(jsvvLaOsv#f0WM*z z6SJ>g^TDuTj&Ez7`?MGE>?y@~t@$hG*fMDRHZDLAEFfT@ZYPb2+iiFuoPxn3ET1jx zJ6{Q()JWp!bJOP6Z5E0WVkwaYY@9jq*fd9*tGOxna-->z`Mc>MZ<`W?rFQWE!1vIs za(+Nw9A~m%LP69!^8$(Oy$I{2R{AH^c(nQ&A5U_=GvslIhpEU*6;}SbA-;0#_?IIY z!v62r;o^cdXTI-<_-07K6#u4%o)E_iEh#Wjb9E%Hs0Lxm|2!)5swBM#Ham3C zBnwXub>PEZRQ8wLJN===ERN~?H7Kk8bVMP{olgR-LmX@7_2+5&G2Ks>``lTgdmIlbmegLo6q&5mb_BiB1#$@2D)siV?SLV z2ls3%L$qT6IQR%)B&wadB!#-0$F^qLS$~fgdG4x*Pk)lF)(%Ox7x-1K+_Y(JA z+ay3YgjW?nm^9{R6)bxrI9OiD$s+vzlOPkW!JLNubtOQUFJ`+VD5VXkk*ll3l{LnE z@N2zb1`Ef_lb*VI{pMu$_(fQPFV-Oc)NRQcgL#-C8O`Uyp1klN)g@zT_+>kLVEpTM zGJCH!ZZ$U^_!N~$cMj}8 zUC=+CT?pY|0k;%*M*%JXWaewxG_7bw^kswf?{tQ<4p!UH@*gTWm%Z@y!NyzlSsUNc zQ3~s~B4uu^Z#qjZCl&@+okm6&Zr-Oma`4sf$C=s~2wpQ}a=C|$d1GJ1-Z)_2e>8p} zAe9Vz2;AnsU4%UFV#`$R{6Y6+6Fd2riHE79Y8`g3w`rBDvp2O&68H}Mj(PQWf`*VJ zj3ht*w*4@g>8NB+p{J4^bK#Q>1Ao`OSv+c-7a9-u>wojIo?tX_Gq@)vkp)WyrHBD{ zWkY$;_%ACRN?M|h`vtb^AXyr_ltd%XN<|r^4$Y@8?O!TCz8SSnr#9D$;4qR!I5mN6__Pb{#$r?VlHe2=*Li-c5M z7lnA9sAsOFx@TU zzekb-NF?m~ z9Mfz}IaT4xg8Fyw#xdDRqk)Yd;ZNr{QtWBwQ!uUM=D7rVSz5fMVn5CvJxhMKC|UBB zCD|>--u}bO6x~ih5lly%i#9ws+F)0;OWZP}8%61;-eTl~T^Lbc=CMH9JK@T5aF0(% zP`P05JQz*#i~#fVb=)usZf>qkXFrWclxXTSFd%PgUQ?Nqn1e zzI)+8t6o0i1w7M@C9j?|!%&7Z+bfPXTxgxvY3B;I*~dzZPNLUO$LE!f6)*{bcs z)VylvU$x$GJqvJHMxcR!_L;ZeYQJ)f$EB`)=W1cnDP_#z{uAW};N_qWkEUH#VMFmi zZ{~Dc^z$00(Ph6vgy2oOIkDb=PZtu`7`Iv(2+hvM41u1WeFoXf7MFY!KCh0Ttnp7b zO#Wd>{Gv!-1f8Aw&ayTJP`rY%GLB;8kc~sl=8MK7G2e=N+{EJm%{%i<`9zs@F1~7a zuPYs70Q?TSak4grM|v4ogq!a2tp^$mKYl^(M}>mA%Kap z<4uBk*zccIP;N{G>q7$cy<;4q`FFI9x z4+?|%`W_PPaK6)Pkt_N4p%jedb@0H7*t(8@R8P6U z8J4IMF!~-UbhkEh_qXtM2Rh<>P5*WSRnt$nOtB09<$KoMOtOy)(4BlT!on%5+K$WW zQR^Ok`e2sLyFZESgN?AjW-2Wd4pl|&Im$tdHMbf|js6o2H2lwzEW$rjk3~`? za7?k7?1Q=DbK>%f-Ez7aLd?(ExPO1awKZ5AXL3v#?UM_^RTG#VTL_`?rJzHw$at+0 zeMD|M+C#nTR8EX%{F^0X6H*)U4h{OjWW84fJXip4+@|wg0imZ$9ZW5?HSkTv-`c6v zEmuw%Y5nWoY|7~^9&11Uf7*}&$@DX5xHb3gwO9RjKMXbwwUBs}7=$RUd zy79d0cjYq?FbQN)vFZ8xoA#Tx>`}Xo;^X{*B9U^GyyI>EQTZ(UfJS`%#SLu*iTZI9 zAX(0Gya)2lnKqP+vU5YEZ)bFwRb8)EoMaF$QVHQv+3-zVnX6ey7AR3}+9j97@S6ZI zXAw-s{U`36@XgC=DKBuw{Fp`f@ylD$$CdPHj+5{53++YPCNm84 zA|-;OYE!{=Iks^dj?f@Qu6og!e^QKjX=A6Tl(h4$PQJeKinxE^gM++%YfoX15y*QONlOGqHRO5}{;Zqxje@##uH_?XH}9 z8mz6Z{ts)jOLHyfl@mM5i*<|Yo9m!S!5*=~jFgHzLDfhD3otAVJ7d->lP81d5U)W` zca%gMK7SAR^<}1Y=i$2jK|d$3=D?+iCkbGSEn#^!(zG8m98N;ifMx*mF<3-H> z3Ve_qdy&=ciw0d5vqQrr|DJt34@V}|x=!eh3xPI7j4rrj(?8)oV$@dh`jZ8jno=09 z{O7*(|5*Sw$FAA#b2l`j`QXVN3u|8C%G_ZrA;z`S%C{*PRno=HC(_O$cW@J*G2PUv zU0wbTpHurZJo52M2kqBt!!iAl*=fOPe&d41VOv>NWh zR(hF4DQpau6CoUMknopaiCq_&z4Rrk^|C5!pF0W1u!C0;H$n(!VfG> zUoXstTsc_mn$#Hc?B@FkWBz%>46`%b^^*hl&BNp=q$%{}(}F-h-|pm$jGs;@+p*9h zWFNyajChV{<*P)D_FUuS3E|yLC;NBnFEmj28ndX5dBKStxaoiu17j^Duej*Jb^gd5 z>T<5hNA=oz+LZrkf*;y|$$MwEOKXKg*NDHAWkj7m*=$=gL$HI@XmvJ!$m6mwgSCXR z-2voG*!$SDW3Pr{7~D5=CvsG)Ea9uKEebjCa%W86lMt`*tYGFs__RRz+6Wyre!9ec z7Ns~A9jnA9MvZZvUc*g%X(McLmsu$zp6&tjS-;$@ z0qmi{!JmY0mmfK1* zx1^4Wks4ePXa#Ssp8NufoWw|MngoS+n)MVhn|ce|0ogQ#7MS74?V-<7Y5__E#i2r#%m{kSZH`*jNal4ag$g zdA%}n_6;ud?@LYl5^PrRY=@P%V!aXE-RIcnf%(yKbMkyEMRF1@c3DTzEjE992}+{O zkT!>dpW+$x6wDm<*q|~@%+h`A9gZ_gob%{Vos^cfu|7A~te34K85Q==O1Ppv-Dmkm zJtZcwF|2#T8t(cQ>#cJ;c@vXPmHU7_;N$wFJS_IU1W^%i-btauDA1FlZrU%|&!Nai zJK(I}jumfmTYrd>P7kd3dNDxW?6~^%-~8*9xQS=~qvoX%rAdYSaKxK&j!R zVO!7$Dssj8eLLqft`2yE`{}y!-v%_x-#t&$H(~_qon>&bclR2X3t2 zREWVU@HW-Izb0?a6N|Wh{SR?{{7e?aUPoGlSw$)yJzyZ~6* zln^uBkDUxzixV9UP)U`%btYR-+*6ahuNC~vGye(?YCL!n%**4m9GNdHx82zOT=cz5 z|LbVf$IoGw!U?IQ;!hakX9V92ev%_?ZCT9L}%CG6tiv0jcbJo3kc<^kBv% zoKbJ`hh!8_sF+Sjw%5kJo9pa)*`GlTlON1(vx zp?2TD4of;KnYfixj?Ku%U~bE`g{Df@^ymyOF}uGsLQS8z?=Q9z7BTzUhPJULq8{4U zVl!F66g7;iDH`5QOn>!08pEL0{wIcI5P1jw3G!M{=o@C4bR>UK`$d4m%8(7Bk$MBiX z?JlVG)!#|uv7o;u0R~L7Z5CR|gG;aTF2Lvtt6_Yop3yoo?klP6F#zkGHZ|o8lE+=? z=iRy4d0XEW22s0=j#zY}fTx6+BL9cc(rI9t!kYNcMh0WtGigGPcO{t1Yl|+#_C9bG~L2m2b)p{31M%eKqOM)^#MHR5cXsQ#EW$Uk@}%x@56u>bz|a z_{>xl1h%0U7&L1&jR@j5c45TC1^m59p##C~&Ht7jviNysJMPViPPrCkjD>4$@n^dV zi7XR_B}m%jtl;4c33Ji9>Mmn7!Q_Tm?TuMC;Jhho+yXh@TO9%pfUPuu)AdVh4JiPK zrG9yxFswgenyElF<2{JjsO_(Tz_LuXNm1)e(p~U|roV6u`+ocM!9rM|T8U%;PO@*1 zQ6G4dZ7;*V$JKk%ZP@PUjP_IS-MB$Hw@<9pNiFz2u|}ts@SLTITv@*%xcOFS4O^`C zN*sNj^GeWC3gFZ={Lxp4rN_(j6f}wttvsf_)P@nMK>Wrl8>)70v&{|BuJ)QDUB!cq zzvEnu=iblfae*W`nf}EqJO?x-wgzrkQwHydj2@)SKg}_T&ZLDM6Zij=y|XCf7Yz#` zo*zoa5-|haPqA5N z>GBK}YI46e`gzYAw+O$7shc!xwgk~}@<7QUBY{M=-SjK&G!IZuVU?}&Vb)w%Ajg~> zQ;MjH-=uqE7-e4AFOXrZA%&^oyrMLP2aM9dl0~=P_te z7_pePX+X}^UiYO&BX;Eg{lB~Q21$^jiY%%>dwg;dpJ}iOH$5TBHhk8*ZM0YA_wR?+ z2Qpi{0xebVHpeG_!+(_7!(HJgogcI>pNeyPo^P`g@%dCq&Q;$wjp`efbJ{&gwCF3;%g@@*=?1gO8%f#0w0q=;^_l@Aov9=xVM-|`5ozPlk znI|#d43A}=6vl)(!YJ27sZ||LyeOD%j}@W?^3pp)vdF|_F-%h5z}H&<5CIWIW%<%^ zZ#}!l-S;SJ+WSLc+bHyY^AHp~U=b6ZgKF|8&dh>tYMQ82MXj;OsSfaT<0s2?igeLI z(v@&cC^{1&)3a_Yo25@rxXKP`22Qvwu?w+T=6`8L6I@Yem}gW$3nRWG;?PYAq1Pys?DKnWi`{6cf$!1C zmN;Fq*B2T-%Wg50t1Qa;cgYxPx^^V-ZEDKsb=N_%o17U;=GAKO=k{%%h4a}mkduF2zHKUNgvlVvzq^grYQifI0LO$oQw;|3i00Qj)4`4@;$ezIg8s9bPdr-KarCON< z5NTrnCjZshRjJuCwU&0eO2UxieddB6%&fkulw0eb63#hrDG`~`q*iJ^Z|@%}dL{Hp zWB;{xLvX*(?hUc7##id4LB<|P&t682Z1D6=Qt3(n3qZEVqj~><_MT~UHL$kQbUfeoMH=R7@7I_wBNI{Z7YB!%(k#%LA`-PLhvLcf~s=lRZ3!qQ&q> zMW+A|B4$tN#x%H1p<3Kfx8{cOq!YRU0~JpHX!w!F0AaGsOtJKCMl()iJDVUw(+z7# z0IA<8pqFCG3g(&wR4Cn-lh~OUT8oQ*WOl6+D?0S`xMQF@0NY$_ZY3#J+q0|Nns&$` z)%BD6VJf!fz;+VS4^O2OKVmG6kx6_vR-GZxZq87ikv6`!l+ZH!la23=3oxTx?iv!y z+)o`PK>?xdz=u4;C|fZf2b0CV8Gkqb%`o-+=Th{X8R^y_sVpc3E`+zyC(!<~BdBjs zkJo6X(Nc)M9p5)ko#X=KwOgI_#6<)^nqp-^dp$`0i{9g@XGeQ9_M}9CtIn~vhohNc zXW)n%89OhttM9|X-Ce4@mwZmN|T(E!f)$qf4y{K0>zbRRNZva-D=8o=eR1} zFLRdnP*lDMWT$=(ogC>BlK)HijmwnFTc&Wnvv%vm-!QxwpKjH}F}gnP$59jrbB z>h}<80*v^oa=%K~pXk2Sy6@h}D2wAaIuoZU3_XrHaSH7M3T$^*j$Gbsi*q|x2iwEI z#+XCNqZf1;Q0d081A7Cv){e57mbyorYHwCU)HQ2}*}Wyip6Fbz-;KZJmuQ9n)tgsS zusa{3lX~@uV=9o=+g@{s4tmjf*eNo7fsD(ATg}c(1)E&o+*H3ZqdVF28$9XUk3v*n z7XZHXoAe`E{4RZ7X6!RTPEFI-Qk(+zpNJ#OebyK^u$$Oe&5O~*K4m)>cKlfe6WM+J z*RMjZW90^tL5K-dk!vwSyGIJ=x%_F)(n)NP$NgP`I#%>hMy`U8TYs!u;{MSwI;jff*k)y_K>+%0Gz5mHT;2AJv?i0!P{ecmee3 zE2d-bnm>!%CfZrChX($#jb3Ba%RSyy12LF?+rdsktPQm3wujeB!&7v`dqx7}u2(;a z?@dyW$o6WsbJ=t6S-f(?yZAm?OS3&klVEyb_LFT9cB~p#A+pI@B(UG zoA`ceH=XfBpFQM#1qTbwfn^U`q8hN&kEh%JdX$r7HeW%;QOma@a^{h5&Rq|=ly$nl z5)FIdQuzOyGq`D`B!s^(n~mi1n7TYI#bn7lvBAT~?!35pmYe))Y5o8zXU{jp{gxlj zy(Io+YnHr)>)xz$Rd~@_-g3ul|AJUcM{3v;1MXb;NRs3V7ZFJrvzaavl;b)ID@c<`zF{aqe!mX#RpwL)>5u zrF`qN=WOHd6VcP~+vP9_Pxj>>x&Ek&v9P zcRH~3Z2Ds=KKpBVJym*L+a=Z!gT1|ew{!?nwQB2P>d5!#G!*NTurD_ERo_cRyM5bc zp_6M^9XK)XGof_%u+~i}*tv^OcCj~G2^bcu;;3#rRLBA`R*RWZ9U_Msg-0c*Fai{? zGpGFC6UBW;KtGShy#V4QWF;d_@796??dDUO$d}VGEc%26&V1<<4P=Lm0^{FDiV|yg zdxfmTrA{H6ma6>29_VV`uyCg#cchNntR?5X&VDR$TMFa#;J3}WmvYNQF1~lxmtz{U zRz#QCsMe#a2Dj!_n!OXCuyVEuAeY{9R1>@jj?NZ^R>|uZ#jOiIK7#zMc*TL$nU#OB zbJcvA8`&~lD424Fc9f(QW@0m0+AL#m1(@;JM$ut)ZP#4K_Jrs587&8+2|2ZHCJgF>j<|MXXDAPeq^&P#5v@jjs zeulE&^p?nakS{g|%lt0~r{iK5TQhyp9DjS$*ndneYvO-Zo$f^iyBa^Em`BsVx?-Z%em`BD3l_x7@y;6@}mi#;SZB`dAniJ!-ms!b}X6d zP6J!qR?@dA=DhT}Z(mxfe$7(1p;^N?L^-w)-p(*Af?TEIdjI*d7jpGtqk#@laXW<1 zqN#sZ2#x3uRU<6WEAuh!_1Pgc;F_AYE)Oj9Q* zME6`L_SilXzNKpU6aqCFh`q>g_S1E1J^jLj?jH6n+i$(l7wSY0KZ%{US(|`w#nfeN ze21E`Nr_!|#p+Akw&yw%;;9^Ci`OQ(*09(`Qmjy^6euH}nnG7$!Hl(MjMp#`yK_Yl z{8jY2soZl}W7OENSei9yeDXUy3dp2dMNTtIg6-e;`5d2f!~8k|V|yE_FQud1NkvZA@j4tJ5oSvd75q=5y9_C63T>l`&#Z#7saOiD@ zHE%x}UaE50G-u@0)t;*9UQ9;>mF;|TkzanvN(cM)?<#Z4Fs{~l;nA0^A|2h>86u+Z z%&7u3A>DqddTy~zJpW0YX}Vz(`bY5bJ=VE4zFj+(lVdWyAv^s1fFk|Tco9Y+)cIXP`$*%X=>={ecsF&+S=_k~x; zCq94?JIkqzS@Z06h(`LYX@55K9J7$!D&=>jTpwPMdOu!x5q@;Egt`p~gs;XB!ow$f zV|&@ZR$+G^JDk~p!+(jt1*Z{v`}by3K4uYjs)-Sq!Pk-0)okp^&p9B>JJH~FQSJ!C zW5N=ncOTm2DbJUtNat)FZr_y{22=PwyJD4N^7>nWx-;J|4ij$d=H(5s-DpNnG;-^% zkkBJW7kM};HJHq)9oPa&)^K|FruJwKW+G7hERelR?8=v5iQ)Qi3t`Gx81c+%`rCs) z!VzIonLl=IfI+ z=<7PoIldxj4{OQGTu49HN8|8i2b$<^PSJ zz}n+tn0|M{3|#Hs2E~M-7oaYWie-`lKPh!q8DKkf83C1sU96`9WkS|CPv#?BdiQOC zV3ktCZ+l5X63gJd0ToUM0V zPFdIsbUH`Si=V-g;FMtrM8p2-OQfw2Da`XnBH8$%);C*CpM9%YB0KD!j}Olcw(RVp z*VF*5Zf}W9O>3(AFc)pbRIa0H36YL`udvP3w;2nxyw?s2xiTH9DkDdei+D3rqQl7F zS4FdO{~U{%HnrkMuaEXHX%sERh(Q{XVM9KQCdf+HtK90v^q)ZoN8e$}P=VJiJ627~ z#xm$D8FkKrh!R3>mHTDm$|)LHn_Jz$H>jbeb4YB%3qa=SNX+=numzL){bD5LE4LoS z`DiLR%~}o2Y3zc4D^L!4(pE_WUBT`jT?EGBKSgY+YT2l^hci9*yl?%7i0fer2a?(L z{KUBI$zNS8QYYSptVriwIsaIGEOc9%A1&0Hlvqh-y(!h35LvegTu3?*K_|~?i~=(e zahtlp>QEt$`w|@!xf08F#uFc1$zacGS~#41j^os$w|)0b3KMh|&~>JCH`kjKL$`Hi zMs1~DD)7w*d$ZqYMb~curt+_J2XUDw4HM+3`Ct=oI!yflu~mB!9eRzq|H?oS+$4j! z_-(>D*Fpy&`X`cgu5g;Y7~>#LGxG>qbO{BYERD%yJLa$d`IuC4!)m%EK4)KRJGJVF z(`BqrJ!k(eP;-gn@GNQ1b}E_$Uvpj`)bZ(IUND2IW zdTMdz-!)NXZ3?W_U_~096qaSH#Cd4PgEpxLuZcRFbf&&V$Uc-(@YLf(>Y0+AIJe8Y ztT~80cv{W+HA5Gkn*t2i@S?NVaoM7h-+&K8J$k z{ZgzlYA;kUJsHSW_xyrv8~&)F?Ynxf(w{-b0e^Yv}NUyWj}A7JmXpn9RY@!mPQKGMK$I+WsvQMi8RQ| zPu6}}S~qJ|r+KlNZky(4vU#)J&zYIv7Zlb*g*c@#v1=X_vfR`%Qy@O7w@>q}n~B#~ zFln<0-6>E_)^*cry5@u^+;>^W_wzN|%#!Whmb9!!&`;}l91NE!Zs|GX%wgKhb`r)^ zRkt#{^|imnhKxNlxP0rlioC~MoQJF-f`t~+4g)F9R=+enPu7(1HBQzW>mcR-ysp_m zTHfcou^vBGl7j2B9YD;WI%TbWH<;<3ACEu_p5GV$yZ2^<*fkWLT+!)U+|?g47Xlrdne;h0<&+wrI{Z#f}gaa!9Q*iY&RVT z)*>TI_|JO?$7oq@KWeXKVsVXVfuHBu;7+b0zd|HpN7*uRQ0fQYJ%(bG26Ah3J%gIc zpBjchwV6-F?U24{Qaqaj6ps})yNtMuz9`JwgJPU@#(gV=`P07Y zpfW8Lnbt;V`+X84Z&1xkH(7t3kRPFv8T^cfl&dKsom@n)mlUmm_sKq6;^^i_2kVZt z0NS`M%nW&N?%~Go=;T$5?t`27t>NYhh&|tpbAHVebm5a5q;mG%{MP6ZiwA2a!M#4Z zDmhAPB=We zqw>taGQG4;WzMFRCi>IS(V)=cGv3OsCp%AkJTIKiSKu*HX^YM1=7X5n@=nj=^?22S z@?{UqjO$cB#E4AGZ72VL=Ius)!5KX-Vrd zd3&&RD(3{~iB<$57Io!|a4 zJ^& zQk_G+MC5x!toTd)mB|nE7t1fCI*W*455)X--^k1#XtYLW@?0H6_R;s=iM_h_?9Usw`{C(5Ra++Cc|QZAzI^cHT~3j z5>m+U9h;u9W|D0m$#+j{Rp_4)Q{Gc8w_+}W!EknH?z4P3Ty}H^SCp6G>^?3bOhl$# zo%vyCJy$fabo||UvzUn`%=A~Jnp{~pU4H^gm-UM;SOLID_BSU5?`ajX@AqRZWeB9R ze9>Cz^v9P5_g0!EM-bBw*Gz#=e11lKmrm*=PmM#*+Fr;D!7`=LmFK82mA!vjS2)0H zpwD)W3vVEV1Q6>-P20Me`=@Rj)u?&S@S2Dx$){gu06EO;`EB+=)9H~wNleE#%nF0Z z08(ZUxzKS5`3q!n?+)6xG|w;s9GGzvM5(<0xY9eN8$anW?5!;{`D#|oSFqePZ}H2} z-mv%a)D_Fqihl(+e%=~s_paqKZ5Dhw=hq$tTj&{k2)Epf)??kK?RjTPA1&r@a#APj z<@qswr&J#Di!WQ<${I&wALI$NYD5Cz1IzQ1bGGIIRHqH{7f1|>pZ1{kXD5Z|vxBBF zm9JZyVUX)jUc$8W<{x*Y31U*_)9j)CKUc)|%3};Wb9dYb+%k@ODex7Vq1R(V)g1uO zPQ$@I?(zhL%lygZ>AAEe^bd#`b!B4n2>wWTx()9Eom0*xhi%2;^-czOSRQQb_|2~H;-}kZWqEGW!v%h}r?P!u zx!*3^nTE$yy|jfg_x7T<%l|%@ZecJu+Y{FZtY7y0Dx96dxR@!+w#Bsf$45h?GF^v6 zcX}TX)mcIbs2a**M%7g%6Bna?{!?`r5U54??!#G>4cbJ1@<-L;vM-sukPNwN-9p^3 zYI^3be9XG*I1Ap%Jo1n;ZnHy74Vt^!HbC4Ns~{TfnA@l5EjD5!=wWU0uRmGmiNkl- zQ-w&weTy+9(D&IOs1xQU5xueWdj@&6l6~#$Ob|OH3(0~xAC^=P*BOFOD7rn7M?mia zl+Z%gb$YKbev0s-i+b5B8GJ|8W2B@e6q|_M2hs~3$(Om_&X;k=<9{N8{2Yb zO}Z0aBLn9ZX8Fy2GkbHTwC`Nj`<}%UVKC#4F!9NRBBls+XS$EdTE(c z(+s|&&c+%2qN2TCUG^+BiN6U?HeRsvK}1|WY(2QJ<&x{|Q1iv?iQoh+r;N_4wTcX)h+kjK?DfgYjeu{@}Vl z%4MkFc606Oa{j{ZP^-D2X_Y<}CMUYkw)17#?e>5akg5-S4)?t%ZzR}2e^Qz9mp%$F z`Yb6Zu_L2p`}_I8N9IzaylBP^I51_?>*pgtce6s~(OuNC7#?WYJRs>PZ6ECk^EFT}=C`fn!!@^9Z&Vj-Hf0`aNNubILM^nwA+`c=(RAkBN zE4Xmykqvd*q@lQeM7NDPVs9z>nbzagqWz^2^(DXCZu`hYDhOHdq7zf^w(S$0&{;w~z;lL>eXa z9|$j49IRiRN$%gE9$y;G65=MJCbfhIQL|Hd{n8NX8~=6+B09yk2h1+(0=?6@o99Mm zy6P9xzQO}}v(N5&u{<~JaK0r!lMLCq4}Eor+q=hquBk6ki8Y60a0okBCBQ8Fmf2uw z6*I>Sj`*r;Cz?KU_K<)Qkf%ig?y&3H2LZ2DmuoMkcpD78m<$T%oa+~qvU<;L-EUuJ z2Kl#}JL4RBZ9CRcjlB>x?DCaeLeHFyS&m)Uf9qY@*zWZe;NGM3q2FUqaqPe`mgwt)Ti3N3spHva6sEPr`YWg)*|mz4Me>3DVrrD35tn;q_6mM|@U>-I4EQNafs zr_+eNdgK8Cy=BJDfv*wt)f=?-zUtvtCwcje^HW^7!QF1O3vu)ugXb>PF*p<&Nblmk-&lB6$d6r4p#iNr{#m=a6QNi_Pu3R* ze$^8!J+9l3Fbt-DPxfOd?CkU-L4!t{R10BsvfK4GA8lLO!`_*G$9czs`Q|D8IWr1i z$jry7B?lR8O5eNc;&LI#Oc>IUuJ*4i}s2Kt9EPrjZJ6YG*q%B(@ zKH_4`-DGkWYI0waFnrPPiGbgGAcjTO$Tl>E@He=2Cda=rUufK~GDJzTGf1nsi!J*QPH}^1S&S`nMzI13j-- zDTd*{TZ7iyaKJy7g0)dgjPADq@Mr@R9; z4ewLg7qfriHqFGNS{KoS?1^iiH=)OhUhf82<`Ywcyb4l{sC&ybCok6y&H{4h;irqQ zNo2T)46m>AD5pN^zvJvXy8be*nA7b@Bnwg|3G}CEytf^$jNo@)zURB zcFM3>WN@{9Nt!r0>?|?|;+g3RgRTo)eqb;IDED+i%%?ynD-H?-?@}&5x{I3SgDmn8 zhW@6O5`a-;_9~ky#dSjAfaq3svd}PQfubbUEKNZ2cv;5s;efU=&dTS3-_s;u72{c} zIomO<<8*KiB81qbK$3FcpT;QS+LXfMDxdi4KcsqL|AUoTbou?2hSu2n1+rUX)P_>` zTCkP?c8POK+Jf#N*4qR64DHeqfEEMmUM;w-@gID7nnVv;H6KrPd-Gx2? z5W~NU>oX`gSusQHljLMw5GMlP?TS}X>1X}p#Ez$?WI_y6yKYa^m?dLJ(4vZ&K&xxB z5!jh}Dl^CZnzd-O7?ZDt7~M^HwMR+;>rZ2_+CME?P+xg#X0X({wWN|5R48PK@ZW+> zyz*%M=>Y$s3?Xb)gIwYM^00Z18|o8s#QDHxFmM?bH>l)2r^9RE%6R(?0`{-rz)SWC zM?NwrjC^R{teJhk8!bNiaC<8AB#oZRfMU*XE3>p^%xw-n95bQtt#?Wod3R9rd{#g* z@GbIP_NnW0bO@d-ncD4n;S}}tv)c)5XlEbt%|ejf#b4~@euPNZRa4?%-!kgp%KPfel2NsjTMtG;YWO!B@F3fL<^7IKieqor zo#z`r!Pr}&ze9udi*bt){u~AmeIB<{=R?{cL{|eyMgAaY@lC6y6x7fALei3^WmE1R z5u(v9<=@5`k0;BM?bkOo8g?a?EgJqPx)kM5b}A@ERAhhEJv`Q7@5BCVXK=i0dF8}Y zXxUO^slLmRV~M{9Rvjfg$Kxo=Lbru>COUZc@9Z(bab2gM2;~-ayj}WH^AV;7fLEPi zd(O?IBOz?@Jzxi29A0;?)O?7axE^Fg@kgZnTokb`?enB3tcgSI>q~D9RJ-oYd#0JM zd{Sqmd`%*Q!5B^ZJ5)<#wOVYfTEQMgdq+|RIwlKJg5N%#GSMdAy3hDz|0;xfe@$Un z;zr3+D@MGNOp7i3T`Dt2(8R}0Ix2nE+Rgw}fgQrs@=lN?#*}FWFbJ~pw*Ab4#YtoMB!IiKMo8+ITeSJb-S`E#H5y`4ue^qr-vYDc zm_1IBYjg|!RH=j0aVZDlQZJ}bi2Oy5wQfy#xMAe8!kBFW5)XKX^|}jxu|f@+BlZ<(;2cWF)oT%@LxT_AMnsebj5*LP-?Q|dvCvl@Jw+I{2Q?!^ zdxh+fur_L-&lSTYe&$rg{ClZ=V0747Ck{h`Izw`g zO=NVAdL&k>ldif2Ll&>fYLzzHy^oa7e)`(B{Sgd~pqKS}{)nPG!>G$;;Sof&xCGIp zM|vM1=np6w5nvnK_^Q>7_xQOXK~HRUFQ!|GvF!QT&C=cE4XF$B5;3!vh1fngyHZPC z!&vVHYJvIaMf<9t!?B`R?&*Kp0S|-jFxz$_T0@ zaZD;FYB1PL(#viB>)`v$hnjoWfdRF)?Q#l97%JKf9Fs58o77XnL6io#y#fS6&Sh}n z6l@ia40udRdCs&%z^gHD$1DflygLx-{^NmV_Ql^GwFlrRKv2_TCrwQz>8U#h@{`&(LkY$acOBPaA{vO8nNM@W9KYK8I=IxHb zgV)1>IIqVJL@WNhx-&OSwPteU$jP5?JYuM#k1h}E$)Sr2|C>_#XZ&s(=b;(rIy)E*niwfz%XRUEI;PO#gQ;LPVl$4~Kc z^C#oa>3^>Yp})!v6yN~=W90ScFVzp(KdW+QKflVPY#R9JedXe;bkVMywNzm-1qS4h z>A83}rRAMl%^RNb>riwqgk(ed91obUyB}`nUek~$} z!Ch47yx$`DN9fil_h&|PreKrE-+eDQeKztQa?W0yMCz(i%G9KZPt{GY8`14sbmwDH zs=*{nC%4pn0P>&Q4WmiUlH!Sm$B@nR{vBot6Bxg8`D2(cheGyoD5<%907@1Sf|8Z0 z+$-Awlr%oA9@*OXsy##d0dc8I00VV~OZf>QtMce&gJUp`%Fl;bACsR&N#Wc(++Bjq z9(}&INfB;N5Y@%3wBNpw_-P+gYpBw`XJ*araVp7UG4O1l9r()a+QmF{Bso*QRDe4B zUE4N|GotSs*5Qttg*EjOCT_0Zxjh=m*Ol91twXmX9r22^#qyIxg;WY(d+xICvkV2? zz{JWoM==6toN0dB6i|8lHsh5p4<>_rV@N7a$TZjOZ40wEy8>bN04xG z2+vZaG`IhL&{;xxN*Xlf`TIrMIiIdc&@9!RGl7{-WsE1Z@4N8+6sj?l(hP7ArC~(T z+oVO7^&%5hYQO5SFr_PlMlGO0&g zs;bmiYs>8RdFC(s`H$AOEyFsDJR*GLQxTdB1|eUFC?9kSJg1P}3_DNkNBnJ*LRn;c zrt8+_S7r4g_aR~TMx|fh^gy6@OQ!SGeO3LXKMvdWzkh#L#^(GQSf`=otgI8XWl;h@ zeZljp7z5Wh^}LH(jNw=Dh3LU*O(YCozwkwqB6_0uy$;Ir-roaz8Q2b!y=8cY4m}VU zMy`Set2iSFD|@Y@+q<<@%HH$w?4{>eKE#=#K*G-w~+*};L(XfH$4&m z?h!5ef;XIB`wnw~9L9xTFofz$k6!cbD()VWI$Ez-J39}fX3rm+QEV_e)7kxoKy8(5Cho%~u>n(T%=?=eqE-OsqSbc-Xv^9c<}ltDJS5;|Bc^HPeY-1fj< zOGY>R%4(p`lIozMRLK_BuQ@2&_AU?ySeFsTwT0z$1Ve8XjCI*J-I`9Gp8|GRI#>m(Nrx zBYs>d*Fg#mp-y_){bv}M*SR&zT(%r?+ka%%lU5@eT6HBj@Oai}OK<-RKVQwz2^NjW z6fk9=d&K0#MqTV1+c`X*Kf)CoJXO=#X~xUQm~fHFa(ym!6k@*p5FmN6@4TZ%FowWR zXYF_c3RjKqZ)fLQJ!q1^5uwDe+r40k-G-%`9{XAi#6Jy=s5lACei2U4^(wdUN<}PC zVVv47+xH}aG`67*E#I$2kg9S}{Yu z>)pn+4Y`}Io^T<8F|%t5%1VArAYoUemE|kS+W3&X)ihm2g)dt-$NFaEtLkZ|Q?Dsp zFL!K4@&nxelWHMM9viB%|Ap%MD`Lqgjh0o*A~n9o#rd8n3BPHab?wqJJXvC6vuEln z&dY!C%cAsaVp!s@OhY6@*r=Mme{1BiNfkMf-Yn(*6dCU*>e@2RDZcvz5V5X|^%4Snr0iz_?~$FdTiD3ll_ z9bRFyuPfPOI*R2Pr@h2XJWJ2?@aqXD1%Q`7w%LQfz1DxspOne}0`%#%-{KFdhW8qP zq5iexg*T_6sb9m{UzE!+6zNSY%Vsz( z&x@Xtv6t0XdFY>~h|J0e7`~Q#N_f!Z`6 zZR=9^{38>#HAD?J?LSQ4ch1?jvvS4@jEmmNJz`2EhWky``pIL9%s|jn3|MpfT24f& z{Bq>WcW}0dw<>O=?*UC6eu*S!7LysksDJ@mj%W8O6{;Hb#K(~nZh9Y9t6t+xT53Nk z-_Z-v63v#bu!a%@&))sQJR`c;60R&4?X*BjbjnYEihe!2%&q4d8av#xqW`fi9OL`y zQciMZANM}`FXryCH0vRR5Jt|6%WgGZBb_yZdJbb_>E;d>KoP?J6lRlI4ID>^bXsVw z`8b=p;Xx{9 zDRM_=BYpWZ2#q;DEjG3Y!uJ2!>@~o3o?Np+qVCJ_Kj0cD?CEs=@gAjVjT&{*uHPKBF|f{V)v(k<<2%Tpr00KRM>7_D!ggA*4-y5OX0YDLhziT?1j!zr?MU{EFttv3>RYBh4Sn=j z>O=Uu88N$>v`-`wp9y2P+B8CRpU@UFAcNoPZ%nCuJ@~5|UoO>+_3sXQk6dKqUP?`{MU2`$JK}CSnGj<`Z*zALW36L(dk_$^|7alJeJvh$Y5)L zO#MjwI>*YoNyi1K-ND)n4{HdP5TVH=|3%8#1p`&mWaI9Rr zI{rXOMR%0xI(c|4n`dy+Sv=}}R#Q2dhfTYiqYQu zz=ez3A^xJ*W*(f{B2Zs5U7eIlZ3+rZHGG;0PcPF}73n};c(k`-7!Z2o)`Kuc!CI*D z-goH+D6-XZHg=bik6Lsu&rE6`c*)n{(_dOY8@R%mt1PBMMv#87o@y?)Z>}AN^n{Q^4BS@3ZoKMPqHn>l$4rDC7THgNFpqBLi<3EyD!yX7SxVfIPMbB3np)+RtGDxA$ zxW}b+w+>=7kw3GeoE;)deW3cpUN$*>{p)S}3CAB8e)1^Dy3Q8i!W+dR^^Y3%dgmdR zSA+yn#!rTJPbsQ@A^c7iHdP-1lof6c8++RjI#8H@tbSu881^lhEquqhZJQ4LVYX^= z>5QkspHO0`YtJ_wY6$+$o*m3TC7HOHVGh(8hhobSzmiNrd;RsS1GPEh7yjCxUNJyc zFH6^CU-kVe^5srD*YPP=wEDrV@(@re+-MB8z^AzqC}Mu$cRQ)kr#pNOCePPFlCnu4 zQ^=_4Ytn)rR7^N;#`56&J=qRSG&S;%eW(>{yio|IAZS(8xSw4l#Qu9uOhG!T`z2CK zzk}B$V$P*n$;7c4gI)Ft8w>5gZBjrby&vZ0cAQYYkNy4VY%UZAHb`Gxk z)m^&)#g71lr&yheZV`^6l27j&xAi`MJj+Bz89cTNk{wOXT1z;>j^g1R=|$NMb_?QB zMMC;MH)>;fD$4froCXMCK(`mm z>C$c4wG8zPPxN+aQ5CPKFDcQj<@lX8Nd@&FatS4eKurKF(Xcdvy_@CDjkyOP5+0*S zNINnWwekJk|JMRIuC3gce7~P#=UI~c?EW%eu#DzRp(JEfSUg0m`4Ii>QAnMgk44AxTTM&E1ZL}S00mut_4?X&goNf5g=_>*~#EY-<>$a}d1eG*~T6Z7BO z{yrmM9!E;_+=mJxu+2!SyTVN?>(b1!Zj4%6cuTq+;yG`aYl6tl>Qw!F++xFWAcrE* z_ns#3Q^r+yZ5+9*|BAl^yXiC;eBVgy*V%{JFTpau4PN*PK{r0SQ=J(V^mZOM+80dY zKlt6^`8VH&<#x3xduHH=R%I47(k363&a_moYj`X8faIj#)9ZN@Q>jtc>4L=+p1p_U ztmXxprGpL^3VUB7+8M17sSkaK3xBniUu5(M&RiKvIY}_eZRx(`ZLdwa2Rb43N4D=X zp>37p_xpp$9-R6e$7rf=36^`-@^0RK@)3-|g&$#`Qvv;9?ohKg>?ZgorMKZa*+W<2 z=F8cD^??K>*rIO9EZv3OESfj_VvBh?o0PW!=e=UnTJ%z0eaphdRu`VEIwh!S!W_W% z*b@7-sVwd*D_-3DW!}NrTnE9&_Cu>nCzj%uy_n4=3v&~_noja8S>-Ra40wysIJ%xF)A#wY+QgH36& zq=>8&@|wxNp4l={l2|Na%LwOvcJg`3lL_@QP_b5Q%S7l-sd^w|F>JXi3&I6&P{>9Y z5rloy5X}aQ$oxH8SunmK-sTIvz&T*#Ns0eh>EzfqGCmpCPjRE@k%Ok5rvSfU#)$Ba zEai6w;QHJoA3V^I@5GahV{c)X?{q$SKWJAZW+#i_lOtE97v20kokQni&r1V zX82r1^oVf9@@m1AQQ-&J|E0>1Iusd^ApsB=;pgP8{lIi=((j>wboz2b1%)brq7Bln zv9Av0;pvKUB7K?BX@syJ_`NxKBi)~ruNmY7fljyZ0z8$spZI(?#?41B?PfW21*t*H z*!;;Ze_eCa3t0g#*!!mQUf;hb2)s2KKwZ5!RW*`~rF!hIg;)N?QCh12DsdWfzG-@# z>?L7y5K62)G;Nqm>lt6^A%vXnb~(9v04N1l`j5apLqDOU@*yh$z7JdUyJDVfyHca( zK9|Uxz8H8ZzQWB(dq2(Y^Toz-+ac;_>+(INdO^b7P_05zhc@SZ@W5Y-DWMbvY)9tu zUnkFU6}cJ2hZrmw4riH@&R(Z;KD~6ne83KJvUb;M7-W8ZiQAt*4O-n=pc}@H;WU4I z7l8|RJvr{n6NH5l!lWWmPU!BFA^(3HDKwh_Oy@+=NbkdD&eEvvrN$q1QjGQ(xNUj6 z#?NVJFSg=l>0a$oT4xTRe@V>b0xG5jH8F!>4)n5V>jqaHSKSy9M(dN~>UAY(cgMQ5 zq3m!uv~xz}Qo=JO_Ee1+hVV!E0en2}_6)MiI*JuZ`5J!CtecrLx$9DU84bL>7Q_Q7 zk@D6x960l-_}Zi_bgUUEwm7-60TWU~eV<3|0rUb0muYZ(Nx;f=?^suLnL3MEmpcTb z&$ZB~<5^~`wMm;ra%PLF>wO0XOEcFLPh|{=t-;QvTShm%12Pmgtlf0wwG7beII+0! z{>PxzlQXWsACdourn8ERf@{0Jh=g=ANOz}zG$JuH4BbO_4y`mwcQ;ChfOK~cCEXw~ zG$IW{eRDtW?>mAcIGNe|+H3vSx^@KGCC!JHk@U)*{yg0>e^lsdnw$1=TniQ`Un2SI z@L+GaoNu#|zEn2w)CylYQ|<1m|Et!ORAP|}X5CF6S}<`}$!3-3fL8@2P8jvSo~8)B zQ@9&OSUR1)hgOb8%ec=fqt9=-;lnj)6a2F=3)KLg71DhVWwFgh>p34JdQ>0t01&;{ zjPico-VVnFIzm~Iz9^Cr{poeym(dMM&G1o z5q-&HYfVdCoikR2cwJNaxq{-`D4a8W=%qgAa}th_Ul@`dz18&1kC9p$9PDN3XDwI0 z{$w$4F1>^SI8PpC%E$I;cYl!X838HRkz^-5V9=QV*x|DrEv^z&Fosk_*Lad|MBdKV z2yOY)0uf`#e`nDOvV#R^huUI#PP%p@iZdn3?W09pyOG@6$|ED2LFjt$qV}(O06uJ# z$hejLd)Nz!-}Wo2lr{2n`{uZJ6QFB+O`q*QIv9fJxUbL|!BuUz(e8lO5_UxlG{EgEi@pB@ zXW=eAZm)p<;lbs^MmB>ovq|-P&})ildB69lg*b%a+g-VwTcxX9A?~AX5g)eW-pq^s z#~t0)wD7>ISjn!i%Y1YKSCa88y)m0XSbDSRg0Ro}kxSr1Hs>q9&~8{O$!d!;#kB0ONCa9LxIK)Kt)65%|EVTmLv_Y6zUi`;a@$yRx+m-El~O@3Tsx!|g$ zMWMTU`{|Uo{-0mof!mD=@ZMLZSb&M(fKu?o)^S8Qg-_7d^as?Kw0`y=r=AGaJP7L3 z!$DA>WN4rx^OGoc{e*F_PgaJhq7cZEtse`b+&8!EixbMDM#K{1z(2qiEMU% zeR`=7-oLUS6X9*#@lGFei{i9MDq3tI1`6BF=>CNSyeR*C$3s_&JAa4)P|@i@S5lev zTpxbVwd1u;(0mg;86=l|!wm*1^+Z@lAgNc#2$+Fn+FA)S#nlyUB*v{<-&|S5 zOZG~%2k@G-ZezVWiX-KTdLB)49+f@b8EJ@M*Kss+5i0I1l0~o(nkmA-RObg*pr1H~nL&|3>(( zxFcCj7zGRvrT52)jyek)9LzL{3c?p(h-aL{*>)^&F=dJ%2`k~8^bvhbke>N$L_LNx zQho?llT2*_eh+b2VR4iDy{fCnn+usyIMjcGD;ebJ4{$AfuF^{|swt|fh1=Vab3}EL zhY$xoxx5%q^9p4jU;0?hE^K|uEjieP`-V-EwxZXg#9Rcwp_*-?HT9U=3MY?!rjC%U zZ>14fB_;&7K3+MuGf!0i$VJTF$KRE^fSPS@9N11Hf}9>#-reWKdNM5t84x3_WklT4 znCz@#M~e?Xjegsf3?~b53%nQ$SM*1sS1s!xt7$sdR!5DYdR$-k!dYe#J$OD9nUh4# z5O{V=Ie8OM7j2Iia{gqQAx@$D+dsrolOh)XK!-5&ug($YamRJMuO4`R zvJd{b#X+ZVs1lUi^k&dT+#V=yin34prItRZeqQv~tX7(*S4t;Zt(+fVz zzsv7tMj#%$UQ6qB`S>ykf1*baRvGEEFa10S_mTRx2Vu;_Q&RN%*H%_ZvI&yxnxg;$22|j~2(cd&#fVW6`(vcuKejtS8`4VBYHvYN%J#c; zaxB&duLHKQ$`ie7Wyksx|5E2ow!!)V8aQ#wk$BR1Yw>HEvHc=BCJXM z^Za@AuiE2IFI_Z8j)}zOIOTBI{X}Eb->&_?{QHjetDj~Ey@0Fi_|gz4 zef&&8JxAWfgWJgq!NU>lIDE^dXR~g6#tyMJnFX@Z^!cf1WAnW24|kw%Arg1}oCcyP z%eyvQVpnWs8!O*W+I&hEORxjSVQNX5@t5H9pYWM)=|(xKFDtBv{n>#LTxkr{g)f#- zCT~FBpM?4&_>yL))hKSxu`}$hM(E?0PU0AUTenMGN(^9`^^>zacAx5YD}MOlVmKfi`yVKz%Iu~Kvks>%9<&YwQGHS9 zh?hy~2MZD`Qn+=Go&K9Zl78jdt_PCgOU~gbH0sYnojCQ4CHZEb-Mw~jot(i@;@H7G zoagg;Ov{N*f7zKtP5ge((wf&TtbD)PO@2vsP7JJX3v?+76yK3nf6vj(Ft%<~XEEv_ z&?a!Cnf~NDF%^*Yx9zyui^J8wDV;XK{_UrD@#n9U?n<c4psE)>4Ywx2Nx2Y9vMAMciwch~S4l;xwe8hKlW>y_Ore zK_k#CoUl!o5ykmF^#NOG#^bddQILewSO7KZ6fj@+O^nT_9{IlMvY-?o)#{~UpzvlY<)2FQD?4|AbelgeAIWBDx!>6LnEO|$Qp$~(;w4g&=J;|_2-XB zg@Iu;MWeoXw}N`~iwCnffp7j=MbaoGOoOQCU;43ms51XKVHTns-f^NT8gfQyrY zss{{Tny*MUK(?xQBs~wJPzzMXA-@=KK|F{^)r9TloN>FEU+yhMB&MoYd><%h*sYzq zM#9j>QO>g{L%)8Y(;Lxe<9#a$=kD^Un0=OcEtkTKNVx2mT7_m;*rw037#8K^A5Pb3 zp2O#EGH9EIjw(ypyB>1o`tb|=!qCZD2uAko8?Srj2Xwi(Q#BPkOv}!l{K(9BetGmo zf_ckwn^rfhE%@Y3Bc<$fh+GC@*jMxQFyo{)yHesC2x#K@oPLFLJTJ7T`Ds$EQn?L! zi|xw`UKC9Fz^WzGUnlr%EDQW$!#__o%x@S!`3Qbs6iJIGZU-y+LF~X8iKp4qwuVy~S*km|W*sr_&%_GX+O0F6&^(6(XGo;{5CE(1N+0K2u$sORM~x}qnc6R7 z$YVFh47uA?`37ToZ%j28-BFn_Ta;Ok5)4Px=Mm~Z$e+NXKN<~lOuq3;x+TYa6`Y&AZu$h%d+|w{!wmwT)RmLkFCU_ zQ1qEClN(F68zY-h_1i6(9C=IL$$q8ut##oUpW&&Jz}`C-D;=?EebV?=s8b;5ugK5o zUu+?A9q4}}Q9!s3T_aMN|L5>LyL>hc6dR$6RWoxRZ`}h*Dn7F5h!^_e>P^9Y@|riI zN^|_|*cMw@)+0FIU9%3a}fgDQ~f-t#=6@P>xx|9lxY3ty?!KxE+F`UU^> z;Q(|IS+lO^fDaR~xA@LJtEucOh7ld9A^g!Tlh8MIzm$1OM*gTxjqvrR=m+tDzXuFk8y~Ft0dB6lU4xJNNyx+5@(r zky1X##YP&K|2BF_Z4Zl{fwEhM`}SW^a%M6V+U`;M*>BiCCQq^SO;QmS?G9|mw(osR zR98ZD@)GeYt2a4A1=2G=Fp!m$k2r+)KsO?qc6vP{Dd0aPZp-TCHi$c1MulrelHc~bVvmJP~Z{TDH!uG5k6fZ9KO+G3wo9yr6tE zWcw{}DTQ2AK?B{`&aHh;FZnA6(8MXdEJ??>>Y&-ASSt!%OHnmkca;L9EREczDl5t1 zGZx!-9i#T0q;5y)n>CrgboM9w<$&^wu`wr#aaAB*3`rg=e3;~5rInt4^N396xk@4u zmOx=FD5I@vbe@vTB#QqKeJ3W;oiL0hKPmIaK{)Vf3_QdAYea0(ciTXCbNKbX5&CC9 zm+N|%qqjeUOB)5x=bQW&%SHx848zt(oxfLU$9PF*u>|nQNd!%GlFE;iu7Ma z`(Y+vr|zRars`PKAuCFSpp0T-TrKfSC-6(l*TfvAz879V{Tg8&*RLwLWgLv5;y!)c zMJ6{#&Mx5WEi*_6@>>6;36^*=(rGzAOlA~>0>ya{M_JK-ag@6K)P|P{CKgi?y%Kb}X0G@^ zr8-9H?4SnOuB3v|sV%-fpO>sq%J-eCyh=ODDv+Z{wR#VYV1DRJk2%TQm0gE0rCWC=p7&e5Jt)t2@txYZIBH$sT4KBp3Y7k3{F)1#h2rRnv-Ud8e zwpetV9#cz=-%NeShEzyOd=i0c0Ft=>e9V&)@;4|vxyBX44UvQ5+gLaFs?pfv8q$Pj=5unSAP zI(7vjDyXkif<8Nkjd5(MJ>?`>i7RsmAoXk6IIsTzYfPVqi$GlvYTyOVDl;L z`=7qry^8$!ZiR^B=NKaY`0ib|mzE8Q_UifC!u?MpG1weI4V}-ZrE6F6)PvkuJ^z{5 zYPgququz8}A-pNo!Lzv3sBOmKkQ3|)>)#!u4r6cIHK*cb9Ny5#^h=Lijru4&DrY}H z9I`v0sVcrdzc?mOziD+9^RdFRbKGh0&_F$wk4;y9kuaA}AUNs;lvb}|u#7_umA>D) zACBuaasjWW`EUON`Yo%gW-j2^DJR+8CqAFlPnC$)+TQ)wEifT^CT2G}NTN1t@x!`A zXDVVz8Tza!cl$OgcxT1=n;}X}o`W^utxHC3nws$TAv5Bgf_3{{Q9-Wt4$8>dbu?ZW zgZPEisvs^`oM7@323#K4I#5e&Cl7~9b)*bSC- z^d?)OWqb&m=^knT>ul+<0!boeCE-84+D;2qVOsXFAa(Mfuxcz5xK}QrhG_D~x!~mS zAzssvB5u%jN14{E^;=NSf$a63ye~rbDi#0t30dwP>dLvyRTHRc?N<;e#p2hTS z2+_FvNj`gsgh`scpt7W6%tV{jw1{a#T?yTAuT~;_5BZIU^r}5LZIQOEQ zQw?j)Jq@2vECsw|o{7I7tnc82Q`!g0HX#OPIFz3I;sMWaxpw!#(r{cyPNK0z;yL-q zzr^9=!0h$MjB#!2(U;#_@80zn#yA>s$!Zx5ooZzJrKb)e6m&*ihen12#d5~&n@cGH zV2(rD`ddsPf*+qX&%YLnRkpm`-xW55aEhVMe>f)=IReJus0!FHjtcDVYZG^+7%1=T z6Arx>>z8G8k=f9l{QXW2VVu^@@6}c1_+@HPdBS(;@~xoqssQMq%|fL=( zq|k0A$4b@C*M_C%C~YOCP?Z6G0bUO8Y9Mt`{`@uCydQdB0QM0Ew>h@KDS+%B?U5p+ zpaoRj5H~O*$+C)5VPa=gPNa{>Xk|W-bD+d5AYz)1sHQrMX#vH-Rly`e$8a3Z4K;aV zb!+#IphQ$@0omL881+jUwq82nfuUEKH6^^GdQ=Lr)?pyNNIKJ>S1+8VGVa#dr55id zcrN}@J0WgFm75+VVjlaj!r-odpsJ+s&Lo{-#KvjAF#Idi9>Pr=GFr4~HcFJ~`z;l4s=4q%7gLhXOTJIx$zwx4%8E=z>1b3LzCvaTop zo6I7wUx7b7JXpjRf4M6^6g^X3+<;V~L{dFp=3|@2E!(KI2VQ8}ZK|E=zw*puTU_cf z;)iKO@f8VgHizy|4$CZ_v}aEC6TkU-RJWexC=|aslKvEcTX$ZOr4*VcmuO}HDltrx zyBcR4Nc2llUmlvGv6bZ+Os}qHk!m<$K4#=8+QD_dmZ~Q{!+0AE@p`djae#e&5fUxl=ADVqh+m4$eY;+wZw5$X_13=hS(+iDLm~qaLfY*b4x^1< zYjWn;klD!0qY~Y<1y-T>$4!mgzDV{bOdUSs+ST%*L>KUKZ6#-2=zt=ME-4xHmEwny zKmt@pORK4EuirO8Try=K#KTfW&=0ZN=c|ykI{#UWV+}4KL}aR%o_Ddr3BOb05j}rE zAg0LCw=4;Sqvw&KiV>2HgnkedL9?A5FjP;b+&VjWwlqQ9m2@LBg;`VA5G)Z=k&uFuD@u^6UDLurti={rJCj~6mK4$Y>iuwN{* z!=$N@`-S;ke(Ok=ffJkF;H1|mRW(UuQ(Y_ct<`yDR{kHqJg80#>d4sp6e9qms7n5z3FJOEqf- zdl02Me!3Z_${13oK#3VT$ia7*L!On#6$=hZ93p{v zbW+;TaSK+Z?f|UcTIlA-A>zdz-WpU7+jpeH-;qT1TN`PG3oAeA<7sJgkq!T=^)9%1 zK*)KG=Y-xAf_~vt2@@w?BW)D(2by5qC@@l6%I~Kzbyx>cap!u0n#zA&0c}S|m9yL~ zZNg7~|F+Lc<4R;u`R*pxZJ0z0R3Pm5Ipk3Bo<&oeY^NsvO~1?HZ=?nzBy+~@jq@k- z^-^}F1DAZz0rwP-QSCNO8QqOI>H^I6qiBsWOBA=QEfx2}dx=jWy+sg3W^p*w2?r`UYo-vx|ojGM%(}3$EXXe2cFP*X- z*Xd6Y^!~8aeIoc2)sFz#vZvX`!zW$a*Phz5+#Nr@mAw(uu77$hpasTa(O7)-eND`@ zA~z&G?Vy4^zr`ql|KTGIOTfuX4{-x*XW5_iwVbo$m!Oz0P01JcI}K)3p8Tk6oFzB; z-E?}q4tB5TzQE)~0*fTm@)ELiA|2bMQYFaDM7dX+O)lO8T~6iW^!nYt<1&GmAkSJk z1_VN^A}_mv*}7HlB)`YNH_6ZPfG>tQ2559I?qtZ__YFJE7Y^ks)eQ?_mIiJ710M<# ztpl1Ub!%f42word0lheO^Wfq^tC9?=i zv}DQxI^MIckI+}al$<;K0IEf|W}=dct)w6wBN!KNQ%#Zpg?5pUiN)(pZU5iAo+pxH zq2tl4a*J@+=qE=Rpz%rm`N^s&tMpb&E}q=NtlP>;6o9W+Y^I8XiS5Ty-}S;11>`rt z@qmz|$!8jO8u9-)QoogonJa`fNCf0N<{J$}iezmO*m1bOu=$RHzD;x}%FnoudX%eh zvf+Ykk>zB+?buMfTm)?*xEXDtdKm|`9k+$wZo+q@Ec*Xyb$7U_;TaxOqrb+RkmyCd zX{K^Sp3^T2ob+ZI6*opsTU>G&A3&e)o)i5Qj=68`r5YycP82s1dR-;|G{5zDN_QL} zsG2}KH~!Sd2|i`cmo$juV^ad({Lh)G?$};f!x1WP~u^x43X1iW2yH;O$aVC;QaO5-VJ&sR&*L zpS2M+zcQ=j93RxB@?}dd&2JW6{ZM>>#Q)gMsxE0hkVlZ)1#N=N43LKT^v7Ktpja$0D9Jl+~^Qm|;0E$eLu~%w))k=JY z;fs`Z%l&Kh<2W`q#J;(u)&)i!+tGNP@keG}c#SH}G6&8kx_66=Wa;)ol2Tl}P}*0L z&cE4a3`)lSAgfMDqf0x%^j_yok1N@ znsJAb8bV)vkGp>US}x?QACJVBP{Ej^CUVFm-iIxE3NacXwj|SWLL+M)L!!AA-%p4s z8Mt&_`#rTxE^TS74F4o!92?QyCN<$v7RfI!@YlhThEO!q(PG}B#+3DY@4gvtJA{66 zHDQ3VWO7`k*8e8uq1&`9ipW9uiWjL3uXuwLOpa|Fnqk0>T4=v3oj<@UB{7x-afU$? zsI7_?vw^bc2ej@5aIHnD@>#2bwkpY3cFr%~Rg!LzvWIcCcL#$$b8pGstHIXv?g?TC z0szU>nIP5>Y}V|Gb=O}{7VNcajgi-00}4+w6dGMc?$vQQeg@QPRX;{?m=t53F(1B1lvkC|GDQRk)I2fB zMDlpY9NAn+L#9h>$6~`m_a!1mj8iILj8t7rvDo63kJTHlGEJ40#UxP`MiB@_z&wB4 zv1lrOLBfeZO7fz7U%0nTK|+f{{jM9Y;}lf=>lN`Bm*#0Q={@0j_Zww&omXow2d$5h z4PB6i&EfclgYJVWt7YP9$msjhaUS96R812=o@nVsE*lwbbWILR5`L_(T>*RP1WUh2 z^5vkL!I}0n4^zL{)k*K0mb9 zi|afEBwwRQEjM9n;mM`2e{B73k;ylGp(kSB$c=$5&H~lRr{+v^JbL`$JV%&zwOfB6 z_*_WWHAxLs2=j$uO8M<$i3WZJHMTL{=evO2AJ7^t^H zAi0t4@<}5I;vCtyV0EWOmaC^hq^FSRXCr3@+-26O(Kp4b1m$bQ3?L@po(Qqgm3Imi zmLg6npWaN}zGcH-%X!ngRwT3GuKA=!?D-G#yY+7(%Z$%O1kBz(f0F0R5UD><^nOz- zr7}pcU4VrjDaw^D%vK{_Y}$Zq#!z^ahq(r(8Lj~7^yBdiTFykfHxZVuAUBKRON+A< z}S=295x!3)pk9M0Z_ zdrtsSmZ`oT^jmqyH#=Aibp)HDix|ZyFEtan)mKSnoyW*+Sw}BNVN_q3ToNTpOhZ^A z*KeleVddnB2jc)1=Bc~j9=g-Wfe=(z+>TIOvGhX5;RLmx-aNZ#5mqH2XEay_SNrbF z)f}N((XC61R4utbKv47(JWk``+jSaqW(F`{7V#BO?LWmpPihov&i8|Q`7qGv8>h0-fr<`vf21KiapZf2&qUeNSi8XaTP+4y*Q0KyjBiXm9-+pvl1FuI)t8WBO zWlT6Ub-fVT+^UYlrKeD(6*HSpp-K3wW9zG0b~3`tq-xi$Hvk0@?dr6m*}VZ44dhqf zH0KO3u8)2GJSk?fFfYAM!tA$XnF0?)*7Y537myG=&3SWwCbltwm8xh&n_ z*RoE<<=V^2C;5^#y&<$qj)mBsPj*zkN5wIfye{7D=Fi|fi{q;Hvs|KvQQZ#(E zI;Qd_6QHCjwg=LMjWkbA#gq%1^P0Te`Z6kCSOQzehMPQv`%e_O6)P!DL8_kT3Ksev zFx7Ysgx2^85`_p$DFWK-@xd)nR)#!?vCCoXF5;dPK7{|!OR|(n=sJQI){k11{KOhj z9EDH@PqW`jnO8Z@P!du+PoW-pesr?s#N^$J!)Gmax9?rz%zA)5i*h?VbZcbRA-?mJ zrO@b7BY0A@N9z1bN-9F7pZ+X39jHr~g;~ke9*V~ad}sWmQ!_skP}qc*qSQR^CQ+8T zHT+S$ublLpF76js%bhtcz%4BC&CtmF@R9D}!5twN?YI|15R-7G`uBKG^9RWTK2ys9 z$6(@Ttdv%K8fWvef6nwT%HJLnY?rh!Cb-`v_we|xwqVJ=^TY%o^7zsc^N7n0tzwJx6|DXL_EtO0qJV4h92dOsRFTq51in03p1;rG9xwHx8* z=2zi|P}Oj5x~CDWB)yZ7AJs%!G5pG^7u;4{tNwyJ_tIzcVTfwLu7Ez4_X1C>RVVSb zW0uIlvi|-vE5jCO-3l$ARZ4YNgP4h-L|<5vO#&snfRf`6C%B%O<~VdBkfUMQP2~dQBW~ zRroh>r3~o1SZ{V(?$HPphutd~Ue`QM*Nru-;=I0A72^9K8K?1{?0n1k71vCZ*<2W7a|p z#36MaV{D#wK;hIu&A%RJzG(_JXVKt~AiOYJkenE7o3%8~hfS6}mDyveuuCZEbpHn@ zo@w+6E^5gfGb{g87bOAw8|K{ChK`YuIaKoGUpKM#~i-8nOto^xft0U4^(S)H@Tk8dcW9NT`c z3z2Ae>`tC`-bRn+v)CF4N^~6_zw8zKUAc^YIz)v0spxVkDW?289m35$j|X?4eI`j8 z8O6}r=1x9g8l_4br**yR(Xf1T{CNgVRP@h3IHtg1XJnMTX33rIjD61tnG?|Re1>H) zz^tQ=I*PhThmaPSBCLyATNJeIGd ze!t}=69GQ-|6|m>lN({-*okX130W_sJD%TfWMl~JQ-ALG$V9WY_(E7~HF4IP-%^b? zv;fFn-T5e;M4W_460!2~oT8B$rOc>O#vsbA6_|m)6}7qDj6@Fw?e+NMsMU8k#tq2U zME*dNOdI76es_%#QNF~rzk-XDl7;ZbVMa?a-ES0O!&-LoYUU%RZreI1mHIBz} zae7H9HG(tPGu;|9Ouyt5r)sC<;x&>hONwp@T@s4=Zi-Y0=|^(ZoT;)tl{TA3jhAeO z&2^zc?zdazNHwdm(RU_!@@3D)V47pcE7s&PZVK~8;4kg6<<2+|IBS(;9sOY%H(bVj z3xD_~82?xAv*v|HRrF-X(V|9vQH%ECt#|ljC|V@^I(S}Q&dIH?I*v*dZS?!g-Bf`( z_J@91j{S#)Du?5~Z5~};$DfW2meW+D_(1cUUpNS?)9&%$Gec}v`i?&ESadB$&k37d)-WZpGvFTTG=VGm1vQaPv2hVTZGU}Ej{|$KlutJr^A>Sls0vb{ zdZ{qt#1D@o+N#}!t-#A@h4&uaa2x&T7>)7JA<8)K^M z%wNXuD+3hzrx}<}Lg2QZya?*iE5!1v@m(fO*MIJ24DifnKmLl9 z+S7si>P+g8cfZE{+0t4p(C69fOsYb_Y#NPe=g*3j#~z0=cv$@crY-x!`r3eTRCuxZ z>+Do*4pr^AmwHskM=xrrgB`V^7kZuMoK@{b?FeclXKJ40#FY`|EF5>ZFi5HFSn=KC1)zDqmUB&HDo@}};qW>~M$P2rd%JpR zpHC{%czK_Jm*-&+GveYqL&hBsNO5u-xm`c)h-@et1dO3A>za79UOoo8L2(dF8_fyH z=h3jOj}nY6dcT?$R`g76E-ls8NJAN81WU2o4{4B5&0o|8OpYzW%og(NUf=pEoZPw1 z+!6F6g%q=;+;*Q8aeZ#q(H>%xCFhn2q+4pBbwuo_!mfFr+wVlqQxI_FE^>pUvAj>l z4;9+H_Q-n|cb$K=&P^Test*+!Xf#J==|YP!?S0IM!bB=^jp-xbn4S9 zuHpbKUQK#a;s@DA1&Gr4!Qy7RDb<+Zf6-I>2;xOohf%er?aLjS?Y$43oH7E~Z$BM{ zN_e;|eN&)@`qFNH{A=64YhdUZXhX9Zsl)pPdB;?;&h^{yjuetoFlgvDF0MMu=Gg1e zclnFh&6h?{ywq88*_PF)od2>ZSC=g)&gq}LAf)@15|-~iNjmT|b7nQD?THB1*5n?W zN!%LjArxpyV;-@v*Kb*uszY%Nb-Q%l%r>58&}KDeId7knIpNB&y}eW;Eyep3PF>bg zd&9?(uPY|QV(B@V%YB*cj^@`(OrDF_6@~tD4{03p zVmj#sKHL%y^IJJOV6Z3>D{*D;+#vrY$%5+ILZ7dW%(oymwLRb#t`-xesq`dt@g3}( zkoCYG>U$;)ea2O67938u6=>K@4aX;|c7JhCiapyyd`%XKAv{a@#rV&N&Sf&D z^}h`sZ0M_u>|w5{njfBh*5-o}OC&6NbVhUksG2T@j-~B~=6pXzV)qv=&Z*-~6(Rmq zfmG-(8XF&&X{)8UkCH)ZMAXzsOuTSdsd`1CcTp@J8+1o^J1Uzg(dF@QQN|uHfVW9S zLEGcz@Uu4*0Ls6PWLO3{e^{(|^%1ws>+W}mPi(CTUqq86MjX4_Y8k2S7dMRC7qC>o zzeh8BG@N^G;dyXM_^a9TV>vr-p*qLxK<+-*fm%xC*MU6ZX-|<^F?an2^qKo(=_Xf< z=684dPG+G4>_~;7ID*=g`>o4z@Ab(>qAIdp2{I%PJy{FKr~8d(VEy&^C#XObZr=mU zOCiT296}AMGaJr(io_;;FN##z#pNQ{AK?I?HBB+*XE1xSgd}2BRA_WD<8rtg&8)vy zl`}R+j)UvtDZtWC?fGzJ1$<RQ{~008!F1jsc++(BWz@S< zEXX)W`#5es9P9?S8ro?#X}Z|HU1q9L+laHXFeMzKH7`@)G#Rk-z4$RD;Vt&V+71hh3&>8q!bN=N96VK%jplZHo~Ky$#>qEi0Qy1b-B~@4#xyZog_a* z5A;U{)mED#VwT@&|57o5_cVE$VdsS%7Hw!c3-_2i2vX&yFu*~Y4?3u;YJ5(IGlt+0 z3N#VJCn(;Eu_*#k)ba0`)_f$1XF$`0w_|RGQUJa!kG!vj14tEMh?W9sY2TzII}9Ve zDS4gZpvzV}u%M-QjrengF}CA|?Nvr&@ooNAhd}Y0Zrs6u7gn;xvxTo@<@ic|dmuTF z6^%j;J?Wu@pjeNU4qRQni{e!L^%GJv_0ulOnG*Fzt)x6HneIOYZDIJn9h=`d0#l}O zWtI9sTokYZc2e=9XkgaA%*#8(hj0H=Peh2u(QCzE`APh4Zc~|@$dVf=8#*L<^D(tF z80HwaEX0v=H?E9Od>cqDe|Bj>$2XT-Mqy>~PA%Rq{oIMnv?wFjxnlZjDl?QjO5)Rq zkNE3pCI2TG@nwXbGCQa_49<_Swb+v18{FlTs4b|$H>`UnZ+=`Dw>#7q)nbu~&3+?^ z6-)ffcho7{{Uak!Aomad`*#nj{ePtx2-}hkV{`nh4-R#JjjArw6e`#=U;~OVt=* ztgjs_(KdFxs4nM%2C0LXC}+MPJ;^ce^(gRJ2I{43(`!36<|c~YP-`_8=mp*7{q-h# zX~u8I1XKZR+%;-kU*S0_`Tt!2JRn4wxY!GeivGDib(#U$a>64f3{h#$Qu^YH&Py%p zoxPs*2uQrB~&M z=6)O@RN)&p3@}Xp>5&R-^T>$)>nl+KvfGH@8%x^}dY7qOj8_a~^j(a&vGYZ^#dm$0R8DF#au=Zd20_^_ z*Hoa%+${B&!t6hAp6GP#OuPubLxErV*v!b|<7gQ_mSLavWS3PEHf;NSv&Czyok*lM zQuaL`p+ss^d8(}4DfQqm+O;y^i?;5Iu7I7-gO;WVtb-v(qU*Bm4PX7c*MjSLN861p zoLzU*`7HDSPOJ=HswF{Sk;EZBH-y!;s}Rmt<$nRcOCdDo^M~WHD=bw<>zo_B`defJ z)n2z+E(c|3eV|J&YLQ_Fsw#liB2?ACYB`#thnpK=%0!#Y%H}Tl<976YHyJQBW`#x! z`DOT}Ob0n#1Pp5~^=Wg7!DR8i=`dYrrNtaMZGVcU?gGq7qWG>>VViksCMR zjUL-z7ZB#|s(+?8@fq#C5Q+2Nb2Mp5W2jb1H#}7K%#{fE8$ea0K1u zr`SqeAw3~{l6uR-GiL?93~lf;dr;uL=lOVe=PvSH~S7}QjR%`V*At8{N zS(mgWtG7RHvn0@;b_-GV8acGLfhBWh>(B8iv?0%hrC_fGxM#C6iTbo3NBFnYZ9SLm z`Lb^^&(8nL@~eV3&ZWjjXqTk5%~WV}rnbFXH&`rP2`t+8M@Pwh`^lsF7=!ylFYbxr z_Rsg69jsLh@MA4p4FOnHt{ukFWSRx`>_`(@m90X5lOEQu3S&n7x-Xzj-Zl~jcEjm7+awUz z=lq{O0@QO!c|zl`(m|(#@{AuW%7>L){(FwOcZe=fp+%{QRJKyW!zuvigE@M3>+D0z z(WUu3##0j4tn;aIS_qlv7^Q{|fr{|xiLQN1+N~PNRKV@Lk%35`1y|NZ1!sA+GpusJ zsE*}2N77g!z3Sb%zqCWrKfeU-jwo7eHr%}Kl=Po*N(**Mi0Ybgo`O7$&KD1kCnUYw z%Q)iKMe0S^PXo8dh3c8fp;b0Idtpzu5)zb%wurtUn)MOsjBRCO)#g2drqmXB{)0cr zs+gf{nxA?`+$d~2&{Uxq!&et2p3)r>`sd~*R7+V@MAeW(Jwolb2-LG~dHanvJI-us z-uai92=0y$dO0S3?Z|W^G`hE1tU|H{rKaESN8Uayma;#5B4e5brsG`Jx_Un}`BS8r6N{qN zV}7|QEz3UIOeFehGsJ6WND)67K99Kw3Gn~!nz>S3CGjB3C-X0;2^=V6!^H}Drgk~2 zb23afg5`qW5~vi=fQC^{7JCFCg$mA=M{*MxXoG3t5USyq(^rbnJu zum`_!*SS-LVBmtnFWac|OJmIfi^dvx9Y1>B!I1*iHPLJ4*b==>X1LlXlRk5DLs@!Y zdt9AW<0sx4_UNw-ucewNb|4a}INM)Ye2(pW0-YMm!}%vfIesP{9#*-@k9BM!8Y3+& zM6vS0#}bppx0nPo&{64}3s75n)-;nnu$jG`cI~~__)9A;Cf;;rC@WlSSE5l>FHFW` zHLGfDx5x>Oh%G&$plzuE@60dXw?24a1W)_c%s^J+ve#UWp4byNHM%`m(R|)VIr^(Q zXpw{_nrB|c=kcRBFF07V@(7L&w;k)CJ4P=ZIM|t=_SzoB$4_2a__BBLi6P$ig~iVR z(OqZi(0pS#u7Lj9^ggcc6GD)0h| zO0@k`D^@cBUf5lb(4#3#&i&Ej>ug* z@>hRe4~+6jc2uOB-FeWR#BFp^FX1`pbRQ0;h3j2CcY~gFYNbyY{pr8^<4^rp(pHLv znT<|EYi<@c+hn+X%*Pe*hDRC6N=w;K?Z&*${VwjVG;L7lt?Z)+bcgFA@d~3DFEqI6 zxCot&Axv{Z(36k&^iAwL?N=I#5to+o^nq7f9#v`6SKm#-?>1LyUCo7|KR3<5&G1G zm0DL;+JK|0GC3O!_nOw!(b!5*?@5GHw_gsz#};L21?9b^j!$xqnOO69YHN(j>)LvI zCGOC>j-m15v;8Ud?x7MfEH|Cmk4a??)TcJp;(8J_hUL+CZH5Hy#$U`^(|m`!hNBf- zu-;AS-O65^sKKJ&z|#ZLh`nc@w_Yps7V$P!WU5XLwP&nV+?rTt;3@L)rZ zbb5PBSFh!+He(|m?gaV`n>xWr9rJnL&vm;(0=~7(d?L)boC4^2KD>6^5#sqyDmoyp z^%>c5TFk{>K}Tn{@DiwVC*2Mh-+uV3QahH=9zS^nzL=}p&eZ7y49CxoAn!%`sBFif zG0#8BquC?hR0OXW+q~mf2KdH|ag_5F1|AqJnXtks#*4FI#&HV74t|Ud9rQIe+M;Rc z|1i&9`rch#v<~8#jrTtX?C7@YxwWo(rZUdn3he1zkx9`=JhIa}bA0RgD`~_yZqX3m zFDo$u@aSL1gTlRqM-)B!ae>XH4)Z8t&kydZLu9uTr!naGk;R|ylXpt?81Kp+j`XDi z_6_mIHPPq-n=$#$_=zLl+e_c)WTLd{cIK1+sE`7^g=Q{vG#hFY%lA>`;QS>O@ZjXT zTfh+mkDu+0zpLvUV5MQ))zkVBtFTPIO5A2<5Q_?|K$Lpy^}1w}(|#CWb`(8MUqAf4 z-~aT}^}t9B&fm(HufCB>L3MKs*(;3N;hZ&sQy~3{CsC_dwDF|14y1Au)+dbeVMFXO z67Ydwl9<$`ST04Ro8m_(u?KEV_Y!hrbxVT>M!)?#|7fLnYIKTFXFw@&nj{GwRbqZ(yzVm#P<5fJhHF%Olf>fUcyb{Rzcfd)NG%v+??X35DlnMIGh5t+J z)V|YY3~1*#l2m(VwC7)8do|9PYwdhGhVnoQ=3N0K{!BaTMiz_IR!=z3xNsAb@#XON zvzvk5KhasxN{GL@k4C&#SnI}`1eITc@Cdzg{D6h~RM^60gC6KU@69@hV#U&zUh9F; zcgH7;l$-p*=*0(~1?AgMY@4o`UY~qodltM=1B3$b!06Xse8OmI)e+Nm ztIoI^$?pqMngsWoUUlK84Sbdm9a;3W=QYkPN|h`N82V3kao0S8d{!Vb>r(ajZK*hi zeW_m<7}8<;EBlP0Nu5~yIfHW61)O+iw;ulkBph5vlra{@ul540hP=1S6Jq8U(f1H) zMShWz_{0{@3*({xY)|A>`}CLr&rvXVpYnY!?$r56U-Ksnc|QGE*AA(~NCCXepx>Fz$`IR?Hc}v6r1X^ue{VBgEwwre87I=;-& z?N>?Y`Jeo=MrS^~Xa7XKQk-1#=H6Z-T_E*2|Ig!R9tF(WB8?-YeLF*b_S2^aMwvT- zRp-O;D;*&`$0DfRb6~dpbKQvP?Yhz%WPVYG`wt&Rz5RCfwMTyPVTDvo$B(T1e$DEQ ziO{xK!^+;f96obxr0UH4a=`&R<1?lqRIkz8>^wd_kUnh3))LEpj_w&qo#) z%-{l^ekg2UVY#Q}>L|JYZj8CSL82IK<&CZ3R}Rb--;qPhfqZDC&_8>Y~udoNQEV z9WNaCU{6cEuk^(c&ab|qk^heON|Ttl+d|y}glOWzp4*ull8?r_-R`=^y4H#Tl+C_X)XU6W!FxwvlepVH?G!cV#S?c zK9QUZ)e;qe%Nim+A^5^tl}s1d3TRBZgvoa0=h%GrfBKi;%SI+y2y&dY3c#?4VP_1C zd^4`TmKj(74kJIyiO2tBrN=&z7>*z2u)d_({9TLC|?2_dbBf@gq$_YR|E$ zJ!7<$i``2cE!#1g{d8}usZC&ND(;s@!~N%aqmtnL`0>M{&X4Y^T*441e{KN^h-gk> zh|4wKbl1`2@e5Qn$?%eX1>B7FHQx$lgY73EZ*~>P&81c5Nl)Un`bcvDFO3>s_-l7q z+YikPWjMgBlB82{VgsH{;`l$p-Nm0Ygk}EnIynED+j0I|`x1qeg})s?(eeC5W+l%8 z_A2&XZxhKfhAW0h&l@eUjc|caen&f7&P5&n#OD6%Jb_AJAVPSJF}i2orE$c*^|L>l zp&$S0pVqdoeM-`izdxBGea;R_H;!NGj5x<}=VQcpUR$=GBx%FQZC-Lz@-$9yW{fJZ zj`#Ube#cPW`NGcyotb0C)dq9=>EChT5W`Nr#3)c;H5*rVWB)aja?O&DITtYyHrr%ND)HO4(%b8-(-x6c1MKxUqt zHOQ;br+KmZ$LkQ3X6jm;!$W`B;9u&2(O=iEr2R@jJil{}bGxV(Q}dWpPBd1Ta*l=P z6gi{8wPx%KrQ>)Uv$YK~Pw<(Kul_?mFtTDM5iz4qNdkx}&=GTpi&i-us5ONtnzFIB zE&hg%?=bqo5AsW|&j&_#+DqXKW`WL=n2ZC8V8)Nc4_#P}A+Zr4 zIx*z<94lRRlEo|iJMCHP!jHxsN8zkLaa7(UG;{3j-o`)NU(_Ur$Y+hjX8yzw7^LaS zLXOGHc#=KNGgorvD)<-8%-76r-5(<}zDZZd`d%~pCx;qNKYGNmS)x|Q_!FNZWD+KU z8LFMz?QdfJ#{6ZPS!dNp`lm4yzq36LK7Pez9s6$>Q)lwpZopAi*$*N6?0^Y6};|B z8CU9*TNqwjo)K5`)Ut2)qF>bH*w^C+lR2Y<0+#p+WNxBXUKkQ3M%#@@#Z9(QoaK+{;Q~0LO;1}yoSFJGZYvL)#gLkl(!>g$`ts}`fxQ%fQl2b6E*p2$4 zGc`__w9YvCY(E`86SMrosb+Q|NB^S-Mu{(tq^Ye&X;dh>E!wTUZm_{#3w6pdWS82> znR&!H63rS@q%gT(WzdL|iaM71|4RKz+I(OnU5+1)9qB2@mcPV=&3qM~ zm~-6Zf3nmiJ!)9k5WXHCoh@rv;QLw|t(wiydisY%A{;jY9QMo!mpJ`93wP#+H)F(! z`5v(lu~EP}NA+ZA1~Ho@;5oeCGR(>OKMCvynkedUzP(o|7iLC@iKjfXFWkghd#)Xw z82GZZ)k>}D3`fSpPJTKe){b?~3gKCa+e9|Su7nHABnC7;VH6LHzWwd*WQ2!B*-?F5 zAVOZsvp=QIHrt0;m9CB-s>^DlGylU}G2XX)^<(|_#vqX3FGPwCh3e1B`yYs_&19bP z464r?CAgaJ+>upiv>8AWV)a`;_^nTW@@IeiX+AKrE(lu&xO-i*d@H`rCgoXylj%OW z1aq6I?OkM(5U3n*Q9ch?`6p)6f%FAdnp0O2OS{I;4t3%-tUB*3DZ;#r+?n)Yp5D=Y zfthu$u@!#C)K)w(RcFFDz>2e(B268|&paDvc1Rk&Q-iCS=6}Wpm3Zyf zqmV54f?NPJKgzubYOFjV{$fJVh6|MQ$GQIzPy9jN&I#Lwjn7aP`@v zSsm$@_Gf+Rbx5U#Bx77@!jV>r(e`;(&Ca&shGgV?V6@-@AIq6X%JU^S4~%~OSAS(^ zqnG7$O@K5CL5Q?<3p)5|LF6WeI00%S5OK{3-4oVOBalJe6=eKqF zdAL!R-kxLc%{Vp5N4cDb+o5PJk{X54RrKjWL@_VsS``pSh|8+BuCK-y4jvl81h=pR z!lFkkXE~he6O*Mas&0|MYp!ODcE`o|yakQdtOG?iYujtG4uDp*$CWQaOy^4eLvM1SuVT z-tgwtw8+HnwBRe2I40vbQQvW&d}w}@7u?FNM&)Px6SLved1ons+|dfZG^4=WvBUqW z)_2mJ7;-%65{H+#bXMfe`A6q24>9Gqi_1ST*<0z&*yC^L9NSMhw5a3r{;lnCm+!3mkX`ev`tSHWFw%FgJuNdw{#A-_ zXkPMPh$^0#$e}kP{*4r8gUbz(y!j$fN!K~_@GC4AakbG<%vnBt`u+8#*Pr*mNE%wu zb+2{lQ`WmUDJCc8DQ7+km7O|h@74&$Rl84}zQr$vF!L{jSR+_Gaj2O=haBXmKkdY3 zo&e|dFSo1fpLn&edae*H;_l2q41nywI@PXo2@T|8#txaWT4~!Y zY-~8;jGb##p6${oh}rI(F?os4nA=`wTur^yNe8UvM-2c2$Ns_W`0AqgD%#ug)g-8B$V|J0m%@UEG{18~*9 zqtzq-%&P}RdPu7WMvD8~9If~fju-sIs8jhkr&L#wl_xv3nmTq3Ml^ZJ0BPg+04WL2&touoRu@U9H*z7hebw2hQg??A^r9nzmlfOd1q1mOyIhc zDdJliV~}hB@;>ng#Ty9669K`pzz_H8zLL4)uI=)QxYWCGYMpe)uc#B(imzK&rPuhU z=&5t!zD|8pTRKx~#yV%+9Hb-AMF2bb%{f{RV7m#znWr>-UuKtnrQm_jXRA@-7%VPn z4R=b<7`aZ2{3q_jobh+GPK_6ms z83TuwZ87Tlqnl{$-8&|wE_(I`5^yFYi#ipU@x*AQrLo-t+%*{Mb>mRydk@6(cJ&C1 zNA1*Tzgt@CHIM9>Yx31rteG>qv-aM^b=x}K|Gd^-#jIaC@8lQmtie3?rag}4%mp9Y z7^!<9h6SVOY^x7OZ7sx4KNZSvOjiXb1!8q_<-cM;?UC67A-nSDmnr*!QCHA8;Y^*( zAIIH(&S*p)0YQW-JXoV%o>cm@|e4*U%{eq-VqBGS@_r$ z&-0FYwZsW%V&p|<`qjpd6&4JZm@Gb7z2sZj2uyDYU7r*+s--%sLVPU4r z0PBZiYAFs@T3QsR8e%1#aWJcs4XZvKKVZ?BH8|#bSNz|)d0-^m$p=68-^4uZztjPb z{oh6D{1+a;?8SPlV+c?uWnkyZ$R_@hfS{I=7$Vy56Gaxan$VysW()Ti5_4|Ik^Y zY9yc(Gy(c~^}r|zb>Yk>Ty@Y%Vrr?s-k_{NG*nC##DEtr6a*y>?Z5y2$KPQTl5a1) zbM36X_aH}61B^?I;a&nC#TW3VF}XM4blv5g*r~Cwa<0xqKewljP$&JP_*4daYTRL_ z?-hRH?%L#3Ol`EVcbr!~X-rM&OsyG1C*yTFkJzsLKn?e&ax6^)4WB=~&)v9>muooY1ze{pk${xuw{u5(s`d ztiu)bzy)+JXjHW$M8qz7ZUq{azuXMyBj+Jj} z%(h~8%j7D4t*^JA;-}`xpK-Jr(^@-D${r6Kd;9@|X7!kf%TBGi{~6=x!qjz*eUwj9 z@UVN~g6z8Qs<-l7H_9WRPj+;%#L>W?StCe&8XZ_LviM~6lFxC!VUBJ&@TO;*t0xUJ zjGve#S<+|BaeGrk&ARsnD~y(z!=4@!{S#x2xo(}gkHDYy%s*plpK;|r^FQI_*Ft5o z=&3Gru#HvA%Z%aEp2YqMO1O!oO}=&fgd=0I>I%9pa=>2&s}WB0$hL41Atp;($4_Id z?zj<--D9YwS*>edE7QV~AB{X(a0ck7#;>Hw4NrKrfvWGtp2l?x#I*3SGe&zV%(miJ z-pH@!ar~Toa-enU2@l7y6JdpO%iN*RA*V1WZerAa6-P(5ac*g?m+Cq-h1b#?_qCo_ z?WyP(!2?v|;A72q&tEECg0T%I#yBgo?s)NWT%J7;fQg-z+m{~Rhh^uAf!(S-W}(e^ z=D}|}>Q9=vt>4JmBK}Q1FhWn+=Zg)|*GE^u`-*z%UC!7?YX0+qk%SO7qSl(^3d|>b zie3_>vv|YY;wmm_oycwd##9fC{`BAN-(e(ppI-MJUokCJ=C-3x|7UHF_ZRT%_8Hx@1INjdvB5})lN*GYWJ1^zBh zOqMwI)c>45x@hsh=zEu6N$WF~^mTRT2_%e*w( zSL+)gx%yYq_=Hi<8x(Bk$W*x^h9$qS?0OjshwQw8K~F;PZM}AUKktJRN6XZ(=k@yJ z_tqkt`PqvBFoycC=9Td-ueAN@Ec1;3-uWg*50sXc0M!AD1tW_OJ7W=kxxl#tS^Gqw ze#dCvM4>Q!jeOl=d!WIDpoNu+FyQBYH8Zr%^zQA6VgGG}GtRsn>uoW8vyJxT&o)?a zr{dgKguQz(BR_mwuc>_I11FA_6(bN1npiMc{IYlR-#DV3XFr;=gj}Jij#%`;quGqT zbr(ltBEb>E3PTJlAMxpvUyEYA)E9>R2fWYjyACmN(8m(P3iHkapP6fZx9)Y!WS-Hv zV-dsN=_$Xs@X@`)-uckRwoapowa2Z*)(M;UAH2_?rdf}C_gdktz~%kVk(BxlH}vJ6 zI7#=buKzV-JC3b+Kc7|N%AtnGyx=r zj}MHvV-FDyo>uA{@+r?;$Ul82hM2HVF=0-vS6DQ$a9?r2h?6)RuSg_#(|#nWI2cKi z8arb%uJ*aSi#>5uhrHAGZu``|)4IhBCkJ2V3;_Po$xyvIm9^Q z#HL0n*YRU~tWRQsdWBAV_SOgV9sC)+Gw;{Q^G#}L2zPq(fzckX9!{RnhYn*>h!^VT z1Ea_D?BJ3jTx;R7NsN8=azThz1OO+r3_+_k_8N`@O6yxO))H@uRFfxT1dQ#-fj9yVlh>3nhkqr`E6Y zz=+y@p=CoZ^*XJ>fAN$(>a;-~$Hf5zGVT&SdX zVxGoAZtx%Bz%Ls_E_gG);-91W5tcZ4d+wb{aWKAa%{>s-YYQLL-CWE(!U=QY-qjIb z*u}5oC;J|RARjJgJ`tAY3pTuWurFl3Rw!#=+z(@>P7*!Ud*Xx_mIp@Pjq{cOr?wXI=;kBK%BLddv2*s#l@=}MIkORO+nn}&-IlWE#W%!r zCbb=hC_Q}CfbH87ke(+=_>F?a*zV}ai-r+i&-la$OAI^XW;bTW$YY*+y0O5Y+zj>@ z(8F%naLyWpIpgrK#CyIRkzX(Cb^fUj7dvCLiIMNw?#S8e?9|gL;KmD!#c#k8pY{3;6}>-YiJs>WW| zsGHHe9SI8Wi6egc!4s!H?To?4f@@5-^5(p{Kgk2!dpF#)z@tMG3r;y`6C>|w^Bgsu zK}c?|W_0Pwi8fX~@?z!FGBNhB-XrwP@4Xd2tW#h9k1&e!5?&gdKj!)C>G+{0aUXNl z^;G-qTu%Iqoo%ovV1*-wC5E*>Mv#quVAMGnu|4px%3zK52)Xw@u46th>fC$I)2}|` znr(7p;W1YF^Y~H0=R=2uqY-xI8}6+M=FD%Gjpy#CnLifAM*(b^t(8&@F>X)U1UJ(1$P1O$VH zk3BKMITo$>$!noEq@cA-Hu%oftsN)w)zfIyv3FQv(=xX6fziaMebIwhv!rgm>ior* zR=ueJcNmHyo2$kpU zzU(v~J=v`WkvGRg?rmzo`z8(eOZ%=iI3KXlTaKtEe&~|yu0}AgVo!Xy8Lw@xwoVjt z0k?Z6;%l@KkI`lck!0py;Y#sM)Ns*EN1g7XtvBYGCvB`Cr1SMyflkyNO}L2_=EQ-) zkDc0TPo0dhEtAl!Zk-BW-am8y$H?)rx+>sNCWMzem(bNazg zTdC5e8`f2$^5<-YCx2F?F~K4SRzi(*d&V-+v0C^#PlC^FA8Ow0J^qCc9k58w7`_vO zA8g0FmBKoQgiBuirJ<+e2Nrth`vNL5Zv`BA#21nH(vd`UWbfi&!O_y2SEV1bW_bLX)8r8#i$NQuQcGBSZ z;SF)lP57sVb+p#v$T%z6eXQZXs*@PD)eo`gf+0J{kKXw0%YIxF#pdxJowNQGFEQ%J z&bZo74~!;mugg93)T0dt3(xUF$>d9`*l%L{@xZ7MNs|UT)1SeI6*zc1jA(e(%>6l# zx&Q2QJ$Yyi_6$F^;Q`3wcZlfzQGYhqhj8Zya`MkUIP^b?d(eS~$o1!g;|G0cFee|^BtYUGG6k>{-Xt* za(|Y&%gqBLeTUJOSNY>5!vmx1#j4Yvb*Vpn%FQcTGuSej({UX^@kT93d9l{`>li{u zmSz4*ntU^+KIyQ;v5mldE9mB*-B~Fr?y`5^NQxSOuPcK8e&*tSmBAO?Ik>NX29d-hE?j5 zI~X)??V00A!?@gA{n2S1Jh=oT?;h{-3jz`=-^?@Hiajpm3?XfA-y22V6zKfu&3sdn z`1CztCuWUA;HpI*0?*73$IqSanSWwt`xQ=gopJh8W7ba`pK<-bNXO5P*?V_6(RyW{ z>UrW^vtE%?Q*FJ(gNub}e6ix4&rg2hw?AJU4~&GL+}SQRlDBqZCfDGK%4q-5EArNU+d6*?}5>+9tj^TxjJl*hOH`R+~Ya+zt|rq1|7yV zV`}3UPVF-;Kg-0v@+pSS$+7?A^zA zR4Yt=A|y6s9zOw|4~&Eoe`-ia3tBU#HvR-Xt_mfiok%YoKbfo*3$^mozmsiNN5NlW zkuPJh@_*Kd6Z>oe8x7yl*1hZR(Y-fy9I;2kX2!rMJNG~DK5&IAOFMNW9}6QQhQ*H+ zmKc^;+QUg#JTVy$^%Li8fR^zeubaPP@naR8+Oxeg_udbG*AI-o{VU(e0DADTu!yg* zYQP?JKw^~f&`~>4^Q2=c6{`0QYrsp!Z4#Oy6B z)cB-F97}!Ru^b%WmR|umE5H7O>L=q9PSO*Wm->Rwzmg`Omew>zK4G;G!^-Dat<%Jc z3kPrbCn0t0FiR?44%o^IZpPKdk3~lo?8Ff#Mt|DY>T}T*fAvV?^tP5W5XXP^<)1Od z!AOCa+H*NE=uG~^z-_&1<#vm4CpgY}WraWaqAX10$iP#%xPpODnn0 z8b5uI@~?ZI3hMZk_R^1rA5S-%78vxoj7|QD0efoDmxVKZuiDm(2Sx>&`R%$_y!SFY zdmXcG+9&^`2S)H1%j6To62po^EYG>o&j%85^TKs!KJWx9-qcWE3-KAFP0YxF@E^#}6Frlh&>Y`f5);vnT)M_=$~i)xCO@1uyzoDj<$M z<0npiFj%-)I1_7MFb-08&A!u^db168`rd7yd8XEkea-_TG;C}C(SExfKcODmkMoG? zUHK4L4v%KK34QhB+A9>4lb0q~K0bX!BE@y>(*bP}p$e;pk8mIG>dL338b;fd?!)o) zz^EBZYP#NTlWx^j%qKs>@5aHO`trZ40Zt5_Znbg=E&=X82!(+^Mo<;ztT}1 zdXXEuISV756<230)`_(mv78T#>|J5H%RcEYzy}Sd?UVPqa22o0x~RO$3+IAS;NrYU zFUO7zicBm$;xv0%~ar9cjuGtzhuiBfYI{AtV zpx*Ev4_t-Uva3yvBd&2I_<9{Zb=adaeT`qwTqQ8`DXxsR*;xp19i!z4akKU{iehjN ztLq;ewNQhpe#h+vlR5yy=2R}cQD9re1T>*H?mqs6V@%eYYcoz=bjbe{f3d}pNc;4_ zh`Iz)j-KUxCK^XYDZA#J`eYj$8nOSRX3}=96=1rzXPr5~-D#Xkh0S8L3eI@oEA*X8 z!Xs9Ug-L83KX=!#Q%~bZ%jbVTaGi6jxR*Iw@=R3M*{Hqpy|*i74SD>A&NzNrb>Dl? z!dh*|;FC2jvgl`#+*sX7h<1OsnD`;TLftN$?8`vCGvBXzWWH)pF)xAw#2{0Gjc zC6LZS%A603@?nv%5nKrejte6XIwIsHz^O++todcjfA_#>P`*~PhV@b-Yq~+ey^G(C z>kfA#dvTEmt68(wzN4H(@GtWU^b)YEre>Wm>W(J?uOz4grG+IB4w@GXj;ja+Om=SEWO~zz=-=AsfaKA&T-v;$p(fzZE0D@IMA?V8`s{(bt`a!_KirxN2}G`s?A%pR_j)i z#b|zyKe1~+F%uj*SgBcuJ>L3$-#t%ictP0QzB}W>YX`NfzBPDbNo=V zExr5m!nLB^0~_yIFMe<LU__qOk8_ITGesrj#!91w(dq#g|_H{di#HWh%V=LiQL$jCEv#c~zl98l^WO{s1?1wEfgqY|fV|V%sV{IrEK~a4g{IpW~YvV0Pv9x<$pi|K8j6 zHu?!^gzNFz$oq<^8;dGv{BtgqfjIcCYtIj#8YYgIeCS}=aA2^+*YOivV%j%4PRkkD ze!}Xd`geqJ=XzquGchwR{~c!HJKn8~M;EPm{Xp}4V032UzJ^@<2QQks{`RpWlp`x1 zo>`IGo;6{Zt;PqU2x+u3FdB`|POO-J`smTemSlC!n!ZkPRRBjR5?S?X{N<3DnJ7(USp=ptG3x_q>UA~-)Al= zLh+2x{jblVpxcrU4|S_g40~e0Dz0sF!G%MZ^Y}q$TG-Z_5nq3P_L5~ zs{2=vj9&?*F~={A=O7{qYN?4`-O}U!f88JZqo4t2w5Id9qV+3jgj+yK3S330*EkoW z0yOyi$aw@c5dU*q95BX7BtBv%lmCS9BHRz^fzfaM&OeIi{UV%}kK#so4HmAv-L5x7 z;aD3R!W6k{2S?*tTFKMMj`|9~N%JH5rB^vG_zf`fEAO$fC9%PUk@JIPa@@vqGrrPl z#33Gmj74Hu@?$3&jp@6?^`L6}BIc1|^yJ)WN`GQzzK`0c-q+E|>Pt{M=l1JdZh7{* zLU|kGF@2qSq%6IuG26u320DJgWPNl5da)>v#xJYIwEO)_m?v5IDe?kGeAy&6J6m!5 z+~sMM*}^PPF2#ahATa2s3k$WBJN~j=<>(4X`1x7WvRjyDl`PSo}5PA_V3~ z>w!`HN?Kv4ESX$j$TaekQ=!U|DA&h&pVKQ2;oVN0B39d5>73&XQj`~q*~uZk32q$` zUM?rPYiZiV5SOJbq%dpiRe)dJQ{iZIORUzg-c$@djW6yR^2-ecP7#zQTWy6`1$K<} z&LYqB3XI<^?Vmzq&CxL0Ya<))ROjegQAjg|Y@S;56h9T3BVI4&tUZ3!COmr6S{uG2 zGZ%V^P$wOsAJ$(kX|894HuK~D)U)S)P1R^IMl3AXL~qVtkFleQ2SX%`iqD&}u57gu z<4U-Hk=FU9P;y#i^_ufYy*J89PIf_gyuh%#1#<6)Jag`Xv-SYwsR4gXxfd!BYS%9 z@?TM%d)F|Vv8+5X-dWk|9p^tViqlqo45?bA002M$Nklc(b0Rqy2AOKj!sQnCo9js|;oM zmE*eb`Zj!wu#c~mF&k%nB?laC^z?CTTYt2*n(5s9mioNdi^Zs=e zs*+}>+EvOrzv%oa9JLY0qJb6KF&Gxkx&WmOPuQ=%g_3Zy%}C7e3U(Axq#4!B-Z9W^ zXGVF!o_<9X8-TZ{nU;i{87zx|8R4WV`Gl!yj^>J5tKz00>+zR@S=)4D&S=p?U z`SnNAq_N4R2W8uNqmZ6TD7sD-eIPiIHoP?T=2w^>3lmm2MfB$(Gfpny0{_tiBjB#; zsJomKvyBTf>VV8gS;8vE52fkUppi@v3J~&>Mq>#O;NI%TM;EZ@<@RE%)fkEUe#d4 z*IU0Dhho|;iHTK~O}g4o)U}4!JkZTqtl}N%p^Ni0F0dcPrNQzNTl2qk)d<4e|IoSE zz~<;>XA<%AUo&c!QhY)i@nx;GUsT;PS^rJ`j45bH+VB46|M2PO{@O2m(qBo_VoXY8 z-=9FJb0qt!_RZQq*`Dl|7rr!ux9Ze>ktNh|GzeuWbuc~-pUUp`d=M)BG+jn%$1;=*eI(67rc_wdX&~sfn{(3RD zE1-5EXv4?CCccnzjN>!Udz$=3RAZkXX_3A4sqeo1LabTS~HcL$3k50C(J=mH+crL1PQ?=YD&5t43(NJuCM{kwAk6-yxPdI zBzNf>F%lNWelxE5so>0~{wY2CjgMA!*7jRF9iwv=_CE@<-c)t1bNxEHQZLHq;+H~< zfrmvJ4z;bJgh%z~?xX!I02-ZedrtRiN>*%Qh+&CgH5g*D(@&eYkZNat?HA#QLg)39 zEphw6s0Hf05PU6h{!l)@JpNS}g;aH`o5Wsqo%UUDnoEUZwXO?HdT8qu>_v^UMnlkdx-9kuD&lO^VN)~FgS9)Hdg z-SSi4bhmzj3fXO1CvIv9bH?SH7*#KfU$NGH7XV>$ZVIgPtdUaieJFD$Jb8?RFRa^c zHIDunSHHCyRb_0o8*8+8iLOw8tYaQOdlj=697)7W+iqQZB_0FQucZCV&;49HF#45% z2&=XD@h^&E4$>(Z3Kc(=^}fPg5pr?W(RAUdt9Ewd_{n%ASkGsINPeBe;y)&fX+826 zbLCl$nNaN%OKXbi(rZ32(g{_68g_9NS5aY?Gy00(J}^??9I&{tl-r$J&pdZ=!HkKu ze-?k?qn_1X(VNCGuG9@;ZrA}|(;T{5-qIfY+h*3D8P_zXc8YiXLiFr+X0^I^n_)SV zpZQEe{T=>Zp&9S+R`SuX7TYiC-*?|e9`k(8Kfo`o1O6*Zt-_pnh;jU6Je&(x^^0q) z*PIBLe+cLauVv}Do9l_WiwiU{P;apTWS>Z0ffn|i<0q5#p1Jae$mRH1l{v%8Ly=!# zo!%q%xeAPofvfV8CuYvSdL^d%uX?+Op15A{RCkyYu_IBK-eQEwhee^q z1>d!KYcbDCMUGC}8u;W%Z=_N-XVf~p1@D173Kx5NyZR$C_GhF|4eDR>?@AITwL`J! zcV1^$r}}_gQCE)z&DG5gi;&mz3(?b>FZ@2v>fyvfcM^JwOO;nWLy1E?W$~Xl{7?3( zQ=Tw6uOYZF(dNCE>+~Bg#r0Bdy&4mp%lycvMYr-=`usLE97#Z~Rt?K#kLu{9_R0ul zi;Y`1y!!*;orwnfwcp9x%lUJr@~5aWkwL4B)ky!|>$xP4s@O3Rslk?^G~*U{oWBAg z_X;h>Y+R$%zy?ml0qdVmb;J+z4BQIoEW+W|pDqEEuhzZ%J+ zcPEAH=U8;G(%*A<1d8jGv1lw@fwlKCR^|7-tXjsh-<5&`wU$J4Ex2+Oe-aeCF7~p$ z6i>|p^RS!yWoHR=)&Epijyg2fHK6b*8*P8W==#7&!h%u{?+4R4i>JPv-N08t<<7)v z^S%Oq+DO3j8RZd1|NOv6Pja*bC

v4i8nuX~7Z$3mQLIC7PVxaHShtnm7rh!FL#~ zUrA#|g^nNc$il~_POQLWKlqNvT8#>>dgvpPFgl$B?Y|T|20)!`aPkibuFduc>=u6K z#Jubd18axe#J)y{`sYlZwY`*Q`_~Ttc;Ovm>61t(uiZO0?8{W&WbmACjd_lna4Sl;XevyGaM*cd<1W}5gIOY0tAIy$!PFo&o#;>B zY;!_9j~`*C-fSx--BVkb#`*a9?r;9n_=M5VfBN2czn631k91lgHG&}9$AMy6<}5`# zK+@j+w>cBm`K$JSu0&{$f-(_0$|t%WI3+go#mQftSQN{p;(X~f4~#_0^abMvX|HfmyO9yTv#E1GuMJYFMCuyz0eU)dY&bEvip+dg{yHDM?L`5Gnpse zxrIOxw1u=rk_Zm%;?3H#op~k_oz5_#;Y@!B%bxQB7i$gms)Zv9b9I{w+*QCHfkiWQ zHXM)jt~a2CAifCESaJ`TSMgo%+H<+Pg64lB{UGI49f6LYyvML$bo)#HRd%_zvFup_ zR7rjM6C)7i16y55hN$bdw%ZQu>yEVEEF}ImUlOdNW}Sz_zvV00rB}tB=ULPiu6T^0 zy2I>wgzH*rTB49y%Zg``Yo;L{S{+A2cF&ss@jq#C{xmPwCagF*a&Vr@=@2+x%pe3S zO;(R*=@Tvu37KaV!s^i`jwRn6<}P;X2!j@}*0N{%h25)EZH=(jiE#a(*baI8$QdO5 zGfLfZt-88C6jWg>1)9-kD16NQOiAX@BT6II=&iMx|gsMD|y#E2$ zdDNPtH$UTRwXv3W6@Pddeb;Jm*XWi$v6536wDbHamiCiWY~nuk7k}sPtk10{G_rSb zV)MOP-#_*Y3CC8E&aUMiTEb1tzI9rYk!W7lbz%v-K^H3yd1fn}3Ix!Xl~gq(2_wfE zU{}^Hs%pD(b387GG5QW8eZuHFe20<76QOwy6@?edyx%$d`JX5;h#qGRqb6zkCAvQF zQvqeyr;EReCuDX-M-6RcHnfV-Jjm*$1!1hc*EM(|;PG>?5?2CY<^v;n8W(`VEKW4g zn?7=-FNT*GZ*ju{;OU>NXs5mvr|xVYjQ=>@F#zg3-|!B|Z4GD56R$R(T)2?O1>+L? zOowxl`PJdm!i^4J?V1^_?Z>_eL*qQhsu((Gobw0&ORe}c^>Qx2BU-iswQzYLc@7tL?ix*gYMSI(zA&UlfihxTWr?+GL z;5RJJuur;;_qiz_82!x8fBNn}`6qF*{Hb{JA1CD?YZj!T^Vgl$NR>@-q^&I`o1Ap` z{H5)u8%AQp`}|m>($Rybym;jKkB^wTHs{H-t}&WF`u#tsPZ<4526)56@#gE{zHJ4R z)?Bi#a6I|Jo^ko%o)}??VTD0M7R-!KjM`d=W9@CN<&SwweOc{08E|xAAa9nc!bxG# z-E-;5UVx~H|m*`g*{G*^Ct;izoR8Q6t61Wl=R9X|$}F(ePBx)lW^8#JGO zdj2L6DD`0PF!&}0KCx-RP0VZy3uf}}+Tu;@Y+p3BT5ao zznsp{$@$AwAS&4f>q)jyD9`%Jc`d}UaoiVe#o^deJKu12Gi@VktQ~XPpGq-eh?dsJA(B_;EjL3@)miZ%| zq9N7u=k4Nro$(ic`M>`3pZv|g`RO13cfa{b502Q<=$;RZn)&II9u)odKmDCg|Mvg> zx1avkKm5Z?J>#pEHyXKz6nWFgLa4??!mNW#6728zZMydC_bj8Np@$N9fP3)`_ zKUT_UVC9=}^;sqc-^J?u6Hcvl-SO#g&Oqnf^SH-;!L`N{t;~ha3z^&8D|39hzCG8a zzHr=)gIcbCCC!-?*Gt^_if%5tCZHilUVp>t8l3MT64fj8+T+&RUM0r+lgk669|xX` zx=4CuVS{MNV?C=Br)FchnRyj1>I#%j;q}1iPyXE>f6^z65>&Xn=_n}FqLg_nf%z3uR1!PdwbSk_$QNz zkA~}SG=8kq$cvSaIF=Z8V$^1tarxl0{pB;}L@Gke?dG~a8MKX=>V++`cW=;Lk5PL* zd}m(yeFR2>;F4U8r0>0k)A9{hpb~hZQ{rJ&iDi;*F6BIsmis9aeB$&)oNe{NV9&TP zPYWEy&4ca8HGU|?aieon&Yw1F96!2omkLUGNJalni#;S`LD=vAh&t!wKY8qbUY{D& z2;-ypiJKK#YggEN_j7>e)pm9BUR^lrwt6F5rK7zqI*LY&K;!_+R5Q9~hI@@92k9p_ z4~(Qh4Or=G!LH-i?v=OW6&M_O6h{*a2P+>bB@VKEWw=?xVya^m0MB@wAx1dpf99 zg*Jk2uA&ezJKlV9$VNs8PrqEBX83Ra!9V!)kAL$wKmBL<;7HN^ z6GpKAPdzC5o!|NNKmPN7o|r%UgCBkRyZ_ZMW*ba)#%k6o_Y>|KIhjNBrg@_GJki@7 zR#>FfSo_JD;*Ks1%z6CpT((4e(ZX8Cy{~v;8{7Xpe<~L*)XQwJC{)&rw&`>&*_V8uA!3okA&^%du}stHG8d}Ae8F7jy+UJF+3 zj1^D)T*4{L?E@poBI<-W-;|th91%~^xvB)|8k6s8X3!+Yvp+D3SaIhK!%Xwko@1C8 z{P?ndI$=MG1AP|<{+0bEwO-}9#7KNj!Ug#C9Hf!TQz|f6aO%4^&UMe+ zJU5H4!$jEit8Jkpw9cRUhdx#uE&0G`euvRQI{wkt>WYj%^=F)ztX^4Pr8#k66b1)o z>JXn+ZDT;nBOMgnKXVupUi~N-aBoxdU}J?BY+p{#L`!ns(lwvjK%*k7m;A)A$~$rD z6T^Z5haaoB7UERb@orIv@a=6bIx{xgj+03}FL^9*BT&98s<0K80fTie_c*??w~APe zp>j6O*Y;0ztmg4^qSZIUkv`JsJLkkW^6`3B3m+eSsm!+eV6b->`Ls-oJ-udMz{5UD zUwLP4Yx#ju2IPnrZ+PI56DvPD0jg4Ckt`ri8_d5g##-Lep&ps;n7by!MWRO{8(wi*RjX>Q%+xrgaFePKKAu&HP32U zD$QkGN5`bCm;A)AA}KcCkjba(w-#{JXK(xIPutjTIgVPR@ifw{kIp~vP&)beua!0u z%-{K?U;gxe{Of;B_mdtReK#H)=@Uou6GnPa^jqUWkqH0!-~QX5{d|=e3r7qqA3W^DsXdohm>HWKwHZtM@6o$ASW##m&k<0Jd7n7Of;G-r8L-ZQ z^V(&M8F`MIf@?-xEg|$vc|+Y{epp|6{T&_{1<`d_m$hqgPdRwG#kdtGi?MqTv}1{$ z1*gN9ul^!_B@J>WtBy93nwk*1yk6ka=7x;PrLLb4sKol8rd)JGm{Qg&8EE;X*ZPFf zZ^tK$ieVbSFSKZ3$&Srrt5cKnBA0X?wTgGfpX9*J_$Xz%|I*rbH7WU**3}0+hwVNT z@?pt?-O;amXn^_>4Y+sqZGC{>Kmi;-JUB&~`tDK)15WWfjQC*G)^dlNH&S<6Q)A;3 zC?{QKy5^ekr6cF+z`VD&{Cnto-Xr(amq32$bv-arKYwj3Y6Zs)Cw(l#k)9LNV%<;h+@ymA@twQ$tW?6!ecfhmR$Gao*s7FIqudL6o zr3b$;tuS+KuqhLvyT!;d9Dc51YmywaH{`_P%ai6f3RuE(!FswZZPaqlsm ziz(;6v=+0b_^j+6-@LACM|O|8+v8Sibjy)@hP)^8R8s%BdZVvvt1fLCRR(&vW%W|H zD5lXK$V$*F`=CyiC=BgEX98;r_Zk@5o7lQai3uC`$ z2#o795*=dls&f3cUQdj0U{>;S%#jOs#s@RQ_p6z0R&kG(cQQ_tV#K9J?P=p2`LF)y zk3RiUeadKZ|KWf04?g{~|H+>s`iJol>7UgniYDfNjz605cYpC0m*!>s5h35?Q4dzJ zI~<%>F)-Geiqd?|?0R40$bo$aEp4i4e7DdNX5u)e6{mjUa84EC*fS0f`xNhrjXd+8 z`ln9v2M9iSKeA{3^(41$sQc)ozTG-?|AqyhF1prf-Y7L46Q|hS`yVjm+AFC2cnH(P z!Z&YU|M2&I_tQ`7ucYaLkqCAFetG^fXoB&QHu_jWFS&6jsHMQ`{ifn6=Dl%yU?hAO z6v+7Dxz8+^9H~P`o@H&*&9X*@8Rdf+n27pzuYgPZl! z7vM9Jpa=b=eiVm?{aie}C;N5vPX5euj&;VK+U$Uy<5r;cjpwUE(0{7yOa0XQs1JYy zk3axzPwIU0D;a)Q797%8l!((mEBamwa>KJ+3hMuqYj&5Z{rcbFnU%(qj=$3!3i5 zx`cSpxct>X`j!C=tT^akVK=7vU^%lW=Ir2Omoe ziyu36b|q<@DA2ZypR7WY2Rj+4VNVQxaZ)<~vK$9JJ5$x|$?YnhM{wr8DqcN%j*hoh zx9+NffzFBUx?zKa`E0E<6r;)ZVgA@os0{S6rHSnM_ugKu7-xLunTo<08{Q;7F;VDT z=qNH-YBSzu-EaS?G27^2(U}S^prMmq|m2~{`c`IqXBeEGh zcaf@e#ncV$-L5}F=t!anqJM!)A`G+cy{&%H@qh>598HI4?b_ZV$x;A4-N(d_uHWE=N!+l+*sjzM?#v0E zI952t<^v<1RqU8uVa0}SJL7^w-)rksIN9JVOSL7g%2J!^ux28XjuuwVKJJWL zXcAQ7-~SO`di|^Av03=d>)djkajqY(tu=(3=fER<0<9I13T_%1oC?;y>e!=Wh8Pcv z#L#x!YhGDL4Lzox*2PNa4(4eNu#C{YP z&zNHd!Louhta!#b*_{vWkte+uN zZ#^(_CTc?omk}^X8_-PV5FOK z&e7s#LOIk9jrqVxDfCe>9n30Puk(RXxN-np`l+XbIKTWb>dE6c zZaO32hemrw-xzfr9Cz$9pqW=Hnh%VWd*J{acB`*q_VooP-8%2OR!|G?ID$7o$L&7HL0nN{Sow%!<VFg$W?xD{ zCYxHt{*pU})zHL>U#w+`rL}}$E1sQNfpr`uW3~UW{jZ~$DsID`npf2Bxx=D^s zI*;S0n$Y}c`|p+X{J0R`IMgEdKSS!E)##bS%?j{mMeJJf33TjpI*eGA)GQV&x})0uvmpd_%-> z+{cgbIx5a<_Ton4qxs7eMR=+UJ>vAS)KCAv{H0&ozRL*zo#oR-Xmq_bPt0EcG}f~Q zcz4>be0MtbxdP()k2dYI=kY_F{8(}lr+?ZLulOBDxU49GvRr?_a#lR`h_`AZy4BH- z>TA6ndDSv<;bZ${jRBBx;M}jR9LbN|aSY|*t5y7fKdUFd0^!E~_PsVfY-FS$W(9zun zs2KmzEDbF`6;5h!-1+^;CboUde?FtB{Bb_5En-uKoeu_oX>);;R$Xqg*pL?Xp`q97|9 zi~hTINGAmy{Mrg^+1y6jh9wSW<{~zIU(!bV)LXk*20GfU`)ueDJ@FsK;k|36XU$lb zxN@@nUg;Fv5GEf&%>UhTdZpo))p%p?6FX@wOa=#?6b@u&N zW0n>4b!Iwn?OI$IZ#Y|AVf0aonFT!0O7?A)FSdtL0K5pR7%|1{yYG2}f8dY;7 z>*TL_(IQ^9qA@kOKq$_Vv;+4C@d=|pt1rD?T0xjv@MWjo^eduR#_Jf!ic^TncgF0^ zdDoaF;p>KK>1yus+gA!MAf>B}oGKm1t)j5|={Lbg+t_80iyNTg!=I`#C-6$QwXq$8*n3$y}j}B{KPU8uEd` zrYAdb4^qcVbI%Nl>`UwHe)T?Pn}dezo3GXY!SQ=2d%Y&<2A&{bwEOgSp0{stBsK zr}O8PW6!hZ6T^P$289lmI2Ql*hV#wiha9$6^3=z8)%D3sjf9Yk+_Lz{Un~!D_UQ%~HJ*?2=H?xjE)o?s8a#pR)Mgu>Y=|5R` z;OU?EcjF)7rtV!X=I>wpZagshxu3017-|0fI`vPDGu|81`{;Z>AntsLtMix6O?e)x z^QYeXd`tQD@9u$-i1`LzglJ1v1?OIq{a%qUIpJO#JWm$E|FPa|QCQsY_))L_2h7ST z#v9|W^T4R9`n=Xj*AI+3(yft4A|o(0g&Ltbyd^N>%lT%5T(+?;M{4E)^IGVMFIFBuVTGAB z&bDxhTc^~*Y?wFdkj~Rmsu|vNp9r7npZa&ycXFmi-E63x9G!oTCM+HZ$8+qJNd=*Sh5o<-6(JYS=CHdw>Q4rwGS;s6v6-I z;v2Jx>(*NPHdOnI$HWM+8C;#QoenAVW%UvT8up-M7hcE86DDsyboHSq4k}2?p1esyKmO&8aqx5 z+mVl3uGF~Mmgi3I^fjkiVR|M4odmPsVrO-&GriX~+!McY-sIOz3okBw?5RUW;`Cox zG_b;&zp7e!(RtqbMocX@rv)=>oo(>o=*xm5#sizg==jlotDRpzJ_sDj_1(&2O+E1w z^7{A@hWujFPH(;ckxXvC^8&OoWTMNewH$Iio$o>@2e_v={!<_S-+ALxM*qe5?jrH5 zHO8;=2lJofc2zT{Ja@H%Ib-;-)Y~{mblKzlr3`-C*(i9^zK%}h-ac!YQEBFk0JR`SXJZMkb}Gb{LWLFAG-MeG{#Ms*vPmT4j%3ex%y0 zr7;I?f`oH<_kqztTa$;*yY?NOcQEAJvC%6>&f3^ma%&@L!Mf))I^%p7HY^Uma-Q>l$8TPx6nrI=QUI)`M@1 za#kzyo6#CwL4m%|IEQnotvqP&RlZKJY7H1|Yni8Zk7!rkd+@1DL|5kdGkS7b*SXj# zV5p}nV1yvaX+D{guTJInle+zeXJ+g?cx|a;_s*O7#pR_Ix&_et?J-lyZ|F1xaYp!#%OY zN2VIj{ni0LHIOEzKJ61no~eGt9Eu@k9`n}^Y(SDR&+jQ{Rq6nDm!H_xX5w_`rzB$I z=~tq(#nDxk=lYX+iOUP$i1wg0_b)~9@67ggs});y_uco-imZ6*GJmt8i9toSv4Q4S zIt7jw+aLU0P2#Wk2glD{4O<2E{wFh64Qiu5`Lh<{_$F?T?Y*b*O+K11|9|%01nj!3 zs_#5i$W(}cF<@6?P*G`>)~M|~;?R;%1&Ero#-Y`;!=rZ81_w~83ViT!YGX95ppuxj z9ow=H=!gS2;D7_-kQfJyBPtPrFo{rAz5Z*h-(G9)d+z(bUsb8~#e2T*p0)N`|Mg#M z?{m&R=kE8u_eFNhMLuQpoczmZhkU{4eEzHSrm2yWki2@iDM!MwPE=##&9R%S>>Pqr zyjp?vsWUSvHDglwI{L%Oz(k?+tS zmcs3yf8{^q@I;ww<(DsPovIV2~m9$tpnQIe=er2q0eDwr2$5?~s zd>sfy5QU%jC-J@`7L7c{_{XoL*=X*3!U(dY%!2$xF?LQECGxFjHUmv*9_kZ@#l;mX z#lsHbgKhju+LQ82uLmyuh{_X|JzU4^`4!nQYF`9*470}<4{XV7fonc_v~j>xz>3++ zcM*(uIZ-%%vfWYJT7!7^YP~)A$g#(ZPyEt_*~8(NJm&8h>0`_Dzh8(K$+O0D-eW(j z4lr>d;_q^tzuEtlZ*ZAI zlij-Qm>dGnd45KVd2i)s!H6%tmLG1U=di|_OMbp&PRBt0z{GxDGzvD=;kaljoo}P> zX%B`P6aR@DV3TA0#?EhrJnNaZVD#G8Y>#=AzVy0!OBxIc6{;MiAw>`2CixWJzm#TW$(v1*}r2I=IP zBiB~N_XzP)SLcmfWE6_6MhP!Jxt9kHBp77attA7-e8WoyBpkGX)0XU*L;s;g6&_l# zME$gSo;{u1$MJg}qN?^|$8RrZ*pgR5FHt&eaR+XHU-`^$@pb#ykH?x>BS_R`NRB`@ zvryw0t#)@dKW%WmPHNx`149fFoZ4WJvf-D+t~{X!yL{jjqYdoT!sy^NqD3Vx#(6&f z%#RAq{Cbc`w(e$(aS6&ykwV!|3& z_KZG!-%HF%pIljKr#y!l0U5itDFK7>U%tri@ zANZOd_*wpr@d?EZX|?mO`^B@DqhsV>G31wAvB!L}AMg;|9GC{)3^~0SK?Al#ZY6LZ z($J-z7qTuGf#*39CMIrDGa(P0oxkw~bnTX6VCKbvGeU8|1Hba>g}q~_38@%=Q;G{pk2BX;eWB<6Hb_^YIn8HuvT=DGud)h zX%4ccTL!qsCkj~bbU2(#i#%&gf?bS6-MHv2-c&_!4A$zFKyPx))rf4BvoY_59@r`} zT%FswvOalr{_)UT^IllsQT^0_^w^-E##4Uf>o`3CcYOCHg=eV8vdS0!IYKyR%mNu- zP(8mO@+Wkvt8$wrQV0w@&!H7h3{cvVA=a2szvT`i0`6Xk0)k28c|Rb zn_!NA#Zq7}9QHo!Gb&oMZH_Pg8Xs!hv&a`QJ+dOY=6zK10Ue6NdaKvibzwSAf$}qc z!)F(Y>$Zz)JkD3R)EZckEG;+X4&0CY*mJj!ymtRDrp3M-A8`> zHBVtFbnwR>YrlSaexgH~C!Kxnkx%i(HPE4qeZ{+p0BcnA0PiRf?IDba4crIPlb>Oj(qz^6j;%y6~?1_$z7p4x?+`rT4X- z+miH3sNI)wRDf_^NLAxzJ3+<=1I)nU?Lg77EF?JcEHfXtX~9U?CC)8wQ&2iRc$oko z%zQ&ArNSr>haV_4l9Lni23s(CO)eOT^EfX|sw+q|Xt7`rgO=0-$mw>mlXZl-vU17T zYdgYw?e^4hR-xzg$Z02ZNzbG zgG>J63-p@zC}?HQf_hX@vfMp8b!&!huPyAu_$|i_|B7x%_Pt?6E0_w9Sn6L zNrNA$Yg*?8*AC~O{KUHS0K<8s&Lr{51Xk^%TPHP;Q!d~O@vO1S@%KqHdEsO38tHHh z!@e@tNGpQ1-&-&eikDc0>#y*|yy!PCuRkVq|M=$^O^dSs%#CF4oYKi9+4D$e>#;HL zVTssVeZlArS}=N){YqM-wa_}w^$C;6p*{R?OJYOnP0wEa@e>QGt8)P^;=KYhy4{TB z4+OYkTD1&b?pG^{HvF{VM`Bkcy;ID)*_?=dq%8Ce6+=cZPf!)NY~o0?$qGhA4eG_1>V;)Ox5K)pPzNiJTLT)6P7bL z2#+loDU7;9n#GNyl@*a7-2Mz_9=_`?&=u43%c}7YZ_pK6(mTd>%k{$_i;3sw&X9Fv zI3A?T%E2qNe6!=mtUv=hg610G=u-NCo1QE(4}15!!*=6nP|&S*q&>IInHCT{mA}uV z!7;PWEwWUeI*+0*nAf;of3P%5svfi4PgWi(pr4$p*oE&gWRXM%({YLqOrYH72HWXe z(eEyrF~+b*Mg`q6S0Ls?i#}!a6F>D+;mkdM84Y|?Qgvewi8nISn$TwSU%j-_`};&4 zqfxK*ZRkeaaXp`OZ3lVE5C9`<>!lVLNqD%*CNFSS_ z?LYOgs4(2;DD>aC4GT{Xqb_WF7<|aVo56mdJunY;0aHuXbI8|P$TPxhjD}7Qlf+4O zf7d%k5&mY}jkeR+D>fzA9`As*1Jg{do)rU@e$gH>^Q#^Fu~)D6czX@|elDHxYQKjY z`McueWU#0fh_%C|H^^|U*cBsx$H>28MAmVB{q)I301t{tCPS_-k8OVVkqY12besZX z`vmt|6>}a(=N+-sj^w=KJS3>+zl3Du(a}ezACmejY5ESM^;gokagknFUgCkz{^-D9 z)^&8H1?o))*ALfIpGU(W+{y!o{P>WU5)V(sre??MRy}O^ zW~iNqUS)Q4_Cv=Oz6=~=PP@xv<0s6@Q}q?Y9tO@pS(N=3&L1wMc|+zn{_rv%tUbUl zNj@ZV^vG%bhleWR>pb}OWg=bV_>SDhmdW82$I4B8oga*B_>nWtkOF&v<@aN2W2KH| za>Ygd#EKxvdR;QFo71Lo^v;a*3+5izV`QgpuMKAuHN0Rtk96RxeQlqGgBwX~#I(J} z&M%>`U7q9^JRz{oOjJ+<~jn{kshDLcKulWoZtAz zVw4BR^`ilGezOH<DlH;7#aUb)i>0U$Dp{BDKtNnMLK{gaa#dc_+muKIqh9zmEVs2~z)wlhg}HmxUQ; zaJ&c2e#CQ|MdbWdERhH)Xig#dan`X>2l3JswzdQ5MI$M@lCjQaOG6 zT&fk@nBy>{gs;bI-sbpv0(VY`X)N#{MGY?!9};_>CSFbTHH&LJ`~XzX+dRMN*!hBd zJGbHoo5n7j{#nAlSgHI`10x!_sAsff(96K0=S&0_9D&yU#%JhDukW=z@>=_Z5g<7{ zRwgsbu7TC~^3^|D@UjG6qjGtGWz%;-DV)AZCRfr*1gB*BJcT9VeaaQL+>)6D@=dh1 zHKCgcH?1k5!9=Jn;X~3NA>zgkBJk_Mgy)ifC2gFE!8`UEw9$}aE&D&JF#pge+)l$o zlPw4QSZty$#O-5B0BIp!^7swiF$PFX4!TE%Pk$4ZPaq_0-0B~l|^wYV!4h-pW79SYfH}K#DoU%a{Pc$RW_Wce4 zQo(utVNySX)Ao&={LM5%q~LG8Pk;2E1tWdwb>9@{v6+{7Q+dQ?W|zKTlYLOhXDt|s zpL*7Mu_N&hi6iri$m{UEeTR`P7`@@u_8mr#v;`x+V9d)%xOP2AvJ&cArWHOGz?!A1t(@a7ku-6?gQ1^sh7vTqsN;m?}H)@pGnr+QjZIL&XmzU5TbAzGuAu|L&WvI zbc=e6Z&WYCc>K33-HP-1%dk~vjgvCLLDp5`kKEyvjy){dlCXgp&qm5Go4@Jdn#jr_ z7{o&f%4J_UBakUV_VXi)EO?CLg84glgs|_PssStX$}Anb+Fb|NkD>ck9_#{lldv5_;k?^833lj?rD3dUOR7eN35UN78U<9j}@-|=_SHF zQZsAIvn3k2S3ZSm=b?kU!-CXt-E2YOi4h9_=z(#X#ZRvA#B-iSj0-fPCrWEz?fJTn zUvXVmdt|qO8o!ob2RDOM{lHr+WpTWFPH`V@fmm^CO%=E2v1k$KCAULnT<6ClS+$21 z3Um1emiWEv9E<|M!R+^)G)na6H^q%uca~qx$zMqG^sSRyOa^gQ2?g4G*={mv4b8y@_9}LGB=}dI~ zW2 zce*h+$MIh>_t9A6hcD}QGR}>`T*?|R&)_JSYXhtEPYnUn$Yjr0Dd0Fy#tX#e;2w=* zq!GOJMZa z6I&p`V@>Da;k@D?Po9(# zd;I|qf7F$&JpbYt={z+O52#TWve)V9NaT%(AI(fGByj(*Ei|zCyg5ZK<&EtOCG1lcT&GWS$;ek8+Vb#9u z(##^GyQ?pX>DNyLRiymduy}ZwiZ1h{|HU+Y z(!n;qy9hKA?j;&`lY!9rKgNh_$54kqtRi$~W?sUp;b!jBdEDQS2WF1vh!B71HH(aT zsdL^z?Goue^yrB{`zyI)CF=fZffw46(XmOh#0A3l8pJUW)??3I`Bg#jF(UJEFaMGz zBk}iIutDB^^qoNfTa{cazwmL6*obGBdL%iEe;k>qBYQXjh(oJ3m4^pAJjg;#=_clu zWWakLdL86}O}~1QG4wPlp%eMaiHd;&p(>kW!y8y5km%YEb-Q zT5_p5>fZ6`fU6&lizBdyx=l78pByU)REx&F7X_r(DI0G*J2rm$Hpj!Qn9hM6Ngmq1 z8aaAzu}lBS4=3>}4nF*}iK8vKVq{x!VEnjPqx(a*eHy`vM_b2+n52ZGSu_M7tb`JNa!N*X57O&>I2@Z$i&&zQ* z+eJ`+Bf7^yUd~_IE_7;qyAGY^9UR0#>LND@4jNrvGQcgW%_=1tufc@Fe)UTwy*1z<6+zXNd}RYOF^!Wg_>0nib9UJeDUm_d>X{<4Q9 zj<=4hJXBXGX>)9}1`_8IKm17rtn+mYI_I1nhJ(zEPOr7|Mnqb6;?+s=sIR3fGB)^- zV336iL?6Fkv>s%z`Z9R@8Zs9Bce0fc=6SqFe7UE(@s~R+zRzZvD zw+uO77~AMmR%D|0KR7tg)q)Ln5FwLgI9D_y%k`uDy-$buLA9HC%?J7z4}H^5&^4|x zFRW--s<+g==8MR~_(Mw_BRIf>v<&8u&ucWr$7VVIVrDHaRZ6^9?Hp9=7JI`R2#&RK z^o=^O;YZH?u=eXGj-c7)a*>!37)O}=$`c4Oh6IjJ5?zS##(lBNH;_K@N0g6ezS$8% z{HkQlBOlwIj~c>fEg5{WFpC!@Pojn|7{@&?iFy0^UM2XmwjlHQ4d-%hB44&KmGFp!&H?|gvDP?%HkFhB?W6t@v&k@%6 zV4UeK@fCIu)g)fbD@TKSV*E%M9pmSX2;?!#Ip>e!a*#vK!aA=z&e?VRT5E*HoZ&dD zO_=cvsFoXqjA;0xg*BM8$6pq-@Eo4MZO+Ou8u{ZSOM>77rVo`oRe>)bQ zZW|n?;P1AM#g8msSh)Yi(AE_MjyUm4XJJfZ(tU9_B8pgQ5{{Q_^6xDe1;+NS*s;&? zSB%eJHvU?_QM~r#K{_sDbX?n!l*@uFhWMBe!)|+;SKRW({7jRv>hvRVBo#6`;ED%E z`Y0Y>$f+=uqoTY(-SDFr@l9I>VP3z(Xu`GzcuYOFY~fe_Krj&CS9#AXAF!HE?b8UH zM&L3Vf!E&hXH&)NyfB@T;q!(QjN-JxFNqJ?{e=u~ z$8;QgqXIwBR<#B6+7(||F|6C_T{Bj_M6MCodt_i@bim^;iFT|;izK)u!L;@>AmNQi z+zKZX5;~3_&kytX;SHY3%b4^EVC+>K>awgg1Zrv&;5y?*na7ke|h`u7r%J><3IYN z?Uk>5<@oi7!k@I$2%JXX*a+yo^?mMhpY4GUeBkzW4}S3W9`E+<+rvKOgSYp7ulE`u z%qYwOIM!>EYgQMi=BSuO%<2xj5#_CgK=X{ii`i_%%syFRjY#cnWB$_0pgp|7+p!LF zaP9E-{LkUGM zQJm;Vz@5>*JW5BM3W9zFVP?VTN%^JMz>vxT7MmowyNx=+Unfe(pr00u7>ScSjlktQ z0++U6l>J<$rFAZed1M|FVf69^qr2CaUTflI-qAfnle%MV?(-R_`6otU-LTA413%Au z%Hh0CDqMDL$lz2RVONk;=4a``Jo^Wq8DDz+X#15k+kksxu`Cr9q{U6Yu3`3gLlxEZ z{Ia;!jNy$vb&5P{jEXuYc<>V+GHz;G9E(kHc?7}`q@$nNbYE8BdStCnLct^FA+C=b z6*!G!`?wJs{DGM{$sE2oi;8Q;hF$yPe1XBw@uv=CjM5=r`O}vVMBXxI@dmQvrN_dL zDemar;9D(>9=!7R@#}stV&YR8!8#u!**dl~eRnvUf4FyzqP2=z6_>>)vw+UQH?v5de#N zJ8qEh1u|1HK2+){A3*yNMgXQn&Ha03=VnhcAl94lZo5fmusvi*n=EsvGCn0WMgIHM zu;;nCtY(R!-4e#Em-i%;0>U5DAGcPO16Pm2bQVQAg5*j$R^2qZn{>#s9q6 zv?ZuhJZKCpvvd(O2fw+3|Mi+x<8*xziio1}-Iixz724e`s+TT0Gz$le#&_sSlQWLWfe3208MP z-KykT;#aWFKMIOy-xF#9i$n0hVi$WrAY6XQ-1DM%AD+Jf51YJ>?G;+owffgv5F%Y708lfX5&7IMu{II~CWwa&VH9LN?>tk4c!DN$Mwbu5&MYe*^&{+K5%TK9f96 zf)g3e%dtqfY&qYI?wzzt`->Sxe_Kf>f|o=;A%zG8c8*bGJ^%ZW5ciaYETKM#y~2PX zc$GQ>a}IZcPbkBqAhjfQ4}CKWmd@c+6H^3k0{natuky6uoj&K~W&`YF@b{oj*?*ReH z4Pl;NCJ~7$$KHQtMOiHQnu+ousO&PS+s%&31rzX1J}cYZXj5-Z_D9b5Ejrh*uu{t- z?_m^<;4Q5?U{ z*L>;fK~-en9!t!hzdHWh{LF&g8BHoWWfDByX<8byv-bE~OAu^U3jb@<$>H&13V2$; z+o{NvK%%9p588u#tE&U3-u|YO&N!}|+!cY(bf+*2Ns0tfC2%m6Mi3pHj>-uf*!WwR!?e{`J?*U71l2y6zT;pPmE-9@Je1g@ZyQ(MR7 z4}F%do$ruaL5t|8u%JxSHn+GUzk<`DDbcF1M^&~ppE7L`X5OY5q*m6o`Z_8-=NYtR*0Sx+0b1|Lh3=PTQAf<#uk0#H>f}{Y;5}-vD1#+*x$9i6UUgUG@+kd zQ*aX48yB3u5S!GF&IWXa51xNVFv<*di-+B&Z(;gm;84NAa0b+uT z!W-UXzlM`s>6B0jZ|Ob;u<06MJUksANrd!t*sbk+)ygV2;~sbXUVQHloq=*UPsG|Aly1n%YHuodm+@8tk4)=rB(4485iV6gi<#i4ILBK%d_%ec^4@3T zF~N&XzxdbiUsw;>eQWZ#+~cnzgG}DHfQH~@eGdn0*z?^E0e$Rfo|hp#v=_{UoFfBY zoFdAD+UY|^&>jN5p-nZ)XFIIt+Xt37Q$H~_A6IXQ;QQ>81TMjDx6#$C^?mfPF03q? z$Np|hH+Tariz+qMF7_+1wGM9UX?|2CHQ7KbOkAY`^kmrr)>$F$XmEp=3tLXg7*-^s zXC237upm|p)<$7qv@<8vfx@@Q^_+){LItagJy}6U6LR)>=kYP@sKVhRVM$_x*XSIh zuF4_dUBE>&8R?2!m!z@S@<^(|-~287nR3-#j9d=SC^&#Q;!vttTS2V z$32gl*-p5mlO+o}l3#V%Ma$`sMWX8d=*paM&UakeQ;||6`$eA(*gTQl`O>51x^$I- z&*lzft3Iy(lCD{Q|E}5W@krhJ_W3n!^XlihOkNEDn@)Fm`{fKt(oLqQ#&cFOxt3dlV zf;~^K`v3G90bi1lO%P+2i}ynb;t`g=gqKvNo@Ii~(zP5}JGUjFo1P)P6~mZ;9I5Dv zFD6R*lrV;%P>o=`=c1*9UCRcK?}b#}I%uP3n^_6M+$6wD{r~*olTI^p#xL*?mr0 zR6db&r$G8Kq(`MQ`M&R!h9|}_)aLAd-V`D*;N(O0a-?M8LEt?ohn@xM?Ab%yL)q%U8pEHE28Ur=^(j_N%!u>v0HVL{gAHEPEg-p$aC6#n>T0c0G{_xH8s}}S0 z({s%dgsa|M9UC(yzn*)^3`fnoXj!%^n@W~OQV53`uA;W@bM}`^+$z_5TQ1XC_vov7 z6}vQkv^jYAdxSh}Q!|G1brdneYFaf)ptSS1AlU$It^IrOijVtTJHtW4Q^QB4UPO}U z{9Un~EQjuWOs~lCCB|$kn^;ThBaJ__&oSP>pM3j>?QJgxWZnxE8Yu z5{T6h$gXKU;Z3LQwUyU8Y*TZ=v44_OM~i0Y|~{;`>H9*w!}itlG{~s;|N$(2IbNlx%8zz<=Oly zZwn2v;oEZ?q!F#`JfP@X2^sV`f=;D0hSCPLVcwYBW7jY4s41l5#uM_c1v zmjf>8{BAld>2AK4@n$P0I8%!lS3ES!GVU+$fO=s$LF3BEQE?SXLHvPdgrpFEGxRDs zK#wEY(z&rBLmDALAp5W&yvFS8vhCjE#h9>SK`~M8;m28J1n_xB=De?t$QAbOc}57~^|Q-w+Kw zVM+eA%qcSe4=mmpRqvX*nJaLvE#aI5^HyuN;3MzA4rNTW-=Aj-zC#ImW)4ykrRtj! zF72UMSCz8Bg=qe_-8K;sJ(X1ZGOkAuy0X=F_v0rq+W=$ak~NRO_3W;AoPp%wF=G17 zo?~=P7$*!z25u0Cf63x2KRu>NQ2ZYIY}LYDV9JE6rp$N0YZ*orCp(#vld&50?i($H?O*dO!2J&EC$lsqx(5iCQ20h*>>}i!C<*8*k(V=hY&nE9 zf}ei&%Dk2F9k~FuWXm94NZR}a47PRM&fTF<)3OJ`vYkP&>vi?n)_GZL&;M&I7N{;q zU2ZoIiV-(UepKoqvmXvpAUP6)Ok2iVd)Rv>ngZBl|44A~c7cv5FxK`TR2eTP(!}m> zc~s4eP+Y^g&Nrow(s@5lR^P*()O4Rm`LCtV+1ghM;|X zjAS|6!jf*^7^smpb;GCbVh?UtJUCgZ3go|?(m|K)Mm&rq(zOfx<+FAMNK)lX*#UJ5 z80@dU1%Rxr!y@`e{mdo&7kOqC*(mPgUsN;3BGa#r=^QR4qaA46KOz zA=)*47G!i6<+IHQr*1|+{-q?+ulXb_g4e-Ro*tml{Xlo6B4QrK{}x6t=M$VWcUQ6G^Uparoo&DKC@+a(++Jjdh_?@g7 zHbuMt`_S>8?-zUTGS=?4(8px_T(Vr=FNRaPLOEGLHp0T&S0fLbrf_r>QCp}{gWVEc zgL`bJ(ii;Fprk$9!4~A4V0>!^QV_IBY;(u9H8N3MYF0gW))Bt7tvP;E|oKzQJ0f zALIqhJcpj~e4K?Wa+&9P+s=^#YM*k0soF{zKL`$d3OmK83GQ6a5W{_kXNHGYXLE1Z zh~N6whZwmAt6{mUxF;j;Jw znPE(s1G2GUP+d)Z%ApD===fGRmrwk6_H@XY%I014rPpScQ(f-wyfmICrIzIfTT<{{ zkgB#|f5hU3iOD_s!H}3v&3(OP*&_zF1oe2o%n7H0MzEwI zzh}3yx628~@70-qSLDCmmM9eL0?|{gTOZ=QB@lc}2%H}%A<8HesOqlM=IgQqK?vy1JLo`~Tk_noy;PP@(baaTzX@e;n2UQftY zE1=r6_Z@%I0`_hv4|;5;!$!0(aH=Dj4&-03asJk&=s)V-F4edFeGZQ^BcVrHqb0y4 ziY1@4newr)q(s89`*Lk~Oyak18N^4gRK$&l?;B2%2vYVLEauNSmy4U=&LdMGHB&Z; z{ZDraxVU58RRx?{XTR`1CuEgNozoQ)-^ScvmcJ+rRu(E`yWXB(yw7=(mI*Lg3LK@QuA57e=v zbdH@!%a`J&{oWfsAiu}Tb6quv8c-%={2i>5)2R7nm{IS^VBxaFVhuZ{BVt_G4W*MtXL}c2Ei`)sfsd5?9P%#7EYSP1K z($A<-I;iBBM67plf zn3I2o8?NlM;vD|54V6QWul^TW@L%xCXkEV2MV#J&f^0`t9%bvAI!hIEiJmMn*;ON_;pO>qC-;uJ}q5jf< zWLlqTde;-c>L`bK+wv||yDr$~!4f-Ved5v`Gy|&&$3^MSu1<~cPU1gD1DVI4Lf4eX z!^=UZ&6b&jD;q{)iTnrTiV<_I9o`%ZcR_Bz^-KX!eN#86G&-^UQeIhyyP1 zCM+dL(X8PnIa!SqQWwIjPKNvKb5HQ-peDAs5=O}}oj#3ec??@MB*)9JlakcRCnt&* zowwO-#zEOEL)@B31y}ba5_6U_C(l4y00ACl83T-Ny}*0mTC<3p>FxEE;O>!cWHM;% z8w!_sGkiEYos^XttSNZOHL|fqi@F^U4|~n{E@_=eNhZJTgW&r6m@^rcz~lwiM#ZO} zCazb0J?~qQ{AOn)k2~je?%v7|=AHl{TMDNmJr8C? zRS$l)_NcCJ5_6r!QIctJ%NGPv^?5Cjin;zshfIkLIE%Q_hA%E&v7^fW1D>W^*ZR~Y zZ$zt~q^Ee}ZkYv=ZFpf&N2-il(^3$s>1NU8W)fZCa#fJm(>U~*RRUn?vD$pp=Id%2bcSz8oaeLH&c{Cg8Dx+;Jb^29MIWbOv+GUfZ)W^x1vi zzc0l`79>!8C?2>{5;)wCa~Olot4EWR#)Al=b84-&D^gj>D4ESxI@OsF`6jNO^MX)` zh_x*aPCxE^}7n0GqkEDPrFAAHIq-77=Zki<;mrhsBES9D_Al7KXqFI%y*QfnbS z=k*3E?wMny#jUZsMd9J^g0sOZ??q3_u$NCQGs@VY{N1mlH7Tw>k(nwHzGJ0oR<_V9GYb-pIj%h`W`_!LV8z9b!Iua%5*$@=gRFGN2^ z4vl>s*)p8nJ#tD&p*lFxjiW;L6=Lk z=WJw?+Yuf1O`}lPBIca;jb2r#H=1L2ycG!UKdk3O>L)VXI8WgEG0#U9ZzQUcRhlA` z4tjXWE-UyS3S>_>c|WhY+0ehLwFv=arv;Jy+caKf@A_tRMtu2h_p*MHYKkD z0XLPJf~HYgao08kkc3N88y~?8q|K_=ZF8JoxD?kpp9eVKL=Eeo&fc)XFpVJm*Zj6~ z*nhLC7U4hWAjZcqSR>WlUS_!t@Dkw?J74lT#LFZ88p3=t1lKdkO@*b-I`TlgSEQRj z8^h!B?39|1oD#^-Utf!6v-#?TBq*~`%Ws@Qx){@87W(aG8$8*gy%RU<e`?Jt=@og53YE^HTZ*^Z(nV6)W)#a{g)x6eC;%g1tglcD?If z5_(b_!>QTA=Zl@7`86-W>_%2=1?BVSZ(CVOAuk(z@ZK-JT;Ksm;FA(is5#G8Yu7t# z?UtAORJ=w>lR`97dQMDS+^-_2ko*4~XASXf756zepN9Yne-6z^i-=lF=8c)@lHml~ z)hH*Duymz%`)|kM8ITTa(E~#dPEyF*=~goZA@~OmM?=?(?|Q$@hI6YxubV6yr~dsw z(m`b<5ohT?KjAFlxoez6O|$*QgG<2gFIj^Zs$z_nT>TPJ_@Sl+RHi?oGUGe^$sucR zIc=fKzg)d++?eAI>SeMl<%Ss|cszxDH0Kr)a6YQ19<^MJmin&Cfw}6j35gP65xdx5 z)h%m<^+O|G5>K}*I;gK>*|ypW@Dzr@wpoUNkV+^{jn6qXZ@&kU%UJ!*&g;zQH6lr8 zVs+ug90G0KNWsgyoI6hOLtQ@5k;{^i4!EYJ<^G*}RT}su{4wisZwsLEoT298NJ$XG z`&+`4Q1C}E?JIy=FcA|{e*zgc$RO6K(<2J=bs&YKXq49#hrkduOkdum8##FZv=Df5 zDl{Q(ktBM>)~4uHLCFGd$EPh0pqML?OJLyjNg~K}$26eoKcO*t_4LyjgBRC{lrv?m zcQ8KJkO3=;uj0J2W2g9zkz*id%TA5iCL+QwoHSnpGqZ6-+t7Nhh@)c<>^`x zRPi%l7#>NJM6d2_(%~>_QV50_vU&lMh|{F#J_^o-MQP%Y7m1r3dsm7ohS2iWm8Rb} zAo<(*b^5knS~qW=#3xqcbHQm0LT}xc+@-7$cp#rs)n6}&^Z}AH5OaMp^Q(xY*0ZYw z+$w7}aaylvH}(-*!qqo$KLO`oS5+4+QO6J7F@BZ0NqRT9|3Hptq(>V1ohh;^ zBk4Yed5h`+d6*+EU(=5i!fDcfNJk3|HW+g_&6hC~doP|BN*_>NVZ$nEyer6v;( zQG#~ALC@&$Zs@YCCI-|`C@~)sC4Y1KE*IT3oIxe>IIk@GKtI5F$Ku)RgX2%OJrZYb zw|&S-Kfm`z!|LYYJVWyCY*I>7eS&{D?h(5aM&dh`e%sSGH_L~)iq49iJX@7W_z@$~ z^X~R*;hOnu3bVL}C+vJM{g61{`$&$VzJ$l|& z);V&E;q_K1xRLSwaDl;#QQLJ;Q_DM<2iLtGvn;vSF z3I?*39@tl(_pzz#J4eQgzkHm`wW5l6O|lmW>cwxmg9o(7NbasJm~`(*K`j{6u5p0T zH_mc?hTk-@4O{o&_frIwtSPUX!zn?}O3d}5x}Pa^JG5_Sx0;ySJP0+JrZUz;%Y3b5YA?6=i5TJT#9UfaF? zDUSzrSJq?CY%VbG=6(4d4ZZ4WMXJyH1uv~qvM59mKA%~W5=e6t% zr(RD!sGOTFuJS0o^}?=>evV<8I&bOXL9y7>@==NKQ7Pd`^Uo#+y=?+#Hw_in3Qttf z!c1Z}?^kZ!<6KIAFmE5NL_LsQx=Qx7%?&!hSir!ZV?S%>wYa*YA46%l?7?+(j(o;I1UzQ=v#@x^hxE5qd@l(VV3 zu=?1>-G1C)8o_V>L1TZN2!{-SCw=eLu~LSE_g3bWm`R(xu7z+Z{49YTFTgV2t3GUT zP|}D&2STc4h{_p{FVyWGl8V1q@x-)Pi$l|pr#3&CAQI67o9J+yl4tKZH2J}F;*lp9 z(`k#dOSpn=&Dc5+vr~&P4|pqNeCmuJ7c!JbWByJxw5w3Z?YFYv+%Mum^F>nXElDNB z#j6j{UXObp)E$|Bpc}jS2|RVTPTs-pq*uK;&R6=LTp4ik~5%BjtRTU)Uw-aK?*1A19AD;_z zq`E5qFr2m7>Y{i-&^Q%VYC%DzhyNj{TP^CO?MbEUbm!_8;!phDv)+5o36$_k{pLG* z%46T*Lj(gUBX(9vv5%Z6?;fY!3+rTrlT68XO0Bf3nQ_bGz3aH(g;vBiM*Xh#KxXNf znbKD+>VJd>Lm+&_5fh>!sI4$~wg4q~^%cvKm+n;bokwQYWABj+?^`+ymN;X*pPk8d zXSNMcMf?YibX`cG@KGGbG_lH9TsuzZjS#$p8s323U{k#8HB#3b&GK1aAs_WrvF8}j z)-m`Ae){wNZt^}SHueL{qN#GUWe|*mxCRqO)t>xtw6@OdvA0jaabm>N3x*WCj|sH= z7Fy*%e{L(6b`pd|5hZtq*oi{1Yqc(7KWi;pSGMhEAtJdaCTkrBK;}?Q<6!R_w~f;u zAtg!!ltF3|Q3u|*A@tf8^-E}l>pkEsT2`O1j-XQTBp;us387x|Dea#_XnRPa^q|q% zznQrqhI3YqmYreoTl}QxdvwrAL~>L}DP&s=S@A-3CG-_7@GYi8=yGKC5hmV(@Ybh;XhNLoI6^BJW=({K$G^M)rvko# z=Ky|HKh@&d?tM}S&dG=7`S_&sc7FIGCsswz?a1PuCw6^J;_5jxBDumTs9kwegRN`-m-Iv3F1_$HR~x+tmfX zjM;Z{-XS{At__w>>U)+cO1Ppsst5ejR+JVvA~~Nz#H~;uMJWE4rNHB!v71@r<0#$u zMfT3_dfzS(^!ZjlfoAZ@2X*F(SM738jxYgq)sg`NWb`Ll?8IyGh=+POzUHl1UK59G5zu*#qy$Muk&0XF7{;{>SF;9(9^mrH-9iz9yyC`$* z#&$u^tRjlL=na2`1H{>JHxR55@BE%ikXW_0l%Y>Vk7h=nfqnt?qmLK7De^j+t^i>R&qFb@&*% zQ)Ka#PWdQw&ko)_pP)>9vs1r1TZ_8D)hrX<@i$*Z?j|gmB+PF>U5{J!0)`BN9t6a9 z^eTNyt|>*RgP^oP27VkVuCRW(YH0Z7GQW5uJPGm(S0=UC9#K8NYcz|1M-EEwe2N1E zuoj0|s%O(d(U1)*DV%Oo5Pyr0;WzyNx$QYFJ?z0Z+)ko{mW`#F%+(IK-#!a(_?*_J zD06fk7&_dy$XqbjSznIj7iwXr$PV`tSW(ak6Ncy_WA= zw>1TfpgM?5mEV`CL1tB)j^s#Xx4V?u2+ms49!MbkmjTuB4KgptC|v!>J(U=t zv48L-0U=WSk$0Z3YGC~pXJDob9-w%NgHV6+9l@&gm3jM|8FIuUDWmh$pQuriip+v`2B7-WSA)q(e z?;T=HJLE;T|DG+(L-QFVGOVcQb{h@k?rV z-_X`=IJwHXVYK_ddgLu+`4Dv-Y_h`#4>~CFbunq{o3FAc50?*H^LteGrxUvPwIryxil zcM3K@+Z1siqv`DU$e<;@O|XRx$h*_LecL~yF0i=G&LB4p*(HsmX;%yDB^nI149wKTDq!4W;sel1c}2 zp}p?;zM|S&gp+D=O2-N?yY#jwMfIz0((!2vgzo?HZI+v39}k@Fd$Ou8u^34Qvl=^I>zo)0GlU6hw7A zc{Qs2mo-HX#BEa?YR~8Z0gO^=ygWqU|G@G9rb`*qy3k@_k(Lc$r{BzD8a#h=$|jub z!=3pn9R)L`U{X&CmCyruo`#6^%r4IEymuH1ajDqD`m;TyP%3-PzAMby=RfM}5Qx*r zOD!$pD1#m1t&IJSPp9LSXS>SFrhYM~WI*Ll`0>|~S9zbdo6?T~0z?vFJJn{7E!}5x zAj|H6P_OoFSCbeM<{a@qQ)c~|RS%n)Ihg};?xYt12{p_fFQ$hmO=0wE0+1_ZA}SpxQ33JgK6Rs5Q%X zuGTl(ov2{MYgRv+*{1BzTIV2zjNgOL2HGK}*brM>uSzq{pCR|r1-g27^$5mG67JL4 z<&P?_-$CN=)~GW@^DE8H7z}|HH2lt#=?Lp{=0o--+S!%nAEThh_>`5`zbwJ|0>kwx zh6{gksOs&Sr_8y6i5-qB<1Ag)E{|<4CJkLG=Dn+LJ`HQMXltXY{&bqC0AeOX{At~y zGur_^ep#x_8kX+T;Ot)cxR(>zjHK**J)9Q5tJmZ>kJ-Nx=Xlm0rj`QZ3KHi2{M`Ll zcenh2({@kQu%Bf} z+2@Y8-MAF;eB(oU>XLHO2q)--HbE|z_Hb`>kv~0#6XB{A79lS^E7@ZxNnG-cXMdPw z&8ItUIR@Q4b$sT4Bsm8L?O*L=w$Kd`S9+QbE09YXPUj2W*^^8-xH5;c#Mrk->{l4; z5~lt1J2@)4tJ@?mULp?XMB{exlZ*d&5L`=vvyLDs zsLSS5iSA+UUeS9GaUZgvoN}6-79Ec(tJ)Rkwh%VsH|iH~U&mOiSwL~*vNt!>{sc76 zbpeZ&$sD6)KqS-U0IBPih@o7YOR=FYgOrFZx}W4`$X*oOS!7NRDAWYAPGCjM0A_ z3Q*T)wB!}^r5#tMsQWMBh>{=?wJY2LgHuU3w8v!Iz1;SQP$!kh@FY8po(UR?k%Gua z_J@sl;RN4uz@I_xDWa5*QYL$&Cj->qty>)R?#{KTFkqRavYeL)VFX{qLy>?D( z@cfVMy{Z;lsHxqk9h)&9wWW*1GyMy{A%8Q4J;0(NFt*pM>F9%=()Bs)XR}p8jrb6S8J{6ew}9mHBdqqnA)T9s;{sbppBhwT6FO;X_EDFq z6b&q-AL&fwAN+wU=+D-7GnQeW$+K~yZ{y7Rnic-fNGU$Jk6lwB9hT232R)_*k3w+t zT$w9tvhR51-hHpBMzeL=m*1C;H;E}?zj9&D#c|}n>1G&}H-Ya6#r_Q13#H^e=>_v3 zqMXhH@i830Hv0T=TgRO6u=hUaxp;6Td$$QFB;re5ZOi?=~!Drtl`mW8z@wOZIFmJ?y9+Q;i*T{zp4&49vB@O zPnXG*JDhcp{>^?H5s^hA3)`jf{+8HaA0H-c6*r~sQ!|4{qj>hFQtUhrsaohF@6XIg z!;>2R$%C2#)W`y`X(#z8^oWujI2+*J{c1T6A)IZD?%P<{h{%h=9^BS0rZQ{ajz6G*80{%bb_IqT-E^#^B(QSNRX7aON=pQs%||1$j|s)BdoelixW8S|>{0MYoDeUln>y4)gb(LD5R<84|D38oSvKv#S<-vBP-{DrYI3KI8;vuNF1IdDk%NeQ?y&KRUhMMs zPmWg+U2vn(c5W79W!GPkb(Ucj=y6nqY;gN;cCBX;PtTJXWm7?$EA$!qYlzQ(;HM0c zF71|7n`BSoOb>%_Sg}Pl%t)#xAATj^sep3pRW+_+b?6>WH#nW-WEk7Mff{@Gl^j@{ z4eZ^KPBch;VE@Cq@_beAe)sFrCnP&8->2RlUGRP-=FxO(eueQNNn<9{mmwxsjDN)) zg%)4%I@HDj8qm(@!wqr;8IqHaWgSHL1-(WXr4d23NvI{A_n@4~7zslx!6seamxi~O z*mevG7fX1ex>wUUbnaljH{JGS26D679(~sE1>&#t0k77JW#@#R2%c!o_YOTRNo`o6 zM98q31gu~3c0(Hslb54S8BAiM2~?c7p0s4>cMwzS>l_;1+TdOEYGF|RxFJUjSt;)S zbL(D(C6(QtZl%+ohdS1P9s#GE<6p3O z*)_@QXXr8SpH!E}w#{};hac_+(SPo)B$0Q2SuZmW!ZGCO!T&!Cpv*7z;9G{vqRI82 zDHq+vjsAl_0ZVJIjh1f6hv!;%Nn0HPhAD%Q2fx8r82VUD6Uo&V%F90ox+X_bb5jeK zGjqhI*JVxeu7_bGlIHuB12l!he4xAaE(VY9-9o5&DpJT7v9DkJE;iLyzT~9Nx^`tg zS9Q|ZyCrARG6g0(4P-oG`fT$)y-Rzua(w#>pM8;-bbq*7h(!T^Ni`#F>(B!w#Yn|Y zlA&JLQh^XIQMiEiIsOrJp&zalNKhd%~P2Gv!D66{{GC$DEo+hZF@d; znh8NWHn(l{G<)#_lLXf#1E|X4=#!R$I;jCb^DMuvX9_H|uw8kt#hN;#31a%ZR&4A@t8cKpEZ1+4R<0?J z9H{@!V=zmj0o~a1dqvc$l$0TXIr>W^T)Fm8pI3&?vF|7>G{tT|*&ip*w_jJA#htcw zR7E%$zut~p&6KEV%g$6w$*qVrB!m@yFJMQlulW%J<`m=M$0`uA)Y}(GuCB@SSOGRW zmN$C3NUlz(L3A(h>{n_b*>T5=EQagp7TesX~W%J@az&N=A=sZTvygsI;4 zfWV0}l64WGuaocR?ZKO5os(@Ml!8qx+nF1i-A4BXE4kG?$~M(6K=6|8oq4`Tv^&!y z46z>N_vqp=G?2>sjY!^8FOC>uM%sRZCuz>i7c{~zbp&KQeRHHe6Hva#fHg|f39w#r z#`|x>Uv!xytfmiqkexEXOVKA?Z+4%WVywfvJA-tCt79Tbc{JrN6E73HP4pajh$yhd zM#m>g0{~DCxtUHasGvSy@!1JGOt{2|DzpZf$C_|waBd{-*4tnrq!^j?5xorCK2cPt zQQh$mlG$i3&{@Y4LqdC!^z$(d<2AP2uoI zxL^gY#jQQv=rv+TuRHf=1cYn*(KP%Uj1t4SM&Fgx<~k@}VpqOj{Gke7efg)FWunx> z^G83c!Pax7EyEigNU+IeR|(Fs*VNjHgGAPE-|`y29PW=DDG*#*JC2kl?q!nb^Lqw{ zQbM~GruA|fR_P3Zqa!TgmM&I-6}=B#HSu3rnQPSAZ4F9UK2urhDs5)aJj7-FeZBiC zg3Eez=VzcQ=wSR}b>uvcOLPCYMo#_tz1}H#;}e<7yd(SSce+h6!*q@LvHjN__wz8qy1&ill?`LK(e}YSkNk z!{%3%sCDb+?j?vvX(vnBb@=T^x#4sX6>atN7T9caXC5sQx-L)D4{s2Ufl);t_)xse zfxef@iS6vD?J+>`YMcFn>f^hWnhodfxTp3!i;73Z{T^UrG>N z;^v}W=7D-10fMbMj7riDQxXt+Jj1gTjxeow;fDg1&3YWTY9`&8CH!G3X&)7nBl*mw6GS<>>kG1jqjGfv zC4Hm2*T^ixk*9f3GuT?Av_fC7w)>JBk)j`oOGI?6cy7Y?hvhtN?0ok8pE0wnN{jzc zdq$F()QBLN9#*+~l}Xw%SXGt97E65DE*RiXkx(gNbFFq89?S81-=7NhxUlxg-4Wo5 zz{da>>n3r4>pMURZF;~;_)~_IQ0^Ol+p-u&Zh9ziF&S*oG@AXE)=GdKg7rliw=RDk z>3`(%h-N&}y-C!#7DGdr?3*MQ;$?byEo(nLM}unR`ivbxh!$*pgEcE-q=*9U+Visy zuEUZ}tsY&f>H|+0dcc0ONmMs_hEzK6Kf*$PWaGPO#StU7&K}->LE5D(SLY|Mnl>^z zO()UOM(XL`tqsMJL9h2?FG3!Q7vBvh=Vfb@(?KsIl8|w8f)?eaSuYmH4>_0!h$el7 zY7B9GE!ludwkSR;`WtM)E_(e8t{#5cMC{(Q$SRce8qwC3o)x)LIa^jmh!iNq#0 zEjDE_7EAKhd*5|f1%-Rw-y4&=6Xc<2L3Tv*=EN%!LEQusv91nj5(3L~>aJ;u5TD1X z*Eggmg#&WFkkYNpOaj=7C=vU4kg#{FO8v9|o6mQ$#E{+2mdBVH&Kz{&VpCb->{({%gf1GPi%y{M(v4jMy0c z_iy|l!(o2HaTgb2+|bXjW6k|q|K@n^w$hRHkAfMQoDPn!v0sTQK<7hPZmcp7b~+3% z#B6)!-oZd7kDN1!GTq9Mt8vytEQ3%fBwh$QDCrVIcW&HB) zxBaJaix2HosdVI(cf(~db;!c+Vfw&T8pzOUnzU0T{|R%73e&HL`I56eOwZoph=T=e zR7M>tWLDitpi*EtXoAP2u4O7(*6KP@sr4^f$K~$R!5rc$ItVMfQ^AV2;5{}L;(rZ8 zTwc%Bdy&aFI}8qnCgqA+pUljmbo#+321Hsion=$b%5KETjv~nX9*J!X->rY0L)`pfT?}4Jy)(NW1N+XPFIbLj z{vS!g^jDI$cIu;7_B0pBHcNWZb77Fq=0nyCXz~w?odhT?vNZM2uR0h7%;|Q zz+hwh?fX2>?~nbr>%CpC>wV7aocq4deU@a1x^A5zi|Mk#Bu)>tlmMHlm$$-*wSGq<)Hlwq@dc&95sE2jaMRunL!()nv<;iWTbP0dPB{8HG2RSR!D7#8zV2Vh9|+x)cS$1UX;ea2Koc2azrFB;H ze0<$Zi-n;&Iww>+rqA=o2%5mlrKOkuSs$lr5{e4>@_D%?fl02&Mza1m%=hFnMxY=f5k z+7_#8X`7r5r{(vYTu=P~9y4CkRF7MvyzDEZYBoU7EF4;s;*d_JMjl3w^ zS%&moS>bN6(rFpPMZKuLmx(*>nX#axOK1m4vlo6x#Pb`>xXx*OE<{n6OT%`HQ~cWl z?c<^SYCJBD#^jxT=HdWvt+@@P(+Z^BZA-{PlZ(c_j$^w&Y1s5FeTUC$1Oz}#?z3(%9sYUp zY(mDJfkTU3);Ihjr|(7Asv27b2iXlkz4e&Px~jH<-mV;L(n;c*B=E?8Iy2HLciX`j zL5aXWXEywywno>o)x6_2UIG9X{5mb8(HD|3va?FP2MK{3yn>nDkSrK zx%Sv0_Ve65RRgWRi!c9`DWG7hIRh4~Fl*b1GVQy4Ek^^}TZup1dsJ2(969cxz+WD9 zlPr2mHfE1N;U*>CaYF5ODfq59snp+&M7#%3zt|QtlSIZ z586>`K3JXJ{m67$ounRj+5a@{@~J(SLUC(F30HdIVU}Ve@=yNyuMo6BIzWC0#)9OD; z@(IgnUnjq4Db*|q4P@?T7&XCmUFLsXC+}%)05yeVZO|>zBv{41y3%^oX2qQX;t8yJ zo*h@Ky8=csT2L()5&oSqLau>JT|2dK8<21Cz_~~@esJAhVFwLArdX=)-rU7gSLa+M zIr2UIa7qJ(H?$m^%mF=dBNMoBKBDnTboZ441cDnuUBR3Dz!S~v;x7cSlkoW}T#6gq z;;IC)?hD!*73#XkUhxWAgv$8_4FQA8Uwo`q2suc%AW28M>*dyy^SZ8wIS8#$E65|i z6~g|RoBOpkDr5pFl8+riZURw#+#^fE^7K6*#V`Ug4hTBA7Rx7wEct;qq5i&co7u}} z{7Wo^{kYXMLZKTj1-Z^~VX;bJgyQFxbgywBsZz{=+to6%mbjIz!uq*6Fq_3xmSDn{opM0%IM+GID37aOEi;bf{es^AFGsQ#k@xN^Kw8vs4AaxNkhq-?>PSYj*cugCu%>!_iV(?h< zkcuQ@%fB!5;6=q9)*lf+H17~S>Kix42a%md4*vhlzXUK;DnCD#KQ5 z8A0B@O5w1BjbR>H=p4mIM&{+BUu{xk8ApjU{Mbrmh`I>6+< z-X722uplB0BX8JZYb97Y`TvFq^zJNENIex6x|GBC^~C2z>fKBQ0COyNP>TU5S=< zdx^OM`$aUpiMCIqr)_(Wg=NRrLmyC+>$7f!q6Y1kwmx%6Cx(vj3UN^ivn_9?R|k zj9+B_4TuZE3cGJ}z^yPTJkKTd=*pVJ{0wv@7XDZhABnGi&RjJiZQ&We?cV-rk=Wiw zcwVTG+(I2$_;4=ezB>P()?%&J;{HDd>JGw%Keay0CA|2Svh+I9Bs1x4p*eH(Lv8(+ z8{OSb$cfd1!(G|#i;J$+y=^HJE~oLr6n^&M|8Cq_==K^X3$CIlu$RaDIgjwE45~~d zY~F!3$Z&uICf%kVxJh&62F*r@FL<@v5#f8`i56diha=L9ROPKU8^B$C8$Zk{?EP*# z_NvkA?6WWA)g62kpFFLUed668ny<~6qONN(!yJ`vW;~xiwZXpDi z;~Wobn&|NPw`N%`xuX}fTBBR)-SyoeR#$%&N{XxV1Dmw>LeO(=wIRDkmoz_}sxK86 z?MC}5h_urC5s0G;bFLMamG(l=R(o<+Hi|uHls#Xqyt>s>8Uq2Phz{3}UmlGcr%_^4wTJ{54`rc5NXPe+4zmAVqcy(_M)=l=XdH?3v%gco9B_y8~!txME#tkhxH)6ywuK^aus%n2J z&CY8U4LNSUYK;zA6i%n+cmvf2gd&Q9*Hn7L43@@<#mo3U{vDAkEUrl6`vlxJ_q6r6 zqmP{aZ=qargSe|9EnD*5=X)ORI%N?Pj-3QO?jLF2M0vX4+rmsXgizQdc9tPotQ@+9P-RXq6CtVcZ`y18Jz1kGUxl|!d0nct>l7ZQ7P}}2OROaJbAt}A;nC` z%DjJqv1_^Bt`-kX{-;WCU)xPe^Z(dEew{w$9L(ZMUBQp(!T`v(t z&`D}5A;~PB!*if{f=mW2MZlvTz3Jn<`!|$NaWI}EAw+h+o`Unro!efnX6x*AwqRS= zGn8t_AW$aV!F-4~QfeNV_~l3IHop8*uqyk`^O888r-R(;3p|pNR2lP+l=_+1*$3};be`AQWdHmtLbQf}9z z-~DlTd&{BnAnLR031M7XTk;pJ3vM%eTVH}`f)d;i_;LU0bUQ-vBTrpY^*&>!yd||c zm5CBNe;Zlqqn5EMB(ySv)lt|>v_6{tV*JZ-6g1l4cKRfBhMQzyL3 zPeS%EMhgPg58gqo7oSdCwuUTR529#bDH$~h*4-CC9-u~pdZXis0=6Mm4Vj7= z$l2~a<=56-bO2^QlsqTL3AopN!Ec05W`j;V+9|gQwCzTL@N`fjPZVB|Yw+d-P?(MQ zV|q&ZpfWRF&Yt6)0Lcz3KvR8@htNeVlEF9f6mrsBX)Rtc(Q1+hyW9WTKN+P2C;KkD zgWAhWP@1b&-eV@X;vi@Lx&8W=|V08?Qiq{3YRF>f5D9UyH-Ww%7z1 zVtP)~teKBr?Q%S8G^`7|8fNL1zdl^noH~~QK1C*@4(EairVZ2HL40z`_AplTddQ*?Ej}Edt?5oVd#*Jzy|E)&_XgnJwsj-S;NhVX z2FVllis^+r0b^FN{-X8Yuqh?I_*L(0pxHJ&V&t1dv@b{x2jg!2s5$`Q2Yq7c*lyFgX`iqEnnThpDa9_e>vqV>we4qNxt^th|b0zj?CbSf07#N)@!n* zRN73^$3{gm;438jc+x2Ik>mz{#M?SLHzUJ9)I zROT}5QFmG%86=(qZzg^;SIt{}rA_2&JPkUgs=nt{q=Fwb5`KP-dPwc!pYxW{e(_$q zpj0(rs)9QG_eLx&#~BlT%*@U61GX_5a1m>i7DXX9_M#>i{Ue zQw!A@@ZD}*IiIj?x&l!tS+1I%Cf&@=^6#;w?t7~*z9xC?IJz0zysXJcl zssefE4A!$AYPfzb*8z}34#E)i{TF7Dz>eD@r#UP;yjX@7 z(gQsM7oE#-C4KQ10(04wm?>{0ijd$7+*>la{3y?`ed$}+UR&6%hu>1))zwgSxAe+IkCPFOPtHmSB7Yf?UDCOF_K!LJ z%c={eG!>Kwh9Ehn2$zYiE_sdW(ztA(rVQ}paXan+Q@$#dq1KO5XJNCX*Er1;G)*B` z^*WrjSLWzkAy~M%S&2dU^wT(6ymuGxQ63_!1ZnlFGYFnMgSY$hP*T@1t#p)!(rm0VCCE$<*P&enU>td+`-)!Wf1jR|%>?Sj zRhE#gKNipQ=%($IY(5q3R8C1=v^1R!Bq+Wb*G*Zk{4>rV)zf1`&}Rs9VPX37B5-4J z)T&xrrQ+X?JN#BPN@K&At=HTPcJ0{CoXvkvGg| zfnZ|)@Xp>-;TC!;X5~psK^WiE7{`Ntx&%6 z)`f399va2ywN3He-h7WRHE~pKZjS~O2uN3ssg*31#;^y9t;RlVxkqUn*J$~vhdce} zK*ooM>H7^8oo8{{iN^}oE+%6Zf!g_FTNUJK{p=aiRtC(Qus}fT$i=6s7*X*(&fA-} z^dEY~vDuuf)BZ4iT7U3!&?Z$ZROGvrsru^8H=oPii%_o^MluPD#;YV$;F)5s5`OcT=X!!#F!hG?O%``gmiH2X8TS{XzExnzFyxI; zh$EXpZS6+1OyvHv=x2|Gtsnj?7@ADxj(0a*`%16k{%bg8o9C6k*EsLtzj8K3!?&z` z%izrS+99(X*E&>!OV!P~VDxGAwA1R@7^a_*v_vRXi=dS)7jIJWn)RtDlTGWnoPn88h+Mw`b*?TZ}!ibSLXbCx>C_tieFPbUAa=W-d|J%n>L9$P-=dzo0dBE6g$L`$_ z!Ag&d_Cj6aVO+>*Mc2hNR4rd_XUVaPUM11C>$)#|@>-(PPkMv5I`*NV{E=Kqkwn+l zPy^w5PKV=<-;CpeNH@zNB@7~=o&A`8NBv?n`2(=)86PG*F$ekGjg(XKrNxI<`K|%$ zI~4=p(Yylg7dpShZ9Y@)_r#G%;BE#t;{jn5C_NreQ6$Y6MBCeWFtOGRAlnn$* zu#BF2aNA*BVH1bQ9lgD$9cf?Lgwm7^Ek!a}=!EWn*2@j~%^mmO>vMvpfUKHi#r5+z zH%}bnqMkyq-tDZfSkL%#m9$1c+j$c!+lsZ7%Ey*r2vai}8C=A9fjj{j2nk)X586-l!L26zA8lhZL+ao_M9a`Tv%pgoT!C zZNEaRgjvE5{R<>}UjDt$7q8m;t$p9}nLV^{Qba`RAe5_`JVWs6^lWY}@v~kJ)JLVm zcX?Wag9Y#;*uc{H{#{Bs7P`)7B-=V@Vq;r$BAHI*eT_fxfW=boyinY4&DmP6`jqgD ziWf6_ZqA9rQ!{Io#IW0+6sdHmX&{KRx`5v2-EK`-7H;S!^T_7rt`NIjADx8eW~x5n z){_`AkHq4%ZpMAFG5svTq^iz}w)r1TOfh_- z%oCi76p#|XP_R&tQ_EPcrGGbWL$5LtG`$V77I**#fit~>-Dp_@Um+DnyBzj89kqr& zr`z^AWI{~1Lr$~&XxLTB{H}(9TJ0_u@eL!Nc#9_p8-EZoAZcLdv4>kD0k`P3nwDlp zIDgHkF8t#S*ghED(Jc;vU0xXRF&i$gx=5tU^JuMT({BQm+lHUwb#5o}k=&L}cBA;Zvpa3^^zaoJ zxUzD>?92#-Az)f{^Aejz1iHc3QTfRh)z)8K!&EY_l3>0DjdAu z!4u0+1j!Km<~J~4E1wDvJBwtb<%!KhuaQl+!rKLr=Yu@Oxfj9a;d3W3O~GbY=jjt4 z8Ynnx=jS)QeaD=rvg$B4;e98c&8UtR2lE~2Rg|LXx7}^%CG6ZOMXVlX-_Xt76L9(1 z8%=M~xS7M2I=989Lg0$YISuIj$h?Nu10;!$iBod{O!J4oFg(n9Qr6r=_{Qq4J`28Q zC-Y&XG|pi+-|Bo4)Zt#LWw9;lNT`EuQ$SmpRJi8O{cHM+{Wl^o*SetBaj=}Ml9tjZ zR{BeJswBMoS-|6M@+kH50#}?&%-=30h~w6QG!%vM{N(Xfz!}5UglS5_F1q$2IH_9X z;{(-3B|pkCkX(oj^L(mOK8FNF%^Me(n<`Ibs=+oIHdQSK`tF&wHmU`PclLY&V!QZO zjGb7^eNgaM$y`>se7kwPzS)7RG{vW(HPYIf?|~ybPUu#k{_W}D+~rEHPXpsK8rTGe zo|nlm=)XkXG&@d+&W)>7CcY5)cprXAtvaH{$ARGH?mM;b5kQyE9Ygt4S`Q3s`LEYu_Sdti37 zS2ghRL@(DE?W)5a8(!~ZL$;j?WHAIR0=U@u+|>hH9Zs;0LI(kwjl*_t%mzPvfH;JL zw%KE}>9W;5=7}9IFJT_yp2J3?YerLbKsc>4exfx4sIY7o>{1&=_!Q$(vZiaUKBAN1 zBm~y!KU93{>R$49Sf{L%1|7?ijS1eel*#?F0|ECs<>kCaR|<5J?!o|%PT3HTiQ1LA z8-YUJhyKj|i=2*7b9B+UBH(j7j>Hx)jKOmBCQMvgS=%C>Rt zOt6QjEaFU1x(7hZT+E~!>OE+xLcLf3bT#W8B_ps@Qup=@{3e2;;E`e5yQ!EwSlU5# z=M$Hvkeg8p%U41Ape1L@%3jH%MO1U zW;=~E%__Q7+fJJ23OT!LFV@ocU58btNWG+#ct{N!h(zbm)B?CN7>iT5}KOIJCgRpRsT{)|U=l{FM3Yr35OJm-(Yi$XM zTlKH#w;$Gj2<$m%$~sQ8-qofZ5%4B1ba2O3LA#R-7fv3(9lSXRmHug@#Vcs5wz<9L zcH0GSMZ(%vY$(Y>a_-SZ>&N>VkA38c9_$$!{u&AX9p#ZrQNY%`gBo^$Aika&w@9Nn zCJ1p>v+nCVhw-a;a_zirE18|8<^p3lHU;nsgBGMZkKe90LefRlnb4o?#&*_=ebLFV`+3-h{$K-krs@V=?ay_#AD1zQysQy} zc>*RjTIh9ZRHvG1Rj2A8W57H`4JLWd$GT44!PmEq$2m+iHwHj6LNqQRC`j{;(~XJS zf}vZ~CEm9)e!q{hQs~;rXS??j!P7A~cIRQob$FujA47>8|7@b*z8dw$#?<@rz6Oc{ zo>jkY?{!_U*jcAM59DwO(I~@Ft46Q#A9z^`SJF6V*14mw9!;jJ9)&vXMtJB3x=SCI z?ozX>XP?JE(H2J3mZ>^7TD2MFVKbZ%JO*G~dHHl2aUa@|>pLJb<>}Y95{{Dq`nd+8 zXWc}uOCTr34TFokCCbQrij#qgy(LD(nUj}UMQc0TJ@Vi+;PmW7u!Cp`a0SDY4Z!vJ zND#DnVpDEIv2mR@OwU=|ur^ce#jb5=D55b6D1yX(QGIsa6XfQ9;f*n=e&lRzCG%%D zv4Qk-Wm4aJ;E2*KVGmMSJU5}$#j&9)zIGJNf6DEgF6zc2-)IB0S$C;go>GM!HYDDL z*shvvIaYb#O)@ArIiHt1dQ8r>Osb>XrCZ7-0)!1dZ8aGE+fJ?snH)yuokx8p-E88% z2ew9C+tgh(&;1daMaukYh{ChwjYU3J7dwqcK4p^0C$F^Nd||xZVbZ6tm8n<1f`ypp zzFTLlk926}dlE(--loJ&{he?E_l|GBXmOnU!5HI0_O1atUUimWWz6fssCw+yRYxmV z`lR#B+^vD4d>*U^3r43NUeC}@g?%l`Fsko+D{xH}`5|du?PeL9o!(oqq`F(>FG*U6 z8Xt2Nhq{uwtj0H`OdYmlyZV=J{s`3Ogk8UukI%H_Qcds5Q+hDNg^pY_411SlY{&+( zb!(O%*5HoadY5ZzRXQ%VJ1NjuCYP)jpgcNjZd+>dqHiLa9OX5>^x_`)SFYScH$SxRJ8YIX_bX{cRTs+FwaU%4VJ)RtRPqkf z?$WEYx042`5U+vp+>Mvjf}RF9 z%WmDgyMO;@KKIgHv?W**q6v_p=qvC32dW0C^ZEhagMl_djo$i=d`h8`4dipA|4eM0 zX+Pwd%|C9?o??+dVO@KFLX+l5!b}sZ_Y){(lekx$h&U5R*e*~us}lbY*Zt&rD+;KK zw*T8~e)=BKoc=-Vsw;hzzx5@uhTw!0Dl6zdZSzLO!B9F+{?3R0=QnY9)|5My48lME z{{Cdjx|4UepBU7+!qixLP4?@g%Ld%xx*(&Y^ETAvq13Fre@pg0iKrAu92j+S@+G5$ zzfnSo>KDzy&$Q9LOFPHjSN4#Erup?y#F`C0y}Qs}>a-lFE-=5rr|y>(a)QGxuDspN z3-AaW3ec)mk$PJa3H4#4f^L*l`7GS;pq>Q&PSzyTJ zUN=MORyVFOzi<~gi1N=y$;3(yavgY#W39fGH4>O+wcZ=Kr@B#%Gyb2#aHd+AnzH}U zBQ{3Sn_V%63@rYW(%@kn4=uWEZ>uahTpMHicAVsxvX-h?b3!`KMRBKZb+;XtP2e64 z@6rQi8aifYg+5M?BSomm8TDHf;yZ1cu66`=^F!t8L>2$-7a}0vneqjD$|s(9-kT;F z4&9cCcSJ8l{$ZRb&U_Vd&a3Y}EN6ZEz3ii>9?phHF$!kp@OiN6%ym;T2qI!%!=Zvq zq9SfsoW9L9uTk%<7WO5retLOJN8XfMD#c~}RK-3vK9{|+4M+dRhrRQiNp90a#&6#W zZG`AUFIaJ$L$bVsPF;CpVy3(Nqv`YmFm25+XDO3z`#1W56Y zQNwVDC0WO4a)@a|L!HE}{#`j3W}W>`nkCZ$tmMG0`h_aRF)*34XeK9K-($JEGFR-3 zNak*zmuatftVbt>`b5+wZ16`pF$2H^Rg@n5kV{g$(Eo>mMid*O$vpW@z}}mu(pw|n zUnX_@qbP7`sy$89&YHoKHkNCm2Og*MeKa&J)t7K}f^;z?{tKU8V=i>h^zwa4*-$(V z?$4LD3NI++VtB-9w_)<7jyyuVfjhQv<4ZcF&P?cNl}s!*cWf;0{n^Jm(zUlx6d#qtUuB zP%u^Z)_nL#5QvzG0g~rf3g{B@-+`ppy0`4pOLpb1{^DKpd8hNDzy4hX4V-|s) zvn#^7i^rew^yp@76dt2F_2DSce$$+_y!O1O9F(-=5f)Mr_*YLr3mT%2dvO86V@@zr zF0{Ig*YKi``6TMDS)RDra6nI^8CZzgilJc*ag;)XYI zp1Ctk@2Gi#?+}k~ZTrCQ@@@?8Cpd0Ucgm&-*X%u1E6|^k*M0c|(O#09&VI6YhU7Uo zYbi)8hXlCY|*e{Y!`6u812o8f-ThNu0|Ex?f7 zzAWRX>iwvzjGUb%o1UJpv66y%C97Yre!$Je>=Pvj-xF6^_)MXFU)W!SY~B#l{wP44 z;J)|!vgYCL1{W>=y{lJGSy~6x?=M_N$5ZgsO!b5|tdhM=Y05Wi)&72iX*Z#X&%NMl z*WS~QWBlgOuS<*2Csy8gj05xwWOHIp+I1v%nmg7Tzp!QMao<9Dklw$HH{vOyEozPa zJ>D#j`b(vw=`cDV&W8LWu0TcnZmZ4(X`5)*Z12n9EyK)|PY`<&bo7FD01CM5RVMav z1)Ix`WAG)HeW(sf4i3mR-Gw?vtucH#aXw)$m$zwUsHkcrntGqPuf|F_b|GAEFU7=T zmQ2%_mc&~ZjeBT|TzTBQ?j6ZvB<2A(0K})oai`QwNbPJ+hohUE@mzwN&1J8jojoAW zcNqO-79E;Gm&0jn&lh=bdXu>@X_Ag|3gsVy5i6J9@hgA-mA|ZqJBBVu%`y@aFZ9E zPIO7`*1QQEvYFNNGqJhE*Y7jVKl^ed7_j@beW5yiT|_&6yM%&bWx*S*Df|wgt)fhjg01HQsuoDkHTaWNI1pN!6*y@yg-qkeoz@L>Whc_ghHp*n;(;I z!C=fb(WI3Ff?1(-)<0T@!XK}Q0WC_v_{mJh;zvd%tL*EoYj_a1Hu&&(S^J=$+Ruv3 zYqkr)&a(Pw#JlfIqUeuNQ0vNi5$I++Dr{DxS|0B>BX0;G@nP{G(dlQ~1eL(b?WZaaL1-~uKf4a4`6jGK4sPUx#tJv zL72H{`(0Pb!p#^t&|*6Otn_dD8uUHhs1ApUvu}kBOojAu529-4_&n8l^8EGN5N!uj z@k?&$PS{<2!t^=FJIQDFB;wV_=cskzkM9{g5JH`7749}oJ8;jlDn|y%Wq9C#!L`(< zDKg!3tb~gVU{qUDvOq4acDn7O(=v6k-dQ+&zTktYWWtoAM~+h`t602ai(wQ@K)z#V z&Wum>huuw09kqE=l%@zK{Vt@ipn)4AFwq#_wU!cfF#LTWUTTA7=@V-!d|S_#IJRwp zefm`*Hnma0o|c4Q*pkMQbl+Qz=u=(-mfweoorikHYfMbzJO2$PKdFNSL`U|--(4u_ zC1`@yqGG`*wWLC_XwJ|_9i-z5y70@v4K~_qP!%@xC)%PsMAITYuD6)G#KDx)8#b_d zp#m#V^w6xzTVpOxa=XJdu3F_4z&H}~`t;{BscgY-oszs0ftJJjH@Ra8503;luwzM)`b71&6M~BI8IwO_(jL7 zz$yw#(n~eo9EI2(417`x*2vwC9~H>XL)Y<5pMr3=4c_mUATNTBRHt^Kxbe00&e^uz z{Y#!%KQ$;xll(2WT#&E(po*lNkvN4Q!cjUcZl`0~(@rEUOq(`1r5wc21o=t4(7|;# z1Q7zxiqb8lL)M(yOWId#uxju7$D3sVJYBR(QJPeyQDap^T`a{p>*RqK`<#Z}9k)sLnyuz-7)JuZ#W?lqlli2#_ z*Y&0JD&K-e^y~{IP`)YK+U!>mzngQh8<=%PdHe4F{N_t6WLx@ZbGFD_H=(F&D@@Vc z11YJdVN%5=h;bE*W5K-|q~H`1$W^!DkIl~!6tj93B%=@zgi$uSteJ=B7M1dRJGAQ` zcV%5TY$O>eTK1~F3h3t#=N7IB7?C%Rrbhjh9;0TdeGlVcp&Juz4^Xl3jo#((-?${t zeTLVQFwENlMyAqO=qrAK>5SUVsRo|70hsE=cr$8-vX~X^`1)VBG2}R{58m~?Uk<2M zFT0-hBqJ}e-SU!o-mKJqY@C=_02>i^`AT@uhb|b~P<{-+FM@V7f`n3@?Lj=>^oCX7 zCwwWtiGl{F!h;$^%p_)BmZO z1}8q?WWcDAMs#H??aGt)Jx3@dt6L>ltH?I>{TF)FW|1b1M-+pkrI05Z*w9BLe`0!; zwpzShOv_M#uz2C|gMzExS0nK2Ts@~2E_-?v%IR#8!wPcO4|0NViULOySZ##xW>~Pb ziH~6Iv(B{IQ`%j(g|#;$H#mJ>57*;RJqxtp9UmM3rQd!| zSzsTtbJtoTC!GYFihAI=bP-6mR+?j-=iy@|#xtzTygCfS+A)Gx@(;Y2u#^T}RR3gR z_Zjaqoqh?MTV&S%m_x-IykOg;AQJtjO!V@e)9|BTfga`PL5n~(%N;tSEqWh0Ff;cC zgQ(T*Zf*lY288QyoOh+iU^fKC=vaaW1Fj$9(7PNroOPIZV#hnsa*;CyZ|%X`JdP3V z%THW75N(KYUbu%&`~lc6Y$eXy1?;TqceBz9KBEe8anVVVicPohR_;;*MK@l=IGsYd6;92vv^T%_%;1KOD`h%>7s}-Y+X4N|weI4Xqb6(G?9zkze_N9;FCPH4dmYzT)UH1`(T^*5Xkv7~=Rg0H zPXL3T=?b{Q*Ls|a&e+|Xvxv7_-n|+NPc#p~dMDMQZ{jleu3!|LeRoyA@%J&leQ~0$ z{DtDlJ9ReskBo!zrtK>_bX0#Ao@`J(GdG-7FEDSG{2`R%&vbD>+U7ySX$+Y8B~li& zB65PIUN5>|hH3N~hAl~MB{2q%uSt%m3EHZ#ayLsFT^wS2!usGe_5?*|-!Xaf4-{Qd12esnlU!3H$sfbRNw;sAlwZwAW z5X;i_WPlfeSS*_EAMNw`VPQY`u>7U{* z(1S}no7Cm4hFeZQlC(aVURbike4z0B0i&pfV&)UvttJw_JrO2Yvgj^#&2LS-q1v}h zm{07rQnXg=eN?Mg;t7p#;%5ceZhZ}&=bGe9mGgzpyb3uSs;&XYXWB<{`qkJ(zI7xzy6BEZz9Xm+_SAc#0$E%KwE51On{Ne!t&;LrG&cU{G zx5y#J1M%JleWBIo-`^$@5>^S4I-(qF?zU0sJ0>sIqw!$T^Os3LhvyPI|h?{ew&t69c z6HS)sRm5nh@6|OT1ukg{X2zySg4eTm;0)0hh3RTQJO#^bHv#tRc!45Ci%WEH^P~y_ zdFdK``HDozoub>ttQqWy!8-+T9UOj_e9G%$@g*|wbwK(^W$k2hT@FTa8Mb<^Ji70A zxuSnf+3({l53cM)CiY%-+`l%Q?FcCPUnaNNRAgb4#h7Qbk`OQIbQMh(^8BGmzzegy zDItX9z*mmTlSz>A&r11rR{SHXDTL)R*e=ikyL5dVJJ4q3VM>OYshw0#8sd2d!j_gDOw#$iIknPBZi0-@~BDm{qRX zVrBXO*(Qqsr*Rg21KZ*VQZbRFREdsrF!2B;+)sxOw=9^e8hd6`eVNy-ITd3yd60n{=dNT) zGhq?^Z%Ja@q)wQx!P_yjZxg4wj;(fsJhAb^(36f!J5{dhZ_{YtD&*MC+z;I09@BRi z(dmZ$>orZEM@R@5AgE>unl4{^{g2|M#a*23hS(2xNaOQwV6crAE~tNjlU+-?vLo2R z5Atd8G8^TNNRn(i9ybXlCUg!6&vKNAXQk? zGO_2#D=|rj*j6dp&p8i~wROVxKh2+I9DNt#7)aB$sk8_zu|o(~k?IG;N?DQsn)b%G za$Tkz?N0{h(eXfZAxX7|#LyHBb6FRpwy3$QSu&cAZ=Is7>K$Er5 z`_7pQH)=~TBbzXq*>iRF?82TZ0t}ZF5kOO<}PhQpDZu* z!Pb!IQ&fuu>s1&zx8cZq0^f!$MgETgGgnv*RqV~1qGQEOgO|1fIfl;2dMn4@z_`|d zXs!+AU%u16&*axiCzdS4qbVmq=9=3`8z!Tt0{xO13E$X|lp}Xu5t8%|wiYa&L>M2AJxp>yw)8IXq<(P8bm~>RfmfU8WB^k`8=o2#YXcCq z-|c~imblbkGe>Q8D|X&68G&r@Ntot|X0m{W5eVJhFs;d+?F&3qKh@dbaO~Yn+shB| z+oCcKS{uTHEBnr`3fcU!cP<21PQgyayaL_gxqh<)2)q|K4>KBgwcXx$grqN+L30!e zy7a33G0Uj^;_(BO=E#cISey?}^l%Iwv(>7JQC0jksJJ=C3w3PFDxVuM&X&T)Xypjj zu~H^}3Z0c0Sip2W`GEo~ZuLqd#tgTD6@wh<$fx*b#??i#sS~Lm&oH=;wC}gKP}2u% zzM0P|LQCdXv&5%SiS?Pj^kP^(FcYIW67TxOH&Ryb%82S}vKS=R66tyb4d` zz={Wl3%%}p%_;<-{oKIr9(OqZY^lXH^=A6ZL^Jc_@@0YJ^T3()%*Y=0kSOC%a#Lq$ z{EP4gt<)Bnpx1n~#CR4Ry~>hvXQKjVjF&%cuf1%*qY0oVsn2J+H+WCgaAV`$pG&+w zYpWjjDX0s%Y-PK1U%V*5sYqvt=5kYvM?6eanIbAfyXbxE2)aA#qhjqN#*+Jx%e|tx zxfS>Kf!CqcznZ#>e}lF&et(ILB1*j<**WlE{FrY&|b?9K|Oe z3V{2tplxc&x+_v*)IVkLBgT3!Xo*4D|7tcsWe-mgQy_fP&$C-+K64uU1`G zD($5<^M2oXge@?p zIam0?zN)eYGx+qX38^wV`RuqE1>ZeDduaoDg3jEkwHfmvCT)ZZiJ~(O?~4ZfH^PR~ zA@R_276>8AtxZV4bNHoEb(r#)>V|X+xJ{oJ;RQR&EQemjS($|s02F=Wn6RG)?u`~$ z_P~p%#sTf9tSeZ4I2~T!5aaQeOiPlGa=6`$o*T5Jy~$?P(ics^3Anz^1-V=o+j>i# zXDs|MavU=BuOp?NVd4FhsvS(C+Hi!l?|9HjIXN|uYCMluBI<`xh4z)qyMECl1lLLR zy=c`9qcQ83&r>#dQzAzHJ35}>HFeg{voD*VAdqf|tMP z_Wc%M;7w=y4f@D_aRV^OCA&GRjuft^C>Yksbo?k+6OQ?f=Z*z@uJE@?@2Ox)P~K&KVi5E|tnOeF@4@GWQ52B0(W!maEfjmO z@{^)qi3Ewa`%b(P&(pLuf03Pk=Wl>CqlQT!!#lkfQ{KBtxv&D~=ouQ+s)`^`wJ&4zZwb{C3j3ll~rRMrklU|O#jhwrNmOj2+%d9oa8Sx%HT z(0mUcQ*&xB6uAApYA9I~`stpVkZ-Wn4~sCv2$2jw-#pT7KQ!*b{9z^;gyNBFZNvEV zHPT65idKGdL7YvhmO8+=e?gsd92ds3sHQfIfz4)j?3qORtI1do4j3p)N#yqFu(BER zpyps=jJpTb=RP<6G^ATPPaF8Ys$FI{;~!CrD@D?JJI@Fc{Xpe(jH-f^uPk}Zg0tpU zVlQ$59^EjKb%YgyB|cham%BA+#$of08;o(aYo;qfL~+IBQ!t8JiDG^tJ7*8J7%R6z zsTZ2)_GZf>GiCB**Q1Kk-OmEyl*XgXIAG9OywuxwQsRSyAEAONmQ9)fFFo6v@klpT z6O-1vFptxMXE|PyiaEznzS3aHyXrIw?@biB$xcaw0Dsx)M@RzeQg#P=l@3f>>@8(M z6FGVD2ECg&OZw8>*lImozVLOke&f z1}MZgD6!9B+hB24oGP;U((C@&f{UM6NVYX+*@BVvlriQC-~OgKi`}3LN#1g(sZbGD z;%9p9rto`BiG?TDcDEpDXy5k1c(Jop%i}+9L3QeHalKEq<6ky(2Nis{+aQ_OF!|9u z=l>(>ETh_LgRWglk)p+eh2jnYN^odPf#RityHi|?QyfZgcemp1F2S`}aDuyg;LG!V z_CISS>txP-&s;OJH;69JT=UZmjF3lBr0rd}8Q9=m+qh|h2L2MvW6hS=AWpLyPPDj-4@y&r3rvGITdxVq;lT^X*hNh6tgQt zNkQ!Jv+1079aS;&dlH8RkuoMRjeoyf9~R-R<@1J@+5aPaCoQ~i%;U1xAC*Hffkx#H zf??AK^F{#rCJ1@T$;-)%gNe9LFMWp1=FJim?YGm`s6aT5^mq@kl^@UT_4GUxZi=j$ zer~n?5kut`WVjl*&}%b@$qt`q@CRL(^r6u7CCyF=VOW;2gXh;JI%UAZc>^8lhOr68a zc;#zz>1Fl@LadVp^_tFA*Z9fRMC-f2+0h5~qf!%!O~tYjT)2?pQGT5~%``pX`P<@itq)D;j*Hg# zT*(1^ci4ewugSgw+AcLC z^Q>b}K!jX|^kYK{1M0T+dA%QuHLJ_jnhZQ~D@r%YeO$Nk=rN_Rs?rVvGRH}u^69mZ zarPWI!Gd@>q)}IoZ6(OytjU+r>Vm%$$+Kx@UkxFW)*s=5U@gBi9hv>GQ#!^3**?rj zMz)F@_JOxr3V;RmSPTkj-f$r#_NvNFU{iw`HC*=d$SssKDRxxE9YGs1 zV;_`p>#Bkzc#LqI^rMb5EnQ3!@;elyw5GJzWR%VYzje&4V{rR8#ncwSF1!z8$;d6b zVAl*_LK;^{1C_)TzMjc8V}D$xV34RW?>|DL6o|Ru?{M&5C4Ch=r@z%Z@`y~+;3j*6 z1AfI5@m`2qk+uif`#HVSx4yLLk_QGp(jKBOsQ8=%2o>`l#W=Nm3O3wZ~tnEe81 zn2o6-i?oj(`mg|kQF0owO7I|H>K&U&8RF(ZpaGZsM-Le*)egbf^L)~7G>yV%gGUvI zoj7mgW~!0sLK#=D6oFAT<&bRT3dSq376S zEQjf($QA88pCi27jRES9f3ZqC!7U6JKO8=uB{*ZyUv%QvEYOi|1m^{ zFD}d=#CLTg4w-uOEu{L;l>f)#@9&;$_*FjECH$Kq`cZvDa2(ZS@ zN$h-FQODiZ%w7y~KUFWIU0ZXfJ>!IUSE21UP``Wk9tn%2sM$9f^P)DQ$CyWSx;`^# zrGsF3xphOv>CEt+4(`x*avFyDsybZ0GD|gurvsqZb2U%-wS@v)aK3XKYpAv zY!@#6&dd#2F)#-9af`SP#yS76_+yGgPt<5}9m4wkN0E#u!ler(Qd5~6!%%ie4ZH0X zDa2}5+b@xk8!(#P*G)s7y{)D?3A{pu>A%(czV^KkcV3&!Qm47=Rpe%E+RIi_4-V0n z3N=y?B9m~$c!|RY{|?x{i-B1|7NH1=q}>vSLkR=i3d<PU;x{pdak32Mc_ku=Ezf(;q;ks|QuQOts z7rh=dN|3?UVr5^_$v=|Iqw?oEQ`hc;`%{>1+_m>i$&``6s%WZk+~4|(O+RW8p`o`? zW}t`o+WC^O4$mG%fr68?2YBrWuY6&!wQENojeh^F@-ixR3Mge($Ui&lcQXNlWljIn z!&T~`{yX=PzODuSoYQcjE&`~}#(-(!80{1w_;xLH$!N>ssm$a!Kl@wWn8={+2>+`P zW$KfTm;w<{=+aYo=+P@Xaw2tzJGf3;M+CZ!bf`T?tMsI zW^UWT=5BBrd)gkK8jBQV;q2(9o=W7FxI9VUc>DCQV&~v~^8_tate<~N?LB0Jx>#Cm zCpMuB7)<4{Fh~Lroe~5)DKy@$os4MS5J?k#ID$m`QV6Vn<@a=mHYk5G`Gn>6xV0Un zB7QLQn&$BVMa$TJ@J#DzE5MrU3C~b_HefP*PA8G+7{r$BOHX;?3m6!l_rSgltQ#KJ zjG|^t=CBk6HH4V|wFO7}NY)W*$E;|zd*MVstQzkBF*1ZJ{aoW+Rl($+plhxP&ArH9n=c4t-v1#T*-O8X7v^G; z^~f2U_P^sVcP>UO$@B(i0@?c-rpEy5IRe~qEhT{!eWGMmH&m@1gEF{ko~uW<_v_j` z-Wz^nq)*gQv(f4y*$$C+cIAqm|!Mx;xmn6p zpfag_(e&(uBh+!vlX&iA$MDmBFhiYsQ;#UeBGd|=Diozpb$lVAYqLU zGPvA1sHiZaf%^;R)94w>gfE=Nk4kTrR?@uHDA2(W;ims=3^6g=S2^{+6#UyHWy!u( zCBN{H>ZDl^LAa7=dUo%5)a~}l)z6IU{%!(3z8rQue(%%rI@cTZjZ_S8d(2OK>yOeR zTPp(RUD;-4%z<`V^ftsx51O1EodJ!8(kTcMB=)phskOue=diw^fx z`F&FnUuD!=w{)mB^#tQWv^1M@g_?IfWSi7{ggv^A1?0X?U6&FQXht&DWjK|>ue z;JKH~@QB8{Bt5WTzqAIW;}umxkk&&!9SO472oo3LB9vK5lLd4r8kmh;_sME^BiGvT zuJY#+;>#fHiBY3pB|pt{VxYRI=mEIG3=z3c<{4vqR|Qv?Ug?d~40nCpt>6#(M7YU{ z&yH4hVMEG~JU%jy{J=Lcs!?l#|;istYaiUfh| z$D{CvZzG|i3|AJSf#c=h{-_YxznSd>Msl|rN00iO0Qh3(pX;CbHD#nd8GbgBTHsLH zpGN|$K3fe`xy~;YicJ+OMd5mINCD$1O}iGQT>GZLOv=vUzf6QQjI0>ll(_7E=Zc3~ zxe52fwVJFv>s$|SeJNRw#g%Fav;gAO{vW!1KSe@B;|gW()H6+aWy3#6W*DR=McamC zTEWN9b);`#`u_WdL^2+S(eaUV1V8*C!%<;##g`iAf0 zW3Dr(_T8VI`w5JNmQ4QE7lmyBjp(bzBoh|iC2c;dj=l$y#Nt!+#zmk}I&U(}@{3>Y z3^aRB>vWOUSFO#x1Jz$&C57}!i)?aww0#_TLD>-Q0`P%A^$q&yEfy0*ApOGWR1B;b ze`1vX)rx1>4aj!+GK}9M42n}7aV7I2nzy3}O#HFgahP#t>MpzfzSLp>I zMDd1f6mNeeJ1PQShBplYd5|}(P>W+IC_)jij8wDQ%eml|m*4pS*QG3TEBOhHPegF( zJe62ZQ6D=k)^0bKEd6!Rnu^gtQYRzpIY1Gkl6uitz+}5gx4Bg*K|~e$YrNN;O6A?` zx}mH-TIYiE&_Xj+sB@)zs~X%^>%BWK>&eGI~8cGz-vhaUKl2Z}3EsG>Wms7Z`-zcK^tX zGlL{oFS0Y4j6@d;`F!n(QLkMQAW~H1*S*chFsi2F<8m%}iAN(@Y~RhRWbD5cpGv;)de8%v3dTDFaa0z=xMUm*G#11~OouHFy7S6*H{gKC@m)_#4pRy1HEIa+6D#_|} zIvd0jj05~uLUVAFnR+xnH+H<~h1u`}qa@-jbXg827cjjNVk?zj>+`hXyo{@cG!j&s z+D9jQ`w$4o#l;q#xBu`mni}GS6j5+&|1rpeq2|DB-0$ZRX{ES)T&D~IxP z5(s97LL}|Xf%H5kt;*#?H9FFtVo@S3UShGLG7=55z(Or|NO@^39%+g(gZ!|;!;xrk zCq4x$k-g!dPEgZ_>C>zr(!+ZZut}dK&!pTAtRPg-RR;{o^f`^MvsX6E=g|Yo4zCFs z5W6ie&YcRhT-4TN${ZOHjU$HiF7j&|yv$|~J9F3c8TUAEm%b$oqTwVY7V-ju(9)t?gcvQ9&;M4aQZa#UopUwq}|vE9})o}0dbjTL=J2Z6`jJ~ zep08cmnj3=m@-|5E@jZ@2(WU~d_+{Vc4KOG8Aoo6pK=~ju101k%qn_bK`t44o+0e- zHWrZU{)h8Ifvu1F2c>$NUe~44_z$Rq13ACQ`&uA$y-X43)H0D?3dw?h%PjuOQVb(G zpU#oM>Fj0?Qgzkoha{H#0HYMM3Y3L6szdNfjq^W_m?5k{ur|L-YtRnVG+2?>(gc*N zy1*Ry48oiwMZGKeCFj@J+y`yUPao+wyEBxWLQi!2&W#R~{RVGJ#VXxd+nGs-)Jg_z zcDMZsMw$9?`RtW4;r5FU%YsJuIMOP`@eGVhh7+++CR<^3~w+| z^5h9_QRKe8A>hbzdJ_=9#r9Up;^kBriXvvwyTA^tKkq1|+-<*-Vd7B<} zn#t!l-ga@^j%a6K`e>(DZF`(9L0z-c2E5`sJ$Y%@2|ogb~1s{GRGcr z+bOoH8rP0A3H)GP)om@;G1W}))`_=!8CBKX-g)tlgg8Czy21|`UJj6LhjaxA*Qi55 z2Xhw>?oE$H8ejq?SI6phu;WGT*c^9cSP@15AYg{Y!QRtUEgxRZ51RaHwRtShYm=fljOSWe%R?}(073*! zQ$v4%^nYGw6({?KoIOP1qVG^oxPC0IduvhHnDkMuUUDWnzJQ>Q|PZe_ym``Et1-;TWm{v#*J_e-9FMPEro;$ z*AEVhS4o|W?DlNq=j@U|e>knMDfCV4=>e8zgMLda{1nMF6Zh#nr{Xf)%S_Ii2o|$$ z&P^xm-^JI>M6JU}a*a!z}5N-1XGfC)T9BxoJeO2j{%Pvi$gM^FxJCyj~j8 zL+557_xy5qvU7z>j$Fnh!t>D5mw()RsdVzG$ z7}3k#)h%YTG*%h_Z!T5y@EG-3)U)!qu>5w4v#OB!!x80m;$y@u(e6y?0G+0aPmC3~rPFI?B-2jl8Y)eJ;ywtk8&qfSRN;?ODufk6529t+DVY94iC4!(Lt(^_~S0{-)&%X52I+MS+(Q(yO^eXnF?*H zEe`FpH-5MNdDK7xIiAd!0khL^)r^2+wI7yL7EaotPF}CJeD@aL_tY>L&EK3KcBxrP zIx&>zWYJqO`62WbTi>=-9J~|W--U1U+-y3A~Km}3M`Drw{^SQ;6+#W{1S=pY}K<;B+;30Yt;35COn@IF|h9PtMuwn z0BJZbgS$~ktj*@@7!~UEiBtWA-}+zO`r4piC0_?&zAKrw#Y}XHRbRtte1cY3uvXFs zM+L|(K&)@@PxsDWWrY%r58Fc$ByK=N56S6Irp% zv0@WSwG0UHZ5tDTEp>z%v{rk^;n1taGGBeKTvHSo!&t26#Tfh3MsMj^W+B4DX24>#UGJaa>QRSZM!O6T~ur$Hz$R>fzAbQ zv#D%2B8DiqxCc=V=mqLovYFEW1LiWBfX}`D>45U~-WN0i&_0f0TH9FeOhDmP=3Qda zUjxvOsj;N7u?+c7H%o0RD4g~@e3BS8m*X=Zxv->rYV|j%@UkSJOx$o1of|dIKKrtgi=B`G9GIigNd53>((-0+O_g9z=rqoR~Lc)ll~uZi`WMJJc7rL zXh+^?CP&lu36lIPt2uT+eAE2(h0MURfCL>jH#n2(^kQ@io~fa(OM&>S{cBy!;opLeVB{T2xxU&U=0yRxvyntiv^@ZP$V(co7kO9-S`{ z=WkutZQ!tK9XUuRKf>)n-4x1kl^nPkic}22_IC46bDW}i)%Q1dd_hW8d&g*@P|N5m zmh_2fyB7%reH=7^B{f{$nB;lzKsmH@G0ybs>5f3iVVrL>pQmEruVi9;?%0EAL-B3q zw6a1MW~@Jdv6Z%{cI;fo%UzgZm$`9ccM-{9iIuKLrSw*vEOit<&E+ot@{K_GAk87P9n0sosF+p@XyTbZerv00>(WyA!Fdf(p zh2d|uNn!Mkm$o^R?7-Tl`V^E1y~yNE$*`3`s0P;10~%SbKASl$dvXl0JEb_%$R)_V z=jNyKn=t($oDm`DZh$1AZ0H4AO-VNc6yF$qOFEBpJr>+gTR9!eBIA_W!Gxtg_wP2P zV>;4a+d&V058pF|9`ZLjJ>gL5N*K@$#}$0a5pX@HdGzxUGi?ML{-oi_jj-NUv_L8- zxo8?D^hKz*H(@wt`z+r2LSdB23-xE+LacOH-kVaCl!!ll{Ah`qZ)X2tMm0a3BK!gA zqS2FOX#NB3j4m#Uggf=;I5`sfY2wf56VwcgxIxc(-^`3rcxgQlIwo<{wZ7exD8+XF z*a3aX$&c2gR{QJ_^`fo^(`W1Ygd_0$`|X*WQBM3N?p!57+47bv)nu86U+FG?N5B^& z34n4^&HECqfiUhih7zuhR;#*|=IXO_`=b!9_cnu6vgxt@A2VPjg8`${_1<`9^0*EUr`xzZ5J4)!`dwKP*gJdu*R>5 zI5)DCo3nAD@(pgzpUBNlRdKgc^C zyd76_Ois(^t#5bz0ls$R!@l#p@2+8AmcNxG-!W5J=ggF^e`&M5sMduE*D~_o+6<|2 zsl35~KCmO2Vaj--GTXOnRt{Sd_=WdAx=2$|2?>I>*Sn+dZPDHNCQ^M@a2^v)Uj*h# zJf208mfghHvhwX&qtSa7++Bhfhs5wDI6SPIOrUCAx;_Vr8g9YVTN%N?1Zx3xy@=@& zze=x*)1ED3#QWw;!?3|jWl`M%ns+)%_cY_;?87!y`K_gTL|S{XnQg3rv}`aQ~3HZ#3&wHS2MMIjf-5E+3|$khUh}6y=c(%J+=J< zO25R($7eZM<<(pR>^0XxC>kEAbJo@O!Q(r-)dFXx3aGk{bWbm$f~f{sZ{0BFtZGtn za|dUx-^7NFEmnBPEqNpSt4GE4;+FB22Nblt@4vXMjX_Pud)gWF7Cp)^8?M^~#F(@y zoRV}*ocf4l21*^{P`_iSlf-@nxeHtzRp>i)p#~4U47+cu6VzS0F%RbVVVU0jq217W zU31#cwBhsZ8cX^)b=<@DUmN-V376C3(as@v=K-3EjupaP5=g%)Yxi#d!*TqN zhEttUQjvu%9ub?o!0;fjF;86a)XWroyM zzu(sezsY=wyQhhA;lxE!vP;zJYNP%sMydcfQbq2}{%R!wqd@QnOVv^fEaG{*+U zWUa(s(JXt@uu}%Inxqu1Q2TCGL*!b?b8+F{2BRRLHS48Iib1wep*PW;;v99^`(V=NRqyRii~Jv2gQG+mcUO8XMq$3Tj`(Na2sO?{r4 zz!{f0$-wHxn>tAplT58k7PolSBOLkKbN=Nq_{su0^xucroA-HM`gAg@DKHj2MnB6G zPC!-HF=yKsY=$#k1EYVSm+=KYbn6Ml7ohilO$ISPCHnm^<`~h;i4$|NMqNaEX_FwEgWPX{S zH>HxB-{^OKDSVM_^rGt*J3@1!xG)`9yPV2GOkMf>Q5)GaPQZI%ofATEIWuUgIZdE zF0ROX7!?EWdq#=LsnfK;e{bI_zwqltJubz*ekoG*PMhX7*!l3_{(X_ryO-C(vCin~ zpz~B+qi_~>Sqpy2d%}$e$a-@O@hF!XXeS$2tp37%;0tICrJX9T)Iej#caW4<>esZ} zNFz6`dTW>2m&LfTac(Sbt+?Okx0$VnkJE?E@Lk!xovf8%GA>4=roF2#lpJ2k*!Uq= zZXEEByxjF=<7==F(-GMJP?Es&%0`cUUHu!LLGc)np638#0(RM<37tQPmGd$iZLTVB|uJ`k$%a-SGxs{JTD|KY3pwhKRgD!{#+r zV>HHlA#&fWlA`kv_m7h0P(SkxCCL8q6kf4}J8^@r-0f?mAcq|#JeHePOCY(BAYWM4 z`eem~^PV_aN1F2@QmFG7|Bxh_@I^wCu?R)|7CMQSIb~Gu-je4teAd)tHs#jbK$^#A z-b>7KJk-kO$GE7t=J$eIepDz8BdAx!kJWIZFno%~V(GOl*tz%i?&%lRbvk7epoA!z z-^m`D!d`cg7Fd+aL2p`iMBk@@7rlIXyEtIJP}Q^|6jVu+G*=L54bWfTuSV)| zR5Tj5s`CCjILHMQBtD$8MKU+An`gkzzM3?~9*Oh;F6GZB?{PD_8I4rnc#-C>Xqt>| z>iEA#0z}`TuUcNpQe(hg9bkT+Tt_vNr;Ih(M%mVvqRHWOe?36-9sSw>qeUV&;hj~M zbr03l7Q4mf5mPehk+={fOEY{mNVaF!i&3$Vg%qOHvei=ETRnP0>}VtK{11P#tL2GF z>!dB>&^yVeAqg$lE8?-*sy%o$H^gs){s~_lPR_+C8h&^Q|a@Mu-{Cby!=iRXODyvyE`E)p?w*MRbL^pN9>h z2~^Ph;*2PL1Dgu??_Z1z*Ezo~(O@L_$i>Rjn!JC2yeoTD*PhwT$Hn!Z`h8=DszXbB z(ylUtb=+k@Tj*rQ(~eLuiONi|ep)sqzFp;vOD{bJ%-UtQNlFCqHeDipFPMi(WsG?<){(kDytFuGJn0qk{%_`SZZs>l0GPH%+Ws0 zrKt~UeSgAPxA3$&OOunVs@r<3*+PE(Q{<4L=3snL!fPSFOk6@_hic&>SR$nqjee=u z{UkTy_%{9>(`n2K-=1OH)Nab6O{~PKSEOipAg9$dE(Jq}$l(_aW$653!4gidA3;v< zO}$Kh!HIL+a4ZdN9CP}!9ipHO_X(+SI6R!7M0E_n`>o0VvPfKG#GM}@H$Coc-WCfM@m-Pd& zY3%&Vw7NnrFEKJGt>b!=Mo@oQvrC7%s{!t;Y#IZ+8ILB-HbbU=#qkSRKM^y(v2Ev3 zRZVupZ^7CVfqE-~R}2^()8_pM(o{3eXx!{CJGQh|7eCc6`&h&8lwH16>%U_>KO%uV z#hSJ3EvkZE=6H7qv^`zuHuw?1X9*g?cNnyK7ZmfgATynx`g?r|S1Y3)yDba6pDDA} z>G!!6^kf$WNgxBJ7^2Q=7(e?_1ih zO+HRyA2CI=J^mv3_C?FHn3jE`2KT#HAgXoLX}*NXG$oErHZ_G+G=>+&2AGXm& zfX-ZsaW`}l@1j%N{MPwC%9GO|W1Fzbv2;vIVkk+{6VY`X=?AG&keBj1y;A#UqML0A zmwEL2SOMX$9~z*Zu8(}Iggzz{Aq3gvc9VU&KC#!0mS!>jy@uY)P?{L*ZDw;CCJP`V z{4_iK%z}L~Jz20n`xUSrrt2E={ndw`o|{!rnhTwu-zNK3=y6@t}Se>ntKJL9J z%o$NXuDoT{cD-WV7voZWoArA?w|}zBwzJj_OCfdtz@&_+qvweT%^+B`VjeHxE$e_l zuL$1mxCqj}QTzO-pr069=&jCT?IhnutBz=E91U5A)8&TL2_ zLA@S;Jn!5wf;C9J;W+6KB5~=wcl^AK3$LAv9C1UVgjz#V37-o*a%H8QPk-}R)d6*KRK$>u{+oP}r}cYMAT1P~0yP|G*b;* z`gy8*avUGPrHACk8`rQ@+X3c=rJ?fEtkf13CTcGgi}ub8hxpvJoivLXfx#>93QdB$oIQ0S*k{@HCN{H@6~0ITZo9v z34K31#BKnC%P@3UwC-ajsOT12d*7(gq1g|p!$pW=OCPT#|HGi#o+Irf{IL^48xd1a zhalcvOQ^^r*ZXUE)+YxXuj)0FSR{YT9%4T9A2@*@OGwb^=^cH3FsY^=pO;nDtazMx zjeL$EryX4UMdfxHx9bQuLgsvZCj{fo&@tuHcO<+1tkh~MakqQ^oOt&nw*6zlc{y{) zt3iyZr~O)p-1&TVVYZAXA6q(BWL3ATXpv9o>^{PeVCCjMBeS#NxfeqLh=kW%fUaJm z>p1DAymy}rZMeL~1F6Me>wa78Tn1J$ICk~P)t;aEsJT_9BJS9Ns48ppYwg(p$;$I} z{VE=t=7wJ*Uv`vb$+SNv{@El)5JXVLd1Ye8y=po8q?AKnUn@o@5h7DMnwLTwEfLk* zl}Qyhk`7thcXRP}Y^r66B(g82Tz1 zqoGAtMJsGEdec<=G{PE|TKXrOR_vrwQc7=VW3Ek#Z3w)I?4eG`mS8{~)q^;!6TIWQ z>50+PSVu(%ANjU<0Qy){4y{Rz_dPJn+7^!;mdhH3{tk0z5*PNrRgQ0U1lo{Ox; zH(agcH6X49&a0$It6HUjKCM*O!+HxpWD=ilcOM!^?%4T-al9A5V0=#Zx06CYf}HKR8+@ zBtlvp3A6vDC$%IHtI>W+Z88+CaXZW@cx2m7&mF@Yrd;{!V@*pD=zCGl%dFC)UjF>t z)Do&WNr@+Y9KlHY_SZ&?sBI}c7Mw7!Y6L1veP}KN>JbY-k*qxiovNz$OG*=cxbr0OC2SX**dl zqQwDU!!^{cW*V7queF{ObVQYKH|hk9yi-?QT#Kz9h`AW5c?QtW={ZD-A~vykpFcVd zTQuM$CYv6$I)O2i!&!AeDNA=q9C*bixQCGL!VHtvOgemTR!dZ6%93$_$G2ZU)>n~t zcdtt4=P$POZjASHul9{$nsCqqA1VzY2*|uAlJ+l*=M9aFSKmB__b6?$px7O%y$*;j zVv?X>c5*h3M6TspGVPY{HCV5+Z13;bPX6FpTS!hiuhIvr=?Hx*84oM1k0{V~`K$V8 z?{1XxZZ+O8&su?KMb*oYuyz$akAH$NOW<|Of+cX{&VAH{;=78p+-w0uZSheHRPa*D zvwpOII`)UB;l1fE_9_UPwN-47{U>&GMVG5SiZH5ZdItg}B{@UbVWrAA{{2L9WJX?x zk-*KY_)15y{-+gBTU0hevfI;6krksvvq*3QnG7`EdpU>N+geMYMJ#uF^MvMJ7{4YL z9GteZGjA%)~%TXOz+*lGJ~ScXVZA3hHn>w(gAW%VLGEa zgM(7PTdcPjTeI$0P$cMKYRuVvkC^#eH>*zlT`0l%uMXt(H{}1UIAMJ3!VXxz7@)82 z7x6@Ov9NNTh_RLL^QI!YVlpsesE|Tyr!xdWzs6GiO^-d z@;M*h$xM`3c1wIzaRiih9J|bn`!Ee3aCcqRvkIUhvHwqc%-M>@f`#~Vb@@uydAc=r zl)0GF0-*}Jt)*D?R~y0~UsE5&AumrHn0**vKiUTv43zoJDp>yK6WQLG*J_ZVcVCZI zV2dNiY3La}>B{D2MINra+KY31wa2BbLNqt}OrtoIXCiy?%u4%26_65BF-2Xv8%(jH zO!8B8||#h~H!;n_D7x(h+7 zQdGhJe9)5T>Jxn()#~+1FDF5s-Z+-d!N9wai>jB-8AzlYZBSNkNPP|sbEEbwbRRhX)~$}{z*(0hyBm%(_{oTf}1;k%PiL&VPru4oHzB&ah6))^ggkQgxRP|26allu10pB5z}C|NLgg zHF_7^4*epy)S!ihFE>w5mF!+F7d*a^znw_o{I7=3ETA`JV|r>wm=37ynpIa zt}P<+2@GG_%+71ymg=A^m`x<<(YSmyx&kd`KFOQQyg94i_f$#}28k+0vcQtL9kUpI z#npwk_kI$&v#1IE&X^(ZK`UjKK`tYp@OR^YXIsw|!TPaddR%3sroKkmWAiYqqQ6gE zFq!-6nw>9&Izi8Pu(6g;obf2GbF!ZwLwG(kJ0Rn1-9PvVr@F(I_=>ro6>G9IPc7j1b4%S(j{G1G+Youv<_#K`LqBaHm8@p zBh%hV(SFF$%LR}jOSI_NuRq*$>G0eQSq3pE!K9Gc9W`m9x861(_FeuM+@)u(!iH%Q z>oonNJ9vU#mvv1Y^aJ55uZST-YisZ6&>6@X1Yq;Qkn5rfJ_+;hHTthg&(=x8Mnlrd zx(2gwO_G!3bb{mmn;X0r=?B_YHxRYaUa_AiE@nhQfv%d{TIMbqUpshbS1Xbg_gmX8 zE7UgIQ=gf9JWe%|oy(IqW7)acF1Y4zxC<43mJY59)NP3f^*`!v>v!$eA@fWZJ4|_W z*u&$E_hvEbLmI>_27iaMv#Ulf3GO7=>z?LrV)$Z6B{)$c~yL0VuKge&~im}O^*B1StaSKD47=n zOP)nr`sma!O%&IT>Wse?vv>hhUyHy4&Xrio$50VTMx)2! zMxF%&lFtLdF4h=$?lAMh0P55ZOMvdA@X2*-ITo_)d75XLWWqplTDpFIv|(|Rt^(3m zD?Rt`yD<6b)7%Z{>m$ti^^sF2$ZIF>;yz7_vdSp_h0xknBDnT6jX0b#i3SOj-hXQG zAf|eVn~ZF3H{Rqr6T+JTJu4iBFDY(r78@zDM0iIRUNfRsb8GcViPxvLm(LU`?KHh* zlIg;ViPcgc)rlg-BT%pTT@3x|B6RP3ofXJO~V z1gl4L9q16ZJCmj>Ukn9AXdh`j{@CoZo2ZS0S0-sI&}W}@c#GcM@o?P7F!}XRN5txZ z@*|EDG2}9Q)QD%^uKiCql(L*e^xlrz?)yz$fS}w?p4wU1HO04y{28*9Sp&2rWKH4t z>!kSv)_#Y@v;mtnhOru1)fWHiua-FRe3qlA0Oc@x+hpO1Zr1H6$m-maV@oG>tiOMo zuEpS(lNZ+4RgssRQ?0tJATMUZF#Oi~p9`$WEHhEFYE>2Fhs$hJe-_Z=;g$n?>C!7F z{-&->4TKdtZFy~~&z9}Dx)rz{m?DpadF0~!yS|&-!p@n)T18P2cnaRaE?$C&X+J;gNgM1(cD#MCjSjQu^e_H$J-&hI z;Jgtl6)5IY{AI&7lNScycF3l-HlJ1sP3rwZc&Q{zdwQSdeyIX72v_kw37;%uNw0PHFR^CB=X3DfZE9;)*#Y)UD-Y+l7vqVO?iJZrSL`9J z6)GbsdYlhkWcTlxuCWM(?=;l%p%BnMT;kXhr`F z0<7z>bs-?AlKfimrCROT8#%TAlBsYPx!cJIm!**G^qBj4zi0!(->wcHajE-%5;BAC ze*@{$lKcn+7jLt94kHI97!?Jy zl${hWou%^2B1h3E!40WlJ7gr~T+(YzV+G3E0pDI~!(i3V^XpmB8zr_~Z%$H>U}XaC za!5TV+8UGln`e%yZL1T1`8r;3qYuJ&C3c@gQI{=XyT>c~a$t(Wi00 zFqo-`p8OkgyD8^Abk zBH)(6y4L1ntRUA_O6k=Ww8aHFJ-)=(d&PxMH-Ju&U!%~Fx(oty85+Pwbq4%JmS(U* zpbl)8(oO%R+V-%z44GG2VT_Dl!~bIIEx+3QqORSx6ll>>3WQ?C-QBfNyg-XXDDLj= zQi6MN0tJc$cXxLUPH=Y(aPoVe@xJGLIDbIKxRcy_@3q#v=G^b3!NqJUNEWA@V$G%P zSAnJ3qV8_201t1|fTO{XhUI}l!Xox7)HZ|KZb-gGC(a0tjP1a{d?)4|$LaF== zxosIb3~TFj5-js-Q%ao8tNEk$aUMm}QnxZC8D^P($)i&41b+)X<}23(+r@~_38+g| zD`K4D82gR0ds09zEwzW?f7=VqJ=>mPj26>}iV%+w6<<=m1cfiuQH7`sXU0r@rPze!X(mc$m-61pj|Wv~j{5EcFV@FN)`)r!bqYp}#|6tKQzrcQH8g?;~|qa#%2dG6Bw!&<%R1QkZgLP%+PT+u5i-`9w6|b^Q;d zj)zFeaGxO(hWkr1)*ekKwD1oan|r3POOZPIC)hw2&4ft(MzW$Be^r=5dV+C$CrOuT zB=I4hZxu^!p|`}6AT1H!uh|h1;&XpRy-b998be!^h`ApyXky%c$B@3W0PQ>#WeXP*Z77;=1a^eM@bHEBr%Q?mNAgD|PcRPCc!K6Un0YmO7cJ zkHoLD^oNU&nTu!l|&lc;fr zci*)j4s6%`^P^b67+I-mRIgSXS)Y-#<3M_5s{P%=O!hq)rEj-agkR9@0ix*AlGDN< z|2HT+-4m~+7Jj_PBb%FwUg7;&9(Zy?Deb=)uxA@dtUFj8s4367C19xTe{j`sQX|UU~e$;HVWfy)> zRRxNiIob^4!@>!>=E;Nx!WF=DmtQI=9M#Tm7%d>~opj}ETQ+&Q%;$n$Bka=tGSjHO zQs1TDLT8YOc+*=AYG8!K9OAZA^{;po@7E=%s9_U998jZl*CBJsbkDP@sQX}MquE#K z<35kNUtdO~Bpl7UIF_;`Uj-IP+3u&wxt(i`_Y5v9C`HaYaL{FirN?>D+7`kohmIC-iu~Vuyqv+@wwRVD9*$78 zqx2}{G;ir1!>LdUmRkBRS$6DzB;0=nUWLF0JJY`Wjwk#uuc$0|hSSY+oH&CE%t}j4 zdxM$aZ{yUv`~}>Oi5ak-4!?i+ruWl}<_KwExg)G>p=BJPv|{1%ql>2M;R$Hy&L|2* z%a7u?@Tf7QXO{Wp+hj5m5u)6j%8_+IET%G~(DCUY*#h?uR`a zQRjO+C9UGV8p9y@rE^9L9&bul*NY;z1;FCZQBtGsK9JCwn1Q0UmGHfJVLoX~s>XC- zrlJZ=Bi~SgBN4>swUuwoa+MRDg&=Fp3qd?}7L4xmse0)x!0-I76!@(hv&$FfJUHFi zw-FQ^`1~AMeZSM=470EbLf?J-4)L)uefB;=Wx30rw^KnTqEDczrc{j?LMfvhkoqRX zJCO3b@GevZwS{pal=X=oC(qu4ag6%tR}a?CU80S|swlh2f+6Xik)-h$qdq#jC=wM0 zlom92+ljKm1v2;-V|whSn1ZO()nvX}a}DCqc%v~O_bu+v9ZGW#@q z&iQsKz|cnPe~BtgQFTw;y-6qS#8i8PVD&R1ishf5(#*Ev{NR$`nI2|jQjS)PJr6t| zXmsjaI>S$Is)lMHcnUcd%vRrY)5F<|56gPL$-QvP>iu-qW8|~rmvL$h4*9kjhkj|Q zucAc9@=!tIqu37%acxPXFna1U&C4({P9m&igjJqXiwKT?2QGJ;PbhF|bqA9C|fl*9I!{6GQq(2azha$JkoCV>bv2vabp~YPsG-^>>I!BXa|b7 zWX9dj66t<63QuRGTm61hsilq}-3r`85Y|oM_)_?Wq=vDpnB+0mAQcGPCDgS?jLHoe zg@_uOpPwJ~5n$W%{lS_(jJ`-QL*Sh~ANGx9{mcH(`3kRKTgbG4?+V1J>~W-2Et`oR z;tdJ9dKM?XjqsB0w_jenNqy@@k;Fdx)bi8)6Fm(f!3DhgcK`$?KMF z^T(C39m?M(LW558jMeTUo05567F3+2(1?b=E6kdnmk-GUnnYp*hZsjDTRqMaK z9fW_gcjDCVO@~I_>}k|9Q_7h~H`lzMcdXi1Q)@If;u62tvzh_;Zw8avQ~jT^cEIi7J#*vY0rGwieQLI&G9%}(E% zuD6OH;p(t4z$SV94h`ey3g3>PTN|a}6SBh}IbiFH##;k%gL%FW*@=C5=pcd-pQrB6 z=42R&UIuUSVrWQR9U&jvkY?xS25QvLaV6p3PuSzDT~vwH?qyyR>vOQ|3zItHHg0W3 z9+OI^mqL&!U*X2oStH@=sHCW?0sF9g3V~a6t_GFdTIn}5DYtygh}-al#zK5NZ{DdU zEn#H3jv}!o(g!ZWP#GRnac zMILEw^w>6!D1wON-u^R47{|Mda7x!H&FrYS^rrEnka=ZIMI}2xVET_9jILW;SbDrS zE&ZN^H`FZqz4&;Fc^$nt3!P@xZ@ITLGpmH)5p0iTYW_O+4;sTLya|UF&%bb}oXj^z zYbz@%ds|Wo`=y&5wy{2Y<0U#c)ocyeOYT}bLiz}#!?xxzb(2an*1f!oD$hO6^?kKV z5Bb?X;$->!lW9~9*rGo=(Q8?dqbxLtRpxbyV+qgS`Ll)fXi=e{LD1@hRH*cSZRyUUl7_jgb<_8r z2^sR7Nb4(oy7*I!(B(x6h2`x~R$-hSfswY`f2AyUODF8$K)j zo_V3fx(mQa^HnG0H_Y#?4MHaFelWFd8t`9ITAq!z1_6^mMy^j=uPoAe96P_=XQ*Q5 z9Ef4~5RviC5HfYdGOu1iug8A5{?v7hTf6d1z;p<~v6-tO5j9A(j>Kf%5of-X%s&1s z$z%-Vd5{r;mfLBg$JAHYbzhTIR`4c376UHJj8(LE(wmU1-wY4NEIHpiot01xm8IVO zB53*;7iUSh$*piA>(_>L;4qD)!Sfief6!xm#VuTkv1iHEe;qAjm~uxw9u$(s-O96jwkm7L(ZH@UI23XY6o(qnf36Db*GHo;7j=i zcm2Wle~%K0b_|4rdL0^&tdS5PYtKZY65&O~naJxbAkCS~1G{?O zDFXlt;|i7U;NH7i#OM^EP3%^g?TvPL`Jh12q@VS71${T?Vr*g5>19+nS`LKbU#1WTltZ@CIV@* z?7K)ai(b0a`qVh+0ypZB?k{EZtR*qy-n`{g@7DO=IPVLBvr>yFvv|Hi>GuyIpFQc6 zIzuN(I0Ho+4I+uR0=BF^+2AWw6@EN}IGJFbeg0X2!=purt$U3h=QHXz(oTd5+wvGV z%^o|)MK90b$p+4LGHEf<{+}#>XjJR#N12~>U$N-^7+DyagM({XU!ISy3a-ql!vJA* zwpBX@rkZ}%t4W~CHC~zCo^tA-OwsPYb-U8L$$$}k&s=hbnH_c6E_y44dU#HuvYUdq zro3Apq?zwI(2-_GY!#{`%;}%mYx)qs25YP%!uBx_1ChGw=HXQSyT-ncMLO2+d{;fO zVkk8sUo-g7@pN@z%P+J{gz z4RjePEyi9CW`j_^^=D(ueew6ea>&Nf+wGi0b{j(`7;t__s`<5#xHDRqx<~Xgc$hdK z0-3VDv6cnUJ5GYnjr@gx^Ox=cZG(Z-8-Xfi3DeBu#r1EGS#{hl-z~KVXY-uNV9KNbokWqda{nc1!mIAnk%$5&ZdQ8c(m|ZQiZi=4bM01;Otk zZCzo_4=>x%v|DAoeM20KuKms~SrVcD`Id$57cM!5k2r@Y$kOf5zV1&x^O_=H+&e}K z?-$8F?sd`8uog{4X^fYEk#I&3LV4h0&>~JkhD}cFBBjEM{7r0r1U-+FEr^4a=on&A z8LYQ1vQCdRJ~*Fmp&~>TK>5x(p0qDEWpH2nz)IV5d`xAW{IaE2O|Pq;ug_u{?VTmD z(VaOO#bzpH0a!H#nbO6eGb+O-o|u|cMa0jRwVxz?>y2z#DoGT}U<|2BqN1@`Ok3<# z?ppz9nkc%k_YEbA$W>u5mJU%gz|kzW*|^-9m+14I1 zw$GAYucZR<8b3FDD>&tq0Q_ z7QnB!x&#xprI<;WuC_guYl_mg2A1v6G3(6(NepU!-2;YVlt*x)zRPbIZ{CL>Ql1A- z$dTZ@=IzKbvL*9{^JDjijh;&7!4`ATdKSQBEb$%xYfGWM@k?%uEFJ@PM}xZ}R9h?W z&CoBvzT&P;=jN_=(zc6fXh}6@M0oGJ-zt<1R_BG#mLE%EqY5y50*RUUeXabGsqBh2 zOaAZ2$e0j9(!{CzPtVdl)tIvWOyll09u+h6|8=96`9hu_F}17$e6}dbGJ(EMuzDT# z)6k3HQ#}SzGMBc}iXBzQBa`MVBo!;r4fnI$1@B3dOqLc~jdoDfB7v!ArY~_SO2qS(n%qMrczy zQlIq}@N~~@$QU#Kkj}Uc)rHrxvS+YE`Hgx{;JvFL8QqzW=p^@o$I+ITF$;CBTf3mz zOD-cR(M=;?SLDdR1O6hXFELJhNcXE{2qu-2S8hj1@bUPDgUEJQVIs0x??J2k)xkT+ zM3jNmG)hEx_|J>;zBN<>aqWiP-F$dj0GvL1xD{9Gv}6FYv5F%oNrzu70xpK6R;UG@YTe zsa7MXto4h^T`4$?*f>C9Cab^`q}WH>x80+n6xIt7CQq!f_o1eJCnxCHgbH8LhYe`j zD@NT8j0$byNmtbRD7{K{uK=*f&M1zQ%Y8G(J9Zvl8^n`%jHT<=HCV`o7%%?m!i59#?HB>%-cKTs`M~F+C(PG~Y%(c3PvOm+ zxc5*YgC2L#R}vGQED$<~`Fa190W!T4Xf>Kf)Q8=Z*xY|ai@F}97^TZLTZw zu%ZY(HkF=fl8??N6d=0t2ccT;+L+~FFFIoJ=}bgBHB$A78uQ!_NbUK1vHdX2z3_wpbFY%*?m+>T7qz*A}}ZpAcudK1go920o~DMUIm}7XW0(0~>Wk zQA`slQTPu8@)kHqm<|IWHVY)UalP4m^LPB{P0D z)za;l##+!IW&hn-xsjQ4Derrbp?J^r_4DBKTn>SNi?NIYcboZwSa8Dg?HutJzN}4S`12SX_EANx=wtlhlH)d`u$0`H z4q(LAZ!{)S-4uUDU58KQ%sgKROTNt~`$Wa~j3;QG&cgCtB&hDcjJK|PBkHUtxzEw{ zNLL2S;h6$gzr^W{YF_J9PpoCoF6=`Tm)7lw_ub_LUp}nX8U#bQURl3|iZw}ZwYYM` z4d!qvaA?mz^_L{*QRlz!$mRupjiIHw3T!Pikn&!X!@(%+K72hkOl@RTO8Fks*)(Q(6Lr*f{ae<;}Y3o_ok7)t8Lb!Pu3^>+4Ne$#HK&S@5?&+M}c-y(D9%Qz~=0 zXJQ$LT$xFCbdehIi`uP`=fnPIG`cJVlbF=FSBt2SYX1j%)%Rj9$WnjxPVd8?5}(^i zhm{JW@@AenWlsbSpeyqW_D52Xl|bUDgEu9v(8A$|51c%-Wiq>oN9%DG(6=Gv0y~Zj z6N#-f2sZo>(;RNq@H-qCoRIXeI0G~&8kqQ;K#}?f@@H7 zQ+9CO#Gijp4(5sC^?(91e{^=^S<~Tf?53_#~{YOM3%XWjCFkh2jakKO(a=b z11IxGbYDb_)`%IcM%%MTxKFJe=Y%OA-1f*3-{(E&1hBKn#-@V%o00gGQpBg6`fZgY zp$}4mY;=H8a}5AVu2vrhPqQ*oaD8KoGi+1d)t( zdl`X!g%K>^Z+l=InAZ3dT#NwjFN&ZZd+)U#jhMW#pso9o_&4G^sG@LHLOyH#kFwPS z-i{wkRa=T)2W!ji<&pSUPo8{Un_sYP*qbgp*3I-wJG@C^ST|}}IAMiA^$|S&5d;6u zAH8ov9}*nsDN$Nx)Fi^nScz>qdIMGXH>m6qxF0%GRPF?dj}`51!@V{mjIseY7G^Eh z8n4FV(vYmLD5IZU%aam2)?cSDp-UcT>wYT4hbsrWlA7P1t4evkEHsjmTvWr*cH%}K zR*`zfDTK+O6v@W|tOxs&vc$(n(@MgRPm2LaG?`^n%<>@qHeZV(*mq^E{(`m}#)hdk zJc>ZiH&(17kg?Cgi+@q6VbfzG)b=%aY1)4<##Sm9+l~+}%{s09DJ$*>`|#jk8O=Xb{pp7q!3%D{f^ zNn>VyJVimEjajzJLScSew9ajSTj}Y2Qq6TJR&6bQf7YE$(14ZhkNgfHU$s$pp|vm) z9ub3jnN+KV@9pQiYaAV9IY?da;K9@3Gr{T7)%aLbkml_*Ed}4OVsn2I}S`&gS4zpxzYn?(F!oQuof%&3S?)$yG`OMbiE{oRWPXLlh+PIIVy0#m=);|pI2IOBSWO=JjS;p;BggDXgp1#k^J?OTVJJLZ1eTAlJb*H z&AFdv6t#e%b8HPAjj52c-#!pB0C83#Yq})6=mC!pW;l&gkjw{6?Kex@HLZx)@l{tKR<$R9YxeY2Fu-4o_T1LLqb4V(SvH zPAUA?$&c&wQRsUI!Pxx=i0^urv;|8<7&16&3QsyZBeAs_7WCIfAk4LB(m*6gHgkmj zct+CpB&M~N!LQ!b{SVUVH0Ps4*|UR{;6{-XTm6yL3k@2yDB^ezc*tpE8Jp?3Y30K! zcwW)0{FCk94rD!?6}Go!_#_;^6|WfUc=~{_8`X}fhMHlsB?qdw!A}tA8k2XZU}b~dR=-CZTAbxz~%7SN(Z~!Kcc~m3@LTpL&=(MSRA}QOC!iI+(zksMy9bcbjiO zjLufXgKedicPF>b`MsU#p-R(b57Te6Uv~A9@)!jcHLz2CKTN{0Kra-80f>9Idj4>< zV}=Wf2@o}l0Qx6y%KP3Jpeh|N0^h@1N&A0V*~8g+1LH;Km5Un7zsGl*+`_PrD$O^^ zmm`VGFIz8p!8IFs%6>Lg*I_}Kn%{Da~4M zt^RQo$-(nM*@CIb+k3Qfeo2s~aLPDuS zpFS}4y>526vlHv9<5XYR zu06sXJ+M%}S|Ubs*)TNnbiMxKTY_o3b|2*d>dW@LXdat>8jl1<0y*?y33RQWN zK@(KCW7zqw{lr{{>L)Tf-y-Wm6cAI~L=-2paB5gl3x%g>bp@Z^=1Ix6+Se@Li}~U?5p)w zjLjVm5}}Rt9RvK0J0Df^%Z0r3XCe)ztn}<*mTBXj$thM$OxftV6QR zhvAy(5;YhOfEQY9gq4VcMB)%@JDsc?z7xl_I7sxLZK?V{M(X-MLW11uF-Y#=D8%S5 z>)TNnYRW6QJH?|1tvP_Zks4;=oGOowo2omrw!@Rd+bec~+O#}6)e`B#?|PcQ1s%Rl zufFj^Jd}}IAUQNsh5KO*!Ackx>HjhY*X(xeW{a$o6CezL>s-1&ipp64TEZVRh0iV* zV#Y~sK29v$Nvg~T;sl5VZ_7dAyd<(c2rxVV$myMu*^}<|%qL8LJwa(5H^jI$N z3Q9NR^pAt-;J3v({uMdYTiF_m=Xr5Xuf=}xF*R<{R2-y$L zQ`K3=sEx=>!zrKA@(S4SbQ$~7xEFUyn1u{p+`03=HcPeeY|Yz;alBAC4em5UY@5iA zed>DOn5IbbBqoKiFhk&vF89;aw}azY0^?{Bz%@I)#p}m{JIDWZH+gnifNeCAXSq-L zchbH-yPpeRlu)wH<_#{Xd;f_IoUF?hl{jg;$I)`Hyx;QAwqqfC8R@#1+S9nUEcDtc zdK&+{@BfuV#U(1(&xHELb**T z?>7MTIi@iYVvY9w%JaPU&Ep)%I*iEws%2~tX_@i9xC3UzIYqbYD~aAPG|$hlmM`l- zYP3rsXKI|MPXle!?>6h))$OwPQML3cP`hXQ37k7bmcJQu-rI=FFDr_L*T;kRalB zsY9=-woTiWRdr`|9TGuh1ul{h&q6VHTridS7}G9|=LBzee?-3+R*v-FlO;-`^u8 z(;m5xGnz|ewG)&3bo4)l)56anFfbWxn(RTh;ya6|qFU&yn_6Z%frbI~{_h4*LlH$9 zR4G9&?oihu9uTJoOGg|yxx?Z4^1;YRgX%e~v)?@qmw3xX{z_xJMlv+@ew4`0K4rCy7YQ+^vqVfrDSfF& z@w1%aotx^VbVE66LA(1as)y_&fc~WN+xDg~6 zkPQP;NY?ABU_Ed1L`^Y@wGN?b5M}z@(rHPx+lEFVQ`WTKme>t6>3E}zwnRZs zcpW<-FI1P_=1Prc8$l-8trve0Yqw3cZdyQQ&{VM8R)abq9Pz0X&7GI6yONv4G}G$@ z%)Qv9lw^{E#;63KetkAmLGPaiz5f#fM(i}okn(hlnvA61mF~0o;Yn^Gp=yJNaqwO1 z1FdblD2(U+Cb>V|@)0R?kXOc~y=%|%>GNtEkYC9(+X7F1mEX=lyzK;)#aWcMX*=tp5;2p6u>D4_E)%9F?KW*@FzC-4!AyOa{}9Xi0d5Jwi=0BUgbxzh zTq^t2G~%ERmM!S%ph**%W(A64MZ-|7p>%`jS`~X8${KKtD-@?eOTL5TGBh@g$P#V9 zRlwcYH|w}Fw^h}e38V}Q6($Z_dvlKHAenRC=914h`Hp_B94s8-gjx*tWGj2yg!Aj3 zJ3ggX;9!jmzfx+~<)31(2CIg^0>2)6V-|UD!$S<=y3=aA@U=8?BFRY;te-A1vWbtq z@LYOzJAUQMds>=4v2;(lGZ({U%WJ|{Q%txr7ObYn_V}2V=5@b&E>YFJ_GAUeGQYER z^_{cR%Jl_oU1Gm!F7Ve#*M3s8Xjye^tLRT>!&AKQYVdrCMT+;}ldSoqGqaq8qFM=VZEeYPBaMf+;xch`&1YwAL_rJXUj)#^wZ=dC$67Q)jxv8t{v zpW9-)1kb2^L?EI_boQ~6(el!Am>=5Gpae#022Wdu!lTi?4gaqcZS%j0JTSeobTmfj zRv^v~lWlrtp#1Xt)O9->V7tNYgJy;GS1tIoY>t3s4KW*U6)yK6=b=#$U@zRLyoOyA z3ieCKy4fU}TxAhi{dXTJe9@ACCe3+(CyiLMB+VpVY^c|LE?SBpF{PaQ3XfoNhYFBn z6TGqKlL+I@Xy_IEJXs21R=KmMxJ3O-RrUZ6Kp|O4&A2r%lP9dPdH%iKnT#v_$t15D zTb~KgDJ=kvD9q1%`iv?_*3mJB`mnr;IBZd`pShJ^d}^2)mjgM*-&8xL(l~^SK+%sF zSr2~-wm)6k-!Bvr@E|ZAvipTM>P!}V7G_tRtb^JIVaK2)nT6Ka2L`~C!S3|d5pH<` z?D}`|wuas^9Fd!~n{rH_t>q*;?bHvMENeNT-Fnfh;GEUkJhZeLYtC-r+cDwaydp6f zRZJQF%IK|ktZd}V&@fcE+Ez2tss8-{V$jp2%;fl$MVGtw|E98CG5U1ZU3igA+LcA6 zq;!ujkT+?1JdqAQdkw9Y29f#pP>JYnsu?v@(PIHDEuUG%;gk2d8gB<4Lcs zs=BOycaXIDwCiFqrT0EIuTM>8g_Lwx8f3Jd0n>>1IiM&6%1)o&rPc?4z1jg#B!DAW z$eOB@JL3~RCi6&Jo-fxdg4>4$&6=5I`CY#6gL}52hNb!qZ3` z>zsnt`SLgU;2EWMrE&>XIBqCMd>D5>lT3f;f+6nXMCcX~JV(fC+Bi!yT~o^sHM?EW zU^#eADFfaU`MB_lHz5}c6o`;2C7*&3_sba-Qe~K(RlfUV2GH$`yn9t}Er_aySBF?d z&3)*6FW)myGZEYBKN9{lg}1Jcy?9Cd^wA&tEW3fcI5kun9e8l# zw!OA~<~3M#CjPJ-c`P&dM0;$}dVUe)CPMz=?Oh|HAA88!tdaHbc!^@c2>8zzye+eP z^oicltL*+JVRF2kE-bqaTzXxgTAF53m$n7njUM9T3iZ7cdt_L^d*NehU(jNcVBZ>N zB{AY1j#g?ee^hJS81mXiPxqEPgA2B(AL?gaw6kc(^sA~ z7v{y?J=U*kGG3~EL60nDtrNaHg6c@I=LEB|*dUMM`{u50>FGROz?;I6Y3T5qZ{m zi$OD$pUkfECZV+`{1@Jgqyzy*&-Pr}TeeKHe*@U4E|ovvNx9AU0FbwtqubATag0w* zPY1%p1nRcJ-f`h#MW9~SQem|%V;I?K&&MJOi@X?65nN@UUrc^xkM7O_J8`kQKi`w) z_-#b`uF{};sZ5C;EYQ$8np!Wct659Ajgc^MM++rtUa{e?hlh}f60oUP3Zgwf4=C6V z6DyXbO)0_HV*~ko|51FjIXmNb(ivU8qH^V`5#$yH?Kd8w5MBLB}?S+HQdv#`}0$y z`b6X*fv|>qECNJ&IvSO)#Hhq*;J?;TU)v#h<%njoCGcMqs5kriYG*zS z?x_=0VNf%_kv^F0Rkpa%7dRXI35007;16rWWEnrp;f27jGjYKT-+R0cB6g`)!~S9N z*6urYnlE!U@)7oTaQh1YQ5URINJKm)f1XMq{-JPZY)X9ViZ7k*-W^tb1B}VYlcWvT z)57f~7|#uIv-f|}Rr}hnDBi$u?f8pGKMHuI#$!~p;vxW^B(4yi&0Qp{h^WDPw1j>K z;L$IQl9$3Rle4fu;>7pH&q6e5Wz-^-HbUE!<>NH>^c#|La!|VBLg!Y|pq}+rC;wKLVTf@3rWN@DV9lsG%4@+LndrihYXJUG&Exue=TNy>31|wDQU%*Q z^`n9u46{yRggF_GX1-5B?kez@pvD<>_l2m&M69Vrw8>8&dKG36qPOB-EE<*oKA0^% zgdo*|lB^ieK447lkCm4DWs(APe22jkH_D}D_1IZE4|J0 z%vj)G9~Y&ln_iWqj&ti#BYxZ~cY{NbH}yv2ccu7~DM`XmdW*uX!+%oLZWlQnPa@Xm zQCdl^8@ZyC6{l5dB`weGuQ=&=N0L=5*}Ekesl|GFlIgFX?BqeKyYYO+%0M}2UT_(L znjiW0-`O+6kA!%=K>7Oh?kqf&hqUt^rFd=0iM72jgHxipqN-K<-UE5baV4#m-=l=X ztJ?eU-ywv10vtaIqR8^4m-|i}GFp4K&Qbr2IhFX!DjepKn)33UlxRSBq$tO3AdarsxJ=g~DrtuELo4ViwP^4z?4vS@ zuM$Xo@%JqMuU{-HXJ`BBYIGV5!nie^n8@4&Imw(ir2dC6OJqf}+1hen&>(AK@ zRf3dH8XTeCvze^=t7m`enUwuH-4+qv*D9yZz5N)E)r)H7$Jtq07c3n%&oeXq2Kek@ zx$ad`O|#D;N#UN+CFjY*;P9^ZWFhl-%iYCxnzH+(V+^EW_`;|iu18r^OM1yYuI+n_ z3=z5rob-EY?Pqm^XDF|3Jv@y2U0nPT^@Lqa$gG;(ufCjp=|>oGisai|FEs`LOJF8I zrh=a)+f-23^_R=Ci!8Ti*afxOliVb zQCuLW--L&?-WtuGZN10$x=ZZ4;J6AVKrxHbo%dXow8ECUg~rUGwd1VeD*~2_B*^aD z0PZ4+%H|(pj1J=OORt{ekqC$r9^4j9Qdm#@AU<8+uv>QmVeZ${#G9hG@1q8#=(ou6 z_`lsm{^SO{0nIREsV7)^mH!*)A_-#=(eLt#l-w!P?XvUI@i+9mJ1*&kA3l3=_vr<2Bq6}~XXA$o<4c@OF6DWildpM$A`V@>A!$$a(4GDm z-uGtt@-PaCq30^!8H+uXbSY2aF?)qpZkuh_MU0NqgBEE@@I%rn)7vXqjw9ivsdn&v zq-!*}enTaBh(qCJtG$6CzTbozr1I1 zqHhXU-&r#QdWCZDDJ+@P?{p`2!h58$(`aLVjbmqi zBV#p9CN9y_jabm5Bz@Ol+IqCL7;W8J((e~BXxCXS67qlUf*H*vmKyXz{*ts9%y=Yfc(JRlzMc*c_jEN-v}FeT zWFqNQ=C!nC4Mp52l9RjszQ`tW9Mht6?^vVNsqwaq%tX>}#qnI(Z`cu@-j)*+$Us&R zrf0RW-g@f0wi7s+vi9PXBJaDugi7`q=sHLFbT6LLdLuhrc9b&ezB&ef4~7Wr=55<( zJau2JtzAey8(6}zb=V%VV4`g;M;b3%9lc}?v8>M_8cZ=>3)Lxt%lAO$@y1PuBGf^N z8S!kk_->>VbD*r()7s50B_`FJ`!0@=`}&uim3;0qflrs)f-#~i*D38MgTssUPj`4bKYkL|8025JWjzTT z4;QcPB5zDk5@N4@m@*Y98E1Bk7WXa^CPH z+KMPil+_swrojl-_|m>%@Oh1@lXyy-c|7p*qL%*;J!0R{;^=ZNq=F+Ojceam0~m~$ z_l!kr72K=tlq+RET$X<8;)Yi8Vkf1-k$P!wTWyFmwfGHD2-O*87BkCyD-7QHp&Lt3 zhtj-}hEIfrzN?*ers4B_OxJVT*2R@KRpSTyR_XMORGaVGjVfVllEqBV`HhF=Vq!Z2 zyxvQzM<%ZvhjunE!bnBN$r={r0}Ft2!6nR&?+4@5aXK}*Ygt72en60y`9dT1VuW6q z{~eoU{5yhF&cB;g&NepPK#M^K%UAE^p&$r`xE}Zf7gS{8&WG;gwm=m<&SdR6oUyLQ znBy3~z;h?Yw83+K4BkPANi4V@Sn^D1zwE|T_q?hFbT&uoaWNDglX;RLRj!@LtbZa7 z&_kCVnLhaFMCT;_WQ4fF_PFrt1N>99vi?hkjPA2?8PkZ}KsaaJM-pzwwCm=UiQD&D zzkkr|D_8@rf%Ashq@((e5Yo}FH8y{bX;^e%-pixVcW8^9okiAKig8QwqgZ&R8FESg&CtTG<`*8gaFsYpN3V5aLLfwe!i7d3IfX95JiRk-BsxFJ%Cdk(#Y+kWas6qqP%^6_{`YO`%l(~x;2 zATrUq1#I@*J^sT*GV)Uz5Xm-+k=lexa?j$Tp=L z4%n>7iZ+`k%<9F}V}gH|RR>);j2Un~n4i7^zuKOEeu&@hn+wyv>NtVa0;L|qO4eH8 z?X2*zZ5E*W+P3n%uLr+fyF8gdX5t8k-rybbeCOyMDf)F5df!Qfhv~G#_oK+=|@RcIv@GM*fAb z01uCC6)d-F8iVoa-LN+}fK`?5LP}2KIH1zOVyNlGL@nabc_u;S*j^R5zC}|BX14De z(_T;aLvkYS4Cxt`=}*imabNjjBRMpOFDTs1Bc^lTEt@41=Ol8+W_r+kuxa{ov7h>V z2rNxXSef%RkL-c<-(35zkBg}8ztKv8d_@U9434{+ZR%=W!*~1_`Irhbj)J!l7}SsNG{K~PCuOf(bN>3xsmE<) zleg~dolG%e)##ZI6oT?wjD*2?mY#nL_^!^$O6=C-tHh% z?u0gs^%8Gh1_}Hf^ylwM^s%iFNX_8)M+IMX6@B@6N^I#)E@ma*c5VJ3+4INEm8}PZ z;TK!3gd;CQ9gbwAubI&Ol$ZOIz+6_L8Q@-H3d#vup69W-n0snw#V3rDsUaz-$2t$l z?ko2iF})Sd08Fr|piz@Po!-3bcuH{pEPqC%?-1>?{s-F~ZFQn|OkYVGACv)1S4H8N z9`<7z>I>v+_Qh9!zy32;G2~?~Fm7Uj7l5Xvj%24pa=yc*{|;B?IJCx~Cot-vue*4E zH~r<}-?rB&;suZL!x?`M)n@ycyojxa~J8C_hKX~Ei?<@-gCDhh}LjEHoJLnuhMl!^=> z0@B?n-O?jnLrSLt(%lV1cXxL;3=A;L-1&Tezq{7G|IS*hHN*S7=bUHnefD1Q9A3S> z)w}On@?~F%u5RexVI=8w<1;H5+aP+GVQCE8`;2+fo+pXj)ZC8l(sfs^bzEVCigu|O zs_M!xQ{A zAkvgMU3SqPH6gIC8a;l{@^}8<2*FMI-Kl`jpP`*|j~7<(M~eHyJ5N-GwQz&4_Hf2- zshkoOdu6^{eNtnmh-JoDJT3$#idsLXJ!dOt%MItzJd+wB-G@*3D2>x|qv6^fNL6*2 z3=1n^Bdx(NixM<@w=RBq``7ue(3NZNv>#aZT$zaDy=GWuqf>X>xBuyqZJH-!VCBCq z*tIkG-Q1AVVaa#x1G6Ac@ixW`l2gx)^f)J|iK={S8`(Y2L#M6z@va(ROvyg)*sCP! z6hKP5QG7kTN|u#j{Y9o+G1L<}myv{g!Ip+*Q^diVnP+hxP@N$b>E$TE?g?;!ObSjM&7bZx7sSQ{toSv zV^y*c`uIEL_j`G}2%*#DUChguLv?G1>Y%&OzxXhj@xy8KKBH=Bpw;J!x$QD&dsOdM zeXuvf#Yw;2v(pOgaXWE&(kODKeYy?+iJ0$himiV`Ggb>P!(9h;+xCTK!{$BPMz%jYwLPI|X}dd1sZ- zhO83c$h05SelLDIajI?W*{g7HuHXuj*0=NA6<*;H0m|esPtvi8crWjYjMGt~9L$%E zZjH2$8jor08k$j~XtUe&MAWV0biAEzwE}w3PTUfaVsP5^PG=tgo?Y-$d^d!hk8D02 z$hRmG;xrSpnf=$qOPth-iGB&>Dlqyk>$Jvq9#kYgw1D*1dl@wvL0a}ZwF+sIarAFK zM~NgqM}n%{iTannDjAjTYa=ekXL{D7oM8&&m<=B+p`NV38tfJNBu`DwF!t_aoQKm} zD!zt-EUGw{P(+wKlk`~MSEvf{U>pvl6nSHHC^jsU6P+I!C@(ut86r7fSDjMS{{DM+OD`ZF{+?eE32otH7+ z4$+ebzOfUj*0paQVLL?^YrJ{0otJhHk>XtGFO-j#>K}F|H|`g7lMNV(P8y(>we*x@ z_7}K7645f*_lco`-;=LrR`gPOIWZt7Xnpob#19rOzzkpxE(!px43YuHB!w?=;Qs_| zLTv>S)5fiqkBRJ?6H^t>P0ui(*RAncR3QhbzY zSS|G4bKB>sd?-;885FrP&3zMy*M#aqwhy%z6X|BVoFTcg+Otff$E+Cq}M^swL7K(RUAuQ9Rdt$)r)V7rN581@roHbBYkf1K%Alij@CGA zalT^T5boHL*AJ#fvfgEfQ zTq^VJEvn4(`r<(Rj+L)v>FO#<=j4{;ryjf8wM1fK3qm1L7PSny#MNItO+9VUK^)ng z02qOuQ{IU;t%u%&0LZmzcqP5DjGOL>5g+)zeEFR_X8Hy)0Y4$;&`&Y-1t>oo_ol zGkLZOjc=04bX$>Ez_z)z9s#{y6+M?MCiKgZ744f0gVWXY<{P;9tzYDgMVNmsI@R+-eC-HuMhRPLw;W zk6G`gIuOfgtYXWpZDWnG)WDe5nWCNEmf$O)ba0){T4-R|HJN{pii1*rrX;Aw6mo8w z$aV^uZ8|0f1EU9IP3mo@lV3pvE&oFJAYe8xBkhMQT5Y%ARhEV7Z@?v{oDEaW+p5F; zm8v`@PkL(vSDmA{I@Q0ojdK??az1>vc8LeiGKxs|DAcb#1Gwh=PVd4&&Bt@S5uXS7 zyaBnsW?_E;E1$^=BJZy+jmd02i_^cRIzH0o`tc;9E7t2TjQt&zYtDDyQ2g2Yj~yVd zxG8JX5WY=vh;k!m=Uy-_vd<(si1fzz@;scZa&zAl;7{jzX5#w=IAVGjVyxrKci9P~ z9iiI~oVo3M_Ultn_v86hpDBA}Bv+9g9GEcYCo@_-v*~mt?#2NNgm8mxuH+9xf*Fse z%72*~DtkDLUc1l{KgnaLFD!y%W=@(2Z*NTJoB6NvRr<{3z>qO53mHVUW>Ft|_;={c z{=%D;*mTe$xm0GSHR8803G2#q3NZCb?7HIaC^>d536)EIb2nevI|{N34bMC^YvX_oUEiF<U;<|U<>Vt;8lpvyoijzEeu!8a z!`K6GcyAqrt)d3EBg#;V+x$O~UE5WyXRe(_zCL`13O+YwN5s5!dLu|gDjYi-a=BNg z(|X?z)L6M$x#1{74$rcw&(DsFJ+RkliCx;(nbxBubHunVQ^egSFAi8&nwPr7RGouI zQCr*cnK$o(duE7Ax)Z9b6?dyAH`IQc6YCWA>4cuX;Hnec^iQ6F zhgo4>kXRGixeupB>4Y}xyoO*rD{9qE9%#TCNxlYeEZwE4{WtAALR4h98CJx17PQHC zOJI^LRi0F<-+UJrDj-QZrq5PpIE&4EJwsfkU#kMsZ$k*aDUCC3zY(F3RZHWe0@u7Z zP$K4YGizDc|IN1_DrRZ=w_NHs&3kUP7(0)`g#0i-iOuBF5{u=QZ0(&)#@=JSzAyqK zkA)82Pi_n|10WjLW1;!-(c-nW7;^(-;f0dGETWEJO4^Nrbg43}=`2^vt{9CZnhiZ$ z&{!mb$Fu#n>Nrc#hicV0&JxiCxi z8lE_MzMiZK?gn%%Q%7MLi`@uRtErBCHi&W>jI|%v%N&|C;G{ z7np|zDs0hTTW8!_*?(|7^-QI^Dl-w?y`eh=DA1vbf12A%9FCr3_?*4RWh18{HO*kH z-3dWhv~eZJ$f!1Zb?jl?o8-8$sGmXY7`Cf4ku<)V3(Z>U2pT(+vsv=Wck*3@NgqR* zj~wVuw>4vc#Hzs@r8vKZ%iLm|3hDD|AI6~ujX=O2hH9TeB#&TRc-e!iQ;N9m&5u;WvZencd<$R<|27UhRWCt?udWp1NY zVe*vMpzzx(@5e2)WNjX)^SxQC0xCfUrId49*^L@SEi+HTdqY6x#nr%A5Z%3Ls(^dQ zi{$a=O*Kf!l6YY0^=r*SenErhvNfBGU!bwP+U_aSKe$n(&&6c2X3i5c5YqP3%=W`? z)~6FXBSM;&=@5v)(zHP87H;wGtZ3q6e+GwkmDTcTB9butj_kXlk;P1+nvvgv(-4z1 ziY`M7x);A)Q6Zgexo_w7&b}%EQ)^lJwT}%aT_YrZCmpBQ-(zPVSIk@8fb3bI?U+0A zg&FD4JjkOegX;4W+v?t>vfW)yh>YJSZVq~hHdcRXRpdXIz%Gw6<)5nR`zRp=hdoA! zAj7$A$-9zYp=!2P>{8vT|LrPls87_Nc4ZJ{lOFP53Qn8l9p2mnTYfKJAj88{{WIlN zNMCLyc(Tp-;R%@7!G*@e<%PJRbC$7vAv<|N=1t46abny3VIBLb1MEox7m zE-Hw7y02vF3eJ_5MNnx$#HKdADgj`zx^MjvRkGyXgw>Ycymf3HVSC70gQu!!iN@jd z5dpcawUDSU3WU;E9jfn%o9u0QBzrZw=^V}Zb)~tjI_0}Cp4+XlJ8iL~z?M?3w1#Fw z#b6I;&vP30x2tOw56mCB0wq{(RU@y+qDsa8%H@Qu{l8uS`8! zJ&O9m!O^2q>*i}jZ54MFm!RhH9RP4OtWXi3R>);gpXrjYqVa|*T^5oei%#Cc2X>`$ zh-4fkem5T_8+3CVPt|(j_en;Xr}CpqWe{+%2Qe zk4U*gO};5NRD)9h_+&~bN`d?bET8nei&~oZXYq?N6z3+BLN5qZp*YU52-!Qyo}68SN-6^K>*$LjYLWwe zT<3`kJdmG8TmC>^!dNeVQyD-@*H=%@IY;4eP^l2 zQvD=rFp)q(qCZc|=@uow**ws=BUphc-n@VhyD6tOWpQt5>r5vPi zKv)QCBhG#>4Mb$z_aKHv*39+REzvG7RISB1@YRK2U%cjLK#y{JmNw3Ifr5P_nH`&# zla>$_{%M{k@4G%`m08Sc`kmPPmPGTlkSe&|+QDUOkF=*|TF5MQC+w<+8%6|=uGB8I z4UAAC$=KOKLEHm^$#jC&v3ppinhjaQ+m!<4yftH=C+OZ}mx&cdolk_(ikrO2szt|F zyMMVpet4Gl0mR8cZo0$uWayIP6080Lhk@)5zWXv(x=YbZbqjxNLK6g!zbVe#wj}Xv_#RoIlAuIgzg=>QDr8GR)}! zp0GzwGDBA|=mRs_V6VuDS?;vhCFOVFk+@-88=-5VH)8}1YKiVXVtY5^dScsd1$mR7 z@ISo%`e!5j0%7jmc~)dbV=r2Y@iDyoOKpjP?j|wbC$r7p^N6>r00O%Ux&1+QdQ2D^ zS$)FN5Y~JUMe}{K%Qj6B5ItH1cw1{7b z03yKW82`VzhDw($4~Y=_xNwa@#;f6f`wQ?!hzBlVGVC|t<-;cc*o!YaP!msZ?F&}g zF~5av?Ac)k2~IMwaiX^9^J79q}AkFdnG=y>j0B>r6JPRIkTP*AwkvrYm~H- zU$L8I0AIdfw46`yvG&!Gy}(k?mY)7GNQe_Er}3QsV%7~Vw7N6u4zt*iMjLcRc16B= zp330C;5tt(HGOw(6y!FkD!LV~Gd>`wUCW05LB^5cd8LucDhK&6sM79KYHf%) zjxPi#JTEH0xE9=ngWKYc0$@Bx@;G)gtTH((52IEEs;%<8++#$%1I%j&eUWP9v2%>a zem@j&{V9s|H@Dj+=W9_rDlxP;PF{pWV~ zBXf^h9XVNr($w+X1)e_B@f_wGq|(3qIT&x}kt%+5r+n(=3OR!Z3sn6M_rc6s{Sh7O zdT2)gIAvL@;sC<}muO$o`oz+Ze_fOQ(;lt!x_mTNfb82DN?y^U9$(27?1k6gLOH@V~S z2@$CKL2wlY#(?+oi~7i>AS??*+IlU>_bZA@C%DvlnPh121x;#{dzmsv48UlXXy!Bk zEvPoTY1cY)k#OiOFEHk7?kBZ?Y$I!%$uRF!wWpYn{C$wZb6T%0bvk49jB zY^KfzPGppA3h#wv{+Oxo6uPjUzHR>@%df7E0hc8<$XrRGe%VF`elg{9(I8++9M6d7 zVyseIzXI)fZ_ic4LW*GtMeon>zdedAwf91c)D0R`z%5VNvd<+(^ogk);MGbLjc z&u(A5sJW5rK1S}Ck{d}JWlnUv;=}l`L!=+q*J-p;Fs0!6#t!U36o=iOE_z1m_c zjwP@~G!_N&o)SITzX?H`L@};mvf>UK^bSdQHm6EmcNAx^khDs#IpSTQ%ud^io?m2% zvC~%9wTxRKlU)V3o*CkX-yq~tEAx7n(Gg@GIU0ooqTE?q2^ARWbGL*)MJHxaGY&`a~;=i->%((b~(GxVD^#-VNHkomJj|~EwabU*AgNXKK zgB;qS-DYQa^J*m^8|&mX)V>HCL~W|_<7w3g=->e7Xy+d4=BBkXO7~p##z<$bDJ%>hCvJ^VdH09Qz8T@Z2h<3d5ke>8 zyKu4E{i*u)Ik!{$k;mdyH_M?@+x%P2jVK04&IOD~6E0C4#F{Mv)vQ#(sW68rm8Di# zn6&V3F8vg-^4Vipmrz?Q{h(8+41{P^2JF1n8{@%g(XgBkBo?z?*X(`7ZsN!r5i00! zgfviE>H?G6t7v!I%Lf^2Rg#KWv+8fF&`?z_n2T|80l%zh8$aSeN$+R3zarJrV9n6z z7&~>S$j>q)HFZxyB+f6>V$vBIM<+O~ zT8N1#_xkXy(GA22GM#6)A5tSX6ZTjyitrw&0O`~~kn`poc5QbD+XIQnfeQzLwp(i3 z)s?7rpQojHptG5u&Bj9aVmG?B4m-(%tNQN$%r!@kKQL2I1+wW}d|~4Z?aD~u!g4jE z+M)aY11NV^JwE_Nxmf#Jck9>I2d8Qp(Cr%3GZuvV>o6vhO-4>b|2<=dzVZJ^suAwA zyrC~6`~clXoI5SEqKt@U*>BHdIc{V8o#!-jerIi{xe67$(y?kx0T@AoVxAGQ;D2+T zkLDBRqk1e8I{2GAFtr}*(yTM?yP!OwS`Rkp2bL&MX zZaV_G3KC{KkJ)+_Wz^RrOJ>xY^qKo?eHbg=t9)xOFu}%cES6W#uW0|*$6*ZvX_#s| zy@cmkt>bqf+G2RKt~+Qofq;hGQDVfkCUCfli6=TE(SAcZBV^5ZE^hzb)jDHC06{C| z`fHpq0~Rv=9L~8I1S6BB6Wv6lR9|v0kV!pg`*mA+w*!H^-uZJ#mZ6jytIK>j$Y1uo z9LQIq2){5$8ND7Y@UJP0bIJ?F=^A`%EEV?BC&zhOkE^iAAaTlKKwU_Ma9yQ+g#W5< zo_3`n{TP9<^Ff40rf_C@*EO+~c}y22BtBeKC61z@qe=Al*FI0LLH43FzN-*;QKF~M zU>*BUvE8)oD4)|C`45BcC;4BRc7QvHx~+4E__+Kw6z^;dF_H+IuXtt(_HG@j@I8hC zTq(B#`8KepDW1N4vacnE-@5dZ<)|sknfHrnjm@2CO~5JLp{L>KUBJ0-Ht6!+8_=U( zkA2$PJw^u2$$6#%sMWm_i{05GcZ!~3^$+Eu*pO4vtB1){8eBc~+~X6uz{$QdWCWGO zY$z6x?_ZB{UKq~XVZQa(NK9qva@P}99(CWZ<6L9Ce!bLe5FoB30#l_hjgTc<@;2(N zplF;uJLwi&>`<%|20=cBwpLhdkz?`UiWLN;1nXVmgC;)uDW@h$TT6SDW8$Q2B2erQ zcO!`~;vU^=B4P_9-`%vnMtpU+_>2D0W3a*e0O9N&U5I5Om^{m>ZS$~_=mnmbBOdtH z_)}Fvm4f6DN`eq4fTm&RIpBJLKz+KPH87p4$;&CO1C$8NZs{gSWY|nU?~vfPpMZ*J zOoT6#p38bpnFYnrIqyWL*LTd+Z5tB8mm*A4-&TcR|IN zI>*u6=N?jR)5pOCczH*Zcrw;-L?Wt)_9A+Ff~?j4w&D~rba93&@@TeY>3`*87j*Vs zI}LF{celRnd%b_LE4nL`!RGWiSIp8G6_xUlTRdk~WATE>=vsTzOwLV(jk*th6v4Qu ziuS*Fg!`mL>U4s?5*H_DI9-pXY8>H9E|`M8v|(@Q-^a`}G~ygR4ZaONF6f>2Ers^o z%!$lJ;#*yR*@Yk*NCxx~* zR0{>|8ASjO;zTgfNtOhHcgXpdrZ5Xs_~Xp4zTx|#Zq_rZ-F{7?vG!n6DD4!j$8v}D zbH~rmzlcy&j2pVu{S|CN!~sJ6eP`?hz;e`XCE4+_3! z_ncM+>TALNs0~AQ$eu?)&c7R8==$|;7r3X{p$mT!Oi|0Z2D?K%`J#FE z>-jU3^ig#Tk5A7(QLFV->GuV`y50cRd&{1+RN5vh65%v)+jdK$jkbPhk4Tb4LTyQi zMn43g97TV&))!QJx?SiTyb}IxkeWr^#TVdNQxHiTsvGOm7nDvZcqP5DuJ3T9V3-3L zU`%`;)wbrheR$%OxoyraNcaCaCS&33dKveziSHGf_p-Ke=^pmzF}$bX;N&Z(FC+Rt zLKj4W4HU$97ZH56GhYG<8JU!ZhCGK{zF;dqm`VU`YqL)wzbu~9>T-&xH0eT&h?uyE zGL0-y45VBP6>T2@;A_0>%OeV~Qe#9n>81;z+Rv2CxQ>D^_(i;dS%;Zz`kZKu?<|BT zB$4*gjI{CWWlb~U)ut(Pw;37+9^c!iXI|4t7%9Er{5QSO_zE&#T4%Sh%f~@ZWS+=? zBXrVXa}b|US%c1lbYVDjC+PJ6=B9`R!YJWA7t3;n-`zWFGKHs=nw!+t?Yw&XF&#a} z%x1oHMcO*XM&p{ZS*@{kNvCBz)1C8lk1Pd@?|79cd&zoP!(GCqhb#+`;=o%#s>=SW<`8UD))w3{|BmcSV`ok`L@%mWG5S&(X}LCS|WkpfmW zoKQpOBxHmpa>*a~bCBiq_o!Vpx$rP-GD7!Dy#^LSv3*xVO!$#&_f4&hmIWaByaQC% zRY3hCV!@F72vO4TZ88p(M*oZHfp+rQ&SfusMAAde4%?TOujo&yH*v4Mmj9`CWFrxq_kP&TYwf*x(^~o%Rh0Zy?@P3%s+8*wLKpY@^;I1#GPeta?mf$( zXzP`1V%6vetJ@+nJuRctqI;sbsCKyfQ(I3o?t67Tm+6a(RpjtC;Hc=I3+?xn59kLG zyBpxps}!&xI=DZ_nY6TvBwIDg==2;QCR%q3K0cXI)7w&MX?ICInWy}-!#r)zVdu{v ze3^H&F-G1IFT#~EfkA}yWtvvqukk?L7j6VU(nN7a7AVv%e7aKqTD3Mpoa&LslE{yd zY?zg2LUtR_s`0F@@lW~*a^F`q#xpwdRj=1yxEfw%T~XP`5z2pCv#}Ckc~4%$`zQ2O zvCA6(IKX*j#5v&W&#A}|ozXcCavH<6>*$API^t4mf3UL!mp658j%*@R@&+5!Z75Tx zJC~S^^*JlsN(;~W>AGJ9Us`0DWfAoe)K;KlQVWw~hA;+?t7C#A4`vX0?dE*`$eB}8 z{UwiEFAiF}_W%GGSR2JSt8|%vr-kc(#6@kQMNz1DM>=?^5A*^BPL z$c&@mte(agEk?Y=hh|1|DjUw~_0N&6i_F|};F`k)N%dBT<+|3-TeI8kgS}iAHIy%E z)}(UTcxki9e@m_KXzx|qB!6flr00n6+9JRF(HH-#<9@lAXLLYj+@WjE@ec*)cS!}m z@aYMM&;GdO%aXz|pA~$e;(H)&Wk-Rr1dABIOivr(ry2kqVZ$#F#%rIzYbSWvdl5(8 z*}yM+W_@>*oz|`B6@PLk_r1DTS>2QIJl@5Jz%De zV6FCP^%&>OH%IjGgN#C=9$<5d@E=&pn#RAG!ytcG zaO1Z4nmRiZ($PJrzIwlsJoTTP4fAX`q}IqF$Fm7`UdY=)q2pW054%FA@tm`Cnam3& zzPhpZQtn)deL(yw)#*O@$Lih{G9FOwOUaNGmBUCR~@XwY<0)TKw9Cqlq2a+D>ts0v=d5;$tQr-O$1a9`P42j zi>;Mz%Cw>q;oivW740*gYke7Qf=Rs!j7ZP)DF*R0iZsYI2+|xSd2OjrtXsns4lBAn zNA`-%V{`>Oj_bE%U1@m&h@E0hW$I?ko`GUd{J+(ZrowgAyl5&q-{HY;uN!HCnaZlR zOg5G2J&3m(qzl4q;+XB8ve)U$Oez!}s81|eKdX;f|7VlU87jwpP_Z2m$olVMjmeY# zt7HEwO55*!eXbGi9ayiKK4=E97_kRX^n*Y(fM|+)2X(0 z8m0>GO^mpeZn~b;e-acsPa0yZnj@KwU3`oTpaBlo1&Dfo)WmB1ldX+o5Ne~X`KtfMPE`MVm2yrS+6gGhE>S6tcI z`9PYvu%HadQLJKHMw+L z+7x4|^GUJglqEh%=x4aY4#ii0z_qer!V?M_jhSx->E*cvHlr9wTj?hoZS)hE7iTAE z%^`xf8%--N?)yAGUe2p^*PwDb-^1l+$ko+b-9dEoM%XsqwM%vH!t8j`jdP^a!#Ph| z`c>M!i;tn%_uVA(cDT#{iEdNM152>m%L26vwLvIeTIanPU@}5Y=)G-sp*(tvyzNgi zH6;i7Du4ZrEmWy1gl&_5mt+Qd?M3TRQkGLteY@bW^PbjWNF=f*;^NiJ>Q6!Tn9GOw zO?&6!^Av8;&+nGj0pN!x(mS>>^$zP1K^q}SiRPgbIez*0Fo~Lm2QppG%B^Pcge8i+ zy;fFNFUKl5PaLDs3}RW z1}}55SIzs-a)UxtP&`{VWLDE4T#W@(_LRa0ssp=eYy09*z3LUQ1d#qd(PrY9U0`#c zKNqaH;a^XPFHiv)bB5DuJ8UG^I}hH}7E($s)F2@JO`+E1~`C0&O3+7mp=5G9)U%f5;$yRrJrJ%p!1D_s5DT7_QuHl$+ zh6evqAY0xT;z<-}K5-cw7Q4%~@qua@%)*16=w!;0gp+>T0DMIK)_#xZ{jiMLi1cHiYdhm5Hd+;a=R39y1Nyo=esK zVULTy_SdIlQ#8%GVL}yt>Hkizm{0*GvJj{1e7U-T8S1< zk}^*XKd!+5*_Tmv`WOr}n8b@P3=gL?Y)lU zQ&U)9Lx)`bZwy0C5%RJ7m5Rh)n@Q<}r}mTA(IIln{K z@WaO3kSg(V8fC~exI6%HX`^;~aXvtMpUz(&%aDzmO zg^O_>&Fp~+$h55m-OUJn-+V^$ee$4<)Y|2&p8I_THd*sRhw`BTv1*CrVj5`lsxS?E z+40*jDPcmn??hgp2*}J&$Jnw$Baq1I2I3@!$RoYkB`G9K#WEj$G(|JZl|X zH$dTU*FEG?$~ZBK@*VapuI_Ah!A=IfAG};(hGJy>AxUXg;=BVDn3yA;@M@s+69YxV zmu;j_%YDcc=GKSxYx7tCmR1R#eY^_1YcyddgJ$?~{FWp}qqb3N;QV$ql2fTuSi($x zU0(c{nC5@XPxXJyj}BVOK#xxX`b5Y$^IWOHQ2gLr zRS6a0->{;Tmbd5%YfxPc#RKO(`DT|k4>}D#^w!J3F+9pzySLy7iVOG}+-p<3zJ&XM zaJ2bv^5vi_LV$EtLA${hSE^}1NPT`!p<7ZkJMGynw1jK6q7qG_?i8c2ZL-7)cHxZA zo2fhE>2(zrIX@M)r&_A%V)9%-Fzf%^kMDAwHba-Iu}_~%sR@fUF^J8bOJ8+5R8`WlJjupBE6in#@VkFtc?Tenx&Y^V#XDpH>~y>68@lI>7*S%ecM&&J1) zymG8IgH#X~ZID}8@5iTTHTAZf)F4k-^(xTa)gRv8-(iqNDNs8{wl{3~w6z&2;A0%0 ztWR&FWbpS_gY3s0oE+!FTZxfM4K4$?9g9H4V-`G)N~3l|iRbEC-{${UDJuR? zr7*yZ^yvn)`MYRE;h(ScG1D|=!2bwrxqeN72f2u5EHNh4T}@vIu)`@x%DpsnS$|9o zkvX^pjX zPgDz^#0n;@J)ryMMx-B09=7JX=teut$?*Z^+of*@5VX4uiRJuJrkG9GyVP_=>Aj|P zW8cMScW^G5`|n0X6}(1Gv&7mW{!8DmEy-0un|FeamG)uGPA>xyB4w~J`0(sVYOPZQ zE9$;KQ-}RFITJ1!kT`D?<3nMSL3eRPYyM`voPL@HSybvu>^ZIa{(1`-!4W_eeaI(x zzQ&;mhbi)rlz5N|j;(cjrCjIt(<0aDOSF^r7L71b)k0)smGy3GeDdO%6H5;hcCvaAw3o2p~EGjD=*2k2Lj{yN6Yg6KmB0tSjTWs{k zPrxI8d_<;p0fitP`IjVZg^PuqjHw3{?#1e>|34+67N-)&j?mg$(ITg3h!0VJCjG|u zwKyBku++J@jL8(XU-ptJ%s#Hw)C}iaAFT?xI>yoE^!w)+%v8VLA@CN&V=r2O<1>z7 z{#?4a%E3H>rqr@oO8S>^r1(mW$z==Nf*|$Omz5M8Vb!=|IQ1?+k zTE^zaB_%SfQ=!!>LSDhEzcnQ+jY?MNK?`o{*Ai+zKlj}=22oXO^)%)f7Y0T0K(-+* zfor|%WfYMiR2j1kRODUkauh4!1^P3rUKKYw6prqr$aimUx9vY8pEpO3Q@-arNzlz_ z!`HBuUau%9RkE=hb=}VzO4x%HW49!5ZFI2aHt)_|%ny^1o>ckMUk0;2WY1*h&wm_X zIN3e6kxIo4kX;=DIjWi|3ffj9{@y0}ybkokm9b?9mEQ){=16+bK3fgZ(rA*n8~o5! z&hMTpnP2hZ$w2ZG=e3WiYN~Mk3-A_^lD1p%!r&+pIm)qmaTe_9*=8{nY^pyW)gJ8f zRO7wUOZ2Vgs21&A#OmH{T-Ea8rI6?%HWts~n6TpFuNVw9o3k)|QpFR*4H+o)Yh&Qt zVu#$~DSQ9RUK{m#&MwML%AaVHG3!0A;}zH)Y6EI|n8hNm*zzUgCWVe@L}Hv~IbYVb zRJ-LKCZZTeso56J^6A1h#i+j9Pcw)-sOttEE^~SI1pZWwL}3}){Ny#&t3D2L>PZ&G zN`lG0$anf1<%*y4rdm)>+pgN+>f{H+N;0oV0r>$n>GX9$llf( z%ysL>Q$j7y!iS{Y;3QR+HwSA;g2|lb3;^)%VlyMVUJ5d;^HuDbVWP?>I<|M<2rEopZ<_n49pYGQr1e8+YfM345ZwT!P%VnI4X-mBmhSZS9+ZakwoC@45Vd z?_Fh_eKqW0M*ZVJf}0l`M6TX+FYvnxy!6O?)e8eHtQx{rH=Y-uh|Q1P%gtCn=ty%) zUm2;=4{Nea#)TQ{PInUiz5Ox7j5{@bV}{GgjC}iTSPSkhT0P8SiOw?z4Ax{wn|kC%e?inf;~HW`@0|Ac)|r;#k)lK(uI<}6n!bG z#8pB6FT1VKGe)9?si5tej&|gSA#2ov_%*S3h1V&@_n{CXdMqxEa=16AytOzIxFW(U=+#E{b%?-(wNSIQV0>A@!Y!1 zO0J-_ko`U);qgYL}Ih#8w*z-uOC(5cRnTZSE?RLDZWOJcZ(2tOG#DFFXO z74R!eL;5Pc0UN(%qnG+^lUEQRsKy2MtMiy(HhzPr^Pt1;_qrqF%?LH>5W1;7(Rycm}Vok+o zd8n&>GOTBGm4Wh?fYkJAdbl_)eo}_qc_Q%qERzxSw~kbR+K1*?tLQ1Qmo(BnyQnrY z4n@uS=(uq%_0w`46xX5AFngmScr)?A5M&PSiK3SLq?6-37ekWg!A1h|-l2Ny%^mRO z2vdPM`?d(s8n&G&Lb*Y$P>MZdNiT5z^>$_ji}snJgxXW;|4wGChs>xI8F1b|+%ta; zauM{K{A{d8-y`a9lMYB=j^2Di#w}QEidK!%%?qkoA`hACc=<9?QFgG>@#7*{uGl#? z=wCd_=aT(yA4zYtL!La`3lucjS&ddg49a_zxF!mG>t7D~_o*8jNN!S7pa)1gEAbrr zPFc@?gJ_t%mh8c^6J#ml6u-O_uk%{06Yprt$i+3(KeNHC55ikcbq)U;ysklK^wf_Z zC>JEr`}3$@RxRh~0&_7Zca3`!qFuIKC%Gru)5GwCJP-I^jGl$|oxJssnCu!#{m@@R$CwLzvYLwxY7=;C5U7a6 z2rlW*%)e^0=Ov5JDz*jHr2WCfy1fVuS9{kW5V6>zt=7rW!@9)x(gE!}B&K1h2>+0b z`D53Z@8LPQdIw1=sKXZ~Y!PUsFTYY{6(-hNw*1C{jioFzIHlghwY^4<2w`0GZl3JcxO`w!Edzk=PK|T z{EN2U+?=4zS`(`PxZ>8iDlIoe1~a4h@Q!dJy3U@)$_2wbDv-Mm)e7yLs^jmYJN?tr z>WBtj&ct#1+LTBBB`^yIh+!T)=Fw}t-j{TO^k)tQi3cI)Uut@-U*FDBvS}4RWysL500CdP_}mYl7-SX2v1nRG8Gk()M#h{HO`5vzAxIqioU#{z@vMr z(K>AYoa-5F49ZwS4RSCSy%#eef!)Nwj}afW_xH%UKEG}Uv+lerI(#%CS4iA@u5Ucm zc&VAWKIp^+F)iJI(RZ5d7Qeao??Y7|-4){!$3ybj1 zSB;5de7|lLcnIosQNzMJ?=!_+j=8Hzp;}ShzT!&m()zpN`h(wTKLh%fX3LrQWzPco zm%CVKZ7^GHAHX;ja%>n;!i|V$5YgtEsKo=9#nNOHiU$ZC2vxfLEicy0yJDz5qL`=( z0HfYzcalQmRCH?JP-k*FswsC9{KaX+p8k_RXPG31ljrzNykuX1W1H>2|`Atwm$HL`Q^lwxApq*$6 z(4Zn2wDIy){Yd7Ssz*20b(~*<(eRX-9)5}KS-Kf{wDb&6HriOFkn%mWTyJky< zo+qT{?@Yj}8P^LMbzSJ_oyOK#6%tXsZ@L*QBie@J`fTothpDr3>yO0C!#Gxan~sR> zt6qUP=FJEZSjt&Fuid6@5y{5w^et{G(U=CVyUXC+y+-G5xoKa+X?EA^%)TxDMHUwOp3==u~9 zv++NsQv9Mij}4=WvE=)YIvOO4Iyu+;sVTNg_*297=k@#`o2_fr!|aTg^L6UOGisZs zm}Yi->a}|j#9F3#RyCoSs2S^I4}CJ zz}(BUOPgw~h{2c0+zPZ_+2dugcUvZb*x8%423r)}1QGvUWWNZP%p!o;xC_Cm3xxW4 z>(1dfG-fyE*ODfi5#3p}1}2`}&k1|`zI%ZD+mipbBi7@JOz<+iI$ybTrkcOPXMo#hCwp41%MXtfh#}pMNH1($Gvyx_r%4H@gl-IP>{A2EMOmE(L zQl2*x;c5uiP96q~`?~NzRXNYrD#f2inJ7p^-@pQ1=u-VQqkXD0e#d{q>0qe2=>KM; zo8OL7#NDO+tA|-hKpjg(TKCTYc%gsx=eF-4{{pUOQ%I5os~4@tQ`Uxw0RoAI0jb*S z0Di5Cz@ihpYa)IIlr+XZigKl(;9R9F9PIN%N$l^->v52AKx+?fzgYjC;rslgG_ zttQBen<4roQibW=ENv4qkw$v4B8to#}ZQerdp`DM;NIaei-jVX9<=pKc(m&>=#+ zkE8_FSQ%ZT{4cau_A~XI2++$Fli7xhxhPr5NNFVX3guihEL)_hC&rld?yu+!_ZgzY zZAJ|bDTofqzgP>86OHsUb)RioX+hSDF77q7A;JfX^x!PX*QVlFnBnvVv|VU7t;*}z zpYCXsrE$J@X=v1Nq~E9Y4*7DoMajUYJ+iAT_nmg~yA-@Acll>Q@FOx?5Tv%WsfT+4 zxvF*myaXzE7~U2@4%}q{0L?e?*oue9-EfSc6N&a%2|sd5r5;Cb^={1PbVgEQZX~yU zF}}@ou!4~gHVPin^TJ^XXY|K2AKx=XXq-1ozxsq%Mc>AP`h*co`lC>HQPaKOnA*qv zxTzh#7%Da$0DIbpg7jR>$iwb^c1D&Ke!O0B`UBXa_9*t$Q$(jr@KqF-g;q9!3Ov!Y z54PaNSmJ)%{XZSj^T?07oJR6Vqj2JVwM@tYl*Mpdj>uG zwp66E*t|_#D^zY!I+T57R^Lv)c9pPVc?FzN`+p z1TkB}{TQtmpK)GLIT#jPcTK(BVU*fiB*2xS?QeVeGgz+xw&wwqTJyLy)RD$(=&r8~ z1XIriJuR;2eiHQXg>V8x(bI*B30kNIZM}{(L1=%`X>`0Q5&IY#LEsIh6q>+Tf=ueW zOmGczdJvnk{bsh_AXb3mR$V9Ln$85^{JYywzKoN`*}nbhQoYfRx{1Nry$G5P}58Z${>Mr%I&KLE5qOTT`z zP3_&|02)M&KbCF(96!f-vSY?$YdZ%pYS&FUKtrFue@1q}^a7v0TVr4{q zRqZ!^>$jFy+E-@&^7XG@{`xI%S>ExEcjO{(CD_;ZBd{NV%@NR>`J3PT=F6>aee31Z zZh!mbbL}giU-l(mvfTaiKR+e~opSctHS=(uBP))x6`LfwNTyi^LuCLJxU~g)$lL~A z^S{UIL8H=}%^}d~J@@Fdu8jpFt;BgSJNW9wnFQMEPox*XzF4Ff zQt=9q#WqeY1bGf-l>Ibo_?w+}1!BQT7`>iQ7mTc};{H*M99w-B6<2LluN+j6(Yj%8 z=|56^pZkxEkvPo03x~c-h>egcHe$?r*|C8=Q8{xS#1~jRhEuAC#o85C>6~tiQ;bg$ zRNTqdw_?K8W1}eYAU|LS`3qAyRNumH>EQox%bLG(>A7O)ii0x+KBI7F!RP{;w<;F9 z{G`{>AsL?#Qa*vGKEeuwFO#|i9_-lPdH)hh&j4S(69ZkJ=0&E+0PuAWrk9f8`37swvK14Me2| zjcgfs_=*dreB*^pGJVH*V9?IEUG`bO=2#ZnFhGfmd2CCBQCUHPGxq%M16jG-7x)luMxjvWcB+7%mhMku_~2ey#aZwpiH z)pN$qxY>^HE*xWteas$jdK~@5C#|)%&9Eykjz;YA&Ri;Q)B^_^J02A??i-M6XGj7JrcKl`>dSvwr?C3FO zpSe{(Y4MVzE{y!aq9-y zg^Nqs0~6bN{?r8{fx$vADP3Qwx4Lw21fwZh=@Wlz>&S9WJEp*fKRg)|jw(G5ceIV9 zD@;ts`1}bYIP%DDju=Gdyryv38shEHUg=(0NoLpiM~_u&{_a}+2^I_*zfpreBwM0@ zg^^CZ7Vj1gAK|M#>SSwUBSkrW zDz~+2wd2^qE%o4hm72_2jF-=AZB!uyyRbB8h>s|M@;6X zC@PgNG^xcVP4d@l(_>c+4oN(b_<*5}{O97dzlj@M>;5Yi@r8*x8ZxOev&vf7XD_Zl zcq=+SNNiQ+ae5GHj9HPrUf(|mUgnP)k2?7ipKRen-*JsUpSR~h`gp1(EdGpc2XLIW zyaJcoe*OgQz$6tV`6GhzBu8QV4@RCyAhAihx)CK0h-T>aL`b!bl75UYtUTj2a>#aA z*@^?pmQ8%vEB7uenAvBm7htxy(iT>2!O0e{K%qw}2QaFKe$Id+Qiv?UnCt?fBvpl9y`17mm2on_xAR9seC?m5K&a^Nk2KP($hb31>6*N&{AgYODIW zp3zm)OP;m&l_}BitTnP!e~?wCSBzA)iGLK9%GgJ2RPJaUb-}3jodivxDlh2ct94;; zpEl3w+FV>6?)IkCHzQQ_5B}&Fx$XnkXsQgQoP*8ZP^jt?un_hNHq$mtC|c9Zu5L;a ziD2O``;FSJVuw|nT!*iuH7<$BP&GVNKUz@^?oSu2J+Rco520)n9W}ciX77$CSgHM& z>xssdf8B-C2>AXNukwc*>H5*RV`Tea6vmDCJvIh1v;!T?uDG&yH><;i#4m8REnqPH z_&|Zv#N7uK4&mJ=9C07cpTdQ`s8-dW<_H*hkK!*v;(Nc2>eMpn@mn8bu4yAA-3jnD zipHF-vtED`pzVCMGICnW{?7{~BW63MF!TAN-YY*AjP&b{f%7mujp)O5i)L9B|1Pk} zHhQkBn|Z-VHhJ>UYSqPAU)FStxHq8<)n$zmptReicJ;B2x1*;r+HvT>uDNY$ImWo0s>WdQZ`rvQM(> zg}=uMn(%K%>_TRJj+eZFF=zR=aXQu>0^R6ctl6q{<{NEliJnsHu2ZuTEzSj+#1)_dO^q={`e4*au0j}C>eIRWOe6UqK z`$w46XAPfOf}sEX9=OaO=@Z8ZG4Pf2_-1e$hLTBg)cmS8_eIr0C0@N$Jcdcx-0nuK zX6FEE@!^oK3P?^j={i@dOV`K58+d^YR^J|U{Rtu?Gd3ucU-OYUjx;l3?+N#cV68V^ z?HgPo+oTBmcm|AYBi36<*3t`)qV~AkJbnb^d#G~!_#EO+vuE#yS85#Uvj=)Pyns&_ z9dDm7YT$|yF76jpYo*|X4ZLl&gY3WFf2e5m=Cv1DntSq^Lic}qG={#6cI=w>|C4j- z85ueleGtQ%kspeBT`edu# zR6}!+|AxBK9ax{^cdl-TijLD-##UH_>~2;UX2xM>TxPqE&#Vl3-?nTb5MkCE~T)3Km<Mn-biFGq z`*y+b{pJLL0MCNagWEHV;FAP{3|q|SGNMN+IS|x`F|I91C(3Ncid8}2yP@slySNy(Hn*(_D5)N{c z#9nm=)!_&~+p1zOzx=o{5Mdvg@Rb%jGHqzuo8E%evEujLjW+I)($iVx?k|?};{FY+ z(DBmNfWoSDnqCQg+g5FaINJHc8x;A&;nL=v;IR~%Au#LxM2bT~ByPrTH-bjhC7O=u zB%x6>Y{;1dJyH~S)oiPsFmTaE*3qkel@Dy?NNlS~pcJryQx%HJAztFMe{%klH7TQ| z?N(-M+&WG&1WRpO7MW->j108}6bTa;$)R5bC|kv;5oY$J*zD>drN3fibUo4gpQD~X z!V^b~wO-QkJZn40KRl{UJQVooCph$DbiCFQfA6J{$Uiw6-W2tEIW7yyrvC+aq ztsM1(Gp;`22Ul3IXWXm>GV-dvFm!t(nchM)b7nenG$#^hMxs}B5Uu;U#^gSb4|Ze= z5&KEXcJlbKX@RAxQOHGpaB#j$$1$h6uz`sq!|S$z11W4*dNnNg${fD7B^ZIq07ssN z3CF@YM5>`g?Im2=R$_3eBQTkozXZpBk1rbMsr^ZZA%v?t1G^#8Xh4a`aP1kaVcnf! z#s(>kFBqME#&X_y#~r}?2og|@002M$Nkl#F@iJOw)<>o+dRAll*4cO9c z49(YC^y{LF%7T$O!q@h4#OmIMR6M85+wkX=smW;hQy<`=6;VJyIZ`` zKT7|{xQ(+7EC<}~NMTuV%>EtFEB$zgowMCj(5rEip(uCa3?>`)$ab6zrJL>26?!$j z%(&f{K+e5lzf>OQ$c@L@;%zA$Nf5{5p9ER=Zxh5EwB|3{5W+D^ee*(&=ib=va$b>W6I0zSJK}9?sqRd!>IG)K1nqLLvI71mh|0U3VV6q zZ9}i)1~z=*B+5W z;*7z96Q6U(QBq=1)ttO(l6%&}`6J#z+Y+J&tovZJk4)+ZcBByW56L)j&kaU5g3cjl zRouZfg2YXF|Ja6N*%7PGmw{5Q?&c0;5gPQPRy=6jslOLcOFkoKU`CJ7_4BY9JvTRH zPUIl^)!%g;&?}uQ zH8qcjq;2m5wEGzzunU8@f<9gt;x?^;`xvrc)@~r2Ncf;&R)jB+p(Kr%; zOLW!(%{Dn&4Mlh8mEI*F;-KEo)Kn0{W$l0UOHgy1{5h@&gqit??BK_qyio(^Jk|h* z#ExV%ukF1K9-s0XIlf4#@kdIBU9yjp3>zgdVNjpQLnNxhC4|SY>rckYT1Kt942X7Q zH2XS0I+Ys!s5%e+4N$YggoGocY4u-5DKNGG@Zg)*@3?CPj$Ll}Gzu5ORSdT0UwWxO z!|3?&vS3uM?cj(*8>#nA;vL>UIXFDF{AwQlhwdPVxwFMScFhMq;J7&8jiwn`EP~j5 zl>ix;l#Le#rRf+MtfGV+Bi@Y;i%xd@FJ!^!$@+wmafvJl0w?TL$jN!Ctqqz~hb@@E zflQjv%sYD(D;;{xM*>3Y`%!$lvsHX&%m0x64?eZwJ0KK?^?z`)Mx<@qmjB@Ix@PED zgIb>Sa_(!QqdPsG349>5By=+y)E(U+;L=%9sZfkIyBij zrugp~zpVu$`Nr7Ww*4%q{Pl9Z?8eQ$Rn_m*WpDF@jw1Sr6z{6kgsZVR?J^508+I0q z>L-k9TyQnS!S&|hU&Asc{V)CZ3#@mXBSSK9!JD@-5X8Ik(~Y1u68+9%xo7{IxtB2ZSq71TSpD;7dm-0w1i>0#wz|PaS>Y4NXSN^PAuBk zk?c3x?OT*>7lhZA@0vG`|JaY_(eq!A0lvhP-YWsJ_k_&o1Z`HZX&|F2( z)pG?4JbfI>&pK;Fu^;Boe)9Yk*Z+_Ha1mRas)s%@z&}obMB!>=>f%HdYRmu zd)v>Wa{dr4SR#h%HE*vMWyO+}Fz6+zW7o6_=WGLmo%R;4V#3~zKRXW9YvzyMV*^o# z&+l;w(z6N${9rq*Rnk5p`uSrxzYa9~h{`d5x928;l0SAyS$HWvIfOI@H-^9FkFDcJ zRu$dP1~N48VHc+Q6H9uPic`BU7@d1=o4?H?CV`%(aPZ9jp1gIy&p+RJ;V+zg@I&G+ z=@|3XHMpO@J_lJ3uDRB(h?Jg?(U=x8nYpk4md#&9p(;(US%Vc>=W>U*Xo7RSxSFCb z7};0SB3U!4^xUG!79QI0d0QQBhI+7pmBfCS4qjx%@2r>j?|4{Wvn zeD@PjU_hd+Qv$jZmouvvI6sDy@(^m}Z-=0laWYTGIaV1W+o3)J-9Np?nmBmjj~tP) z&t~1uc8P!3@xR#*Iat$)@lmhv!zqu&VTHamNRa8i$%{cqPR`DE57Go-WFdk zKUc^Lsr=;J?DahI_gOL#P&k#6ODM)IaO7wWciFJBV5CnNRqCt@uMu<7XouARpXhn>C^{U5I^hJ9?z>*ccD>zuMshN84}KoWMLa zc_+>MNszyGty4(@xe zhV(jI!Q&&?&mTSZOr;(@#0yJo;ZWY0hk3QwF1$}|giH~*@Q-OCqtqCx!;d6(#i6V6 zFuSmDB1eB1!dfv{D&Bh~gZet1Kdt8@d}!^CK`i1U<&G_=TB`}iwathNbma*yL-~K9 z*?+Rk$c#H$!oMHC?fNUWty1+-ubPM&{B^s1RJS`YWpI%v$ zKF6W9eEyJ62ltA@#@68J4X$570XpU;0X%P0OX?*=o;}fa3|NyYFd;)b_W^B4VxWy! z$t69pDkhX>F$d;Q?29-$C;y9QX#8Y5l*f9KY+H;mdUg4$bn^$t~oXW3)$03r07#Ur9UO7L4i~CSG2| zN5ZB1hq~Z%UF<;53SZB!II7Azk6)#<@7w)fynnH{*YL#B7K!f6Z%}ekAb=jss^AEe z@ngVb2jiaR$X~*Q?a4RCVZq29l2K^ly2&=xif)j8z?=n#^5M9^!&@nZ~66jtphV=w$slUhklkP?LRqk9K*jA zUoPnuH9m2NU)s^MU6jp}u5EIo+)8*yB)R1Ni?e5ThZc%z4;zT}xra~0iB|Q;Ugcr- ziq%da|JZMt!^}gL_`BVeVZnc}(IGIV!8-_E!yRdhk~DA{+(iEV09>Hki6#1TLka?0=6j%_E6RK59%q;S&2VViLJ} zD@H__tvz;u&ls7&aok8J9D22hq58~`IAbuQ?2NT+VNVns#ON4vrtj|LF6l>&_#Q`O zR)Pad=*BJHsar1C zinIJ`e!{K`QPvh|W^3{VzS^-NVf_D@d%I;xk|a5=dPcB-U~$9e!t{dJeH&7Q^RpLt z8Vq)~hkPyGa6l5obko$-J~wlZh|H|&9uY*H40AVARWtYSh>VOnKNEv*+8H;O*r4zK=usTTwk}xdK$+ z8`o&!W=$QhAH_9)JO~obY%?=1i~_N7pF;-xSo~Q0W!Njvw3cbu`M=>j%u# zx34?|QgkX4KmlvaL@M$c-SNiteFNuEW(Z*-!jcEuxvbQBjh(I3yxkQOfk?1lKW|CP zd7beUN}cYxHPTeb>n|rxgNs*VJX-H}ev!jm>wm?E2j;>TU?p6Qr0qQ`G5*%GKx2Ir ziDNy>)-YpL!iX~1I>yp(oz|Ew;@}mdO$>bc)5&N6`}zY9D;^r>cWUbT!C7YvNU?eY z)1I$up*kw|_$t7CB&y))Yb^FzVaB-yqQ)76-xd{nJV$eHKN|CGLXqng-gw96jN6Y6 zk@dhRek4sXzX!T}vkHe#PX+1EclP zM!E70LkU+UYR8rZyXk$XCQh-6%Tl-(SYU+HU2EOB>PeT2GFy9eqWR_b^-Hh+u>I0& z@f8j1M;}&4~%|0zQagV$#LFU)7L#@70+uU4vjILeygF5bQLzC;ZTe?bm8XK z^NqI=*$M$!bbk2L{yU7|FIHXHbNK085fUtmrqsL5kz25%VdOg$q}Ii=4q-c@WnAnb ztg=^Q!>gkRJlRE!hGJdK%>9T0fD~*sao_TI z&#j=ay?lG7(U7I>I-?g~>xMyb?z&d>7GGt#TRJ}eOH*1AXZ{Mha?+Re*kIzBh_em$ zIft5^UpY5{H!1QXC6EtPvst8Bg~-| z^@w2=%Xm8CILd~@_ik$LdC-Dy%|8EGL#dl!Gr(iTL#wN@_YTw89W6$0!4x@OpIpD- zt?KU*SCx19D^(^`JTVW~Po$@g^4mZDI6(M*Rp5F4e9rgrA`ce6J|N0>9#`U6c(VAg z;i_3tYkK#HdhfP8|4ITS&L!8duRrN6T#0iIyK6j{f~2v}`*Zx@?s&`vF$__ovWFxx<^0lp^Y^!dhwNYR6l8Bq<^MR2+JjDUX2SzH@ z1!vKg$^GL4BMApM@qWh~tZt67+G*<6|Jn{8{#?UZbssG0ZqY2L7asf#Q^m~MSA6gj zmVsU@MSW?qcIk*)dac2zP#88rAcX+UOG4!Sdk1u)Q%-NTfBb`=U zT?l1Q9l6bD;KZDPv<7r=|ZjM_)ROUpn8y7mw{*$Ex~Oi%;F_2?=UNWn==Su}Lv#Xr}NQan8{QcQft&C`8SDV8sUHNbG>jbvIW z^f=b_wql||1t&eWRdIpN2aQXw2s|+QkM&p579YXk7|0|)*cQzvb1~5fvfRmEJ-k}wfF4X_nvornty5OJT!Oi_Zr~H za@@M|8J+DO#aSB>EOD&!SZiX4v;Oj0_G`QzNB4F@HI#F?ejvsD|1U&VzPS4hi*s1nH%cWg&c00Xc$q~or32)qy*#G z>lAG26fNqp7c>~+&1+nb!KgdV#1NOItvHyxf5<@W^!X=csmM9bee1a8--RgAP|YvY z%>=LBY^!JFjyJ3KucCSWglMjjI&XV))*vT3SY7uO(>^i5S4{hb$$0DFPdoTR^nJ|E zH~SpByjghtXj=*t^qybARa;dpqU&F5H>h>uP7V6Z(X`^Dfpr`dd5EI6ThQ25Z8>!B zGk>YOvd()I)Zk+^f6mmJFDYjs+IWF zB4?c^*J!U>$9Bf2;}JJ&2tnEN{hz$#LpL87=||G?JB)t%63=N8nh%V04#mwUSVegL zqPG0`z(|ZdfAqk}^PtZmszL7~dHj=X(w&a3UA}%P8rt#ap0nj{ABI_sfR{K91!JS& zy9Y*jqoKle#}LKlD>%xr1|A(r>UV+s*Z=4L{IY$AkxPuIIq=hQvpY&l1mn4Zf8^>L z2EpH}O++55C`xr{_~OsFJlba6E1pgt>;x5G?S;v}^Db-8irtbBB2smxeZQcN#{#-ix0aYaWER+NYYD z0ZySj{T@est%dqK>lO2fC;zN`09pUu)|VLP40kB+7AYJd9X66-Qr|wnC|;1_xTr(a$$3Prkf6Il4da zSJM9HzgxQKbM4CQP&p_-yO?EtF4SLRrnpUBPTP+UMVS z9GnRaZ08S;oUAMTEd1D7%})3Y$=MIOXZpO3!@u%SAhOrFSFz6*%vToGX{E`HwvTlU z+ku?xCrxy=BU#Q~14_V%srej!Rq*t zoBC0>j-&axrXKR6hmNe6%9;vw{^fOif^z2Dfev*ne$6!TX6&uII9edV;KwRH&!6dc znPavHh8Rs*)hqqXQ31xF_e2!>I+{qO-gMFtyH$@BaW6~m<){X#@8a;l-M$a#*y$_Z z>`|^Ut`WVcU*dKA@2>x64eC=5EzMhJhVhxr`A@+(Hue7BOlEj`+*UquwdJ)V?74oa zih57@rEz~lG{bFPbllyq$9S#ieICJ#0#i$$+3xT&KkevmVs4~x^` z^|xc6d)Dxt`yU<{k>_bn6`%aG!aXZ$_f~7jZ-4joJ5O_^{l|a$=P&I5yhPGOB|lZOzNI_N*)h`+L_l@m|u_i7Vy$Di?; zo4U7nIM?_y(9v#9m20~Wa|}Q+vnH3kFlP*(EV!rm#J*~y1LoArxR+)^u5T1DZBISC zL&C01-k8*cDRX30~(9cF3IUrFOf(z4d@E0(C5cOER9_2oSz$)B%;HEPw^)I7c9B&VGzHKB&xdXuwjUe}_>$FbcsO%K=1#{LCczu@W_gD;LQ9kfD=B z)NX3D4thI(V(+vp84-QmJTO9S>Zpwt>n7iv#g7wkVWaVQT$e_VU$wx+%J*)|8nU44 z4ZEu8T3f00o!^IXuYbGYXzJ0jI()r%x3Kd1zlA4Sppv8nZ>3945Q9@w^>*X4r33IzsB z42vHtj$$jWgRif!^MR2uMrjWp>qzx@V59Lnpfs`F&j1>&x{;Pd<507md(Gj~`aO!7 zv}!wpr=ZWjuI97)Kh58|jNCs?;K|+3Ul%wTJ(`{uub}%Xd$_ww`)Q#!arm%~MK8Y8 zX3V-qdf7KSuRRhE7kpV)YrZh~oe}bjI?h3Lt=1z$CqiShw^1WBn)loW5O+>;U*qj@ zOL*2_j;+preg4rl>mmo%xuPdWI%D_d{7G0U7bS@2JXQFmnL0UwaH#(6{Vx^P6?5s$ zIpTQE^M|+yjDOE(DaN1cNcm$yobMmkBeBY;Mk$Z_cSKh+Ymiqj>VL}9Y^zQUdCs_e zCq_PPd%kc~&5HQ2x7wYCeS2ixw`K+Xhve9P2&!c~I&(rbN((aI58$yKs+`XZ&~!kmTt%s%BK!-?nf?RC~Onp!#j zs5gVJ7b!WXB|qaQMlR~5PaHe>PubCc^Q1-ml|}1YG?*{xtDPEJHzOhX!?ibSS)VxO zU*M?+8T?)B#5`%pj}DwCJ#bHPv^aj${|ZmeY2iPupNajN_EY`%(IUU<-;d<`xsn!q z)i3_5@e6;(#m|~D(EJley{Db|=LmRU^t<2wF7#cJj>(%)auk*n^dU{~=|NUQo{_^+#!4HfiixwDJX=v+uu5(E(J1%M0<6`$^ zf2ToCV%Rg!iOXukIUg8x!l|N}>P+OO81cOn16Aus7{^h7#hiB^9eK!u zkA9=wB7XYM{*_-I9&m`Dnp3bEkRX1JfIQMzn+|>kVprc`1CH^x-eRqq+tYQ)hsKQ4M$ee% z^}8#=K5egGu78biCi1nvTbe&lK~!^`KTrA5qkhhxr~IAGt)1Ud{O!4KopbxdJ8?QH z`JNYnHSYY+3gxj1T39q>#bM+8Nl%vJDkXbE_we;e23BB#CI5duis%5S^Xnvb^gUUxPG4zSp9nLGbeF3SCZwVB-_$MbQ4zfPFFoJ@m@oKqj)C-&#+m>-!Vj#HgI7xtzEsjQF+sQ|T2>*t?D6pBIb)m|D~_ zeFJ*igmW7l&a6Mc*9*Qu=se}lZ%a*RAHOH;NV}SI{&4;rKHN_q7?E??641EMv1jOT z(uGg{MXrFxnVMSv{*>=I*L62uJkUZy$MG5O>IShN;ze3IY5&(;p4~*Ju@8ZG< z!|_YrBxQVeoOv?JJgO5fVf1o05v%5KZyS0>ZK0(zF%ze@W4->RAb;b!rP%!zDl^Fj z+WC9WK+T`%dJ)IA=P54#=lLnE_3?aW8Ri}OPV>a)$8R~anl_dt^)XvJQfsTEOlbhlLe#p z87GFF@!1ylZeymzU-#^h#n0JW0ev(skpSd(x9UM_fHFSr-yU7XX3tf{Y;Ing;v`&zwKJ}+Ew)ju>QYfAn&q#`7$3EiT#(F13o`# z{ffE9Uoy-S3FH=EYbj%O0X0{sB0?y;yg?F^&tKbj7zry;l0uTjC(OQ8q|6!spdc&c zZ;7%~7#z~*ey5VQz!w&uf(q&I-!TmQ?1~h;_yGU=n4Ny6Wl9| zG+yC|3CD)s8N(0m$(}rL^51FP`QW^={C_QcUA-j$va`dBh(UBV{zY+)&n+GftZxt}Wdxc62Gg}K*G!u2B-+aj;LZ6n z`L2#5vkMZ#Dhd}1V`A_LbLV$_kKE0V5S|A!gpuaE+%tZ6{z&clet}%J=bozfbtROu zD!$hUqcO8SdFM9K_D@eUyLClPz5NK7HMyuyAAH9^$a{B`oMA^hs;;*G>#CeR+iD&_MXq0)Ym{|ilSSfVP=qoDkpg>{INk#gxQ{Yv;DE~|A4?bK9%=0u1|0i^QjI~ z@`NXjB?qVDwx$3Ld#f8F@O8s}iwO;K4Y>>U#5`$DjZfQC`{(FR9p%2$dcvIkC;r`- z?zrIYY9RKt_AbvassFB?nV-Dh^uS1oLV#=DsVi;t1y!7N%qqTnV1y3%?CS$->J>B2 z1IMM@YhT}C)DMh0ADYxdezTiT9~}K@Pc6mpO`O`N4!G&JkH}PL6fNdY%J?SZRGAp^ z$j+OMiJ5KbqIdeqX)WPEI>HB9@rW4fo~lO^i{>ladET=of*Sb1eU~2$7Cu&Wk)QdK zk2qGoR~!1mz?nYU#CEM~^bQA#io1{6r}JOTA>hxvcUjSHJ812^X`%LnCl2+B+danf z@dlQxaXtDOBig=eUh{SQ)gpdcDeM(l6P$aEVO^pr+wR50{a z#5{MBaAmc&6;1?kS-lhogEjLyj(71ZL)tsb`8k3eov2wOeynPP&G|#U=K~{jh))gc z5V39D_EX``McLjx)?8Yk>GwQcnbULSq2LI$;r+y#iSo?W>p(gnupqJcCzcrY4kLVD zQD2Xs`93oA(cdeUa(fyhZ}5vOfE8iVv$DRzcCeH|P z=LoDu6g&B9v(8^v)bnui_nPU+?$tSB_S67(U;Wf``#;wMqrZy>M%wh_;L3E~KPkta zM`*HjtpCGfM>Ah$HNJ+0d*zSsn)b5#l&I>2<&C-(!y8}xjNbk~(&mt34icZNUV32k ze?L4hQZDdnYukta^>M4jp0lgYtW$U4exU~A_-p5(T=LP!-sPXzGfqr)#%3FBS@6^M zgq@g^3+L3se{$(h{1f&ahB#I_Pg>;t8f)FD{%!gw%-mmF=c(}#KS=9>{0p5w)Nz-e zn8tY*pX2`}+jn_+VD$7$uc*y2z#5Lo&g%%bhT3GqP$WA|zj~WAUF_QUarF;HYoL@a?#rF?;4dDOl}{!p91AV$9t4U?z_?v7JE= zRP@!ZLqTc6kp$A6u-kLF+ogjbY%6CA3iokD^6Q>#Z#ms`cLb*tuDB7 ztej5|j3km+h^U5BE^1u^!pS``r#GUO^OMUca>>X!f zH-Qg{XU%<2d{i!63>;FCC zflDC#wX%o_&MM@0kC?1p%E8MB@4-!<+QPDd!4ks?rY$bUu{$vnH)G_-3P&6}xoWfB#nB;#6^9snESRS_IL}z?%G0!)+Tqh+C zmKfIl>g~>8ItsU>E~bz2j@)|}+GRYK^^z6%{JGPjTq&P%#n4-^T7P-}CT9lv|HY-F zZQ-LmbIdjvtavl7KHDn{95Ax@W=mQ+|MOwI&KZA&7p4&RTkAP~GCqxigO9~8iw`?1 zdWU-#m)69fO|D*tj@aMVU%2qF!Z~IIwdHFj$hKhYiC%o=u?a_hK*{7((7f4_hBsRG zc%q=OMz;d%T^_|y19zuG%qqPXcIstB=TB_@{F6=fDi3pr-}z@wYuZmW=0B;Bm-C)l7rjz|t8SX9Nql~Th~j=33Fb*d{wt@R7_D@$JN zQyz0-4_UxEvEjVcs5jKF<)wb+C3$6flMpl(IF`CUvC_yDUSiTe^9My+bWZ(=`UEraXZsa@i)ANv<;*#%b&=KI6e(!IGp!&KMhtt#&syL0 zr_MXyx+{s(D`YO1$`^ki`$MJ}4U# z`vLypPd|M5Z}mHj236(WZiAZf6F1|^pTG2TTmuy6KOq0vLO*byW91``ow(_fUmILm;#gt{Ja(UA*7}J|&fzsIJ{q%iq%kXq;&P2$=g|okCSI)@ z?YuY_a3!6HtB5py&NkIGYL`%|b?!^Q*J(5Ea(A@m!Lw2&1|z<{t9#xYJ;K{e^t{X* zvrQiCjLrEVj_(Oa{A!CKlaqr&99hke62yz|5j*oq4~E+3COmQM)R=AJQ-e4$S$x9H zw)%HCFf%5sHpPgiBX&|Kr7f3kt}UYW z&;6-GY%*wr1%oB7e%B9LYXl_r;>f`1MWIwZDb9dU4pKaG=e{AWQxX*-9~Q>M%l8xq zM;wbEyWSPUk;g~pPzncw^C$D5&~>6=!nGFs*jBIyzAP~;94uJ+Uu)4tXIjOGW2FzK zbL9$(0FglZt0Cc-8Y3{)U(gn4H70bmw`gA9Dny`#7Do*#q)lHmsER#pC&v8!`oVv; z2+Q?=>#TIqz)}O&nCQiqJ_BiXbBMg>)Y!z=tY#jOZZ>i#H1#L+4l{id`=l{3ZfD`H zf31t~<+)>Z52G`oU6;a?SwoHnYlcNR@zT`?YNcR%CnVB5De z;%L9(3peAm7r#>CukJFR;uhh<#yj0JF4?CV6>EI7x}yWCaBkhgbG3ZnojjpF;W{tL zgYkw0Qu?%^U={mxlLj{(@Ym`H2RURrUh3tvMrRZ*vE%$vG_^9|4Cukkcy-Pgn7^jI znFm>gc~LX^*fS1J7L3|AspMXmr+9*Hc2is7C;qz!M!XsF9hrH|x~gEZeLCtB@Yi=H!eKKuPasC2_94n~$68hLl6#^HZ@BX9NBEp)I(_smh3 z2_?b)G z>sE*ph`=IT92J3h#&=!yKr$FHVD)TSc3~^aY2e8}vC}^}V6fm=HFLg&!9TBUaI@$V zm~|7EwT3GomwI3v8F|)B7{j`rTybVT^-n$dzHYOEbkV{}hx}Ny8{<~)W4Cat0mF`q zJHnhliCloyK;XefQsvD80wP*ccRt}MB3}UgKmxzd^EtBn_{oCikVfB&2M17WRW3`bTy&wJYOxj=nsX@iZ$@91^kakn%M zaliXJB;og7=cH6a?~a+ z3nndhTgOC-11`zcK}T_I3*+v}Q{6NZBR(4GAM*#t@tEmoOs?YRgEcs(-EveA--f-( zulR{Q>u3zfdGxmo*=f`@_N^Xk#y-B@%?tU@Tw2c_+w1x_p2TWG`g}lTLuQBEgJyaL zN+H%CIFY1QDSR-ZGvi>^`E|2?7e++E8oBUv=1p4?+jN-f$;{{%p zxP5i~prT69$7M1@uj<^r*Ivn>x&esUde8r=Y{2O7z({}HF}EVCJ~gQ`M^Td@vmV?O z-sIR9=5*(*gmbOpz**l`w0uE~{NY6VTNZqm+)!; z+-84LW_xCke~)zq+~^R``KgpP`&?55c=}hnBY4mwoD`*fj!X<*9gj1noYb~suJtK? zZsh7!f~S#qc7F9LDVLR>Xf*w{)%hHD4l>!)Wlt?TM22P^D^TCF?;CO84J!j0ih>?ehF!n_j0LLwbM z!lHQslQzyDFmSMn6H`9giYs=lxV42bezNQ%zc6fQ?Px`J57)1!{3l-Qsj$W$0_41A zk;mIv>(qLOac&m=$yIw|)K6SGD^e3Uais^C+r&2i5zwrHn>pUCQwm{9>^B}4X|N0$95m8X} zCs$oDdfR*WQH*}_?a%*N4nEO4u45p~qi6iK3KBrv=(&$behCrm{OYXTGUrJL++Ca) zS&cc6bOWsF>56mmf=tlSTz7Gt$OiS&Eyv&l+Og`GzxW)b&#?TB-w2L_AW9($DGpL43|^Xxkp^>=2sq~n;C{2vCc{}rG7 zQ;!^2Fj!%UVOgJj9kU|Gzj5}Xh2P%i{T&{zWA5eJE1C3UP}S*p)-lnw5ll3{`&C3N zvdy(1jfC-Jja>QF&4j|2cu{BXsvjOXv&J}bb^d^x+>$f=(4IKBviPx6Lv0=yJ*|JV zghfLZzvemHRllaTk!pK`OTSpz6<;r5{5sK{FA2G$0>`>14f9dK!Ubzvy*mFzg)cl< zxTh9=aIzCK+Y^Iw#v`!Kqd5LvxkkXpPK}vM@e@OB(+Xpac?9fCe(W@UqKO^nkF;`r zNLE=_SAf{4)4jXBGl0|h(U|v!#HRu{`Bt5rKM|bAZ}QNRof^WNeRy#0;*QlgKFBXk z^whRysf_hUM3{5yIvBYT=sJF#0pi&6`pqHM^_%r4WxpToRiGi8I??cFpvXS>;HVk9 zt6h8Wtlx3?j)RK0n$^|ybJpMNo>kx|NSBxJ_`%6{&I7YQ9~eo)|G}(QAVR=Dv0z1c zwdv@rHD31$j_muHpZd^He&T6!{oFh-f*`bjSy=947eVi9(YfP*3Ol>1ej zm@NGD#(|>}fj!(A&84#dKJL_aT41ODPWQyz^{uBhRM9#C(8DSU1f3OIb~O>U%D5^UcpP78?Z5e-zUV>s!bl|6o-XRX zaprdxHJKJUMA3Hk!Eqv-V>}-)t@Awc>!06PzN?2uyz%&rihRn+UrGD>_$z778FL$f zyoSq@2Xa*->e}Qz>t3D+B;zz~@62`Jc`q#m-H7)|q>uoXPvjL_$QWm~)LOd_tjMi%K^LvHHp>u3qY* zJ@IIRkw{+q!m*CuucgtfZV7FD**SO2o3J_;yqS6OE4WZTdqnj8OQ+tMd(SG)RQbjO zC&o^$(^8@Q)SLPdST}-2npnB013&f4Pt3U4>v_X_@)^I*pS$|O!&eSAVpuR({8;f6 z%XrND-1yEeyl8Bg*@^ZY=JcUAsWbnIZ`ZF{i7R*}bCyqX&Ahbl^Ka2@Y_p6;g;#-` z$55>6H`u4zIDZreBg=Y}R~j=G-L;|5`dj&fV2=0T)U&6pW3_)ceP%&1I!LDIBt&Dgkph}&_qo^;Rzk9~y~rsID9Mo0I*XiS{&(@$GG z#n8jfI2c*Bj(1Bj=FQ$c2DdHQ?Ad!=FZdZpnd7fm#v_on&SP=Gq9cn|9lcn-LPhl^ zulVGkgC&NR`$X;CV=yMt6_B6An#Vitt}dPM$VvUj{o55ka~I#P+P}^_^|HRq+nW9P z$2!8yaljWt?aKqB06)D4Jn2a5E=H~y6L#MU6((`p^?ygs9@kHNn)aE8+{9%wP9QpJ z{_&sx$=_jAzx0~pu9((D+SrSWK1XQYYZ6O>xTQbRSp$bK@%k}t{&DGZK#n8qck#XG< ztxYeHKYw@TI2GT;g$MIgA91wT2g!o~?r=lXQLHx0afP-9sLlD%dCGOdrP=uv?HW!E z@^AihR^(>wo%!gaMSs)8d003bb~tH5+i>N)#c<($xM01~c@?{><2FCQ!-AbTAY2^msFPf3y{5pr4a`;q!S#B2V6=*>8;+6}H8;|qenv%{>cO=i zB#Ky&^ryYfjv#@Ajg_91ib62CAUn z7UY~ie2sGbXAK=!t5{V~UcT!ZzT@^Dxyau+Ue6gg_{y&~{+VT0nvqp?u{$Q6J0AJ& z@+d}JN?@?Wz&QuF@D{+=I9O7;ZfGb15y%BfU~)R`;h=93uIk?2p;Fpo~u)R&G{n@ZRAv&bXtA1<=?lxe)w2n7yIV?Idvz` zT=Ruf%<;~7^2NDo>?nGtmbh~Rw`KxI63;Bxy!Tqu`&oM(f59s^rA!5d@XOK`MlmYJ z%Kyo>j$eT_`%`<*G!>mmSaM=P?K+K{@tHsQMe-%MbHKrp3qHB*pL-ZR60jYiwlmUe zabjni{PTQNTot@$3llk&meh6sCJk-48SnHZU?JRl%88CqVDAsM#Q*?607*naRAnPl zL0!Km4lQ!e7;RQd{z#?d#`UW)3NXie4oAE{^PK#N?U-9|$wGW`X8Sx|n)cRcBszC7 zq5g9`F!~$)NE&}7ZK#OL`Vc=gN^m|blFrhvIG-1+;dp9PT%3&610%Kd>>yui|M&60 zXeJghw{Yr-*8aPUBJyzavkDAI4vInwzq(ex?C0+=0xQt2d!eOcV(t;Iy)&PXKUeQA z+stvNKjS~AhEvB+?{wJivffo{->LJCed1f02$zeFylR=n>7ird3f|R`nxs_cBZ5zs zSGNx5z79MYEc--ymq5x9XJ7haHUPn}m9XYAxk^~7|h`xY8l@xNB%UfEuB!8~b$ zdl#3^Cz$j6L7xXkzpd{uS`TJJSF1L0kwC}&%%d1D)uC-+YQ5md^%G$Ivh2hi*VD|C z5$_Te8rIi$8hPWFZBq}tfBB*RN*Xn6-{JP*bpt(0LF-fh$x}b)Q354qb3dQ(ZD&^F zr)OfR6JH%Ux=}+TXvu!AXQ_$5)9Luk==CFoj$026_wM&6=N0Ol)sy>_58Pcmk6$Gv z{#l-2E@7Bm{qz8nb=&g?CUv4ipH}y+i_Yw|roG}&|K#snVZ#$&i8;<6u#<0f*H0WD zIP6m#{2uSs44?YJPutvX^@xp}k;z4$jNr&eTpVIpV%Uk9In)LNS5`0ld|=9c?^aH5 z(wZB3cX479PrKK+M+zSc4vW?q#}7{NoJ#^lU_ON-ghi7%NX)VRK%sv=YrhScTrg0NJRX%IC+7&P5o+};|T#wykqIXvnF)ZsSu|?qg zX^hpU9yJ*QV^+~Mo;q_xlYu^JCy%%|Ge0=RjX}ZcGj5Lr@QP#>o|#8@{Id-PJ7a2V zBL<(oqu%E@0w7GrV@LJygq&^bTs`2(g9V92jrd(YX=xJ%9v1(!YR|a(+Q>mnmSdOC z=@J7DD@mSK>Z>VB`-hK~h1y zLVV^SdF4#Q2jzDQR$yuAfzf}BzmoQc|Mb5mpPjVlHuK7V#=%b969(+X{w(rEZJR4a z@9LQ;pDSP^C-20*Y)9>ADd|`9UyAR+ zm8aZyV+3;-+ts?qd0M51m{g8mu2b@ClVTk)_2rj8nzl z)*Fc`h}wDJ4?o#ejJQ9H2S#W(+#`^`C_pw#Ez>z`JpJn5apa$QRDq^HE2(j-{)w4d z((ITsfF>WdaZ;EY2z*zhJQJn%M9nrhtW=4gI*OsZ9zg2Cy}a4X9U6G1zI;#dE6yI~ z=~KN=a2!94ssF=b&ve#V>!FanQl#&OSF^4(#K-B#Rft!Tco_xhoof&23Z4wjm2_qy&ej9^+L z*BnLxs^Pm|qskF|H5$c&lwa1<`J)ZCd+Ewup_b$6`~e3aD1~Ie(h9w_2Ch zUjN>sT5E_g2Zl#5;lP$6HTE?_BnLy!JxpQ!NO#nJl|=M!$!A4GDYg`Jvrm>GwI6@GH~ zE|IS}e>OhC@9YgJU0Qch@OjK9{7(AS81L}-?sV_8PTU=4)2#DnE#e4Dz+S2Eh>n=384kzD(FsR!6(rf(= zBW=uwa(lyb*W1@z5nh_9Muth);XBvI_$c0u2SS5hjjQo?c_ZDklLNSP|j;g;ky5&r* z;Z|miX5;WEju&fA)fyeJXPiE9c=+kds`fR8F@9s`ezM3-hC5~BLChX#<<+KXyPNBs z+w*~cT50l1(b|D=*S)-kMpOBfd;ZdFX=uZ$jmEmpj`$r;9Bstl+xvbGZz9paHpa(K zRX&|RaHgKyMEpKM9BnLq);ay07!IPGK9DQiGca6&(G6}^fQEm2bw-U@$aM_5V!_H6DXkxU9Ve!kJ+r%h_mTb;xI(+`@ z*u94?-aa09TDv`7C+0??O77h9T1EKt{GCkmy$_NvzaM>aKnmSimP& z{)r)uC59EA7;VSGJI){Zcp#w+lQXrX(Tdl|>5-1cyXi$Iaj_dSm&)gE^31D2LEx~b-ibTo2Yu<97w1n2T;Hfp=H8!q-{oA)gQR2Tt4oYK#yoW%`q*;LCFUtVLWwek5)E(raNWW=qc*w9i(#d2`8b?DT+-T9i}&^sGMr%0vV- zkNFo9giS49i^bQA%`xlg7`Xn>&I^DuWV5pU@TdNfG&HQI3bYl6;PUKwNBCvaY7LpS z%RA#a3mSteanUt)Bf*{;j^Z_n9#$B(9TDCD>3xsCezdP(SUr9;sYeLf#G2JHe8v<~ zCSSVKiX$FxyLw<$(WQIp^MO~Iit$pP4Qcvls85?bV6epL%Ywzq2TnGRTLfjV*P-jI zq9dE!)Vt!sy*Bc#3r<9Q?Fj|tt|yFV{wLL?rwUw1%>Q1$F^yC))V>N^9^;71$}`!s z4X$$JGNe@lz zWPwj~bo9RG;~!^)PM$pe@yq{**3^Oj?)(8q?x`W)83%V~pSWmryw;%9roJ1%BVP3JdHCY3Zu9- z@?h3!eE~`L+I|qxTV0zs>Fj5@Oo>LLW{u&RAat-D>)NzbY z5}7jfV>DPZs=rshA}v6g0?cN%cN#ilmLyNbQO zQN31vS0C|9?G@1R0;5dipZFO+tC_en_O5^GK52n_iqAarfsxK0xO!mpxB4q-zx`dp zeKEZ*d$d-r5&&;<%+oqN>2rG5fXbmCjQx{I*ue zz2+7Q}@ zSnyc+n-ktFENp#;0XQAK##xORc4~p0F|#V)I)7@tL{P4GH|%I&$)kRr`_ecb7#TZ} zu%GQ?-9^Nf1{EIMto=zH&3Uq=#gihKPN^!aB6FV@7{HP;!niMzsJX{W*~Os z`X{aH+2aiNI%oEq`XdqCepqB)rmkbzN<9Mc=wXFh=TB(!BW>`hf8`Xug3_2ea85fh zaKX$Nz9$TREckr4kA-pf!04>-%y-p7b+_Q=`J;ob587XQ{IK>L-7;mj;Be_^>0i>UfGtTQ9}IV5LLuc^%dHb3QNv zNB!bzyVG){_Y>1zx43$F-Xn|7pIX1lMP7J!T3}>}V~N4X64!|Xrrv#0#h1;6g0`^B zk!U|K`q%oAwBOW^q!ktIrG%pdc#eWwI9@Y{;uAAtvn?*4A2hPKKJ*AvBoGKe%!^3C z^d`*3i^4eI6299mT7Yf>a_N8S{X2~Oz$l9e@$*Cz_9>@Q+{nv)*(Pr3g)cNsvM-;dva%35@<(-&y}7jzC#--svB- z$F6krLKiy;VB}kIoj>XL?4mXGzNNk9tBRfTE*IOX4@X%r@8ZxlT#$ACpdk$TH(dB9 z2Cc-;&NsBtJ@b8oR8p8v#(@sM!)QJ*Qtp|5*09wMtQ#yS(8{rC5Kt1)rQ^a$TE$B~7?OCuO!PTl!z^PTbTGZ^q9Ux?osI6Gx2t z;Y`e#2aN2A>zE_C*POk%;b`7xE$TkeT~Ccx4m~N1E!SLrdhKfmI#~GgMorvYD{9mR zj=n4!(|6*)eCnTCs=A(itxxLc==h-xhZueA#DS5;ZyO#t(?nj`oIB4WSTvj69FKT<_!z@4o2#BnyR<}5X$T^AI6V~s|8u771ewT#r>o9fTwpl&4UTz06B z2DYFwFSn^RxsYZYpDfrJ$A<-&cDD4X#Ixefl*hCT3h(?pxM!8-oCW9d57(H<{ zGA_Ti>a4iE%IdIY?rtl>?|%QgFMnRY^g5j7&x>a8s=FyXC@$<>{7y%rU=`QI$lB+_ zO*vksu9PR z`Egb;be}M{all_`(Ynk-2F9sf=N@hN#n)<($Uw)>Lx(iJn+&RiXL6cCk!th z82v;24kPMGft#Ip#+=Ce<9tO~v{xueq}&_59cM2wp-=r*patT}JrSrSBaT6iC}}g} zl_k&J=$+a7=QvqqsWk+T{EDawUdbvr$2P*bQ7v|;0!f|KiN=Y6E6f=)&t7o#Slpr5 zj}xK57Hs zxck=5w!#y#{Z^kjs1wYL$=7+VTsW;!Etxp(d4%eLgRqTLWcGUe!f7K4?p$9+c@MT( zN-uBBYL;ykHF+gAAlzs>(yAWqu1_9f*k;|g;O)`VYqe8zUcadk40h_G+Oge3Z*tTY zo?|TxdHcObm{t`>+moK7)&v9pssg(0dyG}lR&})DVx>xq42i}WK-SlA71SPUbq)om zKDzMflbNx%e{Y#L_Q(1adTQ^L^s$*3aT?dHD@I6dUIB`$Ma}xoW;R-^75vNw@%Hbn zGdIzbcjAb*->vRbEtOcg*RlFmL5tk2fd{1e5f#7}k_EWqFee#{AAe(*5smY4T_PE9^uN6MrPt&|3+qwrb>W*9 z{5mfP9_#zWefMKLFyeq`5kds%ENC5pQxM=v_)siI@Zc|6>F~hlALFm2*+l|MhjFzp z5r+(2*~z0Vzh7QZEF|J9J3?|?f(!Msfy6h`*Rhfg8rYYKZ-UgK(AR2AF72C@^s{u& zlE1db$)EEjKQ)87)4s(_XONCAa?8%RoOgNT2lp=is1X9Z&72Qh(H2*may;cpfOaAs zAarI3NXF{CC;!r-fdGBT^l$7pP;OLy8Q0wdBPmnk8qRV3!w(L-ssfLV>!vTAX2L%c zp|j#8$PI_CX2O8e=Qv^CJuo`;Bx?-b!NC9|ubJuL_; zqB{|4gIX;dN$9H?ysKxgzcpLv%&iDfbniU#}_v*UL^O3m)pM7sSdwxL-Uat6s+%tixC1sgvY zT)Q%!U>Fk#dynls<3Ua~mDSPML(w~RSiNXD?ma$<^lF%rr&{2~@rN&~7ig@Mlz+w* z!tW@omJ(0|G8=fIW~0MBPZiT-BD); zX|8&WcC2&VwXXd3xP>0o$lFRM`gLrtdGA?iSEe_SYTNJgXT{P9W2za!{DTLH6@!@B z?CoXt!C=Lo8hHdFxZ)W!LO%So>KmUjnR0a8`O_ok!}T*m3L74KE9kWA)KtGf0W)rS z3e4{?`px*I*I*9D%GtR=QbqS^B{qb5T~(xuO$VAhSjq=RIW`LCXC!N4xR|Oc6?!Q) z9Ed6d|5F2wm@Zi9TRM((+&c4tQR-DraPr-6o$(gi;iS_=>y9G~IK?;23`CoW&m}Q{ zz?5gW)y-|j<+#Jh4{pVAC>Wdab3-A_!ez8~(B8#iOx)LU(U6r6eEPCqyZsUx$~kqi z2G)PAd+M6w$0rLv=s%#7j3TEiFq#|Jwkf;0qi z0U4n#baNJRBUa~_{|!Qd8+#{z1G~B^!KcUXtY`fWqxex;ZD(9Q)us!sVme3F=dB}S z(btx7H*?QTo-o?xwdF34QQpJBhc-M}*I|494x^Kg4tC8~Inag2_!>_O3!??HCIqrN z*9POx9xBCZB=K3VQ=Ao=1OD1GOL1eomp3C24GUj24!m`>?|A#p7NU3f%3lyJUnv{X zGr!o3zaLCE|Ez*+SYlZGvgqZFRygK;f@wlAM6(>9QMZ);32SnK!R z{_9*)M{7)vS@NJH&&nZZb!_Hdq&k-2wubW;B>?+I4tWoy0veH>(M)_2c4bx)P(}|5 ze~E7f3{GPY2HgdAV|9xROm@a-6DJpz^C$ZO=KP#^G~fuu^+G<484R|uZcV*2_NxD+ z2j{A5&$<37kV?^=e7B3Q1Zo0UpB&AE4@~ylKEtqr%x3;4ceaHRgVN^QqhP!DTEnf? zuj|dcellb!fRn{fe#QCjB5-`rO+5$##^!6#`JsCUX*_taXk(>COl`J|AJonAFE=TwDUnu^K5u57;&z?HmI# z10J;i#S+Kj$BO4z%w;)!oO}H=1}xPvO8I@n(4A|W+N96>C&w2&iqs%r$B1@*YX+h~ z*UwcQ6Lu}IX(Mn;6WP+TB}T;N_rebnGi3*UvRmnt=Q=vFJ&Y z#&>_oA4!WQ6$gRAr6p4^)#N1wiRC1#Q=1Ov#N_zH!5db_6XOX_jo_kl_rM5kS>f4~ z$6E}yIzHe=O)Lja0?mWJ>$bPCo7MnWEB0!{YvnJ3;}G0qG0MBg@9UVDRmXIOin{4a zfHN#C>A_V0U0g6>wp1^;3Z2+9$Bdt`lK!h3)j%DtA$r^y<$~QS+k0Ie8f)<>l;C-U z8&7NTfD>TPtNu@p=*C zy17=A0)-J)Te;k+{pkaX1q|+M9vE3;C`FTFm~|prV^pqzuGGULzJ6a%oO}D)v(S>G z=wXLSdQU8I=3gLTnExcj4jlcqM69fyJ>FX@d^*T+4e6hkcMj=HeCwW-c4Tjv?W$*_ z{Fs=(`QDT2bsmwQOPo}EU$RzJ3Jb@JaVbEhm8vlA@&4{b9=`3b2*GTPOjLPj%YvLa zrhi>y0WQW=+1t|6aTkS=LMPA8c;9>C<3AOi^5l6PLErpp=@c@Knd|ME(ZDLlx?Pns z12NJd-km>ZP3G@jv}G9!)-j*q#9y61V4T5hROzW`^i}^lKWoh@w&jTPOWeGFOyQm< zbsLXSiy*4iSa(p`L5M3jN%I;fw3vx>-4D_=Um?bP~*2ubBM2G-B=l^ zLk{ZiR%ZT<$O)D*0VP0#+Sdt=-++$_w^k{zJ5TcvD6%x=nN}MLo=-I<|1D@<39%K z`l#&XuUdq zXFeVn{kFcth=i+-AZYIX67}aXtNGLa*a_3;vB#2s4Xs3oFD?0_qZ)XXE49=PkVrpkE z99n;=A3(+ZzGii=(P-!TO464fS-4e!Yq4A+O2=sLR}D%~oqbGM#G)wse*N+Idy;1y znuT)=m4>`nkivMhWoj*?iSgz66Cv?bi?*p@e!AC=I5DgN(JRM3u0>xBba3oCc(3;4 zT@kN6!}55nO$InSVtV9)&W+c&@a%;KpL z#laE7ibL#TA_sW+jWY*7I>cmiivSfHV@Be#8Lja?BW${3w|)R?o87K` zn>GJom3*B|7pZqxnQiu@a)Yh(O#aH4o5-mdpX1NkE zcdcV@Qq$=O@w6#z@k+7fkOKah=Iv&C#5+b#(Y3|7Q2bK)()VL-R3D^ zxvFtCEdnr}=gPF$pZui0=g)Msk-|zPmGXiiRP^FYalJAoI|&sFvoPS!TKl*P?af%9 zr}9|Z`}4X`l&>#p%L<^Z|SZAj&6UbRMw^nl}o?ncO<5#WfQIv+nU ziYk*E>q?q>{VE)(U!r+k)b+8FB@PS)gi~=|awm@XVN-wl5TCwi>w(e#8xM^B{&--- z(O;9jcqKISRQoqRLTd+uOK;zt_Bg;d-z>DTCvJ^D zh{+9Lekcp8zIeRSQBFP-YF-rAmNrG)QM0>lC0fk@+v6jncDV_xw|?}EZ_)G<47WEs z7B-wfupo^oq|6}sJpkmGgHW3~gOXrkr(Z$VpxWTnJ~zi+A+}O`-b&~-^UT)$yPD4^ z=a2l(V^;1Df%E)9YlrRp8592;i)J4T77P}@tX}fN@}CtCznvKeU#={Zqf9KFwIfe< z+GUDyC_*a%FFV{<sPsxHhbNvmQLMcMpt49N`qR;;CyDb2fOmCYBiLGlowV z+>GO!bFTGgOe2M3^`ti(>+^$Rau%H9-t36Q20wF>*rDW}csb>}fx=obVZc#8=Z{44 z8d(@=X09Nl9rw@ZfC1-(jG%MLzviJxXqYqZo16{zt@RkZXCB>C&d$FEtNc=m-5>wy zpTGRgzpEcf%Oc`97CB3+ixhKF%;cvAiK3JAkR7hjaxWhkDd*gRlMkfAM^4@pzxy#B zFNup-@rSaEm8-a{KZqjL=rzuLi2Q!gc-1(cgY7NanR zzt3jgVu$KEwn(e6UaN+^ki92tfWwI@)QE(;(Zu;qAH~^(0tt>jm|bnpCk%KjJoM-{ zbN5V~H2gZ%nsc@$-!Xf%C#JdO{dOsnY>nVlHhH~x_~~OO&h6>xT;|_Zwd2vfLn?Mf z-(_9X(1a*JnkRpbOEKLqP0U)q5d&*nORFx$G7e~=6>YA2z2CF7pBA+stMiApFte@x zsV6>|IewjA*7g}GOyZ7&x94GsqKZxkpyB4)BPj>%d-hFyBQ#4|vj*>CjOGr$?94%%cr$jK{3m8|Ri)w+qqwhcAUnRTXh;cbta9$X=FgUJbDMRX z`ADcteu-XwmluJur{0hX zL?w-u{CcrhYtv&XgDE6<6O-qKVsI*HR@qs{28@a_9Euas8a8(AOSz{L39 zZD_wjg2~Z!r==^;D-D66-r8t;4S^Pc}E-?x{b4 zB`}Yl;;e1z$L5ShXVVCL$M^K$?&4ZI#Pz@t40c!Ad+?-SeDj4QzO$}%=K2kRoqe)? z+5T4B&HT@7pFA)UJ|70j=07!JUgEXB%P27#Xk25*c;^jC))lc?UvzWBR(He5N4$mu zY}x*iwEG7}23ke=)j|_1%&bj)wuzDN#9iY?`1%#m{5nynPkSiWJHCFkjzn0|o`=c- zz1Fz1>?`h2U1NtT3udau|Hwq7>esQI1V=Ux^~ou0u`}PSXzI*1RWMV;7!%2e(+5W% z3+`Q9cw^??Drn65(SK*JdB+UpIW!d_laggZXZ3z!dS0cy;Lry2n%hA%XS~~7?~GmN zTasW_i6a-yM&K!(z3U+kU+2de9yzmsP+jZaaq?iBbtvil2~aHEY%u$Yk~VkL(l@AT zyB_UBS%TX*h}%|A$2bdUG&pLzMt{$-E{@Yhw~M$xGOc)peS!_Hv~Ghny@vtbjAFJ~3Ix`n#dPD#rYab8cu%KU{n# z=2e_JWLbayQLa|$78z`r-=!ao#Kz9+&xnlYxf%Wf=lK=5`M?NuX{1g$mOg;8U@}X9 z{G1}Z`Q$!V7Xul^<3e8{=6}sr z;lZLeWVSiHiyKh&XA-l~_(U5_(n<2E9b}rKlK)egpM(1(FAt1<_u{XlW!)hrJ-?Zi za$Z~ItSZ+HM~{*g5Nv8{7Y;C~Qk{uQf9;<9fBM0{!>E5G?L=C5ddZ6=&&0^Lmj{l0=J!Id#u=lxa@3m$AP@%aM zv~|{+LJf*6NbHGGAN+}#v57IRJS%X=CR^9hp1s9-U>;|b6xfwtjyiw*iJ?F6QVkY8 zG#Rybnj?vVbZ9#{Ey17lvT^!tJ_EfO? z%psk7RA=b8Tl-PG;wC=Z`5qmtNc_ZYm03?KIX){NvC&Li!X_4VfOj!4>k!7g7|#dx z^Zm0K-6BIVGoWnF*rNy6Ot)rbhpT7L7_zt7*g`+g(M*i(hksKsfAjlT`GkNN+uLqDYg7)wTdGxU1%#Ikf^j1h=>e=Vxbl& zI5ZUnl|e`-2zjaXTWf#!UVGnj{{Q=40{s7<-sio0*4}IV)^Dx7?>+0ByYIc{zb~Hg zgXjHgw_qeA0qaxZ7aXy|)epH);V_J#`3Ej9COEY4wl}*L+YH>pXl=oWP$hCFhVa!E zFIaoR%7j-#YGmHrh7*}wKIF}ur;@p)Ii4;b_#7AAI5`{IgZEUCvyPL{@CHqHt_pOxHA5h z@MWIE+Qi<9ci4ha9e*K;1*0~v-IJkYtWM=vwB^pjVS?&Wc%}2rl%oohWadCGo_!aL z@X3-Kvo7bLlXWTg<;Aw<5#CsM-POUk0c)Yj0j*8Ukg><3pH*Q&_aqsY5&5TI`B;WO zV}r41B<#UAoQSrI@4UMF6ob5o2hS)QZ)*vIKlt8y@O;V#Jn@lRe9a>eiVp`qS&n?! zw$@&I{(`?T(L%zrmPf{ncKR7}I9JI=!X3jKF8q-PbN60b&t*o{*)H!c^!j5VfS8hF z+{V#bf$NdXKiY;44SM;8Ej*&#FyN4lr?iUTg1ynkV~t-7_~SR?59;R_*`Ic{)82}|76%?A9LVegW_*2u;rOASIn-|c zfyj1gMth**dBAqif)V(A#t$Dpl508cW;JHrUMFxVzW9m3H=IUK?D$oCxnSgaIY$Ve z*jIeX9o)#XQ=f2K$u5+D*tUo_O z627WDvXy+J3hyS^j4xuH+V+}8&>dD4*}+OSS;lHlLb<4muFga^YGXyIuMM3}ZASO& z2Mj=bJ9zqAAUTPZEItNX|#l7L0oCU;1Ms4)m!Ykfm&6XQM86UtbrZB{l=ee#6y zo?B>ulGk#pjW13d=yN#em*Gq=+kG_hNy3go;(E6xwh@Nva@wH@#y{Qger~e`MDzfb z2Kz!LeMkTYnE~7G-JDf7aAK(G6al?!LM^h|#^p_wU*n6H+`YRY9|^!$0N3O)>|ZzT z*l*Bs3}qrhPb!n;ff+nJOJbibI&OW*3HH)948>1+tx{dMY^y)Ke)Zhc@?jO&1%*~g z7wEPo+B5m^axwlQ$2%?&b;Y7gye|YEgT_2fxtO%C8X0Bl6oCfvOYHC}S6SR`C5vet zNkk9A<0Y(Q;Q&17vUaQrb>sHF|5gbPx#CIgk@zj->>({pO$9p>DPa|&;_pfNhzy<7 z(GxlHm4g-@9gfxN6do7&W^l?U;3mZ{!vXpMXaHSn14*w1cbtbZFRF$)DvXd&4^`jy zM9Fah4&8lnn^o!S(7=*3AH@D7O+e=FsED%XK!o!dT8m@{Q}ZD7DpsoRy>1`ihO0g20(nX$-#L62q5am27qT1<%UUo0ZktNPLr}5X4JS*=T^*W{hMoKA({IO6NYjATF!XUA?suLH8iDJt< zGd$~q+KfoyH>?j*U-@ODw8S5L^al$nJ9#@TnmaleE@xzmZq6OPV+@yA-o%Gi_*i*U z|KpdGics}Y`l|;9fC8M;>=5hQ5 zi#}dG3oGXg>WbzI6aHuMp_j0{0)yl;l^eVl0P&0%2Z{Ty;el3iB*agh#u!Dp12yl! zPHuHahq2x-6G$zxJ9pXgrc;5KJw&n*g#w#KU*;n^9qL8@4+bs#sV_eC4^c6yI;bo* zR9F<@N0_uOD5=Y?w!JpBfIZ=B)~;N&_YvXAp#LG@bV1nXFE6r_`h)mF0b&O=w{9!K%a3#X~ z0-rAJFJcMk`X_Y!uR>aRGcvethq-#OkclqmL?LL5>y)ZEw_$Ne(TiWDMezQOQJtw{ z6HTz%h%C}x{A92?<`Z2WM9Ks-xT zhlaaSY&s?2358KneeZvBTLlr;2(g6VVS^y2!*~7nFS{QOXig~e`P>`=O@AcxveBI< zEz~*@vEQ8KItQ^;1oWr`t}B&attp5)yaVc@Z3!8)W<_7mD6;({7yF3|@P4m{pYBcn zt*kffHLP#f*F$QK^p}Z`D4{Xq0DqD`yyGn~kpCGT&Jv#TIULMWWg~q?*$#50!9VPV zh-29150OW{-J$*lgU|^j`f5wYy?ItBvt{R52++O#h38h6n1A!yqL>1Tk9hM+xy_tr z`fyH1IsW>ah!rTx!aLNKD1yk|H(s2dKi+F!8REe({s8rDC)~!7$F~nFWyi|wbR_yi zoVjfJoh^DNT_y{_u+Az6soR`_&U4k(x}5eNG{Gf5k}is}@Y{XL5E%*=AZb{$UIlh4 zJASPz>hW6Nwa={4=ka?e=koA?TJwFR6lLt_N6XqDAA^!EPSmU_y^a?uM{u{FISov^ ztu^dsrV?EA&&vuW20qggeAoaYaoAiMdYSClcxtxWMA{775tNRQ5WJIk>jGN>=h0 zbTQPVqa$Nl^c@U0NeZahEOdUvkLS@V*A4J@DkD)kY7g$wz(fm#S~?V8t^Mv0R>}=L zx(YUnITlTNV6e3p@5`lm2+}c@fQsi7)5{{Ur}hq$18I5ui+S0st=1wx-gtA!;84Xu zz{QEtgMMQ)to*fvx26g%HtGt4bH`k`4+gA*>h|=i-Q}G6eXcXe^0nwiVu^@~aw{fjg#dp_O zc90r9gRI{>2eSS%lDZ`fK*usawaytm@;d^v{&piuA^sz)gUWt2ATX-uVPhS_yxL;o z#zf{%-~~~!k3(;vCjE7i`J4ld_a{UCDQO5Y?RG8cyQmrA+I8%m+_7wJP%eER?dEr% zAPWdit#We@BY{!^ryZk65=Va*s{y`Y_p_!}bE|N_mo2(J8+_3AM3Pn>`ins89h1$J zz+OnRgZ|%1UX#rlZJ;u>hDYsj-4VHM48GYIIEKrt^a7$d$Zd3O9=?I?Et%z~srqG7c=$CXR>LPxIaY8@dKzDfq$qxpB zB=M}&Rw2t9uUs1=om5BH{+V?GU5TfXH!}$QC?V&`!W~LTv)sv?Gkr?TRl_`y)Rl9e z4RbL(1^_-8Eq8t zd4WxE*xU{yV-ZFWPui`X9KVf|?zpO#+_T3`>;d3GTYngl?{!9`is^eLSFGW>PnoNF zGUR9Gt~{rk{CoUx|4&0Z1Ljf38flNk(O=5w1yka()_$9;#qwBE2X6mK)?Sg`mr-cj zmCH@d&AE~w@3XsR)FdtpRL>Jw5E8qv5sJ0{x>HLpYy3yh&V1+@@lL3ATr+8>HIwUk z7<-zB{d9V4goDFEVlQFkpS0r9&QCuDKpnXP4yP)4EnTW)T!7)0n=P4$xB0+|S^DJVlEzCc6 z9~L@D74=YYm^9+QDvw~0b$M=SKu45eo%{EY_2x)g!|V1(vPm) zj|VL=uk@p{HVrh8L3QU#mWdPA1>R7y{8%hwF4*^5URCRnznfj}YXYdr%Wol%zrq?O zLbUH^QXi;xMMWE&PbhSAWTL;=N@8j)l-Zx^9ocKX7r7mx>=NCbBxf{P#hGZeGL27s zkbT$pHo$X!gZ;OVJsH!kR9y9bAEVwzs7W#aJNZh;+bMM4ih1g)0fum99F@R;3f;#T zFn=8TaJ@53q=3|mYudoD@y@sZ&kFz;Ap+QZD(}a$oO>?*=Wil@mMOSk@ zAFttI+wExpy;jYFS_tz}mulU}vTYtya)XESce;B&Dn>=-K30_&GG7kZK38f1Ft9rcfGnOp3cP*+>7K* z{_=%5tK`&X>*u|YniZgJKqP*~Cm|ygl)JR=PX5{4c=fF(oKlCd2Xgw17-VdV z8u&Aq=(~mk$cFDFPJuw;iadcG;AMM%=b2nk1A+QcrRbW{MraCoBQ_E}GsB>$Y6E3# zV?_S?xDnQlPitW}cL5*k5q03!CY8Sz=Es?A1(4T=PWe73l7NoPN!6pf!qI%*VKH6Q zht*HjBl@=(y$oYn@a)>x@L>U=`12nLSbR#C8_HiBEYC??laPr+|At>mpVR-2TfoA^OndKeK&UmKuPm8->om}NDy=ik?huCUjKKYF_|vU;OO zProTwUCf2Y7cpMFJnhlw1S@^(w%X}lWy6y%*&m3XU-4OX!F)8>_(;y%08_E)fG>O- z0rnYwu-LG01eG&%0yI$*MRbkg8ZLL2x8SD)BJ}5pdu;2LCLLdwIB18B{Slap4UzD{ zqG6v-sfjO$ZLCGPfb6-*sfgY_)(+dPm2jb7By9iUW3t(n1-XFl6>s%igRSXSA zk;EkdWQ`3837AfJLb*UPU=3yXZR7jk-J-#pL&=J8DxhN z)mlSOYMB}Sf(XjdVR`-Y9aZBv}Z)EGM>(lm&exFEF!}Q7_SEpj3|KOCrQYM5%fnb zvW=JVh70Of0#*v9{;_L6cHf`65yC+&aRw+lUL){dl`?PQQzsZL>{%1W4S3r?{U^N~Mw-Ao)L$P|Tm`lAC|pwmcW1F7X0aYTQwT9oeVMtV=#$_ACC zH+`ghf0KHZHom)J|#MX!t+eS|3p^iC}D z`3xnJ$c!8FEC^v!_dT*5~UzcODcuSZs5#rPuOYm3Xf z24k`=i{ZPGiC{D1{%i4BvxsOI0JeV{5{J^zb zfjAowaM9;TsTF-l{g#X7OhI__L!Yi-+-I9T!r~doDDGmas}R zG0u-Ry60*>diRfdX_u*+q9g9mMf+C`%KHs=AA@U-vOraaE4Hire!XlYmojaX5l_@{ zZ1gG171l5N3Jz#v;p&eS5_{twJZF%ol@{8*LZ0^F7bo{Lt4D|)VYpaEy^62ylu$7Z9 zG&bca>~&92#XR0%ohbh5|G!6{dI7&mM5a34r@c z)pN8vm+)y$Z@3>3y(?z~9=Ku3V`WcjoC7LmcOUpWANxaWAx!ROHKpMAa`Qho&mAHd zRh>pf0uun0i{3y9N%%dSeVz3_SKVzvZYW#A4;PqPpP{pc? zdhr}J3B6yes^h-<-CA!xQNa1dVmu>Y(}Esx=L=IIJY7$vh2>FoELVPatQFK_kffL0 z)SUVjJ7{gHQg||wD^?lQmTgMV<#MWAUk$)~RfhqBS0>lsOu8`qt=zCOv?pny_)j)#R5%QZS} zHS#5EPN9h(KTvcdhJldmel*dsS6Y|Dbc5@zlOICs;I!Q53;gMLn; zIbZPwd@=e(eU0qiA^?+X@pRYC5jKeaSWYzA0S^Csc&g))HOQ+ap@S4B9-h~vWP@IL zbk7THIXb6qHf)t!o zKbHv5Ia6oT-a8}+#KFfWO>QcCqbZa&y*eJE-Ge(lh$Aa*8t*3`Ow4gu88n&Ul5?~r z$xFQ-3EzH7blq(84BC*_5R5qca++!^5dfp8y4Q17TFRbe-iv2 z|EMbLLl-A5GI0$N!)xfu$JLZt9Abd5UX%l!Ls@%5wI#s4l+X!5>h-a?@G_ofa#C^# zm^wlB#x%A8MgrSzR(|mz{Kcw+E}i5xCGHq4r$eFh#mkjQxe1?h`VhGg(mX8I>wOqO zDA^(E9n0KgPV*DJbWb}%*O1;dI+Mi%v#;Ay^xH{`&-=B~_tU}1bRPvEVH^&JB*3WA zlwSAk1!Do(T!&XIlK_c=^|xjbK-#g+c)q%T8nx)qdBp`wZXz~E2geJGu39cYlMJ+d zWAY|`bFacNxBvCp;0Qh-fyV6)a6c%|By+8T0p-Lo%rt(Z$~Cd7tJm{c=VF2r;UwD~ z%6iQk#rpsAnFg75z8sS?J{ii35EnLWqLMY^@78~?)IIMKR=pFL+|K(Gd2o+{FsMI?>UKAGio{xLj+h;aZw@b(g>HpABLB2*O7ntxF{N#$-N@kHsT2B|5u~8xu z#UPsf>LfX!D{J8#32QB1T}5$RrV+}s*`}9Q)av6ggtc)7d+&9~l-T179=epUu98w0 z_Y|mGz@rX(kF;Urtpn0&{;LSn5nFJ-E%!YJMq(<1ct}!`T_e2&YxF?qq)<|`7@wWnIBT1Qbh&b9Uqt57O6grH2F zt$(!!5cQx7V@niu%hmzAD{#M$$7GE(*e=$bCqeyAp3Ds z=|iok0=aZQ5(t)DY;UQMG1!vJaG>)QOo!L7>4AO^{2k6kEWa_kI>XIrR{n*yD;QgE znFxZpNw!5yDx#{2e)>E!|52_@$mrDgeTVSFSn=zupQF=PcM;?qR-bF{FVqWBWwwuk zCqlilTRuVHU@>2Lt_0cMRsYk5S4>&i-IdaV6}SN8+YtTEo#08UMt08#&kuzCln@!g z9vlDhuAE>?ah%PNNelTOX|vttiWbJgZ%Ux_?dC;$>)$)AyRu0&;1UB322RBv=DX+o z-8V{&!^zWsdkl4{&ziu=K0M)}Uj8Dh>YoBm`Xxa<8PUt`lwSl&+Jj{r**66wso6U% z(xgsHmf8_wD!n`X(TPkqHQQ9+)AY3Yj(?s}cA~q4qcqJTwrkMKj$@>CJop?LyLv)2 z60v+5CpjQE@WxFlZT=i!!()ctzZzH-(97FGe5y&B375Y%MS8zmcyA)&a`z|M9~ku| z;EV$86UnR53u<3`r;Dt34}@#ZkHaVlq4Of5%*Hi0;c`1C52(*(`~uYg6*)t~|9$EU z+q^B~tRKL+^>q|lImiQzHpBE8=6M%8+V2qlwHG?!(8CT$P-E-}NS@u5Tsi%_#)JDA zv9L-0#l=RZDfn#Bk7Yr2LHDz&E?x;Ap>gy&IOSv&JqHuCu6E zW&B!=n5aJC3o{=RS`bYi)y>E|wUaNM+36kvM$Y1z=rT^c#>tFqEczQSrTT?iXrA2E zh%%j18n3AmqFJV0Z5v*mbIzFs`wzKy-B1`xnStnA%~QyqTl_irWi^M4E0P{YYh(m$ z#7uvNJxM>BEAo->8?~)3>kN0bT{?2Ht37l@ih6qwf}8?g{o(j;`Ndxe zxpXectO!Ye$1NSNQwk9TQ8(P6J?B}=Wq zXjD1>3q$*q&Cz+o;i#vq63mkXWy-VZ6viOO?V_N0WB&?1>g~LumiPhpANjPTXuTCO zj@00C|9+ijgaq@4gvU8(K$70;QgYQ7jqwlaMz&nG z?;`Cko6TExP5#y9cq4PR4=w)D?mCRoWC-z(cd2{1bZmxSaywt&U^`B4b`WXl9sBAd zYIlbd>7BkyOZ0wszPmh{S82|Mx#zv?UIQ%<`c1&Dku1~Al{JHmh3+gTCM|FCK%@j! z#}vXZ*^x#iOtpFvIVtqmpQap~kE!`UHHs*jpM{X%KS_b{e% z6gs4B#S@LWS$6haD-JwB(p*}=NS8rUV7Z%Y$1 z@m@yaR|VZ7+p2C0x7jW#4iGEXdhA+(cXQ5nJKK`wG6>TTHM9Ny@*N+PZwf#d(|Qhp z;?JThm)?82l^o>RCvNo0`^zijUtDKUA-5lEBsq=}+(tfRr;%cY0brB*8h6YUG#@*A zoJ2nU9Es*K{yZYE&9s0#;yI?)m%jWK#7+GjM4&`MK%$LE~3%>?mXZGBA10v_akdPs>hS1Us_e zYNptq-MZgE390uDNaBC%DP+?76!0AV@q-<{Mw(PEqZnK;&%x#QvUA_(jqQu+I`)78 zZ*IPzJ}!FXOaHUehP+FOk@SH6;!EB-FVERBnY#p6$)o z&g#GNj2vu=`8W|mrLQWT1lr{Y=qF1NW7s*!Ug1XRu3q9~jWRcw^taZF6IHQkLd?d*NJ#GK^7(59;knR}NaI(pz1`9uwQO zIjmI;CU(^z7u{YKc&ak`dt&t%g}$2uDI>=7mkqZU=QIEGGOm{Ew$(VVXBG2mci3tq zAstCLD!?y%xJxe4Pq^X}R!Th+hj~Da1fO1)rD#yQSm3`f@~Cji{$;GzW^2U%OlDzb zL|~|#L+*X8xreqP|Lg&hkI;MASh?8|s$#e1XX6wB#mP=axeq zAV2aQCAVeA!Em{}S7D11>NRc|_>DG&?26VCypO+fCpq(Q9_0u0OZ?d@iT~TPN4m|Z z!Y=H)Wwx|7zl+2KRrMl*wB9%>Y{1^s4~K z0p3FR3ticKk(6iOsG^KHj93F$>Esu(l|e&OY5rcyq*CjA{o~2zm51DY58rwK`KreX z4XHh*n%#-z(uk&wNbR=x_9>9jdh#T&Bhm>ubFN<*KZ)i3;{ z#up7{LR8V8uXn)XUHH~`^)4QLns+lu+aQ#_X?l*ASvNOXp1v&{Y5^nrbT+A zPW{L@sPAV)K#AGxDz8n;Kao1XyME|KIh{nq4d2ssg!t!I>l<&y4C;?X?$mgC@QiOh zi@~Tg{+5-Xj&dGVT2dS0@|?h*-5uByD7lgp#-LS=42%B^5taK4#9k7Nz6kuDPUqhn zPw(J-?_Y*uU^?tRmGWHWe-(~*lPX}F~evXlVst4vq}l! z7189v#>>u#*)kn3d*S%w;k8sbtZlSSIe`qDHYU;X&h*kF3H7E}mY|G$EVA~2BI;3F zL?YDUJW}oa#X_U?oyh&@lSbbry?#~6&&ray0OF3V_=OJsyJhm+wcmWV<@nsPrwe>} zvSK6T!b|RWGAcqp@+xi-wz9V9F#Yx^z}=tApw8u`Jw(Ty&vG?RKm{1G@7nQeQ-_DigBM0KIOO*?$;o$UF0vC2+5c~FIK81FG`ULYnN zWbpNw&eu`JJff~w>yB9`C1A#Z@`598qB~C5J>ZRZUe^$WnmINF--Yp=cm1AdZPJzrw_d2gr=Q?ig$&)**GV_g^*9l% zPS$RdfX8^V4~i@gP*wen;c<~C+mm>j3q86wA%sZ&xemqG%x>lV3|%$x{^S3AC9xIq zAA(N?SPfr({vLjJ5uz=%`-g@6r`S*nFLi!G5sxjSVHAU*0LS}AMTKdVA=P3lJq6S# zfzn+{cKq4d2Z}A?AnVBRL-V2`iGnJ2v${P0Ib;4QsWHd#F4vskBiusjF$3G8pTy;h z7nwM-@oIVJ_1ngQJ%ivRCCBM76tP(JyAN{gnff$YSaISn`tfTiLo3Mn`Rp7XuiwqFa_;1r;RN^sIU65;qbI%2eE+!x zSr8GIu$cE>NDY<3NjJUW8oBH|nU^}ylsy|ruL^)b?I@_GTEZa%-j(FF+t+~FD?7cJ zWW7u)!zDqio%p$Dgh1S8KAC7BUEq%W1uCcOqIn9Ds_~o+yp0k16JOc;uu)_H^7(v# zq|_M|K@!o{KobZ{u-&Vzp3gy|YJ`Ow#e(K1u z+hXcqA?Vs~d%Li*^NavxdtSJTDGYJFrOVs=XH!!aHQeEO7bQ*_W}xEQS6K5I&>6UW zRtbjQr`BH+0q6-(7VVa|bM1*QpepCz68=uUI0+_Da)W4kl?zl)zfb}a^TRR8&r^u& zO79NsvWSadU5tOoaBp~j;wjf}mK^@TjJdgPJI^Ph;|Gu75 zGGVQT)EP@1j}(F)cGqJ#lPBf1PMWCDgCKbb*ImF=WY4!YCk}ULANu9 zoMwLlXdbIqsjdTe)boJ&((&CLEVH{U1}U~qjk|;ad$wN{0kga5&^<*|uDhj;HvT)c zY`>6Re92qwxT{H-3x6ndo^yXs@hC|q;EL=O!G0TK;BuetE|oxnm;jdKA`L&ZA}1y^ z9xR`5JWO%`5-4rrkF9yL(ezVamm7_@f!k;&8VGB7sh{?DS&|j7yGZpjQ-02g&U9*X z_F*2M$g%8^^LKtRFjCbV&~n$=85osUluB*>EVaJ(X<9e4{9<((0bX{|A&n3f1bH}m zkrTWrf`i3Awr@g=sd{VlWY=t9ds8EP?k~c8BYG&tEhQJLdtw!;@-S0+Y6P;~d&yp^ zV8(>Ezf$wjuES3lSsp_=+zof(RK1#0jCPm&_O5ba#+KAIMC55U&XhH%jW^ZSbzI{%%|oM3=3j@hmI{bEq=FS-io3}r0xbjki5>)*Ij(XXTc z0`Ug#qc4K*2bYgrzFn;o1R*Dk8ie$nLghcpN57yW%T}vQ4|7)pI^eJq!+FevYx^6^ z>}p`;s>t7U7SYHU#6R>X6zOYTI&n<{uj8H8Y}gh351eUd1rM0G{=An`N1?qPC%<^d z6v|%ID!s^0d+B}qZu&s`eEz`pf&qP^k4cjc2v!6~oZ>e5ZWzl9nzW^AoPgwtI@!cEWSZHNZI!L|m8xPH>wZ5=_8D65Uf0aPiuM$n>U?IBzxw z-!lZb!(OYeIE>SX+X$Tm8}REm{_uJ^U}pi_tF1Nv77g zJ|D}~lSx1LrL-lQHygCPb}gcLci}rx@FuiQz;+*TlJH2^t{Ogtj-=q0o2Fx69>yEgqKBc^e|R@1F=aftKF4I6Sb)Mtp8NWHiSegb0qu5E zdhZ^yROi_}i?F4e@Ds*1r|poAf|DANM1=x++jHi_Jx`}A35)GH@$q!6a$C$l7Tb$r zX`wcntG7JV+E`No5Rq!#G!}AHdA%o`0)5;+!TSo|6x|&8d&T@oezVy6ja}M@-}1ut zV%s}+4HtMU15_z_^k&IWiY#expByu*8IgIm``r)J{dqJmC zMnQ!wte0mh~<%iW%h%!x;(#a4Cu>;gA^>ah@H$=RV;nM zkv?}Fpj5_z! zNa`8S0Ky#`N%RvSlW0)%Izrg**uq&Mq>LZ=Eg}^47xwiLf`$tS%4b%|7g{$i*gZTr z))=ijnNd(6b&@1d0=bg46E*)_uXv~5kg6O$HI$Kj#;D|3X=91S7}>#rln~@;<>_n| zKOv>@Ocj6syRirLyx4s5(r9Jvt^Vp`1Fgk#tpK;TA;*}C@P1VxO2|i9!z)r8sERl% z$MKJoP%{}n2!YZ<9}F)DD?MDuR;q}sa#tR&kEQ#!!c~98<&BoTrMZX6c7Xo`{;O+0 z$%a0h|G%tEfzU|;)7|8@x9R=0V%}%jmVoHxIIQWyfm)TxaV&jPZ&Uo*r_R!0R8@4* z!E@J^q8BGWoT)_$P+9nSE&|~O{O^uEJ29m;eVsp-2H)mrYvxoh<@gll1o5{z7@i(Y zOmwc^rkUMq%1-6m*!~x>3&n8E&Ye45eTP7%n-zfY7bun@ePAbmU#D_ zkjcysXXlwhF03|d)wqBISjM5A`kyGlo5K$Cl8#dXOE>qbBh%-pt6Q%{E8YW>@^AsJ zGs!WXmer<)!bwNHh8K`@DzncQ%>j>sKz!63ymZMbvLh|#DYlka^h95UC3&eJ!fsWe zoH`UllADvGLUm_EfM;|+wpgdLGU4o0ly1MXB^N8(eOnd}ipw}(B7ZNFrHk*m6Gl*k zx@WTRQNM`h)%U4(9Em(^XWJEDAF}i&9ug|g$}14acOMwLJH3tCPOb8rMRwuK`n249 zKW@jMQ`4Oy^3wiS^q3(G=>DnC8+6v##KY%*8Y6bk5>GbXV7~g$VV12Kt430Vo3q|t zr>^}3m)bL&hvSP&9rSd>6MYywImd8Ab^g$a@`XmsXrg!NXV^(ty^8oF98HszLk=iD zT#uywo}l`RsvI$sx<9c09{ax+UJwRg*htJO{SiR5K&WhbmMH-W@zRXHmidZG=7Y{N zR+JX*7Zhu`6J6zfe_7|-Ecd1t6;Vz{U7-pTvl}~0EZ)NMyDvX((&lIsN+OSYGCP0% z*4i+K4R%_1j1O%WMD&G_BKRcdPj@zJ!XNJ?VHL7_75j97R!w(>4cTtDCYwuRPrsAn z4~>_8bH%TV<;!}q#NPe%)ryDV#G|~QeFfr(=)U{ihtJJqkHq(?ib$QA_HmXTkdd4j z^M8KzD@Vu$WAhcakWQn+y7;xd@Jz^Q^D2F*WG`-8I@fGvs-rFJ3vTkKt>AGm`Jwvx z-<Xr}v?yQAv9I*AV0n7hhwHnxR z>u|M{yC%iyZW{jc*Q8v5@k<7mFdqkp+p@ zjDrgatSs50WRR6L=QV?TdleZ~{P_EQZt7SGVK0Lv@+sj-7RKuq^WG01By88N#=2y% z`G!Ey-s_64nRh@9gTjesbK$unfmdGhW#p+XJm<~KF9LfWEBn;`m^q~;B2dHD;HfwA zKGE-uyi5gOyexI5UiP^~jR20=?GWs#@3d&fh8l;dOjhxeGKEq(zPUSODX%Nd^;5C) z3}hfD&F)@|%z6R3n68n0SdJ_>@5uE(x&eF1$l}PN+xX1xmfgW%-*2;NW`5)D_TAue z#0EtyM*+j9mlqaACN+Kd(i97cW6&}f{$lko;G|>T$@($A@?gJ0+4?(xse_8R6}gPn z8kQ}-#?adC=tvIenhErX9iKYWOwyiFDX%c1w$1*b&=H-tOKAhLQZgQc7mK<7Mqg@;Jx+^<25n zGs3QD@+>zXF|3Z1R)R7P730IQE(Ghach8mt^RCwt7trPgp#pLVR+Z(YcFp}7VLLq%~6Qd*`ndN^rgO&R@4orunTneESH>u<4 zIY`C*Vqy5nnA<3=F1*vZy@`-X-MqW+GTJ!p*PHDA^^VT>fm(&SeZoB%1IGOJ?pQxs zytpMo`bKk)1)majOOu3EIZ-U;om>;}F+=mIgNR^bNovksw3~j0 z5F>TpuJJ2u!QlL>6iW?Os&Mx{ixR?!I#GM}h>yN^rg8cGio;99kkTt%Jta`@KG((w zgouV0RwRAN=XXlp>Dh8W0EUdZXaBdT)T3QxcL!DJ__CY$l*{a!BqM>4){z|@e&du! z2?(_kBc4d+h{h@^3l0xYn_5|)1Cp2L=WL~B`E;a#DbQj+N}u*Md-3NL7o@-pAL05= z(jTCrt-O!_G}=dhHo7dZ4&l>D9e{Mb)(VdgTIo*zcJBR@z11DLmEBL~bi!h8Y!M4C zbn;vFc6$2^dJUpnwvD?k-^`}Xf0F=`gZ!F5?9In2KN{Bsmvxl*82W^@US;{4lEvwwPl;Qxl$a%>xl#M^cJ)H2PDU1k{un&Oa!| z3=RA|9qBjwPGt&=9ljx&x7R-mdv3fRJ{u6p9Oe+gP0jJ6Wg$og7XD{-!N8VKLFRFE z!ZS)uITYVp%xo$TAGK=w(ZOaODmovXpnjH;=Yak~AZ6MX*PuP;|w-iN3v&WcR1n81M_P zhJUCDC34S~vl(uu@PqhpLe6j!fMER(if@g^rNTevtSge`xev%pf5$WGqk!jF@sL>V zNlF-DLTZ`tXZjlKaISwI-KzoZ8@A9&|qHUT#}`ch#GEP2vVNmS|E@P68| za2_#tE58I-SC0Ai@CF`|T5T0(61ZPF0;9TF>n^lTMikcfDujEFQlMr%9jUCiupCCy z{w=#56@pPI&~!szl)2)F>eLPv=?+ZHjklm^%vG+=B+HE?M?Z@(tD}p+b?@l%mESkV zJu8Bg=RCjt1`HX}e7NxBYl`UwCc&v*&0DeMKhn7O75x&mGB;a8mV7VAAWOdm@)zOC z|BuRVP5E^$=vOPe3`XweS@NE#_VhLK+@g zQ*J*YWr9(Nns4AIEm`Qlv~w&YZXmt>{75f#rFjvW>#wYKwzJI?bcLT@mpc9Tf2WC3 z$h=9~Kn9Zy?d#OLm${)VY-~?Bj8TO2B4Q5AUKiYzD*blWxz?e>A@x7l(z`7|DgSe0 ze%tK)Z(Y8%tWbW_e|Dhv?GbUglKtEj6FOTCt<#qfRj}#4D+sp0*beQaSRN!>JGxkB z(+yWH;^g1@IF4d2AaY-Wc0S&SUYtFh-Mz&8Q`-dp1ejm^uSlIF`mx}Ad$&Yian z90~9Sp(+^eE5Jt^+2Ne>(!+N|vj>!KPR^Axk6c(_HiU2ZAu86XXYm?OYlQ>{vTA3K z%?}S*M9sMeN~WDRx;l+Q7TQ+I!@s;9{?uq*lvjjR9d~a<9B!n19GmTHS#soeHL4>- z4fK60LxYIe`*+eFT%%uK(VK26Xu2veO|oh(#3;U5W%+h=B+KeFs+u`z>C58N|8>uL zm0C~jqX3Hr6V5I6w)RmaTcuTNVQYiKto^(2m`N5JPQ&^9Mg*xK8BzZcn4L6(1WMoq z-RC1(9vEGI#2%rFiJbD#4?h12eq3&Q4O%RPcQrhDFYLy;WiNu3778<9BCj*2IQ?X) z_Ps)}yw;0|67t@@VF#C_#hW)REq@tiBo?;YFmGDdJ^_9DTvk=cSyQ-w@#H;YCH$>6 zTg&5u^Q5F-^yYH?rB&NR1B>hvo_C0#E!jD7EEj~O&`vXg-8)aSr6;qUn)6wuX2M*l zDIV|R;_`M#*BU-QsDZQktk!Of7F;k|en zFQpwMoRdJQ7!#PlOjLaN3}ZF6Ne%H)xnE;uW9?Yo{pfmlA9cA(aZJ52mcp^ItYw&K z|IIHMz^G}%rnLZ%4CH6acxu*hZZ%ZyQj+~~B$aEGKk43rj=oL2Zm>kmysiItCV`SC z=YK>(P4+`d$cm+`s&49r`F#FyL=a8l`~$wHQ;kG~C?bN_hf}+6l7f=(ysv(%q<=2+ zXm_FWFD>+}8SLs!=-fyS9eIfeEl`8Hm>1+ur+YwX+AsXw^rm@AWGq@P(t~_`^8$PJ zv|BAoNfa0xf}0SLsTJ=)pvEctX!|No&}&V{acjwyC$5iP^)GG{4H@%CqqA ziwZKBYyUPVA%o-b>|ZXU3HoS~%6~c!8`ckqEwfhO*F?o!OSPrp_GjJbwBomb{;#6W2Cmtz5aDJ06JUD z>qX>sWwA8l!~^VtPlZ6!#a<9FY|-x#{-j9MQ+qW1WBn_z>`gvQ)5APrDj1&~2xU}a zTIkOQ?`Q{s2;xTTW31##%R4_lXo0NUjV%_8idKT$y@0;MnydU%np5@WTjIKSD>nUb z?Y2y|y4356b|h^al8VU}-K~qN>fy+!{ZaJRi!s5>zc*w$iq6@uGoEe5{2Znb5$YbP z{C??=nwKbt!`Z{dK@7Qx*Js8Y(|RBh+1!_Vrt2W{ z@FbN!MCDfw)<#*Y0rxDzbIur3#hIZ_pm%YqK;%9qGS7SLKS4ACh{!gra!YwU=fERo zZJ7=2`Ypyx99MNosm7gKy6H~l-YZs7Z=QnQ4N08X`8!4`gfONTi0viT8`0V(%BT`J&&-?fZYOPgeRa zyxe|~>w8RLt+a*z>O8Qhdcd=3QOz-S=39LU;}THgrK54W(1gxo(AZewlH|b|CVcXO zUVQ`$=HK98IW#@ic8vw+U@X66jb~;TdA)9VLQ6KAKNqPZ)Sm0{sXUgJTPLkVSiPHb ze-$F|t)%dclRGR~g3j-r^0ctLtQg_u7>F0_F27vyJ&1mHsE>uvm!-_zehU{+ifE0> zr9yz1DqLviorNvBQ+1O=DV#B$IF>Hi@$v3o(bl@ir!(p`=37=o4$v3e>|qff z|2+>=zgV~qUZi`~VNqT8f>&kGc0Mx&Hh0F0Q7h2&4ABkw?S%|KaH^ z1ETJ}XkkJ^QluNCL~7^;5h>|z7^J(qK|&e28v&)eySux)OF9M^xWDKA-}ipx+i>Qb zJ!`ME_S#BzzoKjHZ0IbV3XMOa{qlOErEYWyaz5QGEYG&kvv2vnAPiZz?l4>kQdWE1 z#>VahN3LS6656qKCdssIe!Se`*2?4zh7JvW|H#t+X^3I*np6@+;gJjAapO2E>%eNm ziest9StDDmV7$4iIVAqbmqd#J1ZVMGjeYo^qkMJOAUXw2XU%F73R)N4 zrO}-4cD{J+B{aDNRhU@bYWhGilbKuWX^UI9KUt44MH`n16GGt3fR_`> z7cz3Q+;P=ac9NGTtGtmK3YzL+{nZhdboM=}k-xdGA>!cU8qWY7+(bk(%U@_Zlh7j9 zcqeT>w{s~j@A5g$YF91mZ3_^N%T-l3X*`w10Im#I%milhG*4_Y7tQ`E}>|E}KwBM7?qVz_Zg zc_XIQ?{4F9gXOv-(Ixp6 z8vC<+$}a(!-yyX99&ln;{;WX0akH2O-eG>`<%$1~D#eXCTe%D|JylTb4r?CQf6~p_ z|D>Bqz_p}0r-9OMpCq{owAS{1bt__H%Z@d!eR_#b5 zT%4>W`__z~!lJToVa&!?l&zEj2=TnX)>1C)h7OJxEZsH~{u#ZAiakt5STkEAQXhT4 z@RUQu9(SqkeMx!HcuDj*E8)^7`*W`iu}J z&{REORDxaN*BDrMqMk)AOV#CbL>|ZZFw@Si@>{9?kg{)1()=+WdHH?9ZQOQJc7=+` zcN)iUU?)W;bcy=4-;q2Mr+26^Nn)oLivkNsQmH+LNZAwe5H<7V{!$7c&+qX6 zXGDU0pgbaj84b044jd!c@3sj}^?HNwmie)1;dh!FUSd-f#M(aL}%*cCZNRf(ama+cMknPg) zoUtL{k2wPMJeebou*V&@l+16lPiF@b>a-PE#)7>4Z$C)I%NI}%Jlr-Y5wlAU!3N(5 zc~Xa9&~r^0|5gzGK)r{kqw?Q(p`H4_?}C}Atz-43nf$~tzoDd&$goDrztj-M{1v7$ zXVDOPL|n+wmhGkGCB#>olO{DEPR~L6qJtDt;#f;?t4d;v=>5?ad%)=KSczowav>-d zxNY-6IF0?d7{*!nQgGY&;;PB!i|f*jLvD*MFv^jyp;e+Iq3cCfaN+W1VGnOwbG{CD z$8P2K8LIR;&S+Yv(5-N2Xv8&se_*oAFJShq8YJs`W6*AMF(XROe{SR=P{UmX+|zlC zj3dC1otHCSh*g)iCo!lGm04DOkV&)b8cN13vPxPg_%eu~2u_POUS+e|P7}WNc)!Mq zuz7GhgL|bvCePZd@|0#>&bR?;{<>!8j74+9&nM_D!GcT{dqgcRduurGhG!-GLs6r1 zS2#2S&@C?010A~x$xqZm+bJx61i};J2R3(NfF;ZXZEdc^!d^cXEliJK8$#>kO!YT7 zMB7gWC(WKoSovJK%JsG}HP2`6-S~$|139kBGJ1pLV|Bz2N^C@zm^4--)#@F&zw@EF^dgJqcf zTrF_ttN()-r{)!(h{~{(h~2E?kn@a1d5Yza2m_u>LMTLtqk$to1;L|?n*-|KkI!asW6EUNY7ooVJOE6Qz&9JRHAE~qXO{#+tqVW?D*if~*$>bJ8V zC6_OohP89v1hJj>f=!a+llRT* zZS}OV`qyu`kD*gdfH8BM=wyT_S)}?gCHI;o^A{5)Qy@k2u>?4M^F`3KVy62-v7T?~ zA!uq-`V4TrC?}Wt^b9uM46q7UzyFv%a9ZOgiv>^ela0f8O2({J5@^Q%pJQe%C|0Ec z#e>VrooHT*vNO_awQ@$vZflCSW70x#?A`>k(>I~_ErtJfPC+)EmPtRr@^^k+@hT7J z)#Gm5Ni(;ud)}>Jxh~4CI<5&64-zlmGD%0593A~nyW0SMf@bZ!*7`rQ#DogOxMD7y z9{z}6nu@tZr_)4Ep2tShdheGnw>a=$gcJJmkL~&NP3;xJl_tM3g*xhck_|!2WM}DP zbs;U2sL!OjN!9G6RbZEC0>~+;q+6`tcRf&dtO*+XaT(tsN`t__5?1oO-wgoq1`oe? zNu%q(xD9P&@OgS7P*LLkjwWZ^$RKO9>yjScefO|kw~sMI_4{+%z5qZ>Wd4ts;Irtm z$liI)!5P_56ruOMqzlvrsjTU&gvVWlE?g`ztjrNC0J_%Q|I@WpflCw? z89)+{pG9?v?cL&+kSPT&Y*fC!kh&AP@k*`ID=0DW67+A8j@@}RgN^F_>V9|(FXP?5 zzqDNMsvx=mk+y?fqF9K8Pd_v~HbhJ>M>x|CxQU>;J-peXj<{Jolts&9S($Y=6ePaV zY5ODpm@ zK_Zj^Mu~&2Uot{Lhw+!;xahFrbn~a7zcHkedVz0?08I~2d&>Z3Bqe;G(c{C1pDa}b zcuCStw+bD7M4sXt_Fle^&V0iA7(RvOfC8xc5?`yq&euLc+i_v*DnrJ6e0mkM6DQ>R zpcf=%t2$5_vT5d9H!ZrBNa0rZRMuHi9a&l2%D!D`CIqCbn2_5yI2z)ISbtm zV2W<#W^xCPlN?|FNBYprolXFg91gwaVC$}rlsh}`Om= zS*TkqOoM8JrtX{V>c=|t)kd3s;kKk`^o1XB9+o-W>rCZZu3MPc>eSgAWa<7ha@i7R zV?LlXp6B*)i_5e1IV|B3+N?C?HS#FUr%Ow+vpjw@)N2RRatKq!j(#<=wZ9q*MH)x)E2s2O_3;ayjm@fX2;I5!AqKdIy9 zRIrqyAqR+R;T_Ojdba;J@&(vh9J|8ve95Fl9z(XMGi-!rCv-c!VpwLA4U%_$gh0H| zm)CXMDowmS?(n%*-1)z7e#aW2lCo&8^*U##pMTVx26U}upQ^!f*hpn@`Z{_z@!cZ| z+OH2Qf8P81Z4N$Rg%8}4!s^WL8+h^j49JG4f&w~^ao?qz3{@2WC`u>x?8-6jv%fL! zIxM=w%$o|WFd4|kSTd)?C})@eoTG-$JDkcuzW{KN?5Tml05W|uha_D1R(h#+ule_j z1tt3;@bgjnOp8geM6QZg>~E!#;;P~(=j6}=+B5p#XJv-gvXs$0_nM$UVJRp}bZZ+X z%KoQNRGUe}XNFTNc`rFEJO8&Lb?cH8d07jre?}+pIQ7v0LR~Y4lqW=5BW%9uF$Pdo zt28?~r$ukhTF?zz1vzZu^(nN{4nQT;TmR$p%C z_Pv=RbjH_hRAxir`_fO5F)q@U3HgA>=h2@!X#eZSZ;=UmqQ2)b7=Vem5IwsrO3~S) zG}wLF#VGZX;6D@*4+_2D#M@dKQ{I+QhWC?~1;e_RL1k9lS>cnQsR5DW0iRhyWIg@L zpEa>PJ;>ea^)z2@p4O{@JUqcBJ(tTqzME>jS6p2cvRi>{hrenyuOTT$XOXDvnZXJ_ zRYVI}edqnLUS)KgKFSj!`e#-`hHUMq!Z!eI_<_@H=s4Oj2Hb^db{=Hu!^_4n@hIH@ zoJbvUdNa^@dL!TIg0}M)e$nOS(oY(eP+F(7f@uX)ytY67(rRT)Ueh@3tMJty;ssI( z_TOz#yq6Tx!6>G5Pwv&aV2`k`>m;Obca)`l=a^eISml6kvh0jaW9Nm*v8cQDCt3(& z;q;6}ia1EQHtE5s0xCfZyaM2)d`ACwQj`~f9#&dk5@2EesxGQimpsgM*D;^s(pBbJ z>g762!2R*=Fhh&nUF*7d@Hft74HFbz&zLmV-IKxum%&X&-SG0svy;S17MJHFp)9LG zDFrXoTTA*(s6N?5@MKrUejJI$?7!()z-RzS%7xTw@sC~kn6|CRrnOD1Il<;*^cZdz z+Mz6W2Hvw|#|m-`SZC_+wV_ra{KGnb^OM}SqwKErXOU@uEy}=ghwVBVK0e)mBVI`GPEx6!$>AZuh%g219n3yN6|AZ$bms2^H+y| z5AHiwVWKs$xZeRozS*T$Z<`pbQ+G`+V@EeY!(IIs0&I&aXW0rT^^Qt}>_irt}Pv?Xrz2s+&esxZJm^4w%{qyT1n&9}Ov| z7` zj$_ebNxh;^o<5AHDuz9Ga`IMD%cM~kv}K2k?zSzwWh~5ga{$#kSCj%cZU*cCW@LB+ zZmw$L@*GhGb|qkqpO1&y;4O$Zi%C!Qd}KSq(ov|WEEZyU%P6#&`~y*jYpGKvjGU4D zt~B1yxu&Ueprb7>itgY=Lo2bM!IkyB%B%d;!wNKr=%UCFfb67_t?n1s84!cs9avCv zRb5IfqE|>%x<-T)k3^JzZ2a~ke@!E*>NT4TqdPY*A>>!Z&x=KJ1Mxz2j}jHw-vkeb0mk}Mj{5^uLg-Tud8tVOE(D_v>&qcS(nl#3^*RqVK~eqWM!yk7a)L28CQp&mis^QT{ z(n;r}mmk;*A17OHI=qJBun}%r5Tm?M=)MrCd9;k-a7)Q`9`O*)JHt%Gh;&dRns(F%o3-ekotdUs;F9yk;u-YxOUGsM$Wsn7=%+ zI`0jN6-PJf-TTFxuo&ywRp{3vE%si_6Cr=t$I80{%>VkpUvr3Hd|5`FC|asI`tKad zq^JD&&}-|D^g*pp8nZnS{a0a87%0L%Y!%g4M%lUrJ&! z9)-bCOID&YPG?-s|LZQ94~p&heer$jlcaTBRldsRFgFAPU-Q?%A4RWarz@-jmSjK{ zTv2Bj^DK1+Og49)8)X|xo-^kK@JbxY=O2;tEtb>ajNcRA>Z-S5d;(2P8AMwfI>_sT zKA@O5_Oy)$&LlU!zcfRv072}zNQAV+aB0RI@(s!Cysz}^$FlS5h%?H7_h5>A1rWGR zwICX7FP3rn_O=%Q(DB3Htg>4R>W~5NvKORs{Ty5r zf0GM~?1mto!0h~|#d4@*$ue4V7Tc*%P{*R>NQontu&B%orQ)EQ4;1RHGXb z5!!K%v;Kbe)~_#J;oW&~oybHycZ6{M<1#7`I2A&TOdk+DOd3_g98ZRa0JUJM$}@hN zQlKPzk8&EuLKN6RH*IYB0T+x?tRv=X1JsJKjpDa<=%BHJ7&qx^Aco(Wx}l9P0*nR_WbKfE(7v?6YgFZ_h^- zB1Z>nTDQrPW)pP^DsJjxGp!#;*(U|1hjf<`!{wNAH_IYBon*Z={fYcoHMle!AB8&} zB6%pMk9%=rhJp}_Trchg+rm16|(Tmvdh?h4I2MrI9 zy>}xGq9XKpPN#8-S%~mygAacE%nQ+_`)D_Yd=37b84vg2C(NmZa1iqqb`}BfYAj2e zr2OHOccB7SylpQd3$JHacMa{tw~|jRBv|oiR8V#`BuL_rTRYFKo@JKrp*y^*Y{u>H77r zXuF-~!AiN|KgS9G>gV`qfeLIF3~y$lHHTxCEMMXFvaNwuw7=SX+eL`xLTU`no}R<; z+4aB*$Lpk#XHv=jPuuTLsO+vj$0@?zcsT4&=zRKo*M;3^b6E)k+;@o`ad+Xe07HKf zz7VE{P{@+b13GbAci74vr*z_C@W^0R1o~??4=r=ej2D|)ZLfGd;^KM}!klyX?$fNWQ4R>RdzXmDqjHo{|QXiZ+e?~Us z?od?cAxoL})((_U<(cj|WBu3dKQvQqY1 zD4!?&JMa~eG1^^2SIhJ(#`zxSqX?4HeNxt2lSAv!D0Ub{|{ zVC0?NVk*qa(_RLVYrM@}K(HtMTrT_md)MJ`HMlpw+}qZAOT0UHG^;y9=)|sE;FCYy z*-yi(oe_JH*SV1e--Vu}ik8z46K4nh4-zByz9-H}Pp=c%wa*XsKC%?uWy^jOx99qH z&6j`oGMw!@GJIERnC=yQ?kku@z53=xg4xe| zcsCo*Ba}J@FS`D%&nv*{1fQ|^_6Gt-a-@S+a4&mBYhxI8$w}r}61|;?qUY89V$c+H zWKRcl^}R{M$PJMZo1(h8BtN02e<*aB>~KGRD{@9>{bzi7HVCipg%?BatRjC8DQ|S7 zkan-5J}@?tmk12Y4~b8pi|#CkNZ2=`NnQZz$nO8Aju@d!mMQcu9+pT6E+qdYKlq?T zO!}fmyH(?Q+7*_(B#SUtqE{PFNFiL)#~Hxpk-qcrwx#j`a5;VJFg5Ng%*5?2EbCAi zpnU62wPxfT7G0gg;{QQ9%n#5v6?V7)eRKLhaMl>b+w5Nf9^Z$4$PfFRRVgnj;Q=^G=0TRjfW8ak1C|uf&>cJXj1ofMCYsv3HC(7 z)jmCiqVlKxq%d690c?dkZn=6XJok;QP0XRzFy-{@r8G_lz5jgCs=TX{e?~7i7~42? zbZ&DJaq2{L61$>Y(gVruhbN6A2YuOG5@*P?xtz>V8Y1j3&HU?xlg^-_`Tt;fEIYH< z89G%)3~n{&QL+?^UFb+szg-{>USRQ{r9@Czh{1IPL2EgP0a7!4t0R16}Jv#H!~LysBM45wj?Wb!@ts z2)A=62ZG684)kF@PoYyod)pB}L$cJ7-fKL5*;VmQNA;vO?KbWtjiFSIRxQHhP!!(BgvFK4Z z7k&zXA%Gn*GiD{KYvTJ>*3h;`ncQG-KfV`5>G!lpao>Ex@=acp&F5aU-MuLtMt1hx z;YM*omdyc}Kk_VZ+qL$%>!{&`ipL3W3p8`N|>OHrtaZZ=C&R5wohIjGqTHFrLh+}Ok z36EklvF__}AUicAdmXrGnpAu?laC-+o+5BLd*+M^6}o5{2AU9{l+RehmQUy-rTU#< z71&hI=F@%b9odCDAkq!c)pZgv$N14#MwIu3V?4PZp|MN*fMI;TzV~E5=ch;Y9}wNo zmq@G^>1R)4cPM@`+ro-4Qd`2qiuEvwmy$7p9G zFhUPsm-DH4d2u0(Os2$W0+=3OzghB|lXio%|WM9EHw5CwQ>y(@3%wvYn@1ZXQ92iq=bs1P{O+GVs(#^bz2GVHJbpk$)fIY6KzklWGK_3 zB`Onvl=tB2T>8U82vB2Rh9zEvN{K}Sd5{`*fqqZgqd!k8Q3~^%*igIhF@TxKlPubl zCEXEo2vJ8V4nFTIOc@VQK5oxVy9VRfyxM=!Ew^^jkO#9GbID_@+xGl4Xm0h*A%v{M zqDYerA4L7VJ#Xzhx;p5b!kq1_d80+_p+QnUk5@5lF!K>xd;gV;aVP8bp2JLg@B6Eo zjeF~nlh@5x-IwKdZKvdxHOr>tH7;mL7Vv8;wITR#Eq{{@mbk#e)^r!0w9&p_H?jdc zTS-O{dhpi~*vSIt z3*U>Uyl@=Z$-0mjv9)tgw_|oOuv{eU*sL2NeK{N(Su)S8^I1kbuM;}_eeY@q8Z7aD zbWawA!aA-NvmY$)wfs)6tdlj{9srvZkY_bQL-|St7F5~jhE_3^>yNjLkSF^)@AV{U zcI;w~)YzQb*N}#bJ|!w1Ulhboussh(oP};Iun>yu!;did3=1c1v>tQwXug_y0)a!u z#t#X5hR+q*N4(K~=iciLLIeM}ZJ*d5Wx&;a#_=fylpXzul|K0pF;T zmwmQZBWUXRlYerH?+w(CEq9@9n=Q%Ecf2G?)brMmrJ6308{32*=Awj3I~|H_YY z^p)x#Fod>m!2{Ic9b&*cpX*0OyuK*aK{wEhXtxVrU?sj%WeTGjzB!ybCT|UHNMx%p zaD-VUTALhxm*~ui>x$f}xSlBIRYtFEe1~|%PdxStJ{N%d^D)jX6@LQq>ldozD1UBY zp~pAS!D*^<=bQI}8`w$7v3;|-+nw<4XA04o z%9*vqmV4W=@djbx!W}ft{(ma_pVDT8Ar7)>R@mcH%a8iD;JL3-3fn@Y)ST*1v znuD+|MixZKqbqMPAA71Ck%Q96r$t$G6JdJF#yX?CC=pi%I`EGb|2uTLX+d}oKRm3v={Gv&k>34X^;4BJyVYiNNBE-;;)GXz|B11p74gXYI0{$FL%_V4f zB*a~LqGfM*=G+`rd+lUe*kq(_#WY~5*}HCg7)xnEbg)9dZTko5#Mr(g414+Ux$NetN6 zdhPaxS{vaGGT-s~Y1=3N!N#igzDUQ%y!!M~q9=p8dy<*Bc+b8H6{L{Y#}TcQJm0HC?;%5X zot{U&ca^gDAd=@(hou@AarLfKnMqhMZ_GHjxAN?y^85$=1D_myP=q z+A~x{=E0aPG|#omk>y`Xet}4v@%^PhVG8+u+sT(81NCpXJkILFzcaN^$ki(UaQm5j z9+$n%^SO!h7+DO*#_7V+ksu4=DU$6I{6`PY_a-XhC`SS&I$y9?YBgt3CcX_TeP8~A zYT8BPm*=f(tq8L;ZRj{61;4@(B@)X-#!f6F{Q3OlD{g$SGeo-&t`1X*5&TBI__r-Q z6Fj%Q%t=XrvO@pscd9I=BDpLazQS1$0}n&I3q2jUcEJm_S9h>pg)b<~x;~_#?{n3a ze6?*X{%3=lmnb0Kppk|dfu8a6l~i`z-Q3~1$%(1gh^C<*6RC^AlEx}?{b~@$_Ly=# ziAkr#D==kl_&7LKk6?CFSg{wOydfRER-WV@=4YX3u>y|3cfFll<#^Jo0a_53v@wC2 zQZS3L5jnIk!agS>PRi$krlsXv{Qp)nO~b@_2F>nI^X%hEUtTIlxhGDY95z_~ z_#W)JzGhJF9X%N)M?9@s!VRns*s+FXG@JhIxnyg#K!-Z?7EJqHCz=rJZ&lCt+sbOv zD#r%k`1~HdN>0u9ExOfUUX{81!adAn#f}lx&543NA zBe=XY?az{2_0v>(@x;h zA4fq){<~vWw<+hRFwui?#3KW~tef+wJ99gpyywz@eFE=NCPB)1G0hHnHth9>AU(I^ zw!i*2se*&7&C}tg4Rs31ukFv}wFXXN`eU=}8ic+#>x53G9VmN|nvz7)sOhTj!3m}x z$0(C}j&~_-yds4ma4vaTtTO^*vNrJp%Xl6+G5c~{!qHx&9~wWuWfXi=5=SNy`;2~i z^##?p1~GLKdE@L&kcJY1=szPoJDD1Eq+4t=r>-E@)BKP-F+{e#f-|)++e01=a4;e5 z99mw#S;@90=j4qU*awbg0zV6wI!6wm|$X$Rtc-~7r!M#CCx9m zd!QrXNfl)b%Mfa(qf`|xqxypnPh(J-KQ(17&cmoWq53V_E!SVJea|2D`=-GBwq?wY zxnDUP&8Vt+F>#8M8Qj9LL}z#t&3*_U#g(+B9I{V)4x(@Sq#`cux4}!u)xDIV3mNtEyvN(5}Y)^(MNVE68_@TeWQ)6dIn@ zfan&P5Pc!ww~1Y|c`rGIC@azEHD=^$^H!6^c2s!v`UE=-EGji3x=krlzU$N7? z?UT6>0{tsG*yWL$cZelfgdPru^>soS4Qr`@_1f{OePKv{L5MD*-j=Ze^C)qObfpJ1 zNDEqX27ks$&K0$|{oV5_P^{W$L4Egh1sk+3b?(QyQsxs^o{rC%a9YRc5_5dUqz7sxTuKu( zalQIt+i~Vd6}wg0u4CKrdg|ax_|Hk!<&abK+GVlJ%GP_!yX4fpp{WDY^DYU7W2w-jWo&61sI^9b;(Cwm;JIQQ#f3Ug~Kx| z|8vKdQO*s$t3fIA0F#g%Mnnje!|0OWumT#vpS{X)!r>Wg%<(`)?6sR*AVFe}V?fV8 z);dc>fk2yMeLK7Z2Aq)mI>}Wacpcf#S=q`ixBjFIw0+QW2!Z~UmJ&U-U~)-GiMaCl zrjxzyn}LrpgKTC06x*RRqdmlD6+?E%gA{>nHp@ST8oLMC zB1f_Cf<*UWs6@80AQ&M^O`&9Jpe3c5#h{Pn1ZNw2HDBw&z$VWYBOReEpXeL9$88E? zd~m|J*<869csM2S5(N??F2q2^@^G&pE0UxO(%JvvvTMq9A#U~74dEXSb8J|et)FvyFR;sAm?>Y%x-UrJ)CHFLHhiX^LxT6{bzKyJQE-d0dUT{uW#Fx26-PWj@;k^g(v zWamMP;7@vKTi~^fFa1(B4Ad^18;2JB{74YUm!Dp-sY;RL1=~co)>8OQ@o1p#EFMHk zFL4LmX`OUt@*cD@bnjI<%}i)OJ{6qS8@U(4P7Bz+)L?_*D;3OM;4{y%St7|sqrUAq z<^p0fRj0>#uHb8+X1d>t z;f-0XQq*4}3=C@;w{_7Mk@yDB+l#YX3(vqABlA6ax)e97+co9$k+B0^wH#q)}TZBt4TgJP(`XR1s zoD7HSO17hB#sO1n_BUf^GDvi%-nAS3=0h$siVSSe4vxt5jzXpZICe%Y?u(4XjeYR? zt0xpq+qF+7eRWAOCD{V~gS9o+)alX<&JB4M8n{YnWC1v$&{7;cXSfsQ$+*xXcK?u8 zb-bs%0OceU^w8-wxOIeUF4Gh$z zd7S+Go80f*?Nk4cxO;g6orU;|sfFc_!!!O~vS~w`-qn#b6bYh-e8qe() zu}}&Hn@-fHVVmVgEksnHN*UQr5cea3tlK=X8de5X9R2_!Cf=;D>!lq_`V!BPRSu?x zA)XWwlWTO7O&an19iEaCKcN!yYRjGJ`19ljW9gP;t8ceR*#bHy~ zum5swoL!@Ua<0>~rsA}>Bek&D44W?4ofR*Szk#h)+p?`m`zG8pI`A%;y)KUpiq3aj zkITAcGc!RRpyrYA+Q&u2+C4wiC~f!O%|^mYR-Qf<;V(Kc<^SdR9USzWdZ0RD8#Hju zxa+7AOumeO$zj(MJ=igR|18^i*nW|~1t&RM;NGy+J~Wcm9`g{ua-mO8SrS)Ml{0p# zc=4$x(&ZhsO1Np2T0pG4lxv|G_*X_CDv>YcwmJO}Hp3r|S8B@M)-O=CIIarSsDtg6akH@k4Tu6Oc-j& zt&?V(2_}QjD{rL3{F5V>6K=9R)UL<`-}->&PV^DJ+VKWd?$iZ$_BnPbngRwID+iz5JJ=%0>&a(~x~F>$D%{9KWgC{J{P*RExJ>K4@x9}GSN&u;m2=Z=doZKy z*})_nHvtc0{{7aoNQusm`bO%1Rr7H`-M?7voHB?1yR}+T0><7;w}p|a@j#y#SzzyV z+e?P+|7sc#TYefpaPB2<*K#A{xpm3wlpfeH?VB$0WKw%}#?>C6+;*QDfEh$$*jc## zX1c> zY8Kbz${Mv>^;fA_k;LTUOTzephhDA)jm`ufr@v$96%Y8Q+hI?cl)#gO$WbHRSa#fg zfHLwMBzOd&BFe?#-5{e3HaU7d|6a;O{eZ^|2y<3v=Tqq4CFPwN2+TI)rOm|ZuX+5D zCU^Z8q-VKM!Z(4~@>aT?s5*upcf089eR`a@u?3_pk@&MvBbXq$nTH{MounIaeUx~o z6Cr`0Y*pYT8^Hr2tc`oceP6~*>oAiO@aOer@4UL`$35_%o(!v>PHd`M^&UEZFF6du z;g)Z9#TT>A6&P@HjBAYnTFx2ytW-)FrnTTxwkfpc!UQ=$BknhTjo9=NWM=GzTw?_#nw z=a4B%c*rBUOWvv*bmz{bc5gvTZ+=p0iLKcMUHf}q(zI6*fmixEdwuMkD`Lw?bx8w{ zCgxh)yw6M1?iRMSJ#NOk|1PfX8^}AgrzY&=^LCi>aamS(|AsEhigPu*WcU<^)S%;% za;f z*sF#={fvZQGl^xr;0|mHO_x7YqIb<4>a-n}UXGy#Fhb5Xl4F+Ol}|LG55bl4Oa~OT zY31}Id}{n%UPgmOFppS>!2_jVsRL=uZ;`zSZm2{{0!;@3gpWpDf2=cs8}Y#pC{+jN z9d)PO=k}xTjt(|Nud`H{D+)zAT9yVQHeRcr|LCc^!BSo=(z$Iw6*=|Krrdl6Ir79x zg(Cuc?9f1|9J)D0)1q^OXN!$l{G7q<$;&RRo5FNSRXdtGq8++Kw?ziar@n9-dULS1l2zbs?-MdMQ8^!t^X z*xQ+p0D1i9-nQy_8(%$pRkr(S?H-BAaJqXent=y=(>Gu!GawPgt*IFK(lRRWG+Lx< zEn4EuM7|0Yvfj9N6VkKEQYdC2P(sOvF=4PLzk^9tX@nt79C15ILqCGKE z*R5t^c1ZJ#CDUD(i5GM69`*Bj^u{Z*JUQqCoThztDT z@_(Bc`Zu5}W*nywI%4vg`G;@6d_7%J!tVVy+ELw4I|N92w_xRFMnM31O?@OEYR*3g#DlXQ^Z(=@4YEl~4GM7Z?k{6 zm%*3^tm1DQD4;hv`lEU&Wx2hyg(ai|i^6 z2)6O)u2`U;9O7C|4uPR$A6VWeEV59KsXX}|XM6O)?XUa_E~ro*?yw;`cb;AlMp+6-gQ)){)^{7 zagG(TvE2>I+ljkLqj!s&o?(fP!@P#RYzG^@m8F$;-}}?l0;YcSbF#zJl)rSd{G2$X zUz!;Mkr(tF6HnzlS2Ysjb`kw14?HSA~dl(qCirPM6(LKJamj%W5 znOPvAtJ%>vuP&=w8o9N;=3TtW{AZL>n7doX?XZC0JPFc(L%qLE>|&MF>Si zo$Rvbm-HhEEL!3Phuwof?vhZfA`z)MNrYXbARi?COBy=C&JxDcJAYLg8 z))IaH8?*48?KpK|^!+tMrZxq4xwRRvZ+H2;r_v%t3iQRWlo)Y$u^slW#UxbQJW(N& z9JFnGMlCK}t6XU83U+FbSP;$d5C5}3P>V=<*M0jFZAt;o{tAGMBdq>`^jf`}xzs6l zaRNvoQKh!yWT30~MBR&@s@i+O-QBz&0AA9P^I9UbnszmQL3t)5N$=xbPy zXuL0vkktJ!R3%E-oMGv55|eY?;N&)W0mvG{I;;yhU8qr&JYZ42wbTC8LT3w5-KH)0 z3iY~IzZ#s4D6{=guwn$uy}l4=`@lC{kHvEs0u)8sqz64c_-0)<&*Ws!Ye{RO(b0d& zM9`mu43#F#mp;tBO?usX;NI8jIABUPnyGQ_lC10RiaExj;{18__d5!Ml~*JFjwjps zY#~J%bf`GZXFDc>edO8QYM!MSuE?TCegn`v71-pltJ$vGj>$a1Oy;f7vud8(KM4%# z*AKInU%r?a{J`)3$@eP&XYY%aYsoed7&k;wriaDSGEm4=D_BvGF%k%~!*W`ocsga` z#UP?OX|QKwAajNorhFuw(Ctjci=*h6U?0c1_)K}I9)M%^d&_BAJbIAvew~e!rRD$7 zbe3^VesTXNBqRi+8$`NBcL*}NySqzLIu%q#m*kM{4(XB_-5tv4ZZ_)P@BaVqr+c*5 zc3tOu&nMmowC1%Hf{U#_u;v~DHzYaP&`vXSpl@|a&#{@;hL2S8O|XD-v_osU%lz7y zAX6`x7DT+!duUMQ@mB*w6nsLzT_~(_-_EJhfi+D^#?&T4huab&C3g&Gy*4T!Mj`rc zm_5i0i}_qfo|H}mKoKP6A9A=(0Xc)xJy>uO0iByaNx=*gE-}Y(k3rg!QdfOAX`>y* zU3A*Wz@J~sb@%`~MdkkD2cDDH4XO^b{ADhG=(n@@l>%&k@&ZdU~bjkXp?YB=w zvjyDmHVQd626#zE-UzR8S)pES!XQT}<+B!HNEZvjZ!x&DlYso*K%{^+fBO7 zNlO=WH6A_}l1#Lx^_P`=@N48~faH6QYp>e`bZ*)IWy==X%T?7g3G`To)xN0@h%T~` zLG0O3>%D2SyBc@UHNU&bUhscvKL+TDZSBkeWAjl~Sr zH^nAG;U7DD;d7l5t6+M>W0Fdxm{pa^;pm22!CS<$I-X|&FAwAk&>mbn z@Bd`+dNq*S=1DfWX{GAgonu33Lm%c$3{HDXq~0U#HqV8oQKHoSBotpx8u11jeQPQ7 z_)6gNlR7xwqdo#xY#Q)4!d-l6WKY5G)6@I6uJ6>k@^*N8)x$9)u{dkpTq^IrloTRt ziS5JagvM zI0sti6l_OV=!J|otj(fKTybny(jIy-Vc^XhZ8~%ppf+Wdmh_)i6s=oTNhb}w6rJ>w zGH0MJD7GpfnSDFgSjZ_!qToNSxX9kLS1Syq(ofAItRqZ$ke`E*-bS>1d5= zVFUII>PfXH$9pZG{4;b{LdYu2kOqa}$R^grO=+j#RjI8UEUZC#O*DP*TB_N2UHo?2WcuAU~Tcwp<=~h1YY5bA=i^_d(Cr%wzDSEk~f-*v+DX} zA10wX$R%j+^^`jXHR5*ZLeBO$olpC(3cow{jSb{e51-re287|5&8dgZ)8VN!dgM{H zu>kLisb7=1>ely_dYm|X^UY{E_&gbWn|S0-g^$*Rc*q0TU+oA7tc0WZ5nFu8aQGc` zd-D16V$po`m~WCJMi5+L`M2z6dq1)MXmF%i@`p~f6puDYlaVF=!3!B;2z^u_I zGYXCPl<^XEJ#OdJgQI?G>Yp`uOZUFD1?=tOv3z$tjOc^cxig5O1Q({(J=HUgNW12kQc_0pn@X7)N z<`P(ok$F8%(Wpav0wN_7j(^31^-tjIB|>FWS8Onz(INFV7mgGxdh6406#V<255@&v zNAQLGRVi&Bgu|0wz1Tf1BGm6$z~GOLfyM|LHmE4;9XV~>P0>HI#%*LfQ|Boj(N7a4 zg**;#a_t-n?o;^phLX8&){Vty-msEb?-Tjk@+Q9pM0Bf5Q8Cm=@R@`>!}JZSo|5XN zj~+?kE$Fc+b#P2iagkWBeqmaDdDIv62yEcPCIO$+s<>pg28rA z0IpOaXFP1`o+DnjHn&8N7yJfjv=1ry%-XHLWCb%q$4t?735V{kqYanDnD|S6mjf0TyH#_rh;!qQIHtYFBp)y zAOk&D48wOE{JOnT+|3Y&)*TscbJ*AK8|8N04)%O5GIKRZ1&`UfolTwuj{nC3c%2mv z_SwvrYrjh(@fv6327dzKP~RA$k*O}}Ag-4q0z_EBS@-Na7V^9v4IXTBM4|<*9of~j z%U>pE0Nst+aXu80w%imSaevYM9K@sk@ryb}Tifh?BVQGT={v z)Hn0l$Ur-Vb%UH9|J@FY5!>X_6bsMrl*_UoLR681xx`3x9Q%NCK%q-QD={#G@*$Z= zUSpI=8OG7rFB#l0ndvF_T!;?Z?=G((zklIJofN-sLn1=CZ#ms<3PW;#F?Met-gyjs zlWQZ(jS9-=gVI8AIsa-rKFTjI$yA>Q{Tj+UJJh5M(`?Atd%Ko|&L*8XtSi9SgU+qt zux_)=BUzYtw8FZC+U7M4A#KSiF~kwI7_@B56n=WYbz%D)`4=6T!RzfS31oYZU5UNG z;^M@D!K&eV;)f_o_c3h7&VOHD8}uIOH1+LQ(1s>8(D>`vWq6<;Tr9?2UScrp^cc5^9H_Ce?!TI*Uh7omKiF!>^eRld-t4f4LuH(V&f#H z6Y))sg?_!0O2{$~kTate(Xz`82{3HcW&bZ6TBLf_>eal|YCnABbfxNlcg1bfv=B3F z*CsV%WV8u~(D}cNaueg2nUZ8$`~EO~8WGwopB%E8il&7q+(^B+G&qi7kMd->aA1?z z`~o~)O?yrgszbKorP8L`l$_K$JBwng6Z*&mO+3G6yN(UyM4~INl3a~&Rn#!yIk@0G z-QoKG?Zop2c|p+GLM zUVjI{cby&iCdEb5$^9F$HX?sjbgCdEA=`JfKOV!ttvYK(%0X~{i zsb0nd;?)!y_1qV1mNF?SO`|J>+@WcPv|ttv6v~JY=E<(~-*ytg7$7^{p3;UKMm|XSVrR zW{{TTYEc#Bf-wWZaJ7+JfGBR3S#&pH7VlnQcCRQEa~xTKjAzk>uwK}-pNbZv*0{7K zx?AhMhse`6N>{w1cJJ9syN2;>#g2+V+mr5#R$lm34K&61nJMe6-fl2!2^R&p*3Z&3 zY2(bV{3e&4Vw7-_?_U8ZSv5c!8rxy3cso_k@VC zv=_rha_oepdYY-9C8}22x@Xqo^-)d68w$=Wq4ZE{PmMSHeM;G~?yaV7WcMy=BstUz zCbE)be8Ak`2WtQQ;eMO-0g`bWn!qF1<-4dXo0JGemiw%~n_I^XWA9!>AMp+Ac4RUh zb82wYI;-3CLa2o%9QfGj(gyE@&_bWPo|-crQ&GXm{catKqAZ{0P10R9FZ*K1<70iU z^KE-J0+p`1=c@D8AKEB)cK6H^O5%^0*tys0KlJR^mtZJOwx1@Rg}_HtCJXDHsCM6Y z`5thILC}>s)LQhUsvdfe%N`hl-wvwpiZ@Jh$+HeGZpXFTbQYy0g;EF58EVU!Vv~6+ zjeWeRU}n{0xN~3~1?Oom*nDDP9S(Swrfsqn;UF$g?=^{P-R$l!5AegnztWOjp}(|H z2*fg4#w%8`d>)IZ+blpNKBDlY2FE<&TV7Q9`&q@_EIxKX{dV^V`>LE8`yWjZ4picZ zc_=R&+i|_sxggwS-cq*pU{0H_wO~TboU09Hcf=4nnJ2~Z`1X=!MgLk{)Ax9ZCwa3; z-@WxFQ6Bou)HhMyJV-ODhW7FFc|D=$7zjFY$jJ5Ykf`-1F1@Na-9s(uGB{h z%fI@^QF`PV8p24zh+U3bp_r)$lako#^tqgJhNjj`49&<0QXspT#OHG}ZA=^dK8AZr z#!U3DL58JKw`LK2&o}9@6|7I$b4AIG*&VAHHrU+wP6o2SgD-6T7>UZ5`wYY|T-W*j5e#npzFRRL`?laWZ zXTYVtDWj@JU{$JHIyq{?I##xYudbdZyN9iqZ`0gEA2r_`*8ZYe^svs5Pq)N_-4nK6 z45aU%n38|JyyuZ0IczJ(`WiNk6A;5MUEF`YctovdYPg$Sl(F^#Kbkpv#@MN#4N0IT zo@?`L$x)5Qc&cWIxf37JbL*TQ4NWNjti00gliTw9{Ov=RC#d>z#aQ36X=+}R`v9w| z3-3`m*UzhT%)Oul9IoWFcsNpT|#^O&Euy`R`-&VNG z+t%h_?_tULHM07AkPhBymZxRM{l{fG?SDq7PIi`b$~?Sb+Gg;u$=yMh^F~U>?q$>* zI@XVuOwHe|RWafbx^S{dw4@qj zpK~rP#%(Chc|8t3w+wu-=AeH8cGFn$s9k1AT@^43#qnu%q^>kjRbOOPZGj*2P2OKpW@ifa}ULa zFYzbi?)8?EiC;xTMDg}RqeY7v#zjB6oW_JD;vxw~d;%xKhwr3f5!m{#1coHPeS@ri z%;J>2R3U`17s1ff+c_~*+W32!3Gsg<{=0wadDZT?WqSA8Y_niR+rAZ$-V!M7yd&TGQq14!$t#WaiwUnTVg6NDMfBDlH7QgEn}rEs{?rqS?XK$PD%_EwNPj zVQ-XV!wBw8^$iW()|@}T%pgUiHPAlAU$-jj>$vKyYv&AQr-69=9$ ziB&b1Px}jh)E75>n&Vq|S|o^u1jKOLxz4N}ZXbe0;h2uaEpv59KMkMeW{+bgm6>zt zlv{q9(}ddp^cH=QF;rQNOid#R6Jx&t=7uGS6Q- z%&H4ji3hs~8HW5q<9iO>2$2_kd3v|}`RUek!`GOy69>?jUeWXbW|zHA!`f&|BcC3O z?|x*Xo@kdwXAh0{SsfV=6D13&ofs7mP^t~#?V9HV*aqwrm8qe4;ikOHNd;}wAAc$5TyO9>Dr*@sQDUi9k%PKT%@GX7cx z<~$+**2?$aT9i!~bAllzsvqdwpHJn1In!9AEwD(EhTqalDj!`f~09zOXq0}BA> zv-7{acj(>-xP)q}XIz!x0@`lf10l`qi>IF(He-cWPpq;IAbG2Oki0=8261AS@373h zjgN>hU`7Njl0lc~nTwdz?1WOAD5iyOoke_ZbEjI8`E_#>5x{8cCn1!w6H_F3wQ6Sy zcZGBwdf$GU+9LI&@@BoZ^F;L({*LbYUTJFeT(45L)v`fvp*^wAVW%AFYUt}nA^`k+ zY_@)0qU`hsmM^Y$UfHIg@yK7wYH;0~+0~2cpi4)8Y4;f8(AV9ml{kQZu0aN=0*&sp z87ew3^?HT9>dW!ZngJ5F*ZSuX|I!gA>YK9UW-?V3zRj>F_ll{vl+ zW?@wJkB?GroIBw%U7Qy4lJ_Z=u878p?8xYEqLbP#zDmGhIr|H54+b2atd+iH8ysQ-Rmq^#XO`R(&-YUs^>Y6qr*4# z(Jm}ZbD0K+nTX)(#)!P%`Q_@58^KW~{UJkL@o2U^esA)Ny4dUQB-S}AdQKg>y>g{7 zg|0;jVk4i6;|WX9hxT;kDR0@o9A>zD00-P{1lDY&HM$xucr z|5&7m!=+v0-Q6snwe^ z$iv_7J+Zi>`VF4O4B66O=h@4#ZM+8ji{Z9ym1L$oP~NfD9g|$dOF(XOJP@p?VS8D# zZqX5U<_?}=y$pBzqS~`xP}7~r71KgbKzRT?>xd4$#7@;4U zNpr`OSsMM9g~bSLaX-_lST*r2dUQU=XFUy_)3biwS*+tVx%pk+iUWX`Y$E}|5%kXMm8AQgtZQs?QB`3tFtTio1babz+W)-*O zI@MKZrBtW*omwt=&u8V0B)wOlnV=a1?=hiDR`8027{rgAOU2{+a7iki{dyQrLXp4; z(|c#2SlxhtNMWs-Yo2Prv>n^f*-*+rzp2J0Z|&Dmv*QQ9ce0{R9eLP24oryJg1VyY z_sy54$g_VRSg!~v8f}&EzjB$4z#_LxKo_zYuKOyKPWn-}EheaBFkHrJn;&eXRvxA& zzsm!WENp@xhVq*lY^`MP-eAe0qjhj0?Dai01MXN-k{Qh53Z#P9S*uDFnc>9%R91^W zlitgF;zdOs+R=^KR2d@q%4x{JGQTuG-E42TbWRS7BF6=h)=(d3h`%H5cuCv1iJfY=9_=tAa;xMZ`x zlab{YT`4elR_e1D#ol(W&_;?YcbAUQm_!SX)N*)c6e1VCmo2*?w)4K+UO;FGJv?lh zDPVO$x-D5W5%EQ8Mas7TKz54XOMhH8!Z7-p(r3| zve>O>A0z86G|D7os>?pxK(>|z)u>B@y$HzJeE&7wf}n83I>$&R%R8M>s3>Cg(=RA(Fg<;q+~$psIEgn8_|DP#sUlM$^x;IP^Nl?l-92@1I1zxNM2cd8eT(KxZt|W6)g%#EO_;1U=8ZDaC_+eu+FD=d zg!Rejk)~YHlE`sS0d$XhJzEm2AJ~5RC3_h#2npFc zs@zuqReJ#k9{~T`qV5%b)i5zB@*wi7aQ3H3qI=l1ikBkoxbYUf)IWBTmGe|avm!Ec zM;38cg!D0?8rMsIDy4R_9#HSiU#c-U|L7n>A=!CUo(a}AL;^0Ie{wdp{9~|Y9q`T) zfr?%96=EUw0lwajHFpJ4Ns zQwj7l^6rZL*2kK^l}gsNlp+1mG&YU@{^n)5r8jbaS)QwHJ4}tA%caBw9aPq>m}M`TV%{eEBc%Y?Ns)+g-hx(%)!PD<^dS8R+-V z6pf50a3GU#kZ6G7nj>0c_)tgYwc=uTz;4s99WG&t1+Ec9-FAwLJw@GBI_{b`CxCxR z;by7Wb)ihsR+{~?cN?4YoV8y?EqE?gBB3WdXdP;#Pegpf-lM==`#N1R-JGk$)Te2z zl>MFiS!l7hiP!jy1K#PTdG+sYHPj>hMdn@C(uYOP^|q$U+CTs5bt>d+eyQa^W4MQg z`4B&i(^un}xJ_qu5l1QWWh4Hs{0K$aO=|iU@VvQbi?hCs{?mvs>NYA_(I#3V`#H>> z>%+baA2#Y9q|Q}$(}xJCJc85fX% zf0E03VsMwfzC~t;a*L}Az~m-E1zn#+d3YV!&_oio8@e_($@d^`mS%{1DI$6zJ`s8O z+{rke{-lox{&F{%9w2(R{&sX|p!1B~y=6i-{;t)ie|#0Z}7_w|=B0yW69 zdPazxHH<)TKc*?kO4IsODBDVR*!uifDfmz{o)=I+iseRkWj!c>zUx^#io~#3EEn+Y zA~UBidOhe~>fWot^stgh9rs@c?`i!QrE6b;UUf7s?+9mXPb?FZZ z=ndJY7?ZoPp0(YnGB0u5s+$W5&>v#p0jJ~QQLd8l!G_WR;^saHv<$&n<6XU}@lt(} zs&0uBSJGgVWm~Z`4Wjf|pQ71@wwW?S2|-$44;P!g;>Qou8@)RDo5ka+&nud@jxA|* z`lp|ZgPu=X&@+oDzaiG(MTwVus!!{<^_9>0&wU>9q~pfj?I;?1>`3!Y8@rcwXM~J7 z$(r^#2o|fjb}Y?a6AI{3_H6XYJtMp^|a zQO5%1ChvL_wA&GdOi^8K2$BC*JBZejVdXn#<#T0A+ps$>lLnOLem(d3KOq2CJl}OP zEpOXm*6G?bl_5{X6@8|WcE#H*0|)C$YcbUND@I}`dvz| ze({H_1A}jX^Cw+oBH-A+9Q4k?TfM6fy96T+~UmgXCF5 zGgDO`A;P)qF~=577j&iJZNO9z#-^U3OY31FpQD{Wq!AJK4MeP?LA{v_N?z)T`YEiJ zWuuvoX3W`a4HK2@AcW>f#H0r?RGOob?HNY$hXQxyXJxvI2c(Dho%m+;^AB4~M+p;@ z_#}eu(-OAp`Dq_ZbNCk?uv-dk+g21#Qspz0_I7%EN4(>^PZ)%V^hi^mtW=fD)TjJD z@CvLaMR}OlUa7$q?pDkr7lpT7U}wXLo1}x_@JNto| z#=N2LAc+~A=+?64B#Af)H|&28hH7{2D}ZI}WCHHbD{5>9lGPHyAy2yKG@&4lk}%CH zhWNVJhO_xp8EWqV4Wp2Sm}I>nu!Fi2dpjd;y?geWF-b;XwT`OfG;XW@@vMH=%MR)P z$=<}`x^AgmB`ezh(2qOi)w*y16xV;{XHKu5Z>e_&XqZz_e5ri|=K0#no!Fnyo&$oD zs33jJ*D2h@DQymjbk}6l)!8y0+@?ujLRc9X1b1xeW9dM3V|CgXqX|D z2$jEp_NgWKZiKvAG}#9|LF(wl5*jW-)h9j*4L8w9!=s=C?G;%N_T7hMlBlYYrzH3* zpjo*omwo%1W{^y~{oeNRFh8fXD7$A4BU-M$`B)xXl;aOL(TgIZ3=~jO4ag)2B=eb9 ztx1KDqWydPNMpMd$vucELt)#7|4mR;>J=GVgKT@LR~P}&cqvWkmOK0V92wil^vYz# zKOf2*UV$R2KGKF5>|+G~?60CGyh`JTm+4>0Y9Iq|>`D)&+}pOKJa;uwZEkMT)0AIK zAFgwY2_1s(VrsiS?XefT(@*?qr zfl&Xa_7l@N-`3x4ENDMU&Bd@Acl3&#ceA|IT*EZI)&uunuiyQ;5hoxMd6<4ts1Z9$ zYxMzx?c*hkjLLB!e~c`zckZP255aubqjd>h8};bY=!|HfiRs?~P9_FV zSLc=(qxs`|JmgfjizRKQQs>Jqa&H`B?nDhGrzL*;$kIkNS(bBe3G^Gr|CWll5=iux zCjfdL*&vY{LN27{tK0d2(aBWN=*SsQ>+?I@^vPQ)U)#q#pPguFSrH*5Aa=1W_mF8R z=y#!L2;4P2Y4vVZ9^nW_so=I0oOtXA@ajEDEItVv3c4tT%sk;iGT_(zm7;BXJ*f4} zGqLl*(!N?S3^Pg>!Qx$F;Py4E=Z#XPY(7qdV(zGW&R~&(&c8+7A}AjzV%T{r=c+YW zcDy67FIDqBceH56v0bk#O(x##A+aEj6QYsjeXFTW>%%q~gA1`b%V$h}95aseI5KoX zy}a)CbFf~q8?Qw4bN<(h5!_XFylV;GiBy+N^wBd?q-n)h?;nelc1&>(KJTimjXXQk zTgwjkDLL*r>NYeicGm6AhBzmaVH`Yv+&hH**kM>-b=5}um8TEYL?~)!I9u(E$P6yZ zJtpCkZK29IFj?|JZQvLxO&|~Lr;lUFh|bO}DjT&H>rx3`g7XdQv4Cw%@ue2e>ATJl zaN-OqVPwWFW=e2Ty7KdJk_&{K$)7{saH z*M?hp8mUu86ifQiUC7(az%=-dbFjK@Z)zA>k5R&TQGLG@X67ZaYK_c0zDIwvz1tHN zhW%djs~yfa-;PtSBq>HtQ_GZehmIiw6<(ZpQiq#H`?B05J$I)}oK6%_=9uv`6r~KC zjGi2@?7){!dC5UZcrTJF^UQK+AC`!+&O#0jD81OcEcyP$&q~APLprxYgqqnSF_Xrn z?mx23efW2-+*3b@?!Z{-=s$$E8c6m_V-f|Wcg}=>XMx+FYXT$iFe+VWO<4MPA<8;F zHIC9p!$}d}Ztf{-n4H>{(+S~Af!F2g=eUiVgY z9;_NV5|Y2FNpaPIsl1ZoQ{7RZ4kO>FaUZX;x$b6b(X@$-|$WdZWNy6+a^Cd zOR*gQA3j7lnt2z+Y5kSa_i{F}VTqG+BYWoL3gFjRHu+o!@{6jobXPfE6cd7TK+{PY zzuF|fd8U&HCmwWG&tZMf>M7+!@Xk72PCC2No{AB(QYj;gv4xR5Fd^oZWGgS)4;7Zu zKr!H30oEc?I${eV=7R5yx2LQ6Zhl)IVv1X<|73f>F#}5Ix$=^kd8LP+s0Wnc6Ud~v$d|y<4!sKgT@Kb`ihNH-y??j z;SRVga_*PrAE&O3)hIff&*zeI zM!oM-zF;UeX5woyS}nFbVqD=5jD6(OI4sGcTZ9=AZ_J|Pdowr}wd8-dnA|XI-o!*Q zS^SXzSvR(yi{o^Y5Zmq3v%mjCngxz<;|p;?>N-MsVMscs+sy*W6drfmvh^i;ah!IDNCG)vH?8+kC8A>W=4t=0kV_ zCAxxqPNdk~W(n}_g%6=qxm$kKd>lOo^@^vA(&G2AzbD%Td*^D26_zwQ2<=iqeyQ2! zf~8UgPGPd9VIw?dRz( z3>pVK%AOzjR&|z4f2>|X5ywQv&YY9paMYdBtBHo#n$JgTiq#sluRw7duKKXkAuzP<;dH+JW5~pH zo&W95x_Uyu6a#PC@`{%>m-d9TQ^|As=me~1(Fj<9%d$BMzL zCcPg-gR*V%DZBR(AEgb7>lnqjOcV0TzU#F6pa643Ki?9j=OBH$)$Dq{OO^Vnpo6}7 zMZk!}z>d!o5l_%h|2n9ng{cf<>Zu%=iWB%PEOD7e9|@#lh*q`_)LAAIJ_FG4Wl(GN zV1A^;SJV6ZMXslmtB;riQCy~{Nx_f%oWuZo&B|;HomE34E8<(=UnRDfKQ2JZ>mvf$bCi2AFATd~ZHzI<=f*M*?PGdYRL|WKdISF_UYKIE7V?rHzOQ zh}ohu*%A_N=ycysI!Pt^y~BZtu}ong7gS-fBcQ8c=o9*-2Z&xI08OmXwh1;b-*l8K z=rt)|-xupSCCl0{U@kZerqCxt7KhzNc)7+!g-RObQVFVLvVUs!%@Wk3+*TL1*ufX{ zF>{Dp=5w!VyW`EW=e9=W&=%fY?0AsjK|V)5hSV|*&= zH!rOVXY!ZDD0&8lufm+=OkJDu%`9T$@s3OQOR9tZow#(^x|cL9q_F$m7)w3xA(h~x zBES%qe4({=&*$`s5aiufm{}FDJO@(j3aM4|=MinrK9Zl$d)seag_3KjSQ^VY$$Dm3 zth#Q~TjcR^edQAh!hKqa8##}DsCC+%(!EDyY(^H!S z4S`W?Pdm(;iMp%Y^z79Q2BuFxrupr*;*o%ke;nNu4ji9{lhD!L{-?F3-W`{V4C9mL za`omRh}Ggoc@(E=Ep~EZ{Joi?O!kJpo6YRy*TP zCE0c@Ik-ow?Ogae1;PZFay<_3er7u`Rr~FVkXI?g#g5QO?gW%#&q^Q2)*lY(MCA0P zsh2GyQ*K_m*6}TLB>n72Je~REIT?2u9HwR1(wj%@b1w6JTSHMqp+>IgmwM7APuVUr zim*AglOZalBB=OVg~nbAvvCqk_&;uF;@OXD`BAy9*_gR}y&#qkP0IVbgPb zmoKyo1E1&KS?FXnU4mgj+}tP2@4$Rhk#R@!yEXq*-qu2rN?3t4bvv2%&Lkn2L^9Z0vh!m4kI5LZuzev{~3I`$vZ;^n;T=rsSi9B-+k3bp=O|+}Ew z-)?X(jBb^zTO|r!S{C1i0sg^pL;o zKk!9;&%NQ^9xZQS>d4|03X+x4_jkwznZ`3?VRmYw9|sqbO4Du;fl(Aym4@SO@2%ps zw_~$tKD|6#A1Ft+xS}T`ulm0;vdp6ZEq>eHrKEGiaqOH1tcUee%Q#>$o{mJ-e(A#D@or6{A+!sis5C+@d0#a&!5R_ z(b(YjJGkY!zU$Yo`=ofnuF#)pPJ zUg}Sq&7m+Wi!-I&lF>S+-_q=$JX*GGB}`D@OgTpKLK~Q9FFBxfXS^YxjwQC0Z`u1V zaqYpgN7cvx2ucCr+y{c&e`GB%A_Zb6i(NrFaY6h30@-_~H0*sQ^hF*a6hU}p6 z2n|D;K{hmBfka%9oDI$Xn}QEMW@H&DRPnV%%Cc1fa`2BS$%mUqA{e4`N$%L@Bhlkf z<1?AEm`ls#nr5QWW?tV=S!+iL3%I_!yqAdF$=-HDNLcrBw^DT#$D77y7`QuS57EPa zjm<1Nt8SgRshf=ycO7}6#r-!eT8m#$FGnY>cZ8l2bixSiD;*aRU>Vh67LF3S_+P(X z59nc|$>O2{H7-DZ2-AxC&?9PtY^zK61bKTf!|=@hx*)TrgMbI90@ugHg({>lv4*Vx zgezp^b#zU7u)%hf10jMzmxX1cc`HY(V?`>F7$(-h+U{P z{gN*-W;3fqK&M#azEg=$p^jdNlY@2*`xdATfnmY)EL@+iR-oYaw9Gc_7&H|92-Q)M(l9G zT`ss80q017WO^?YHbgOB8d!56UZS&2wVLtaT21F6)&#CS`OtIf8*A*3lA@xLyqVJ< zDvT_l4FDe+_8tdF$mYdU0z2ps8K@{&+q*0E)U>S*8#5o~x@h;344rSFz4CLa6Swb0 zKcQRA00M`P`2M9RX)YwtgMA>%u*T>W+X$P|f4@zWd>zu^g@k}UEQVxy2F%N7oAc~} z47L}`F!t~J(LU2s2W4VrvPk^p%h#D$j;>g2pn%MSb6^GK(;4uJYq`M)O&ABU1&kmu zGWNeYs}d7I0xlOX3Tr#Ds{JN7qB3FVTK4(l3B>|HROOS85h_gl#`6!VhHWXE(_OD& z&d31B+SO37AmAH@d(ek~V~?-=52ItAM-`p6;R{zwn-{n2H>rQ!_X}k1OwJ9XC%+p*_inKK8*Q|9~KJ0ix~44`tkColp+>-+?lS!pD-WgY5rw=3a=AmN2v#49TzFHr8w4kYL=f6d9p8wU@sB1zTp&w0;c zMo1zvNFt)&pGZV@6}a7tJT&qZwgY=9Ja25W%@Fd;s8q#Zh{Vug)M88Lef_B{5$#vW zmuxRcCj3tpyZ82yp*{Qx>^GZ&%+Mhe-+9w~J*QfOffk0@NbaLM{T7>pZpdfNul!XE z00S}o@Tc%&2s6^eh|qiH)7a)y7rlDtUu=(}$=0k@vAwD0THs(G`sys-pbS>aj!4Mq$h7VQ0P&B6TAMu0+cc*hvxh$|65y3 zhwfzm%X)tk313DMSlvpTZ2LuCn;jvZ|Ke1$ePoQpyj##=(nV_Buia&!0&LeCLS0@a zqPQPCHs0luyh|HkDA|Jd^I7{PnelJ7I>#I*Ds`HVu z$bs(swL)u-O>aUq)w@aD6l#y@aO^jJ{b*5LdC=wp`Nkz##I=Wgc7H#Trl=wZb$rrH z!2qRT-mF{)`JdLkKyDvCCv0=C)eZ993a;eA5--`bji!Mb4&|V)VnpO6e;5c3Wit&Y@#xkZd#MUp1Anl=xyBc7} zJ?HCeq`zJcPcw~^s4X2?@z?A#685+0)+>!ux3SB$??b$L#f46YHk{me4+22*ll_N} zENWEn`|iyfWow>L0JdSZBB)2wY&h6#zpj$h3miI$%Vu=Isq=bxy%yJDnvxSLK(X8I z23m5Vv$uYK8~TV`u!?Avn^^u4-a@T;&$udH6m-u>Tfa4r3c`HW024~LJt!eDphZZJ zVxcM4VxtE3_Z$CW#>oyV$ZqdTRMVVgacX9@_I!^nUyGsTt8CPLss^xZG8NWDYx=II z8#eth3!-ROiU90*mYH4lwiiIx#NMRcw5M?2Y`Rv>?sRi^vjW}`1qj-n{8&SC5nO>)XjC0em0BkwAO!6EkKumxt4}I+1AJvS2pq0Peq+CNaBd;yy z+T&N>L^-tV15k}pkjZx?8h<&3CGv=GMtQliTG%xIB&8%nQd>IU9N-BM_8##5syVH* zURNrLjp)+nuFz8p91LVO{}8aVa%y9&5`Hc>N6=>^Q3l-2I_xrYY z*2>Z&lL&zFJxXS_V}*Yei{T{7TsUO3JS4KETZcz@&y3eR zn6;;y%w4&)US2=gHr=d1Y~!hQ%e~ZER-y1KT0{tSX4x^|58~@;^1cALixQLNjWTdt z|ICK%iA~Xi5gj2UAnuClPX|PgQi7-&@b8COp;RST1cxV`nQs!nmO*Ab@ zG3eXCy{g8^2lOz4E)GOK?w|M+u@libOa^>y)jsyd#$ec9HK1;AO+QT2L$5pp5eb<< zL_!c3DS@)VPyEqgh&dGEbXe;vapvDPE7)Zi&uo#hqWF}!p`ntX2y@JWm#H>tlV8Vp zLK$9)eyf%nkL@*nFdeQ>7z)uGgduqBo1NN2qVbQpZ%h~Ot`rgVK)s6p$LvHf$JpXZ z#Xl1jiOU-Rr=by#`Q}2{jBjTiKe9Qq6a`WI4^dwk7v&dqDxgiVPqE z(nyzpG$G8z1J!GxMB%_Fj9Peb$n@;oMMh zo{aTDr+cH01(iNdjAt!uIe3w#C0mIXmg6)h&R+6i;b6UJadHK z2mjimYP$b=b}V85Amjg*+8vN^lT9MflEN3tOeB5lMf@Rx{S44xI57I)QNnZcq@v&2 zGCQ$eY^9O^MJ)>FM(LdMU@dX5?qg2dBv3PM2hZ*DXrJ`k;etMvQ~tJeNXEN5ka_Oe z7~iYSfF|r8kkRX;oiE$3Fx2L~GtX%S|J?nN-+R;k-O^QuaIVjV8Sr z&BGc<{&bx=`}#9>6#T$Xh7!2a2Jv?OjVJ;xZ2>Vg@b9*n0|7&XZ-Z}!+c?@A^I2cQ z><`wy?{F+cm^brvWq<#@`E_!a3Un5L&gdvCH>n~3g<;*vf#jYK;V7skwV6(*&BEpN zl#ZyoC^$#|(}~x>D7qdY~Vi?;q zU&c)7{vBL5_{xo9X`>ryTAzKLrzP3*gZr_7+Q4Iav?^^fP~Y%bwnDnxUxJ!gG;ltv zVa}U5N@C9VGwqtR{Bezs;SML^9~)i)61N$Vd9!Sf!66QVFko=zMA>o6=l<)4jbzClN7>~DC1FnQZMJts1KuJEP4FF+E5-#u&qroi~3~wrVM3B2jZ7jSJ}7j1cNKO z|J}E8pefb>7QO@bFIx6-)v?_MX_I+8FN6oO^&R&-RrS1ihm|XCJgw7iY;_dVYP2Au zR&sQPH@>Z^1apjzk`HPToCx_QGL$h%7seiR|dpy#7+qz$szf z3&-_QIy6r&IE&n<0OfBhH0ong%Lp7~6dcc#-~A2S(6cg`cFJH6+8PDvGMe>Y@TdBh zA1N#pU$cyrqsFseBvqCeSsnmWzyKOX)+h)e7&17(irD4w+$Ss*vGYZMPWQ(NPQXdQ zNRKlnA9a~*-8^OKxW#chg}a)|yc~beFD{>#;w{|Z^x5uPiyDsGM()+unOdc|UGqf6 zBQnRBf5JSe%O^7u@oCdLY$gzDVC17A>04vkE5l#j(}YgQFU$l^V1h?@Xs4&ttdFwI zct)5WH7XktINv-T3iH5NF-S5u5fS()Q`py^BUQj&o$HwZ=kD?4H)Yvwd&T$!4_&kb zg#3b*ZF3RNO9GbFn_fMi47(#}Y*9lKE+fs#eG&fdi@dk#e6a2-RjU0TL&-^iAHn}q zu>+7JD$&E(&|Q{Gu${~MAqgifICGoHf?0xXlbQC4xfY#fb0@OZFrzsO?}EPfxL`+Bc)r{w zdfmBjjeAtW+AZ_tt^Ndf8s~|=W)y*kMQ7-jSDiy|41Xv&%O{_!(?!!OZg~6Q9e#eo zw{YQ`d@Daz8HNaJ{+c{j$DC3dKT(Bh0e}YrM?c~B}9Ke68&tJ$1D9 zN~Y=It%|@nxRAdE_RZo93$UQLLwx&I5>N6t*o}_J^o)+4QT~UjSk;$K#q!ffP|+*b z9)B^&)E3M9*Fy*QE%sRFreAq_$2_)m(IZ|x3{oNh2O=+&3H@5c`uDzU))r{K9dFt5yNgK7CFJL>ys(p7>NlJC@Y z`b5e5?(TaoQ_s-i^9Bcgzc_X~b2rxP&+#$yGq2B;kZA%LZ7*&NUv2uUv8`2-xN&Mm zZ3(eJX@#Y4e^0StST&4|(^@u%p3CS>pTHZ!N%bl*ytV0c(7?M;wL?HUG4+3>VlmtM zUOcLnoYNB3S+yvA|KKomjj({&u!hg$PPhW&ma9xGZ zGvT)R9{#=OdEWg}$f}4ESMhw%4!J0#j)^7`nmsE=M7Rb?Y^OVZ+&^|;nc+E|)5MUA zeE(YK{0TWOe_WubK2Yv!3XQ+4eaMHkn=Zc0NmanSHVtUhHd0HBdOZucyF7oLVSzbn z(OFhJs}IhW@nRT};8zQ}cggv)V9a^b%OE400>d%Q?s${sC6dZv5Pl z);%M(eRnGw-9+{!*RgbDjIcwr(5Wwzfyc@vSXj>B8r+;(^e$ z8b;lXG)xP59C=N<42!$4(>_!X=+T@e5g0sN4?$pe2)l=6$F^$PD$xTIn#Q~7L%Ug0 zC=#l-7%qhaK6e;qQR7;iezvJgYspPwyOp^-iIKe;_S93IT^!k8{!3b~XoXn3OFms) zqh$U35rasizAmLL%_HVVH=*0p3+qH+rEdZ)uSrw{ZJ;^=+X&}{8(vsBM?p`X0nE?W z%j7M6-ZYNMIl;*4I$r++Hp7-(!YyuUI{BFmhlBSeC`qG;lS$W!3$C|hZ>PoO__6%#B<#WCr zne8N_#Fb!lh^jEkk4!^0Lzu#o<~GIA^Gz^+9D1zuiY>43wB1IC7}NWqH%+Yz#}uFy z1=(25j5^&*GW|l%8S#c57=6N`eUXxZ#!@ob=lz<+G!Wg!@i)C<*sHNKJHO+v4kS;f zg$j)owyaB6zgccrpKnVp#GUT8^vZj|cd9lAiqlQ4c+9rxX{xJR;kUSH*BBHxZg`-VGZg zN7_dypW8XjX>rG!>1|*jEezGq#fduWZiC&GuiQ^=vb7yTv~P|=!>lleUEl@g<$i@j zRsnLBs>Lpoq2B3^_R~y9Mmgm+Yd>hd?;7O+Px^S!KGLv(m47e{+9LdfsfnuXxfTc6+^N#x>a4k>vVb-{&yh#{|{1c(&OAMm{19P+HZo3BKFzRTZowzX%TgA5qMngiVF=fsWv?z6nx@hBZ}wO+uG z&M1tCNy8eTDNFMlsnO9Y-sk@=xJ|Io4cV@rL`;X5=j{H}2#$7da!S%NkbGlj_6AcE z;|W(@2toqo}-M?-mJt=!z9A3)Zx;b|>@JM}zno6-$R zJkEKC+7})!4hK>9$IJE2Yjw-bx(_G8>=}1Sv@M=p2JU^8QlH_I>tj+j%DXZ0>!ok5 zz>NddAg{WS_QGT|66o-f(@fM==0;oFO~uB_K9)2? zQT+D{imzh5_55T_c>ZjSu)mLf7g!7gG<1)gzi;y4e_GP+5S@_TaQ*tI5m9sFg84}_$LIrZry zLVvoY_ANDJ84hA(GwZwMvJuz0UtwhbAYb=A+#UKf#Ah7s@o)pR(TKlLefvyx1$8bynh%*EimxW9KL%N}@V8yBjp#n?C_oyUcV;qbJ;(F! zbK$KnoUP=5+aopBk&znDR$@X#hwwrmwr=Y4Z3CBpjd~1*$39n~mTA|2zlM#*TJTLH zivs5x`#rv58KIA2R2FRq5rGLB+HoS3$n;Ebz=mDY_rIhj?Y=3s5)wty^AEfVoXNXU zKJYjaGT#oCkC#6Nc2)PEgIh05#TDE?IAE&nbJ+s8@{@Py)}}mAjgTjn4?nHme0p81U{fERw?{cUmv=WenZ-=C{p6r8Nea1##d$Gg z?5CBAe)qmP_7d!3VAc4oEm&ExdwJe`Rc**;`S=1bU!}z>q^5Y%E2`3HscEnYz6RzAtL6Ddsw;Qf~q&qo8%<1qL(*jZd?V;M&;v{sU!O&l3%27FjrNHr}4EDk)ZJ ze%d0ClD!*UN5$yq!#peQqc47H0#3LFjMzWy!yowK+0$I!R?vMAwUoH__>`%dPIF|r zF;l3C1Ab7#0pr_tl}NHWRL{i{5#=?RUx`={mCByIN9J;at!s_N3FY@?%=l6dN9Cx( z%`IAwsWa{j*Bi%kl+(T8f(Yq0Zt4$s$c7h9x5j-z;v8p6eU0IdFE??#o*@(W$*%$q zTgKd}P4Q5)p=Gv}tMLc0-}ZsYx%>9^v#^7XMdTR*JG&V8bd;irNMcr`)8y-CBFjbH zzNH=|Ndn1U@Q|e0e}xo+x#u^gO9_*i`HbxI)GgJyPb?nN68MW&_uwd;)l0)YwjLaS z?V-UhDwQXO+=aXqv5sRFj4jda&LXnZlSk}&&+nAle4qxUbvw&1boi5ki) z^7H)pkxbzAhVDyxE)i zlP3NliSWCGs1_}pOM6Jk4fE@Bto)GKpAm;lb&D|uazvb%TOcFXlBaaZxk4-? zk(NR7Dk&yf*_Gb243H*HyYhf|A-2!%9eQ8Qgsxp?KaRgzo~H;;I%9oCwOgZVhqJGp zTJrt=2SGMXwqZ_onAn^YgNNuV2%byNQ|yse!-zwD4?T5v)<~CHdGl#pG+ns^e z-!trtGvz^@hn=n)RsWyL8!H_cAC=VKJZ z9&u*`q<1!AKCz&ZWJ)-OP|`d{LuJv2#Gv2e`x4nh7bY<5@Lfy8V7^OcAmuu4_p>L+ z3c_ylP}S7WmHWNM%JWQ0~ zmQnvpblnS9V#`E6YKDFdp`86oTFFxqRI;zI0FN(j`EQZEgXbZ0Y(0dFAo`yn(1R`Z z?-vXauBId^IACNz^h#%+niE|1se>SaoUD*dB9*zwrQ6dYe$j|;UUjvXn^QyO(uMF1Wfr^kPg0SX zC#DRX4raJo{r7n@PW5m$SxqcfI0xCGK+BP8 zlK_B6Yxs}E$x{LsajT^CHSn1fNoaw-4aS3<5EXQt0`7q6{F+Rb%<}fZ7p1SL0K$@=jWhBtEe*A#x^>tFO`NAA-q8hAZ^p~lL?Zq6bSFs)ZEf#@e6?EesVB=z=p%}~SygAAE~k}6C|D|qT9fE;5Iztr@PHk0556o$gE^^@f*pg)OSme8@V>Ji~vDTh^{DND@MJL0i#G+Qme zV0gwY(d0dns}}8?W+^2t#S!&4v+>ofZX5iqs{Lrj;ZAW0*`B5u&#TKT`K2wNasixR zNVo!}j>h{pCMS7f8>=bFdwR7DSh`cF+(Q~LJK3e2{zN?{e$Fd%ooq2-h!Jr{@2a(X z{cvhEC9zu;EVfE~aqhK{EY&8I*o0Fogh9BPoxA`>8N0BrSSBz#qM8Bt)6$r3W{Ktc zyZ>8{sb2@7K$#!HwdNq}uB5$fFdRk~DN{7;jZCiA2Zg@pXeRmoBj&|4skL?G6PmlV zluPy}PEA5jRUL+PURs_h8(#5rGlkm4T#q*m{%W_W14gCF+#DA>`?!PB3>7^lZABGo z4TY7fcLxt(ndC|)K1H~H0ax;EQ@i7TugDMx-`ehdM-qR1(wdX9Z~-dIDQz{~$F3b> z-tBDpKYHitK=y%Dje0+ZSaCU-6^Dh5I_$hUQDH2WC|}=m5XT`YwF8d zP^0(x`wg&cZAoO;@v+3GHPLX$cX$Y+XN=AEaPwyhG>Dw04|3-iFQ=Jw;gVSw_OAzIiEy6^wwMz3RoDoIa!>{6Z0f2*z1g|;biuDj zIAC{zM~RYPc6D_RcGY@o#6xSRxTQv=P_)9nVpLc?VP8q`foO@)6AfBa>4&-#pnXeo zCdf{i7lQ_j<@w&41tdQ*4n7lGImQ7W-%uUl-r_d5nHlPXs{FVovo=Vezg&tz;!pN8 zrc8QY&yhmQnwIZhstcBDA_Hj`L}3P%GD|TUt)c-&44C)Y0!wC!*Ep(&dso+%yQW5Q z>D{m72=)Xszc)D{0|-erUB?yN2o1!!iKb7eaw{_&<}xx`@SzZm9fM6^dd36^-##7OqTut zg+E0^bxG=&WQzP0jR?kIuyx0yk<&GeoJ(gH%TLAO9mFDqCNt~{pFEfG>Rxp)(+BO9 z2dTe4#D4oaRT{I>bZRh9!5#9)bI3u1w$AXcoqOs{#}NV}8%HEVWI-JLc<{fD21sh& zl_l%rd!NL{s0WItUoEz4r2LqzBY~PmUTaj@jP3kkLco{KwLq`pjY~)Zo|e$TN)Pk{ zH;TW-fAC?~Ndt-Q_qU#>fo&BUY1WwPE0NRPZW*GCAF88+4OtofSavzHI-z&E?@NDkAwhFN6 zp-4bD=uau$vD}ZTE=u8O1RMWe8Gc*awM+_iR76Yzkop^Dgbrw+a!2^B$5?0)xJ4@o zhYG4z+p_6TO<6VS_PxhQro2`2ooaigcA|65w+x{GjKEaxixHV&-7_XG4l=LL0BkpM zQ=_^bo+`Tm00X1~L5WfH(Chz*TIrK&JJW-u=8B?8>s%IPetaGr7Blk=-xFns|v4vUi)4!Lm@^u}F$DABr2nie-QedHRDd(^M#ez4c1r-E5m zv_ZJjzp7)At)Rwr_~{Wdl>8XM1ZDq0i!quBS9sNG1sYMnG0cA0>)EPb=;bmv! z<2e446S~u0MF>n2-^fZ`OxHw*Cj1wftT7z}=Jb}@HuhAP<~{khy;rK@Mn9x)<9NRX z3Uzk;bmC`d8c&D6KBSq_NX7H~b16}i?E428-Yr;Hcj#%2o-Qx}yxnYhiK@a|n7E|0 zYOa27^~G{pqnM!YR}~&VHKnvmg^@r*_d$s#P%$#Of@orwf)*e@CnxE(jkI@5pfBKn zi=<26rmiyPE2k?BJ=ezpn;+k~~JH5XmEd6or(W>*Uj z^;u{(R4=dc$Tvd!_}3jfBBzo`77xfpw2;iuvT22KBIdH$$C_467j)5oUuJ_IJ$tmo zpw%SY*xH+05D!$Q;hyDX^VJ_ilAL5qe2irq2E8J^%{f0NF%lHMaR00(!90)6wtQJc zobawf4(ya}lGny!(#iHVObk{VP>016Tq^Kpb=5!3q92NBsy&%pdxG3-AYWVRi1;+d zpgsS-S{>C5{LIZyhowNhnE1ym0FwHVJFfQyd;xw^X_A<)!R7qO(UPkNzREo35+b#D zkj4;+c`eb`F6qe4=hm`!(d+F&T|u~F8d)Fx3L2#jx9qY!=J+0lfHdOCq%da@9&m?7 z)N-3OdP{&)*iwA)kl&2;Q4~MgK(djqgDhaie~GBJE!Upl9T82e25aRlQFta{k6$!> z>TiOfCb7`Dj-hvA8W-`>0xe}%pjRJo&Hvz~F$&7nN9hReHZG+ex$Fju93Wi8+B&Wf z;^;uM`Tb)y<5COpRHk6aFX$bPCv4?Yl_HydW3@K$r?EMf`Y4uszquac$f9fz`>&?b zIpU85b@JN5L~IX8MrKxD9t&C}`zNmu?>AqIuMYx64l#qW@`TIQm#oAEo*D)dClT^u zp5tyimwi-IbrFpJLSP7Ns@MN-Qw=ZiagsSS%a3bt{HCGfGd}ROxx_&S>lhJuxFRI9 zp3T5~dYC}A4&1grG!+7vT!K}?S>#MWli?Xn8%^XOr#eJ>XvJA%H!N56L4K^hFT>_1 zOQ+3mq2kqO?&=T?5eT2e-nqfuUODd&CxRRRF1&wy%2fB6p}%-!L@p(r+9R*CezO*0 z5#xU=Vt{EqLNjFT0>h82;UY6t1rKZQMHy3LMvKC!sjKEE#VzF{_G}zV_FDW} zUj2Z}iq3~nw#?Sy78fa;h`|gm2+LMTp+UQIBETYsk_%eGWziEO?w?}w?*FvFSpM3e_3vds9A(jlK9#;u!FkX3D??&wv}QIOJ@sx)(wIo6 zrjmtnbK?1n>DEh_PJ+vPZ*;laM{lOm^ou)QqnG#zkg%t0DvKM_>le7ar{S}mV#`qn z0xjY|Z}=4e0#5(xh|03sv)V%DK1%}KgO)Pw^K_5lIllZ%b~daT9eu=_xIEJU!B6Uj8GVy{g{3p@HJtI2pe-C{iL@LOsZ% z0-~=|sxv696LjInCg3@#AY*+H#rpoDl;=kHxy2xccz4|)lmt3y>bDUNW&O1KoJZEZ zY3)bMG|`cJKe*>=gjczFH|C`Z**#s#c}>pC*!oXLK>D(>{@yS{DT?mvSl2su>zPvw z)HWk$#4u2Uje#z%v3ThsIj4~8B6{+1x$LyjyIv)&$6vxJ_@}TJ3RGW~m0~c6ZW0nK z6eD)m7?o)Q2ix8RlZxX{DC(SF!pXAvjXM__cSTFNvbJ}!d7;jInS~~8#S(+>NSME!11~~Q(qh&rV zn0YEE=DhnAIyN&AIH4qwYjrtf9{Cn~95<4a z5JEVhB$005>G|Y>wr~qYw*99ZZ+xQ^n2j#wXX59xvXa?xICl&kejmS)v24lxES5^9 zeHbtArpn`nr?=j(N$FNVom|kg4AUdn)^dJ7XzXJj^NM$h{`n9|kY$9dxca*Z=!>6& z@kEz;T@exmWHK%kr~jC@ivMBW(Dcx0v_Zyn8)Fg99Ok;LI`}}>tTv}fhVE-MHRGj| zFt6H!ri?;*zbXl9HZv2(Xt>Hc_ZXNlwmzJa+V8eEn}7qmXq z-E$kref%`%Jo0+$l&uwzAP32Lxp29Fl$83M@o}Bof}? zgz4}k;3(b;lk@*srt~ztU|WG-7h}{o+8*~^2c?$lkO^e-xGibzdTEW z1nqBdFVBWGa)?(NO6}2rX+7t@4tf$~Kg{gJ@F<+N2P>{TA zrtxM95B$CN*9Pp9$-SchO%^qc=t^c(=?rJ~?liP(KG`J#HyE1I-O{qoic<`2JXNlD z^V=udHEY{@ihs$qaBTFWE8r;a|E`xQ#aB9;MnXa412!Fb*Nk8b8oy){tkG2=iy4%wTtD*WG0FVTB`PX?Q`(~3sVc-sK zr9(+!7}>Xv5{4y_3NbMIFKts{rSsvk#MA-GA$aq#zj@jeNZQ`{4QHT7z%0Kta*12QzBXWxHHBkn-S<0*kYc^9HD{kxVA?k_D=8+N)wDeEsqzw`f6!mV7$t} z!9uVyM;T3-H#?7sbLmzKwX)4oWr@UQ#V2{eli2oC(INUJsmpcQHH>X21VmC>9T z$~>QV8g`tUb;MVHdn@hEIvBBw4aPNUED{pQ5S*IPr}E|zIj;m#^B9Y@VT@D;2#K?F zp=}^{URv#19$qpxPJq$8H-m^`SbZH?r7(I=13nMeU*u z4^6vC(4#t`qQJmG-YS&yncTwu3jm-5L|nycCOb6ic-uewDQ2l~tzr&yV24u`N%oUl z6J~OfeI>>TKrB?6(o2B|>nagCB+=EPj$!-OWFg%0Yw%SZfNw~4pq3!tSvIkIZqe%z z=&-zK1{cmFd_gM)sfb>tJ@+G&Y>_LZ#Q)IFDJpKbIQ6TnE<%O5LUG6YthW$jEu07k zED#5bJOl_Bt)^UPm4+u4=(PkEUsor1*1KG#=<~4s#~S`U4ir8(=vgiVD;mG^V zfvHR#t!FAW?BgcTXUiOesNY&;l~cC1 zgOPIwZs*&74UH}pkNs7{_rbYd2*PnUH-YoGc>`Z5bi->n%H4-%v80z%j4a2R*nqpL z)L{c~14KeS*LNr$3w?Wl*Yy0xQ&X8za<}gK^r@zTEsKBc1(7hqFnXu%Z;Kwtub7ie z&LyvfUfFtNgd*ne8!-YaWM12#Pn^O6d20;G660FHAsyO%#$y3fliy(y&UcrjbVk>7 zv78ils+N=y1VrM&qjE1JQWIR!SbEv71Hr(=r7 z%1z!Z?Tpk*yHzgSTw4i$ZPb3stQMj z19LFHen;MWb~C}l%X@+iwRx6F;AzNrc<6QGUsXD2+%qz``%pdtF}`fjOx9+^ zzl$z=&-(#cYZelD>F-9-o_*}pSLXKR)oSG9jwhNYg%Y>&`Qh9aW1V#jWf00N9Nio` zTGQG!$r3?_3z{tj?3KK+Uji$RgZ2ardRJB{8!qYnpHa^_hy-(fOi8;#e&q74w^0hB zs2dCe%0mPRtLY|rKsqb-pg@2O-0MeZcx=`DWBy$EKz`s3h4Ncw=%NJHq9K;FoKgEH zBg_=xTfZ^OkIw&Usu?CgiTt|LB`81-zVK@Aef+o(BfAbO`Fkzkvdo`vw`(UXSuO=Xlb844=~%E_&l1Z}Is)fD&L{uQeI~f?KP~b0rk&vC zX?2BF%MADCizB%Q_w}zDH%Z$rk2xZdJ8g@d2jTy{zxs{MmM^d2Rxe& zZmhlhJ3n-UE=c$d{tiXmmdc^S))>?x!h~jP?s`&2Vt)U;7;@JeFXMH_tU3R`oBR~F z8BynLb(32Dkh~ck>G}8U*x)=lzEIsq`-%2_A>VsRy6R8zlgxWP{rU7|kGl6458Y!V z5>+l&8N_es?_-hEKdaGq&J#`uu`n-$mM5R`Y@A9(?2Ha z2YHR-H3NE?T_4-auDH#6x)`KGZtE-dWkr?nKiD!Gf1mXnH?ltQ^fRuMg^+LW6Q@_W z9{8px1>C$a$N)7?00;zw!G6>~yN#)MMDAoAIPQX(IsPM?h0Rt+{i3V#X;N!QE(lXh zQeh?HwSIFprNA`V%oRnRYuC$94oVc1x|fP4z(iQT~f91hu za7N#YuLn5W`?uG{(MKS=@(FfGnUODLl^Y0i2r@s&dLi-vxoIMHR!B4%^=m9fk3bJ4 zH0^wbwcSXs@}qvahf#DzF(v|o!IeVfB!o$Kf{`{E|FWkG1io@T_$cI8gU(c zJmd`cvY2z}lr^7yq?Ly*d`7HncTx4Lw_Y<;{>bWxzE4mgx$> z3R9Q^$CTOU)C8CGdj_t(?}z#r7i$lRz(`Zmr5J{q+G)e$#`$N5of>%j$xfg}Q9@c|7AS9V4|wbyM%p#+yOg*j&XZ?2QL90e@#fz9 zb@=_$+H~lbt5G!DSCgFS1DVd{iqpy>AM@22i_?_F`xTd0Wzs8}nQvp_lG6jaYmtS* zS|w6Z$>d~={EY57Ar9e> z7T~$dV<~rgc{e>XI^rlqIA$99Wuyn$DXCHJt52fJ?u8aSzwZYk6jP%J1AJntSVJ_J z7)ow*ZlPXWr7dRK8&DKg(MfcWspv(|zSw<0v@-SL%WB)dzzOEmmpR}krIwM`kK%xp z+pO`m;iEr8wKJ14BP`&3cJ4M#gNH*tUiOyWvjuKbO0K}z22K^Kuy?P}vO?llApg~o zQB_26Ev2>#N!Py#LBoueX)jk^yXE0DUz;PSEYDMX%xk93X~gyZqS1Ex%pJ0vEAsYk z#6bLvp5Tl3RnI*h_IceP&|M(Nudl7_beHeq>)6}Q!rJOZr&}8@(yhPWKPTmTq7RZi zkX@^w&&Bc3t$x-nBzegX^(_!Ro6vBZ#Pa{8xInKv|Eg#MnUdRiIU#4`BJS{ZI+1`% zwqA^(xHAH#tt&7b+Gb|Jr{>jqmr0=akdd&YLhs>`;R`%@o!s?0Hodv|CKBa(Go{&L z>2{GVKEZR@*`nDlI8eL2BlApjnse>{4SxHB`ys^+fYfs>!sRDQWND|g?)`HD zB|=X}&}mZ4R(9%apFPfhtv&Q^>~e9RglyRMy~Ay308j^K6;1K(?eMQ=DdXfhD|rX} zl!Ga9J>6tShQ46C{$#xzeG_v&OH}5wh?_!6Pj$SH!7%ve93UxqiT*>)slHbLQ#~WJ z;AT_R$ke~Zzh7ZEDPa_VX88R*hg?ds0Q*_u3uul>3CY5MHwZ>#mJWJVPVyS^>vwnL zI*q6T-&5lu9B`VQB<)en0sEEc+WunZ5a|NcJ{2Av=DM_?Tio_~eS+pZaZ_U63#xxT zWzgW+Z*Wr+ZTP?FMgSePF7R-FMl69CiI=IlC3Q$$S@=*rY^snV#0IvRb9;XE96U9% zUy0TnFE?LXK5(Q={K?ZjYLI=?D<&Mt%88Ql#hw~sx!uQb5&S8$a=$Amv#a$TrdeEH zA+z9qia#!~%WKS2b!3u*yo(iBsuw-PX{+c!{71Cw2JoD;p;92#cj4xG`&V2@g(e8$ zjDf?hnV3ZL#=y}2XG$rW%gjD^BaZtkcluJX8tuPq0%1NW%@ z{ojX^JFdq8AER2~!Pd4*is&LZUR@AB-?aYV% z%^|y_PorU{=+~6kFTu!mtt6j+I45;9suc=0iQhz0!X%6s|4d_I!K4HaVsyj-N+>P! z9~=N0R-87&oKqcYYp4FO&7NN;Y~N-KT<(Kv3)HT(Ny(i?-2GoE7OCtKlEvx=Q=-+@ z8&kj>;q+X4uT+F>iY_`f{gf9WkEyEC2yuUcmrWvFfAV=Zot-rB)SZYP&_bhQ>%yNy zgdj?{+Vk6<>)b*7Fi zWELKF@Q>OwC)xOlQw^qNBxCQNP0`Zn0TMgIPgbaX43X153AYNU&?QlSRAIirfnTr( zW>Ww6F#}f&hT9+g^@SoCBscj=TN%Q&Auqp}a(zBqxB8ZGyPMH^w=7!A{qE7qRf~Tby#Nth=cY{ea$L$sD+=_PVGAIpM_+UDAQ3Cm@>hWqRe$64+}K z9vtagvyZ1Qv+I&;VAna8I$b#ej+q$#zr+RK0`8)xQ-bz#Wo~{-uGW1g{@*5tb@$j9 zyqFHf5pgwaWz8MDK-F`n`Rzbu5nN!>#|`{rPVq;kh{R8mY^?6xGg;A|^$*j2aR$93 zIN%ap!uAX&lT%544H9UGnr6+b#K+*zd>3FD)k&=Jbn9+H#gv7>TEq=InKV`YPE*KK z6BZ>OOcKDm@WOdKpwGJCvHv$3thI;kGzzRTJ5a{2(bUDxO8Ly1_KG2%b?XJudD-c} zVm_ZjvTE8hDpGHAJ)i9q@ss9s=qPE6(NGfV@W&}1c^yy z#A6_3lb(-KdUC99L9)ADT3)=t)fd!%(@w0&0)5&_lCZ z?Q&pry{owF2gcQN6`wsRMAsD6)ZOL*%$=T^K1eL}Bt(<58MSL5$0L2h=i+3q5=LCA zDC=K?yQqF32?TVXd_Z7(azca{$X8e_H~2Z&yNFZwRDC}dk@&3xsEB_ zws4spEU*0gHdn{EsUj}6H!p8D=T_V{C>MsxHYTL0dZW{(GPET_qmk-E=OG39hJVvv zf}r|_KP9v~n~Ybbo<&l#Qadv?D6a&L#nbv=Gg7UdW>jO5vg_Oua^H^7aD6k7YdR|` zo)=;h6>U{^5Rc7Bymw!Q-V=Rj=-8h8oAISf)=?CH|NO6@*v;;9JB2;JXlv2GoK+83 z0qZR2=vx5{&30zRRC{hJ&Ux{F1Mt!oJ`s$x=z!kBz*)+IAd@Esxm!zOH{C9;=F9DTqGG zjAUsd?d-JPuztg<_l6fnq4{V|{1zLvab|_^y1gfz;AQxS1G)v!@7~Uv9JZRYo!Oym zB+}!)Q7yE*{cxt?pY_*NkDLvMI5T2OtjKuk=;?+_P}vUN6}iV>7IKH|H=6F%lfT@1 zqh3{Gg>m}9(b2O@9IQkNtEW^Po^|`sD*h!QVE<-Cf$gzqB0&Aqj?n&yQ0MnV@KdWV z+>ss~M9tz7OWxdv8Ha6&u=EnW>XG(aH_X=cAj$;NwzIfFbFXQ0j`D|_4BD!~UdNYM zD_D7y9@)1lCXF2wwMYHvTzdxHwzU%yqCe?{&T<73Cr#zWjR>|PY%Uj2pwSq$h*_RU zD)ds!5jrt&L5QXUl-4^HZUNu%KO{2H6yN3sOw`-Mv`vp^aF#eSx zA$IEy_bV*khY+IH`0i62@YX?pt$jQ+mZ3kZTe1T5ffifWQBgy9Nfc2g0oivvQzA=B zxS#Qh-TG9HiH!BJGUHFi-AlRF>FCnsZG|uaVz%c7Xb9L{#Oz{1d$@s~WBbv#OVykN`gMJY z!@{7Q&a&Hf+MsI29b=fypnS(1!`rYi0#C6G*^)6^Gji_Ge)t-_B8&rnjzPHKtJ|R! z#OsF2Lp5vRSZpKgVb5_RPU+JIa6#hVuG+`d^;lqw-)=oKxXqW>w+WNJd#f!rEmWi@ zb3c*c;q_JLGLpW)h=uD(_`4q4@_M}5lv2Nwd{d2KN_gn&EWYwpO#ff;$z4N6`&>6W z&*sV@L+>tCXXD+Jn)IYSd^*RD@}gX?7vxHG4D{2!tQXrFsxUf!==+xZy! zDLq(metX9m21LTCMn?CIjlrovLQ3nyK6(XkrVPvn+>4{HTy{z{i*5&^JR2^?$FG4Ftl4VPzsr%#yhRA(0)X9037vH>;VyYdYj%5+Ra<` zi&28-jh%!9=9{w^y;ZZYpkqBICl5jMy8lDdclbm7`0-alB`cNfDoWWSdt5~%E6K>= zY?)_dk4q|KlRYjGvd-R|P1a>|_MN>sj^oDfKHuN(_Yd59-0Sszy`Jl(s?w@_CDDE- z*C9G0Z&1qWPQuMT_3SvMSh-i2gJ;=VkNwNPrPMxGnhL107+;Kf?)rt_Osr#d=OJK^ zeV$adUN`7X(aBR_D%<~v<}sjUUivMAhfZ|r^6Bn2v#74)&yhu02b5-`_5B53SW%ye z3f}kT?4L(l#K^mf=zI@7s=w{=J;;b|KZc!y&F5J_&$eNH*`%w7L0YAs$N(WWGvZ3$ zyO!5T{YraE#GDnP!gH~Vmm)W(uH7dnE)6b&lgzgx9hNpw-`?pxW*X{g#>o9P{%4TG#g~!E&w9G5Q+xhz5W@EBm6uA8 zT}FR|={o-!R(*7i)R=QLQVw}S+HbwsI}>|@-Gk1!>ONL&w)A+Rf9vLT293{JslMwr zKbR5=Yisd>4hLVQPP>wN#0@Ovm6yjc5H+Ox4xO4@U4RosNi0o8F@??nj(Lf1cHbQS z5*ftg{rd&$zu}mIUKeyQ*_g~f-%WVybYr=h^qxLXWQ#?uWTJTdJcC~F%=}@%t}yS1 zAvsvgJy#=Dp4YiWPhFqydpnfk?AGtpTC7kWWGIR@MB(_GU-hMIOSfEy_{KO^>_CHa zkLJ;44?DY6yE53VK* zI+<@Nw7Y z;X-bG5?lwu~Oz*KReEU(0gRzFslQL65D%K**P4H)5NQ+Z86VuPb z`t(Et&%1eRx(rTx|DLtjvQ9DnI4^-05YK9msC|cOsrOB;lumVYb0hmDbUPoq1Xj^6 zgc5=l>jHCwj`shFYh`sVY{T)d*ZpT-4qbaGDn1!rY(91>oLcx~gf17gH(TzAJnS^r zyUgJ1DP$FW(>l6o)Iw;utkO)cMhCv36FOfLO)FpeGB3+8Lj`+RhfyfuJgJ(og5uQl zS~ZxIYS9to_f-;`0qFYz^HjYOY!^UIng{0h}(| zYS&IFI<&Q7J4 zCg%J$b@o}YWdnf)$O;&RRiZyER-k_js6?Lw{zDXGcW{}b~ zHPxvu<7Hh5j~MBW_$x#LUsEf6lQ`$4(z)TF^Vfx3R!Y4uf1T4iEE>TcHS|O2Wfnh2=3QwU6jiu# z?!Oz6TX4^`T)v4Z)+0?~Qp&K>UtPLZ7eCqHC33^WI=u3{MDZi<&UP!#DY+o1oz*#U z7xlkIZ%OheSf%NZ8n`7>-@1~ZnGDbma)k-We)$hOVWE2k{LOs&S^2f3x6p$5&W-#dK?jzu zrg-fLnUDvF)LQ^h#w&|6itd)z&Mt+5$NgXDCIyD?V|M#j=kcojom5V+*DKz)0M57h z6zjexajci{q(`r0RY~o>yzKoom*c+7FNyRDavl<4{0l$3jL@={*RbC~7#_6>i`!B> zu%${Fo(FaE{c1usCSR`MWFe{DWXTtbTr^n&KLKf1KIu+x{LK%rQ^d?7<{{NZr2lRC zVC)R6MwH7z0!MMZoQ~f7kh7r9~%#j0Lfw)Gk|1WmKb@&aMdxsbJ@3(0Ns4%ciS8 zwVmok%dRj1wO}Qe(O`S>RgN$*rZRXK&Q=-IC&9ozAdUI%;1@lyjxgH7t_Szn3xVyS zslF)IV9T%nkIVWb`Sa_qb;NP4A zM`FF3#x332a>)Q|ngiAGISO4Q(-NFOe2pqkQ}GCs;qqT(lDkKXe5EQd49rgR(wJ** zAl>C}-m_9_AD%1C_)uth;p=I+r4wfp)ZW+L@F7nmgJtyopGQGrt6#@0OP}{qTAYnZjoK5BD;9r9rihFbA(eMPW?s+7y2Qo57Z4Diejm=zbKn5%vP1dyx zT>$WL6eIb?bpS$f&l>2BACGMe#7{F`l&?Ow->-wZl4nh1P1C;+m60#3{NP9~ko+)rTyGrBwF{1x2+tWLs*^ZDP+&h>kbVDTtd2y@@<{|%qRR*RDA3=9kQ$!hq z+og8k`>tLnVS)P)O*ZZ7R9dhbGwv5X3|DC+NBJ! zIKR*k-o_ezTs;fr;!DcxEWyBT-_i`QrW!Ut&GfKsW&Fa!*Us)AHdta<)I!qYnYZ%- z&IEbXHtWOmYW68}6~$C&CMgE(DCEI-@p&p;lG#4YN%ng#d{3Ayt7ShlrPacQA$4`k zk5Txl)TLMRB?#B!xC4X1xw|yV_^+ZjL%xMkm(98c`GLi5ko!@}W_$((o6$G4svQ2_ zzXW=p=VVpqE1a7xrVOgL^}Co|+&dliJw%drjvA+07!D{?CS|z4f^6gEISgObgyEhd zj7F&TpyUvx*!4FTC;8l!e&cpdZd~$Q$8TLR_AV!Eg&b5k4DlHa9qZ^;bg~_Ub-Yoe zCx-9tEn?ptX#0fh*J3t!o_70Y&Ke8*3J+L`)-h7o7ae+)HOODmarhxDg8h2vnJ=lG ziHN8t$3({YZ)11t+K322K9=Gh`!dqgk%$tfmrAH^XunQ-Cd^pw8=7`f zl|@Sav9D?F9SQB&xB;SwR#~kexKEaa$i$~YSC?tHGzeTyVxN4P7J4@bc4Q=lhE@ZM zd8EA?y~vfjMiG=ekjf;cw7720I}f{F3-~=KeYPr^!Q<8bxFMZ0gvn_V7PZ*vBbD&JjqJIe{R*UDU*tg7@q~Ztm zrZskPb3yUW`m(h@UErLx^L8Kvy; zcb98Io_&+X9jsz$lqK3ks<*I6#FoLpF3ur*ac0{ytLHyZJi`9xs ztFds2HeLDkCnxW5i3cx96|BnYk0x{L4os|<+H?>PY{x!R@;;14DH7D(w`k0U2`A@O zmq01a=e9mG2W<}>2J-MX%qG4ingb7CKJ8rjJaUGyRl%@y{i&1; zO{UQ`T2kKtOKU}7$78YY^Oqi}&hr2MBEm$=sxOaV1g&(^rDX1Fo4awRke<8pKNi~k zDV(mO_w0MYOsd1d^SZU+y|RZKqn_=Q2SyIA3%FQ!wyb({Cn~W3Qsg3)AvNb(? zcj~J%bLgvFqq8+0LAZ<%^Ca$^C8?BV7z2Y%wIdbSVQg=6sbhEJc(^=za^Qn^cuAd- zHMs1cuS~S(C*YkeEk2P0Tyt8d(rIQDzDh7edb)o4uco%uz10k4Z>kdZp^)64poG%% z;j<{#yY;L3l7;7Kxi~8Ol7&7DExErjjkP;NL((HKUMd@v#4pAUhKOt#_P#r*_Ec5_ z?l{TWb=UvC)0u;z75#P3vl_Ggr9MK1#bb@LbcV|36$W2-&#NpZG9%Dh^UzU?)OPRP z>Q&w>AZ>74njsQQovdSkLE_aLj1*klJOfXPwTY$lJkEIk>(OgS@mfyFHjM9PW)JV* z?XbBe)Ah@cd+pIEwp^y8Mf5eu>@2k9O81!i!iG|z**EZt5ViTwv20h%&RgfcDE)#{ zJ-5P~sOe|F)skI=Q)gZSh6N7)$RA8^VMReTiE)L}n( z7{n4RWAUe$f-LB*jM5qyA$0joV4%ga&~x(y(qfz4PA?x;VUAA_5 zAnT6QFK5M`b*I4L<)Wkd!vlH7%xq|lQ2Lf`@$+FB=9Xq*Hl-T!>-0Ajhw?}Exlu|z zV(+X?`^&HKQSKbiTROfIMGrK^+&8VosI1Ec@S}56oFyfVSpeQ^`gA<%lu(GB6=e9j z(AeIx5xf_(vrMg^^!NCkCSK56(Iw5cc2q_w^kvX;+w0HE%}TSDRIH=8H75HkX_hrM z);&Lw2QjkYtH_qbZveIxu2|B2xr?Ft(sdZ#DGe#IVBgH<#I^qP6W@A87k_c!TSom4 zx02Y(6Qn3O;@2}rIpn3n!LFO^1XcD-c_zW{fzsO6YTY!vG(C*LgMpf$mMn?qXionb z_LjF`kVR3M`iGvr#kD@FRnsB&x(4aAi@?7A&m$v&J+8fsbY;pPJi}9${{*ex1FV75 z{;QSa-lzz1r>P(Uc1AiXzIZ>LUd*n1O*YS6ryhmp$Hw$^g;52>Saeyc_wPSu$Wl8& zWv^GXh+Ug~*gQyU&98=P(u|DlmZ-}o5O0?e2K|i15{jRU@ixraD4>zKys;y^_D++? zGr32&C{|NHB6CBc(JHJ#ARpz}WF=s<#%!`bd$os4jTnt(Fqea#S;gBZVzIPqd~zYB z9ym=<)hl0pg#j&Si6cC&%dXh(_k-r?4MNPFWp+es?kWSjh z_N3OlSS^=2_ksd!7YQn^{}w%MzdPhM>BZ@MDAt zEn~4Fimsn)btFiWGh;`}sJwD2+F$V#y%|vVLn}ngdiyV_Igz4^qZeyv1IL4gJ18CLOw_tNpd5_Xf;y@gc6O z32{@0!+GepyN%G3%v{i@8c*udGb{BD>fmG>bZ;zs7%(|aFVf7pI{fPoZqEO5nqZ|? zd9yufxYTo6Se|U1@0xMP=h@1?!E@iB==2`d3nhUJ*??iCXRVVg6l;<0%dTysnYYj} ze_2miM&5+4lTq|B0jvNWjl+c4^J*tW$ZRAdsFgI>o(-KLPJMs*uhy^q)z*^3rR}z` z9kz0M0X^|8g%l2#PLV&fI1vy@hQi+ri|96^UnBj@#X11WBaxwD=cu9tFmfm_6@+5IM*Ay#R8_H50R8kLwG~QvoDbXXRuH*;EZqHSM z62DAEf$%qj_d|#E@K$4lEWW82QNJ(|;q^}a4?4FIXs0FsaMw3JI!d6dHnyK-@lHPb z@wvV}qgX8B055{2*swmJ^LJVDJ&_$zQ;f3KFQut&X{(ai|_OEeVa% zrc+DNrnppd{xU{rliXg_+4=~5yP1t}2!dHG$*Fu7uk+TbAX1P6C z!bBdTpnm^I1|yp;lyAwiTm&ybPgZOQFg6M-XSgepAG`y}D)s8sQnldzu4l4Y@MWz& zYdb%`{D8vkhG!^yZ*F~S*-{L}y%?44aVk4VB`&G=2j9{{&T*vyJ^n~FgFCysD9*kn z&U_j_1Fs>-fQS&=&a30rCENab(@N%RKN0;G&^OEmN`!|Fnn>Qbie7UsMB>5pIOu7s^ui_A%VTRL8XzjU|B{@l`Gk|icY9l{s~AT zQX$S=T`9^EZndHeB(6(r*)4%0A(~s_y}jIAK`kVLlsyeF4AdSu^;33dy}R2WODHXe zPA|R^8<%dnCf*s^6{9K|gX-b&AQA%B>`IV8&P#2tyFZU;BS|{HH@O&G@@|D9ILDv^ zkNOPDhHB_6w`#kktiLRFXzWvi>1VeRxzpW&SROQ2OKpIgMu-0FMs15|NwR5||6|wg zs1eL3pGEI+0Gf?hNh^Z^f}yEP4Rxf0d&?BjahS2}&3<9{zw73)FyCA$?tmC}IffCI z)?e+eGi@O`;ME-&>nGYrBZrykYcG5&Eu&ppa zmJ?p?UGlK7R$_{Pcl!B!+5ToiwQXn2PUc>$w0m@|Cs(fwCXso7&+j=~oRR5}(FfUo zyq$NI%n>q^X4i(eR_~^=HwJNqEPPN&HzX4+ zn~qq6#XqnN3$<#DCI>GSef#X$``G~S*rV%-7lRcOVJTe0Qve;`aYP-Ln}%U)Q9;Z1kk0;3 zy^pCGQ>L9rwWc{qKZ59Z?c5xlsARS~(*xZ8u!GJ39a+a`?C)bYT&}7~I4 zU5PvY>HQT_%vR$$V#Z@R{J;K^saW2w9HR2PM8jWx%q}Xs!}W5b-P#C{!RK#N)7wyG z)+b$1&(j}yV;F9AhxV{LH`@4s*VeL+;iHO)*%YC;wzi$?VFV)$`8_gllF+^s`bG1W z*2GXfM$D2{D!^#%1KsmhU(0sR2=42g-h6@h@1K}sBR`pN~U+$M}%4je=O$ZyV+kI{zlM3_g7#3*_5$>yzrq<8q= zZ`@09Vb?lg|HP?Su}OLH1j$-dE$ZZia+b#Busx#dZs76OHML0pv#sv1u(HeJ$@Gw# z3Q$!Y6|c5y$6=v&jco{Rnw|tqb#+^mGAUqVTq|E}Egkn6t}U206{eO|)BG46Vsq|0 zrlZG(1>@AAKTJIVSz!n1HQ`cr<=uBGIy>;-)``Ssd$Mqgm_KRh5{r8eUe!q4^QL7H zH$bDjs--npARBidF{~mUL&rBK{Sfn2|K4F&t|ZQsDmI7=h!Y#B?*s{0v{NI~Bm3N~LaM73Xj@Pi`pja1k-J-*KCVyS&y`=>P{4L2K~ zcZyQ{J>&p?vkF?lQIhdkQb)W<#|1smyAQmtWu%ttLoEPuGf+FE_8-q@F`bUzh#%d) za-)gu9AGl;y>xcoN2jxTpKK9I<(enkdu{X!oR8#KWdQ!zm!|3Y+*!<^!|QfF+5w{u zB<3dJKK$m6jaw>D1o{tsK~1Ye)?`cWgaYBzc!Pk|%K>k-PO{|EcE@Q0qC=oo|GwO= zMUD3WB4}m#4l5){)z%t+GYjlu67HJZTY4F*6>zT_K5hDwCZgq6m-jIiaA*h16g6>D z^|kw$s>X+9HT1X|nf&-kMU=Ax>|}|E@BHw0s27O85Di4i zvs+3?e(9e3&0_tGB_BN>i5X}uRrflY3x9I++oKPrt>yN6mxVe2d}rjzJP)gHGki}h z3LX-7QbVGso@juCEpMfia@0ApSIHJBAN&{f{CWD7*sGlKuj)I+O5(g8fnLtocf(j0 zxTh@QEJO#kG*B`ov8LRs^rJ2YCLD8&G<3TvUo%p~)nR!G)=&uWZrCa+mew~8xOy*ZKOZNtxoCwR7z}cS_8d>=(xth+KV$wDPNCVy`6QH&%yr zYU3ti6vz}0yf(!HzpX0$voiOjnN;8tq-2eUk!)Ac$J#I%#O z>F5oo8XqQ^UO_rrF!13_+dsJbKaJsfM@|&(e1mKL2}g?)gGC&=P2~MaWkpC1qGu?5 z#CnHX(|4&(&4QUjuoBhx2c1g4Ovr&Rq*rAp#zioc#aAf%yofP$nEw9z=^frG8ioEV zGNc{x9t^=!Q!#zerYdbV0z#XmW4Wne#00lUrI!qmv8w(y58ho-GXNOBELrqNe z)VzX{ySF>dr_tf+KkEa+|j4A`e*Wg=pH`Pp=fwN`3Sh^na6ye%k$hW(T&%BJ_Za3sZ^KIJFG7E$$W z$FI+Ad@}O~=p;oms!m+0otj^FG)ER>%6t!9Kc5x4nWwl%?}z1|5}0UZ;><}Hcxu3I zq~)sol!m=YGk`>N6wwISHfF#_mRvr`uk$Qu#7@xEm$FT_K`d>s>xdAkkLI+M1`IWV z>MnEvt(Q8Rse5DxU1tX)ZEdu?Z?=m}AZP?)TnDxuS zKM>9wP5Lm&hWsga%P0GHzho~HHrY&8$p&Moy|~mSvf7tp|9ce$X8l&tzMZXq zJvrt6F4Xix_+M$xaKd_Nj#-vB;o%Nk7-k-Tg z@9SKrGE?vKOxAwOf9dULt(6#)I0%4>xkZLWOB=_cL(;!KlTB6ZVy(H^6W7rZZ9BxY zvUH6)cyb`-zt9EqC#LKsU}x8>2R3S(bo^DUrlV@LUDu*NHds0_XMI#8KQt`mK$tB3 zkw;rl%$R5lo-@2ozNQteU>9rO*Kagii3V<4UJD?(Kdbk(A%3F*)W>buQ($gfY4Eyp zZ>(5z-DRLD=d_a|Se*#_m&N{d?$6sUykYjGDa^Bv62lE1}&wfC!01lWxenkuv0Z z6wx)bKn>$; z%nO({?c+)PHeQnv;v3{v3c{{S0oQefDlOYz!NiW7e4v+~{#b*p|1AEtj6leBUDJ2^DE@O7>GjmSpBAb?b9Fy3w{xj|LTa>`Cf4%YVqu@VU9zjo zK>rLP-_DLU)pG3k*$muGuUWBbdN+9aV|Z?uv#7V6+$7ouWG&w%el=a*?NvWCQbihy z<1ki%V;Or@=bJ#8^S)ZVR&=UA!WBKO?s(K{__Tsn6ZgN5T}n@ksZtxHiy!PHS=0X6 zF*2>^ifgyL87*Y)-|oRR#RQ9y8+y;xTa}c)QC35Vk3v%V2PNeP@5>Gdx8b)AYM-oKX-i(J4c8$5tG=ce|gZ8%Z0`-ooqWiWxN|z=0K;mCtpdgbaYi6dWn)g zbb8lFx|g%QhhcFS8={Ejw!Z8!9?!1h@X!KXxjBY7O1a{J9`d30MudrWK zuRHm*8Z+|D%D4e)KF7z+UU}pB=C;GggiNtK9N(b*_FlmF6xg?tRsZ1330}i%CDJI{ z=grJt5{=Scnz+v|b5=&IDQ55ciU`rjGM!BPWhwsej@jUtJamRF6a`{+PEu!cNefUm z;W-37VP?`g2k59`3+Q1V_yJsf?8T(y z(jy8jq7LdV-1jd2y0PsUnKdY@`|c*eVkMWpwP7AEar2FM`)*k)<&sPAo!K9H&`udS zb($l-7ij6HZ1YdH3*+84&_tcIyuYIA83kTarm5$v3869Cn2Tuz>gOMvV+?2A?~U$9 znJq`JL{z-eC~++I_t|Osb|tDcK96HE4agKFqsPQ8+!@fa^g!8)*YW|)En;$JP@UK) zy{S4kkd#05?{G2i$aG!w*(x8ShLR0k(@Fa1OO6|MHY_@zKGKaB%Ux2n|OZm zr<`!Jwqfco);&Y{i1CsepWa`<%x;kK?pN#AJ4aNMHCDpXq0A|(txA_+y7T}E+zqr=+ZBzDZoCi*Yr&_UCn9boizs?QeDShqNxns8~d{p-t7o=v+4ZOP(^xm)8pLZ#u-xKV&o>&mw;fJ{E=m8cxi&6J3>+oH7Dypq*| z5u_AepVvM3ql7aCPjeHO)>%p2V5=kA7{k~P4% zYgkoi&6ws}mS}I|Ah~IMW1kJ+V&gyZ8BccFpHCKcNZu(~n5X%c!?~bIxi+Ef*iQ8~ zjtN3zdT*DAS#%>mQ1l!!FlYijl_)GsVrSm7zIiIUVmT}yv0<`b6Z0EEudwY&q>RAG z1@_giy4&jYG1Kdzk`?09Qt0-qSFG`dS)c!m6~u#&nO|W_XPYh?1*}9=(d5L|+Fn#d zRP>O0oVi?XG)YhCFk=JbYF1ws7emj3=PaG6x< zsevT{w`*$PXDUt&a(pWHX|nzMc9IK%=?Q9cBiD{(vi88)B~yAkd{|?ymOFD;V!Hb7 zn3t_Yj48{KQ-U!I;o^Kl6r_+og~aIYCk?PE@T17C#N4U&<6kmD6?X!2nB<85^TbNPJ7&?a=Uq=Lk&s2sTrxq_0dWRl z+hs${m02i7%h=bi{IKl(rXlgL>p-NWi1-<;9X+2=e^fEO^@yvX(eGZ?-L__#hTF9 zH)+Ej7{Z9S1uXpBHgHrA6FuW+N8SH#w!mPRFu34A0%=bmPt;-^&4&Y&_hDOKg=^ix z>(_vagrnsauy?uM_~ydIVr%3KfUmK1A8%wkSZS~uA(dPcd@%_jz|j%{eeVmcCO8!? z_Dk(E!KI6bb>?+-U(ZShS1&SWYN1NdwWFn>KN2ifO$LZXo)uOlbqu>A0@6sH z%$55)bSCPOsYVS8Fdq_d_<8C6_7JUYi3}z>-~zD?9d#--NpY`y!MRQdtea~8WvMgi zXq-6Kpg#%Dy}`Ta%Ox0hTdG97L$)EP3i^Vl#KqCeTV!@$t9M`Zn(=9D@wkrHR}(1f zx&k5A*TbZ0Dj$hlKLvGit$>_ZAw+9(;wr0}+Z_35BDb}~HS6Nm{G%{YhD4XM{UgC( zIoQYGAGuH3$?UGy93a8BS3VH*LZ3=2lWEWP%V2O59|muorD4k7ga9+jd<*LgENu^d z@-SI6N}%cV0x{jWNcFhD%&jTN_u*OfET^QIYpO;gissiE(_W8&{?0kI?^w64yNcI- zHhGtPJVmLn6nuki@S1SZE|Qubi>z1BAx>`TaYlL*q{2|YOw}$d9>YGCXTZK7a8&yW z4C!~Rc!Hu0pxi_clGRxKV)QnvZ33VwjK^dJmz6I*5^Np3LPgHKVidf7zICN(}96L;#m@%{0N8f zv2Tg zsuTX`QuP4q6_-IUCgrqP9vv0W*b?5%eM7gladxgHh;y=(b><)-u?7ws2!7T⪼m2`2vrK z{b!48)*tF1+KQ+7(iLkRS`JiWyUNh)mr0r1b`>WIh84GA6K`M~DXvbQJ5x@015K`< zW|DD>?9~QQHMz6Kcr!54#q5i`@dwDDEBG7sQY|yc8spkz|LL%E6Q8)7KF6^Tqdm(u zd0ra*dog&51se5txf<3Nt8^FfdiYM9-WzXcR8Q?fYVH?Hv`37e};Y zcIV_K7w5s<=L3+rsoYJotJmZ8pTI7u;ro3})^4N|$8G`FgJDQRvUvX?qz8#we6kSr zvbuZ0r!k7>ru`9q4`0l15`B`jWfVn#N8!#INKK1E=lz@m+vj+W26q@*fnlo-yr{5; zbeq)$ac_esoVcgf{=O)VE%wKt8Y$AC8X?D4v2)0oI((fjHyEmp&{I|?WFGu0WB{|- zp5}=dE?7n}8D$whF_J9P9&(Csg_r2C96D_zzL`H{%)fnYC}$}8Ykh0(Z}(+@GC`l@ zfDn^#oLV7@#O%9}zXVyQW-gdu(pXPL>mfHIlvo#hL<3=ki>o(rDMuu}^T=AC9TweQ zae^_Eum1Vq*CXC7em@u0bLP_~EmB3G*987YsIrszq$44vXwjW?ztOR6-o=9?0&#~R zys^G#1Ab*N+15oW(E5ii+3brRNMY>BB>oy(t~lijk>3+J)yTE2Vz2Qc!%X9KJzOc@ zw#JTIXIm@Bbx~eux=*pEpj&L*Yr^tg=FtCob;xyfCB?yYyp#E z-4~qA(+4tkyoZW(=J<(?w>?O~vz0yb_4&IfwcWLagZ^lToi@3Uzp@f$0 zwR-x75+7HPdtKdUm)!SGizi0W=9hPdW+}1QUyWwF#8W;2d|##w#5i5_?8sWJl1dij zEmlF0*6H0b8rSs~0MiSZ8@jCa4goT)n7I#Uu8=w>Qr)#9+Ue+B%G(J}A=i9xGGCOP zh%(W-*|_x1weU%MsC6p@I5K!Rb`&vl&7FH{!kr#?h@vsy<6LE{H7ikG2P;SxBhS?X zSr^yE0&7&nPRVTLyWXlpKH#j@b|DMeyl4mYr>(V3L7qBW{hFjtxg!xN8v1X;ix(p}y zjt?Yy-YzTyCBTur@|3@k%gyuP;CqFR__#w08Hy+Uo(qZ9F#rru%!{9%aF@(xYrL)q zW#{*8m@L(63A?;4^;^H77Il$-~nRhh*7b4 z>)4bimSqnV6{E{X4Sl1@dZe)yjb}DoY$D|y#&}J_1kYE0kR@PZ9vSECzrW=n3q9Fe z_2brT{m)?>pjpuju<(g2TeZp&D!`<^h%Z~^#4l4&ek*14mnb$#qvm)M8R*SN`RtPL&7hM|kv-q@% z(A|QU88(=n-cE5K;+&IH&14oH>w3r1a0-+jzlVUbsC;!r+!aP8V^C*Lev4h&n{8_L zJ68>gp64N=A@kp3$f%~1nA1tWi)g6Zm7a3MGiGbyrx=*(pytiBNX~z`{s9>cc*e70 z-dTUFHweYgrM!z?s4+piDuJIewjUz!u=CQKgloJrWcOXl4W>2Z88=c9HZrBG2G28v zx(R?-5-UBYSsL)s&FiHV__c>eX|PE%8Bivo6Tw`|IcAUE;c6?qAAGT>EDkYr^J=W4 zs1x5KCk-Klj2h-e&%?kqpd506e-|7<=2mOjEhr6AM~=N$zcw_==k4T>yAV*YyLey5 zsSYZYklE7|kiD~IxpXf&???((XB)e$sQweW1jO8VYztc4iuMKBGR;b}^-^ySwLhY? z(P6xkpA(yAo__)G64`RaUhP+7tBKKS)|Ag-g$%in80> zxwBn@7R`Vbz&rSM-8*pgfhtnbCj{mPql`K`j+8>3Z1n?riXdDd z@DTD0lq^F=Q-<>0W4ZEPmdyu6U)5q7H>&+GwW8lrZ|{zC5&3@{Xr*zI@#^r0Tsh*O z^jGNetK0OCXWZLs)_v+|1D!9QYH%aO&S6ss%RERs?6BwX`c&@XAk-q`FFY5VDfpHs z%WxamBchO62jCyr_5mgt zR0MJXnkdwRBR`If)-0Pn;C;}___9$d9k%uiQNyWUOSND3Ee0=WmO9qp8kSB{3R!eMxBkREJ!Y`uMM?? z#r=kfMiy?iGTfV%JJ-E0wGR<2my2j$g1tMBJuw<3jbY$j&Y1`2HTeFX)B{hPkPAKiV|O{ z;nG|wDA(_#(bmwa{Y{q_1h$yXxzDm+dT~8;H#$K5vI=v-qW{H$i@4dW{s8-@6U@CO1O@?)N;=BzLU@sgs)wX#<>s3<5o+t+1Rj z=0&oI2igA7_StoNBmJ6x!CR#FFpJmy+v|Otb`iBP!)QKH=zBS{o-rS&= z*7Kv_lzqS0Xs+t+sg!>&euXts7g&TUt)NuQI_q7Oq#u*{OF}z0*$k=$L_npPTPC?) z7O_Qrbf4dGrbDS zOnenrcDIwF77^`H+R85LWVR~50dun2^S6JAmu&8Y?sAAzSLnY*L_K1AhR^J4E;Rh7 zE3aP6LCT$?3`uE)2fY5e-&&JrC1lVA^Ti)810DYC8soW1Kkg1Jbz1=I)v`R1PQ2|a zR6jRfzI95A`KIi|JyKP*ec5|rT^D;AVr4h$H5tSSTx}p$Ma^b9+a=i%>lf1( zk*9XF7egdH_5VZHm;Xcgc+o3GMN}#~l~C`LWwI|*5fzcFMMh*V`)+x|AFWE|6eAER^+Mz=*F`+ehZtj$nCLmpL@WP4ts)_A9Z+Nwz;c%R6?} z;--Dz818^}yN2p|;?tevvJ!bJ@K5R5>+|Gs4hX=aWV&bVOVW8TkUcql`L*PHo)4hQ zkFL&p!fvF`;RBvGHI(EeYq%I6yz6oOG}KE$e2*dt{DCq)Tk(&&)6koTS=gZYG4;3s z{~wUmY!>{T$i!Yryk_1j!93*K+PE>c2v#`6MRo#amN6Sg@4n=8U(7~8(vkSI(0;+U z6%aQWhtMy-mdiVNb3$fm9Mn4h?2=0owI>*Wz+h)}+fCI({j0uA3^}xTTw5K%I?R7J zbd?=7t_SVuYIZ$Tj5sHl! z*XHT8)2laL3GYi|TwGjmc2i6;pmkPIN{jAKKX6sP`RkYVJyG8R9f5dsL+2|SxNU^i z>cc2v@8kIHV!+brueG~1PwN#N=&1e1T90CLAiw>*secRA^SpSZ%BRI@l5x|8nl>1s zEN-6`Yh}t)@w5BSJqSv*LeZ7Fv{B!C)EmVbZUA+To~$~*D1zLXRu=wq;S|KH$+>3R zgQ>Mq_7fxp@RmU-^@}2VwCW8kO7b?Pkb#Vk`>5D-BA!7<;Gtbn(y`7&cj3!fSY!a( z)dzTn2l=CfbR{66Phu%p>@ViW)UM_)8%!_!*>fYA46kZwOYUFF6~$Is{6azbPv z(s3dG&kGc|*>91PGeqWz*Vhz?@0w?On)%}^K*#$i;g9@Yc|l4D z{5DPhFN#;26fxuF;jVjKnES2upGzZ)xA?u%U_Ufmgaqs7ZVFo?A$!$TE6^-1@HXgR zfN?K%RP>P|?&7o)*e}g;-YKX4%Iy9icu675wwfDS!JN9p6uU2rKQ{ATOy$w(lWmz8 zgwNzX7#e|!K_5w*-_;W$1ADC|mpYrN1y__p=3YeA#d~xHcLahqZnq)VE)|4YiI(Z{ zO~(r@{xV}`@$4!xR@8^E1Y{Y?O)(^y5H-<`EJ;Wd-Y1=&V zpb3-2OVSnyE%m8cMAtG(=aV4Af>rA5(9w=B%c&%-mWl6TXnt)9^#9)UzL$sD2;m@{imKal6ry3QuF3 zWl7)u#Ec`VswNrd^HnD3^V?LnThgIqP`YDbE4`+Q#9k0{cf7yoHx>MFh}}AgC}LP? z9cU|hF(6L6`S8*BKezj^rtd3#-*82vUQElym-K%S+#q$N4GxG3<_3bq1dq!!$zwjO9r0>6`VL&C>?^F}GMBK& z9U3Z+&_;EmKZ!P}jk5%b zZ7+PV4HJY;Nr&!|I|SyA+x==zACItvn)g;MR@F@%xedjQm&MJ|6kT1;NP0Z;4KlkH z$hpeYQLm;d5M%eYgX5w;laPz2N9v||dfgo&{H6VvAL7@011OSp(J3AIhaF1<3IH!^ z?|!Y;d*_5kYiW~tBF&^ct0nVj2e2ho9a6z?hhfD!DiSwzM}vJk!%I8V7iNQ`ha*L3 z+CIS|PCTD=w?3S6Uc#tFoELZ6eqQ-;Vo1qVLY((S#qoE6FM@fTkoUjvA!rZoCCNDb zCCQ&4_dGzJ>~iYBao$8X-?EOmEc5ABg0%CsL%7u52c|%S;4xg~V#+q+ZzjViW)Cc2|Ce0Jau%EVIiUru9@G5EE!bxiR;h@1MS?)MO99t@rEP8x$ z-1fuafn@JIVsifQ`+t9Cw?bkNoGU|6+b|xZN8mZ;6idZVr8aBbAbsJ2yyMPKT%&7a zyRT2OU}aTyi`TM!T>ib`j!Js+!n4SC<@y*|lX8j5sq@S^d-Q(4EwtD3lApBRw7-7# z|V8xo*5rTY{_uh`S>X8@GH_EMF{I>(QkQ2AXEy5cF^3*K~_4S+@K zE=sXQ6z3WCl7n6u=FLoB_fw|^7ux?mr^WQ9Yn3>SO{?%-38WblDj0b)f|9>TbO$tI zi4P@^F*_xwGnWhXx~vAU{8W)`9Z`vr$>(8i*Z!MuG0v1_9o#ZC;%Dnc< zZmlbF64vv1jd~u0^2zUT^QM16guUgT>PYdOw1xymuv)D~U6c076mGew_;-dSa&BB` zCJk&XH=4F*cx+BE`=?@=<>dbDOh4oUSbjjVMMyfmVgs2xW!}cG`$x&P6@sxGe2Mr} zb`G=+do0-8R;FpO?Z=2aPZ|mid~@Kcj@)vle{V}QsHODrj?J=5M!tTkG^yo(X_q`9 zlDJ@$fk)D2zMLIl+}?G;7v=5w1s*etT(Bc*!_1U@&Y}v0lCI=C>Cif1H>LOgGLJgR|L0LK`=&xL0z)pbik>)yiGGjwe67f2vI;_cp3#SM58d z4x@k%W_ItY?MTOlap$bNLLxgO#~a~%Ki!=KSUj&a$tLjtvnFg$`t=9pM(Wbn!?*>8 z=wA-PcnsMjM@<0Wcuw+49cs_cSh-nHi9s;m9zcgq4#H0gWdrWXMvdSY@t z1=-3Z0ja_MB!#tg!_V#|D>)Y8Eoa(-fH0Md6T4!HaWgz(89sOVgf;EdvdVtL1Bo|n znjsd7yxvOL4RXdava|mjP%!hvsg1gz1KHy%%wKj2a}E}w$Gu)RY;v6)j!u&kTcPpO zAH*Ggj%<2gv?WpJ@8!QvBXMPN)9^T*`O9T&PFWMh^xlm*-?`(_xlhOFYmLW5amg$t z=QHA~sKj`>^oOXeOq`@ZPX%QUtsV;Xkj`CrGi}xMg!n}nmkckTu^9S>pit*fIPj$RG=IjN1pGXHNb^cSUan3-b6HFYK z61^uKNMFF8%tjjx<_6F-3CUF*nsEBqq0OP&Eei)1iuq8!o|3L~24&uQp#B}a0L4It zPrRX@@P~U&&9Q3*Ev>~e-+#G(VnJ_RK4UTHxde=Khza`r&+((MO2S(XuLqhk zqQT+%;?*ISrWiRTy#0P1h?r^oHoJw_HNDFXkaV-;MV6g=dt*hJ6Ls1#!9Fsw;1NID zbaZohN~vej*}W`6^PvvI!LILAv)6APrr<%Z2^zPy7b$kwf{nyo^Cr2a0k2aQal%0A zp5#&GE!EjTgVE*tIhFSw89Y>%=#O?@3eEU??vSmK5Hz~aUGlKmv8uNisq(BB zB;GJ!N3Hy%al)6bK6a8yK=YLfIfLsjuZ4r9j&vnuC9Y>?N1wK1ME&dWmqgeZ81=ns z`F8L4M)NGD{Rh>iN4&2Z;z(a8eJoGZ`FP6c1s6$UXjmXste-4vJd@U5P@^B8P9UDv z8)=0;(4=Ld*29eo?!%EfMF%`r&13tBH}ti?G#;;qJiT7)^>QITGl(8?RhPxhxaRJR z;HvSL)V;LO?%x6$0^C_L$BEnvoF(4-F`4RYq_qyvQJ;Ttp2pFtY=jP1jFy)V&s}Pi zsb`Tu+`!XMmd#)XKYfI)N_>_k9`wh_s;I3O{_wxO#_N*jeTJShMJG3^)B)r~w{^8> z<3VX*e)kJ%V7PSLKs5=t2}ivC=Oj5Y^*p=#tL(io?iYV4 zX`I&)JZ;$6^U>RHLM{c-bGZ+C>Vr&NIY4OkTw~{~@F0#~#D`-~B(9&eh`+s{auHZo z$B(F5IfjS`lele*ZY}af8nW3`A>g;?7?E@Hqup|)S*0~|M=u_)KZag42nI)=%6-A#FFjm-z3S?o*JWcv z1M^s~TVgcOF4V61UIV{}kmj`b9^n`O9FvfvTj2%al)=E^<5s72w-aSH&ixOUdWB`z z9J}9%_FhVZG9Nrcn6Q3VpNyNOvaHz%f}eS-hozdk2TEKZ3m^!IVLHlHKCw4Rxn%BJ zWOXi11dyqQl}s=$u!?jp#W@&MNjYQFXQ`k?(OX9?VIz#Fiq-1NZ_`ruyk7Ds0*58J zB6}=1;8gTB+>*Eq>Scm}Oxk8V5Gi@Z;Q=5y?bBdd|C{~vhSL5eG*5+|w6U}XA7}zT zc)w|@`MPYA(?tJSQFGQW@Ql7^3mpDe;2G>5NcQs1?DtQr`@iLH;? z79gilJv+IWzYWZIHVZWu+KvR&1y~CHev;~S{ykp75->9$^!L&&;%o<`)(Tjr8Ugd# zTa)PPQjPDjqR}iB5aZ`L%>4| zL!i~uTRaf%;2=>9RhK^(P+jd|csE?&fM7Io_)^YVBJv2LJ{S~~X`rR?{zy&1Z{G(k zxoh`tU@W|8`GHp21Vn}=)LL8qEY54?Er9ern}DyZa`Ep$VYi|6%;WI(mowA{gt5QC z^q1SwUKLbRHcWbgleHbq6EquMs<2(^ad~v(^?g+%Rc;5yfPO4k*5eI5G!nA5z$hL| z4H#5f!5Z6n|8Vo0<3z-kh2@2uKzQ+U7hK}!!nY8X5#NWHWiU5Er$p0 zrbP0s(E{uIcwi}?vdMeq&~ZV~xdexVKj>(g1*7yO89l?jc z{PdT8G74`)9tmTY2A2ig7}0>)rL~{6-2RKwNTP!=dUg>#DpydBZF^-1fifD2lmMRr z`159$C--mwq{l8|nen?##xv|qq0+$KVR#zHCFAIY-!4UODm|MoTYHZlo%J)V3*>{m zTGccpzE*)eOo?bHFEE>2yGZsi+(1j0zfZ9>exlZJ%}cT8YtT2|QNbL45Q{!7ItCuD z%ZEPn`&>5OAcFJn>S^O>GmwS|+FIEqxm4jZNfHvZ?0d>40+I-5Xij&U$rr)?X_3W3PDHKQ4q4Yz@@eHk46ARON7oL-nFIly zT@2*iZ4`)^f^|@i%4W_--Y|*Upk>_g_&eDmSigQvGs%n2)pmJDkOD~Ax_n|b;YH_j z9CpCWb+puF`}Fz44K$Q)xl_)q+CFhkBQmT>K6Gn|U5z2o3mM^j_Zw~=B2|_KcR3;a zhg{$_aOwJ`ak;(#>ZNai{o+@`(p0Ft79yuoz*zi~nwkBv}=EK#fPvH;FO#E}PJDx%2FnvOz^E5XFrFA}tOy{20-1hA* zCfxv;9|6n>pj|YN+0g%xW>k)x2?mUfVA5Bz-w9XH!R7Eb;fo6&W3Jbj8t_xGfgP{( zDcdE52n6)_Y`-M+qEnTW=M>s1^sbk3ykaXt(}mq2*0bQr#*ckM_u3AZPV&Nh!~5+IbaZK zWZgSek_O&9h*(udkdOo1&7(%asb;k1_LR5CXsMxc@5Y8`s;A({vANCY+wDudL6(C* zr1aZ{-qD4i=eK2TN#{H~J?M>)MTq<7tWj^HZbKXxk4Wkd&zqXe3epxAZ2kFtXQ%-v z{w$6g3jM~v8Y}xvIc;2+_h73G5!kn?f>>wOexO0J>T~lT#VVCD`pcJIY}OtBYQb~C957+)X5C1!s>M1 zeRek&J87>-z3dF#f`F%R;qYEzgsme%0{OpHLq3lwQSc;w^A2*|y18G&toF$m(GJ&KsADqxlU&%nqkh(&pDm zO$YNg!ji(TllD~0RiVT4ZLU23hQa~}B+eF+wB-Q5nV;9LPC3*+)DN zOsap{%W*VbjzmhgpZ{4dzuu7CVZYVCAg|bdP;jj&ruDlGnLEkM1)zVWV!4OJS@Q>}%E=g_^87Nk;|+$NPuBMP&Ovwy_%# z(}Z%-)rof(nFpc$7yFgoArD_=S&HwLSpt$QezYk65O1+4mmc-B9L$R8(#;$kIosA; z(wd?E=k;C4irF{VI4VT846dBLaTBY1y4ztVIHLD+8b*A{r))-P%Kg60Hak%PMoidc z1ql~g3ASqE81H7Z1Km|QRYXpDmtOf%Uo6aum)MvLi|74h2B-XgKXZUz`}I;rMnxN{ zm6WmmWP^@wLBpY`yn|j3no*>5!o*=0Es*|Ejbt~_2HUGfba(iOhW_5*B|;3Gj4>aL znnz}@xTtW0%KU2jHW&fcUmd_W+vuF5A?PuHi7%adkcu1u~TslR%VbFKQlvRTm$>ys)i`XRx-w<8AJ$3BIpfvxQ z8R9aZ&rtF$_Jd`04H%BDOm7D?U=-c95{oj!6ME(NplMrSWmb?EL2Xi3BI}?IzI|ip z&WJS}>dIFC-M;?j9$Oi!Ti@K0z5R&RSXM15QR4uG4>ZkUvYtuOj-59)=VYo6>8{pY4- zwi0D(psDsx9>0!;hREY@zx6a4pt{T@d@~D6co3A+D zE3GfSd|+%zGP(4Y_qwo^w@POj^evDyig-M$lcW-3&m-Q`Y{hfzkEHuOq`_!8jk1Np1Le#efyGY0ceNir@ zZi@!^9~_5yWR8?9m*sWMQ@-Tl^xx^bTer1X$050eIIG|mNM$42K22oHB_PEX?qfq+ z`oVHo(x7iadSp{-1PQc)Rs4vp^T^j=IC<-5Wss6%-n;(SysDpkW~(&?tMf*JI<&l& zq~7>W7l~{Ea&m7tIu*OLDi{w_omwL11Y7@3P<2%!KK1yz|enVGCv(qhCWhJW` z`y1a|y0e}!@+Vfpy1Ug?Sh*r-xISR9ufnVMbrAtPDW2fGYqnXt;f42?1n%w0{Qjk3 z`rhtkwa`Qmqg?}KS2>OQ`5~?;t(t-jRhP{*;*6M(Ew;iK-P3q*^!hcnW$)R;|Mgr=maKtDWh^$MP)6tpc8=c#xpoWsx`A%Jndi(@oFS zdm}HQJ+u9Yv)hD1&`a14?Fd_7zok%H zba$+rtM;t$fQ96=uiZAHg{h5974s;;YzR}<+H4@zz$a3})#y%apf#9&Pc(hv9m*g+ z6tKjm1SKhZ8s+&MX;yniY_ea+gX}*{hp7!{-Jcu2)?PrZ5Xh1q$nhxejQkx9e3T#I z1GyU8e6`N!dJ(O>ev~&R?M0P@Mk;`bVb=XH|K+;>{)+-4kme5D^vN{Ux4L|;k!0QY z1FHNEh&ss}vtEC+8O-bt&L{8sRrcZ^>Z2Y@dMGPL$qzCfV^yE4g+0 zJgW(+3V2BekhJR+bAaXB60@0ux2rQIVB1WOsuX+R;TCVMJ2|*qxK;DNf+qD`#dP{_~?-5i1QKCWIGx4 zd#tplgTt)vTx%1TqP6LsB`|^1&v2lfr`ZLsH>I)n2W0?4Rvn# zoR{GqUa~CyV*4@^e&*DL?3FX#c%>+k=eVc}^Hi&b7b=ce8p2}U{@ zUzX(&T`V^#Q zzT~O76`9@@pZZx<{H6a>6~(&m813hUzs$^?kA)$R2!byKGcH?;t;N4?ce|MCa5~x| zq#NF#4vPlxvUCM(4V8goVUpZmEtRuCHBI~)du8K!x9N39;p=}szKNe0+jwIEzA_v? zpeQ1+l}Rw0A94DVKF?jwhkyBkOGlTn3K!h^#*?=#IdvO;NfNf?yW7Z5&XcPEoWis! zkEoeONcMFWx&(sols)2ss)xV}uuxeVa+0GyBd!7v7cF=lyidI#vXe2@Z%fbw&TpxIo?V9Y zu}^Yu?Ly@hIj_>ES6LO>LC_-@D#xagX@B3)0*LC|c zO@HmG%-N3jz460`en5K=60UUK^&t)Ytux^EB#YVm3e_s9azW#Wk3d|?s^^t#p-snH z-vuP&zLr~7|Bcy~nG<<8putoWN36LAMxK9;~5)PW4m9sous2U%aUKqQ$v6 zeuUi9>K%JHHD?R9whvCsvoHB<(06FNv*oT)ts(8|&5oj$Ry&r#mekU_Yz?oB=m{)q zoRbb%z#fkkvdOBXPuc9!2y8O^##H{I0EgfK4j@5O*lOl&KJh1UVlz1BJ0&+0YY^!a z=M=ujXGF;$ElHbt-l?o@vG-Y=<`Vq?F>VXU%NgN^t}Q6X-O8}Rd?hSoUwF-kWu726 z{aU%GeG+^~SN?<}>Tla$h1hAzJ;k=2WLKeXQ11nOe0DaYwfF+0bi_+OBHjeD6>;iQ zoSZ`b$U}Q(Ad|p&u;b_yb!9?9Nrc?Nz*2a$J9D)L-;88*u~ z-^9A;A7WBoK;8TC@nDOiw7;x1#eD^aS$(4hS?ejDK*ktf_LUNK+;DGrwfS??KMMY| zi~O&Up&D=g_hta6AGuqR9XE{k&5Fd&uZ8OD{U>O^bcBiJ8GPgv7HuOeEzLv*pjkg~z2b;|t1W*X=DZ$a%E<>zHo%oWhf`k4@(@QvInra~4WdSPtGepB)vyT(qRlfc`ALV6lL#D7VN;2DH zq=wbnN_9+g?Qc>tsP9ZLEr>nC_;Za%ig2MHMlC$=vHJxS^xxF8<@G1Gjs3*l)M)GZ zSH)`J$-doCAn5bIZ;zV4Vkq5xh`#4&Zr2)M5{a?Q$e?FX?ywPMo@W}Tib77&<$jnY z-fHza>tQ^|zwni{+y}X!ZbMd{6&)WNnoX()?k!;ZqWK9cqFo8lBiCMwF-p4`tKe+( zcxivF%1HmI5%6KHQHK>_Vov4LJ@IuV44ziacABR|)miP=^8P&^Y+D6doY?qAZT$xO zb1W-<7@g`ScXTB0cVYpotMS+Bm%tm$*B-`Z!dsWww=_L`kOpF&{l(z3?^)I>mdx+! zp2x$VFx^{5De3x0hoo?f6T;jwt;;@$h&IPcDqMh@?u}UNRF-UxYVw!Vp3;l6>0Xm74>7uhwwK z#+EjyQ({&9H(RGd90K~28XFgZL!vavixl!}3180}j27qykXpj&t{$;)-4jGf(6qb8 zgUk`SB;j8!z4Gd08qjyE97*|B4^nN&5^NC4)9lNahrHO56)?~fV>8OCS5HpUeAUJz z1t#;NZN9({?aJP#KGjhnezL}eW|7nIzBT~1z@dt@r9OA{PtSZEpSta%m0K~zd3&*V z5>`~=7j}`_r)Od+TET9#^*z9WD~$~cWRg4QO?AJO6y->Zpd!wyRsPOQql&(BdYxoyZO{4*$=f_m5RzkNzkGbdaKqBw9cRYOOX*3Enc-Gp$95*S4>YXk8DMUcVQ%x%@~ov@y!nvL zP1^VU5S?>fqXu^$a33r_lxMWaBo_1%&c?jK@Ebs<-*a85^{2llz*ypSj5|!kG9czu^kG2a72PE)nKh0cw(0y8acr?emvhLbk2X0%9R|V z-8yqPeZ*OVHzR6QnNWwoKWCaA9y0D(++}?~gP@c^i*xqQ3=>i7Hutg$o zF2zaKDLCgA=$Rbw_wo1pD*BGR+$38e=%FVj@$n~V?7ew`E2!CiAPQ;d|5;6|R)`rc z>LJ!bu5cw<5z@|F2=EbeFn|pQN6@tZRdS;>L5&M4C}d~!(?Ql6m1B!#n|6V}u_Q)U znN=mPRgH)$HQs{~%4>2pY9Gqrz?Nh68t9EKmir6e6R#0=`{#24K^p|3tk$wwLCLF-puJ znp~-unlz4A*Yi-yJ!Jj7Nfo3d4F7Lll-2vU{T@4fY-s?1mNGH)GXG$FgUg^=P@xxI zdabC@k9jhzcG0R?uC7OX9e-fGfmY7vEi~pd+4#TT-Tz8zTsoS$rNN{bGIa&+VQth? zj?0S~e}2*2tHS+di|2xA@$@oBEvk5dxzxL06MVh9r!zvHI)K)u(HrQ8JJ-Nd`wZFHs# zAfjM#c+>3W18!mL6$)(vJPV@lZ(U;#(mVqvv&=ms<_=ys3 zwyf47?|C`o=I1OO>Y; zLoA7ovK{bVmw9e$PU(X;BB-v~lcclcGr!wjY-V5^Fyg$_so_?Y1HuA*RM}q0PB9x& z!HqJ4;trq&Ty>O19f@-{giNjgd}_%a411cjKzg4(kBq)DMkgxQb<)`dRuWs(hgV7k zZ9CwlTd5uAf3LIaI=JsSX^=b|{5;RVmm(wVZhwt$k41A!of3*|6|*Jt`2AkM+#P^b zF#ns`BCCI`Tfo_)`-Vv)nt2&%hb1@C;BUJ`b~$HXO;iMgEOa6GcgYz|bN@BFdD4?E zmp=|m3s{)132!z%9O+IDjh&agWdfu!Sa+=>5(y@A0>s|Y-bNG)kK#?GL~GMB`ia?d z=om%%WgqCYXRaAXpVDa0L>_T!KFvmcC|tTdL?o9M0o3ViVSQ)Q42V47P}~--gr39TV?q zEx_)M&Hb_S%|q5`agTv#6Zw9^+xHk5y1<6drA8cQj{1%}3j=TrKH5MPOX}!;)F~VO zy&aqm=b>QA)%1vZqQ&87jhc|DyXpSysgU9Dma{Lb3GLJ@B-Rf>J!(4pm+|V?v-+aW zI|gP5g4TPI|FKo<)PcBMcsPDQCB~z?-m&i@ z()rST7b5?+C5`o`ucf^s34irUQa7QS_iY@K!EP+47qF@aUfho6pf0;y!M(3Ioxvu!)GZmewRc@ej zoNZknDyQ4J&&$ASeE&sSWaAlseb{8ZC9`D3>tl1O<$x}dndj#Pp08wXEfZrXf~v;7 z?=BLpZTS59*5}vOr#O0GU_MHN!%wi?1bF~Ddu{5ksej|NMc96KBfiXx7XONeY zc<9k?j48uAk1Fld@<7kU8LQ6)cQq<0Phv#cteDPP1^U~+J!xOm_5qkEQBzGgaWm4B zT$VAc8FJg{Acs#4g|{F9yFyWLp&~P>6`vv5{H<$C$Gme;`P|BC zky%AtovQt=C<}k*UTII8PoVnj7M*>Y`WtgQf3&#&?K_NJZ1`BpuHL~6kQHn#GFvU5 zeDMM){8d`DbAxEFifHNubETTY=5i8rlmC$>2-Jd7yXx^;N-JseXTSy9*AW4tu`Lst z?)g^5X2tD~*hN&qwK5w8Q;^*}N`n2DH;D0yDNTo1b?Jo3(u=aRKck1;+J-T+3ik`g zDhL@y=PJe<;|HZH-+t&2`3%Q`CVLg9#-=W1?S2LY4~&2cYXzqE_N#l5cUDeD-|?TL zII#=;SsH2;qJalh`2EbtqjgI{cPH@~#(FB;btc$8HwSWOIGH+%c^ z71H0a1rA!&l!GpN|I*M5S?Eb!50O`M&boN90#mO!uNP66C*7*NBT=(lFYE>`Wm*Dv zRd+F0dl7@Xa`yPVjTLGQG%gyXaky+&;2;X&_3$jZ+qmfDc;|*Rq6P8ZhS}kh{rPcD zCM7OUxt!J3j<#$Tna=YH54ZiQ4NTbTi^>@ z&JJezh(7Y4m&{ROM~ww*={nG^H$b8Ty+lb=PyuG_c$~B2KA4OvxV@^% ziR*^LTt;2&=T+IOqRK|5o?XG7fD+k=pUhM;K+b00^OBH`%cf^j`PkYjB>An;NN#++ z^->t_$bym7-5k2OXq_kvq zts&Npfe>!j#*MtJJmcfgn8Ub_S!t0x@kIk&*(da8qS{``7(zulRTYxh#{xymbO_Kx zb=log7Q{)6&O)N(ca+rgW@(|fe5$sooWjg?2iJQ)Z9@V<;IA*iR?{|8_7X+cMmR?H z^6a)J?CwHfTY`wbJr)g>S9ezRovhN8<5U-2U1r3ZZEZBN#wS=w+TwHP&=Z;WM;-|MiKeYk#C zz77WN<&2lV4X=hIAV=1W1hc`k3}8`&eT(UpAX2_VG80P{LnkdOeaz=GbM=2ER9k1=Bt7D}MjoO+yGU zPxLqyn2`3dXCdp9$D7N%JdT)%yF)WdK?YYt0LkHPm}y9FCv-FW^M51(Ezy`1@FuU6 z?7(Q1%z;RIE6LzEv-siB)U5g08YHBHkTay9EP@4a??xl9jf?VW@2o~KuJS0qlOv}C zF+W?D%yb|}V=I6k?vg9#=g%}bE@FE7bQt`s-?y;3w3M8gFB#zIFj(Yi=)@UcJv9`% zHPy2z;N9Fj%&*?snG5jmtscWUAaTn7%285bVAk2di~jfo zSva`IPHxJ-+vzpBZ5cQ_!|FwEAt9&-WtyU8U`J66O-8edGQ=i~P-y?J8xoAhuL1Xjo>CMH=%z>g~O)5?OT+othY$H9zngQQThg??0#Y&q;YY&&yuV z9qQgTcvuJHOWP6cHg0vhB3CfnSJ`+F49d@XJlQT+0Bd!|a{evAh4Gmcpaw04Lj;E0 zuefI;Vb?^Iu=2eZ9thm>@g*GH1c-RX*&fJL5?zyans$Ovu}SyqU&9hSqw9B`#LP6rEAD>XV|0qID1S<7 zPCKXf(dvv>SEdw_S37}^_mR6;;p}i;!92%E!^$?Z&_2S`_*YPee7J_)KPTHP+xxZ> znZ#9T=Bgy(9E`P`9o!2Q0l(Rx>QvJePmP4Tc|HewN2yZG6e$i5*xW_p0U`BV1l5$r z%Qxig?(|z^cV-oeB1%OZ%p)zbLymB2TZhvkK9Rn`MQz__O~ZvUF^$N~asd#!iYF@C+T)Sr!`p0DI@6V~-gJhf;OA-jdin>Gt?`Xp??=@ltW&7O*Jp@&V_6NFF($pvI7s9!pO&R%Sh+d{-+sFGB+7ymLBBqaj$PoyAFFCc-1L9<5EoKY z#W(1@Cw^p)@h2oxh7PhIh)Z;hzxES4%j@EZ&k`+W*uPklt62+ZdjnE&6GD+hq{Wmv zP{u-Wr&0)5_5AJFa8P60#)!5jP?;BYLM*}4px8u;J=eR+G-!VEN{dhY;S#S;kWx^H zM;jE%c(35BVa1dBfP`mOfEff1vFnR*)>-G5jL$M32v+aFLl$Lb0vq3#zTDE*(tPRc zb9~}-L+B*0MZA%6Qy9H7Cj0*Sy)d6*(P3G6T7d&Ga_6GKzqa>nwb9=*K26Ra=rgiL zG27a60?TH45453?vHmR|mT)OQMEWH)!q+0w=mQ)UZNKlR5z#ewek+G&4UQzo27QTD zuKBvy1Kx2%2#~v%kvYpxKeS1usXnL)H1fx1cy;lpg#!0Ex}ZS@$|@0Wg2v{{IslA_ zIup3O1}wO%8_RmpLFUSU&P`@ykxEA`mE8V4RJx_rcX)N1L-HYITJUkEP2Tr_vR)jk z7ykBiO8m&!9V?f(+|KuzmB0;~4GEHK>$mN32m42rE&#JfT@gCWCG6UfAOZrCQvFhTf3Y3=O>jS@U-(jM{*Hz z*}m`2Ezq z8hy7$cW7=94)5C|ls~^>zZYtFdwU`uJX%#UIaOi0l^iMzZC_AvAn>sz1=cG@)CJ?j zBsSOD^|;=3mBCO7lyIy}_8rak~TD5?n zi+GuM-ZwK7R#SfXVVmyu{)hL`&QtRyi z?&TrPuw;kz)1_~FA?V2A;#_;nl~MH5=30nLCAp0(zfmE?4ULzn650P5)A=6m&pu_?oqcU)=-1xO*5Sq zFw_<8?WMatMxj#**2^WxCx8n=TO(RYjdL6G(j1!2%ny#W*p7k4L53H6H|Tt)$vm~D zQ9y9k^yHjZ$z05NTZqI!`NC7yUfhJr&&r%!pnMTyt$@!{C9p6rlPHAOqcExg{eaGc z{~slVKjiAvGMqXt5d1#@@(m60W1aLfe{`v9(axVR!pPL`qq>JjJ7t9KGL8Lh7+fBt z70d-cFJ4EkiN@a>Z6N*P#i~~}Zuo4tkcMB?gbWSqmx~Hd!yniX<}~hON>nCm$1jm+ zC&;`o6vOy2ABD27@t^$5-?(SQqcIKUkC%=24Kv^UM5Fod3zB5wsXFoJ3nTwZlj-#f zRP|OonjYZLImh5y9Pg{>AG3LVr)OOCdY(IHZ+2Yd&YSW*>KWrof$iN(Rdv3kXy^Ne zsm@k#j4;%{xEmxs9o3=f4O6aK?m_24747z|!6~iQd;f5eAN^|4hU0O4ZG_+RT8Oz} z8o${@8vZMOo-;*5^HIJS+Di@;>%ff9!mJxMIN~>kUh%2}*ZGv+FP?M#7bCb}gNOz# z#}%#7%gcIiiV8npxK-+&=_OPzthP)T|RI$gy;T^Vj0ldAGp9^%Q61yrkJNX;tH;&}8|8g&T?tdtmf4NnowM$}D{ycwqQn`Kyv-UDJ`OX10;=?Cn_7HP; z{mtN2&bm+{R9 zTgRt$3%Fp*Dd&2>Z}mRC*AKb(9%>H#Pep5TVNRIO%Iv-V*-!Dbio8#npKPe^^5(ze z|L6-Nu<-XGzyE~M{lduiFRlg3zMkp3j`7vleg6f|nBV;1>g0HizZ2-|k4?RvpUKmW zQ5~S(Luv1E*ggR;d?@fJ-(5e)4+fxyY5t{%n7n@aPX_gvv(`n5e^HZL8C&mp{U{$y zaoBI@IPAWwHTPl)I7PnwJ+7Sx?afavWqb2A=vTLsKfOcDzF6y@YiIThX!7s6;6Ca zAKWvddAUxE>-XoMd;OgGnfZOr9_z|lm6J95{rM*weeYlIpYAX2CzecGb)94ExsR9O zq6YoN&~~}jfqDI~FzfyegZPaZes$n_AD91M&F7zbk2kbom@#3VV`$ZzGO=4t$f zX1Kry^Wlbx+KFH718E5jv$(2(V8RKT|lt28@s%m_URh{AE z%#(O@4i*PWUB1^8vCu0KnjRn@Itz!-?rkhh{FBbXK`L-Ra0@&8#~AJ&@z(l@leuJ4 zVyqKC1C-xUI=&j%)De%3VxHamdR7BI3)1UZdawO=;t#=nyl4O_@~AY`|{JL zfJ>}tV>?Aq8Ek_8Qljez-tt!d4_!BMa5qPEz%fquAGZF&sBcU3y?;6%*iuICbWL&$ zhGm~q5{m}$s4eK!nsCHN{!82W2Yxh$OUOQ&kKcR;Ql#fS6+1Y17WRMQmm#ya;y1dH ziC;wOUEq8DJmRA>@pruFlmNAMYqhP7^Y$O&0P%Ck`)<85Rzx%3#i#|!wXP>nI>i4}fsxpX+4|()I=v3zzjy z-PN4?{Q>UT|Fm;7L0#+i6`%~>KbueJX*ibgC)ZDgTE8lFe6xlp`f)( zXMZhw18F_+%Ya_u>zukVSHE$Mk1rSS2$TXgF-V*|aqLnKOy>);YA44s%E9mWp+@)Z zFO}$o)T5;NKl%?p2DAJ2!hysc&1tiNDhj?*2Y{7gFJF^IE1BN_wa)c35rhs=BZlUq5& zC*11L?A5P2dEE6s%My|HH@*!W@1%cXT>0CIv;!SIt6s;}cL%5)j51}U>HVNJ%TW8^ z;%kluC+*8Vp1>E^$i7(~)eH=eI(n@2nSXD?_HGQ}9vaoze)GVVLomcDy)NDMy!kKn zFZVx#wQOHMtT}lz-(EjvrM9&(shiieAM{Hg8q6~m`3l z2S*(6r#w27>o_~^3s63^xqlji`FO`SE|`m}*wu+*-;mPbA-nfy;!2uaTu;rN+fy8$ zd6{+Qb~JW%_FoyZmQ|~5t%f}8`e>O0Qa__^z77 zEBVYf&kr=``Vot5A3)yM54Qe|iYP=6Qs;By2FK|`X=XVj4d%F(iA?*WK_ zO__$V40`6|Nr9WpSn5UwIj1< z&kTFDUw)wc5!5}nz5wUpFmm90+`RvHok-S>*h5elkK<1o(?-9V$|ivlz#OqZHKeX#vytoXcH z2B)5+@~3FG`}U9hOEo;<#y|OgAzVXK>rBJ#$-G5O#l}B-TjR<1Ll!SAzfweXE*}`< z5O)x7g92lx>(QJBxV?dWx@p3S&s9~|z)K&KdbAU-3>~*uptYX3VBb0zt6(-{>%?64 zu};;~J0cC4^WDK1L|7|>ch`o0dIKK#eM zy+J4M>7Ja_#J&%t@p=bh&P);x$q z-*oqS2D$V)r@@X4M4CQVSf=3WWRB=ln$qV_V{6psbF*5d^WfYCp9hxqw@=jJyN`lu z%sChM;Ks;O)xY3EN2dDh0YMo!;IKQj2%r(@zf&&L1hdd+zAM}!E>jSa0?e{+co z_cm19{pR)OpDfsGONyo`U7eF3F~@05_uwRMr?w5$CT?_@7vA46qj|3JG9<>}9FO_q zi+}Xq*JZh%en-JPPPr{DcdXYkI%6<>{q8H}32KgDz&EGQuD^KGdF$@nTB9SWz5T>_ z_pwo~?v4L)q^p4Vt%r?uQUL<~9A$i9r}NhNw4cz~B~*zz%82PkztL!UnV3XlA&(G% z`r-Kz=3GDHm6lrN_TA?XeE(#Z^Dj{yYcK2|2Qi(89ykZoYODRafmzqw?Toi)AeXh z1H3;0`~CUnxtB9fulFyXw|jas#Tt5Y&-=NG|Jctz|2JWloZ5NfgERmD002ovPDHLk FV1ifi$>jh5 literal 0 HcmV?d00001 diff --git a/docs/design/imgs/tidb-log-dependency.png b/docs/design/imgs/tidb-log-dependency.png new file mode 100644 index 0000000000000000000000000000000000000000..04227db9c86831a7cff1dbfc80131815b5b93a0e GIT binary patch literal 360548 zcmZU)1z20p(l`vI#S3kL;_grgQe0As6?d256nB@DQlODfD$m<-vZ1o;89ij6- zijSO7P(^Zl;AX86rF?gfxt_fF@%IaC?4HLACJoKG7+tQIRr1jv;vQAmf0*5Uc0u(_ z0eZt4*4paE-P{_}B@~HE$-a*Dni)j%vX2mdiA|{r>Ig<$+dz$|K9{j`B z!eXVn?WAt!{(i;l{{Ft{Q&jB2ac#KJf z@#F!*e8|$6)c*@BU~*t!{j2>E21b+v#^e9ss6Ukdq;C)TADVxcSc#DsI1jZK5840M zqyNEvZ1W52f8a+H4}BOiI(84*Lyeb5m9-{hXy?NcSc?q7%y4>$(Tyo zEGG~AXB>15y$#h=#jV|3cr9(*tZaGxUEKfSgCXfJ{s6kzdRsF1yEwagiTg`2{R>0< z0sf~Mz{Kz`6mKUfCPOt%23a>xTLxiXK3+a1X?z9-21!pFJ8^9}h5wL0)TEf+dwaW! z0|0)0e!PByyl$TM0DdtsF#w+cKtO=!0fWaYz}4H*pU2gU`QJ?bhmV}Cm$j#ZySIaz zE5kp0Ev?*qyrr0!{u$_h)4%_xt-r(njpXX}pT~N5K)^pe0DfLR!2j}oAeH>5Rb11- z-`3ek&cWqj&K|~)77`Se{1^WJujl_p{4Yww|D)vR7kK^Or2p0R|0UJ)vh|d8b9ork zTl)Xj>p#T*-T5CvNx(mk{$Eq^?|J@L>%&A#<4XemchRKrwNm+`AJ&n^K~6*Wp?r|C ze-b9)!w<*5%Ln+;LMN+eavBE48w@2m8C`$OgO8`b_lC2cZNI#;P<=*;IXS!a7xYn1zc z3hR+{%Kq-MUaO1`ks)!iwvQbg0XKpHN3t&@cFMJd@qL zx(G(mJU7B>YX?nuPYk92u$p{JuvZgA;xmDa#ZBzgR!g_6RVm?POD#lY$_sgBDzCL$ z{#NKgJAY*%xt`F>vV<|$sf8fsDLx&|f$cR21Yz`ZGkQy1JX>qg+VwzJ+o^Z57tNU( z9s=)Cf?TVPx2msv8u4|FjC9>@VK+F}TXFF%LByZKX6O$*Ndb%Rc1cn{T^J0- ze8TTPTA~Hm-LDuu+Tu`F_rE_EOls&T%>As$x(2OMU4W%=kgdx01-zmOe4M}w8E+%t z+AO$-BC6Du@{^9Mi##co7sw^Su6z)_dODXSXXrtH~BV|)27&=JC7DZpP z!i0{Ur&i(cEOV^G-&}A#d;N|Yn-kjI8uz90b%Rf@G;laGcQI+@KQ z%FR06ov4l;r)5KOyhe%T|X8AP#@M!{u6bxB|tCA(PjK2 z+qai^5q|wVT+}P%!>C8QXBpw zfzcDz-J0i4hOzibDl0&CBBmLg;?{jFdUW&jF7;FJm9_$j9v6EC-`)soob^oMe$I$( z{iW>eV^u6(CHgcLe<_hg?`@2tSqp0Q4@N3!OVI}$;T(BBBxK>LScaBy}rd_BjN&;eEeI8>^AFi4G6ri ze-%o!&Eh?r-v&3;1eYG(9|P}EQW8bAdBR4~+8Lo0C}VoUsnAW#eV(N!GXyuTPPDa+ z#Ii4YPsvn14~8)&qRv(&PH@Cb>#BCUpU}HWiJlDkS<4~PzrS|qNVsGtoaG5PD)x2n z%Ocl0lg*^atCUrAI&CX`vy+fAA8Tn-Dz)%Y{A4`_iaKZ_9b#k|`k8O{k1rhE~q_6A{ z1(77A!Zd3SN@n7J!Cwe)&{&>TTth|Y1avVC72zNyZ{J=3ooFNli68k-_|Z*wx1qWH zL{?+g`u8~F4gJwqv1($&4n5g;tB^p^VxYe}&+%0Zz&QVB4L^t^nXh)+X>K2%n4?GJ z*v0-J3VTso_;|Li^md&sF87xH(MRqSY9DLW;8<%3x|84i&4OSl`CND-fnNb)Rr61) z@~gP@>CmdGt3y%^lx+uuV!2WXgY()p)ALVl!0SlakeUU$!v<)N^!6F-I?N#0G}4kH zOZ%7C$;huK`C%a1(#Y&b(aK2fC*><>mly$|rF6mfy6hg$6t4+N2r^hgt}Wq1%IsS` zhUCbd&ZOWkSVW*)t3t*x=CRxBDP>rm@KXv@yW=Zr-`(Cw?O%>Lu#?3F_thoY=5D~k z$?kP_yU$3kIh&TyQdf;ie1(c@OY@${%7zJQ?Ev}egjwO~Dr4%j*YuVpur*C~_w$A# zRVa1VRX_P{mz(xEi8YC#5x(~$KeIY~&~E*xl69ScKkyO(f7my4^r!X)#l120G!}HB z(|4a&um4O%b?%S88ZTnhh#8;F&Z+TVUB5Z$jVoHBP}n;-zd@ja7c@4l3aI#^zPTS! z2>En*N#`{o>JiR$u#b1*at6V;3(~UUhd|78)LHuPFnw*_jcKC5`GNduW-fEEX`R4O zKu@Lj9vX#8dZccI7L?FGlk@A+G}KrS%#b8?lXZjpsh`G!HE8!$`%qk`XF;TrL@`E~A8>-UnfrP^ znv|8Bd5(5!J_XX{c8W!YCqyF~S4M4N@GwU6Q}=#1P-PY(hkcx03>XrbPiH`P8)lPv z<}=b#Ry9EbMY*`K!mY>VJDU9q2h`Dz(ur3sJ6A?x^+18Vv1=bd;@ztR&-oArKDmXy z2o-pr8}?e#4+~cbx<-mBg{~}AbchZg&Vg~xtVl6M&^zNkf+q+3fQr!Emry$?@VMbV z<#4}wy6Ll1c1xDGi$B4fs|j87GzV3O!sxueU<58RH9g3gBp*@J;);HYt*!J(c^)rE zvUm-;@y^BCb)gepN19(Ue|^0H8Z7d8#jEcHlYJEgb{djwqh?MQpX!2o{fb}u)yUry zN7FQ8U{lA6`}btkPi5n=qqJY%dAi#ukJU(zWCy`N$5t{{GM>)8 zc;#tm+`dojTmOxObi4+wy>{F`NkKQ~3)=WQJ{=FqtM|Wq9uKQ>GQ4;sfnyn8fU`c# zJ9=W|HaFSYOeddFzbC?ZC7fg4fRn)SYh>{xE0~%|3Zh#Ju`-8M1TDYK9hNR*L?=3S zx$kjUrv|$?Wyo`{rVgDNpdzv;a#W)L10Rk=-!@NC#~h4iCr!@vlMB?O8ecXV!jA95 z)HAY}GJH7w*55^El{8NDS(DO>pDjHFF5hQT^skbjLuqGW8i=4}_VI14n2;w^2;T6E zaTp9(l(KhjGT*{`b$sI{Z4Na8uX)T4oPpAhTlOSjRanBeA$uj_!$|!$7QsqSp$65-DFc04iPouOHYy(% z=kqt&59^#BvX8P*DvW8xH6j^j&xHw!MRZn)pa;Na{so@OOawWx%gvK>tbU(D(_p@& zsEUD+(*+NdbCQ&x-CfY^&WkN~IMYUGr*~VsWAnyrCf}4=RJNzV?cY$ERUHI^K}+pB zl+y9|SVU#$C(_!?pV4E;a}tV~L+%yt6M94VMUbW_>tNi&;mu)%_!wA8`XY``R{Pk;aQ{&4%+ucTqj#e{ZvfrKb(O10m zIFBi0`q!8lfJG-LqkuQ3s2_Q=F_Tcd($9<(q#NHRO{Vfj6_E6FV&zwj@zQH3&d99jD|65d zSoGQ`ENi(gy3Q;F^{u0L38l@!ETw4n6(|Ao7(O+hF~jki2~{U* zAi<(Jjj!2fv8XAH-wvR^WeMG5Cu3f|kMk4jp^FVr&PLX?x1N%#-)6!iB|p{0V^6KF zhfkh3paU;-?;b3ZqT)hLHy3h5cO3PLtA*>iKB?9?O+w$;0Q{zad7Gx%+%Th>s(?` zU0mKpsD3j}_uSnv)&T!X4iY>UUZ9Cbzj`m$7vUs;X%;#2xDt{aLf&CIw9}y_TJ(ie ziTMYM@3v%0YEbRQwc_DN%*stRkfV=MLdy7Iif?J!K{EqxPLvkiJ6dRSE8VYcE)LGm zidaf!0?_7?6XQ@6x}EuetZwzZ-XgXg6W5<`OF8Y)q_W(8s)ZcZ z@~pf5u-&N`hm%e7{wPtC`|??+sm;b-KMeh%N4QO>7R-MheY@tK9uP zLs35#P8F405r?fmFS?#Q-WDa~HTx|Qd&aF%ig{}Aboj1E3mf70%qkp`>;Ou$+sGA( zbsVRU3BA8HJ{{=1EsGh5vm0R&rJ75t^jg_>0eay=RKyW~Cn-z>Ln^})(pwx0Q2h!m zBpG*Zs!JEkbtcq(A^|VIT!z|C0(Lg5ymYe`Pc+6)4GvLujpM6d8Br@E#HKx>t&YDF zP&{nr#0!CExO+H9%7;%BXjDZaX;^m&2FY=?iqW=*_VJDkbf5*LxGDc1MHI@)o0aO&mfry9gEg)+%?wU zNZO%8$HriM*;yiXagp;)uB}WJO?F_V~x{1}G zd0xUGZVyC6OokKsy8Km0Dc% zo#9jJ@3_{t$9d5VZfi(<5gEwR`*r|G%6cA^MrE@y^LEz6kC%IoO}ES(=ry_6JtK2a zcGGXt(Q;&SwKYPR9)5c0h<*Y{?TRPT&;46RPB3`Vx~C+E!f}LqNl~>M^4515NWEP%iI&)hAtJ^9mh4+(YKxj9iU3l=x%?K8 zmPXfAxovTO(Do}Q`UISz)dVIbLZ&UTg@MOoEi|82I~eGAy%VEcoFLg}uN9h)oX!L8 zE&_<_E`kGgxbh!V0lDbz+QUM89sL!V)RGPkMQc4R1S-FLhe^SwBI8ZEI z9<%j1jN9L8b|c?_c6OT$d-V~S2C>v@~% zW5RmV4AF2cuE%blKl$ij)adiYcutPUik~%*zX6~ef8(*?3?OH$esp#nk(2gv;aCmS zO;FIDuF?^z3+P&h_=Z^ZFS$?2(Gy0SHSC1FcDYo%ZV=1_1UQq4Zrbh;rz3Q_ z4~*B(EZZPC3beK3?*kI&mX^LM3xK)de)E=@yt5WF$-8O^O}scJPTnKPX=rI-RAy^{ zHU2_rKI?r&_+s_~66?hxZYgQ-v^Zpf*II0~9AJ|>jz*@

I?N>)ik+Te!7uK%tlR z#o%YQQj#6TVvrwHIgngHi*=!n%f1j>t4ndirSmRXley;6RY#Fa<}kg}0X1_A{0~v2 zuMNk<>#1DNF*oHyfQ=2jPJNQR`_D`_PC5T0Kt!YIY!r?O<&VM1%FY-yr){b1U_Du( zi&onbrXTcT$r^W?U%TAbjsZGyb>&3ut=hp!`UlOFu5U`S_&3LCf;F9WP068NMW=uD zaJFE6>fv~jLEFMuFnwlC*A?z{m8;1e(HOX7Qd|yKoOL`YI$j*?uW>j3 zZgPhQz0_F%gNS;O8lqKf>nnCn6syq7&<;)zS$S#}P!Cw1Eo7f5 zl7-R(560zFo`va(sg6v6kp?kkkVf9nFCa&N7{p4{Cv=+F)jwh9XF9$Y{F3&hfBZDsr#Wx8mvk1oI} zV{!j(p(rQ}WGo=@$kB~2?4<$q`V_X(9-Egh48hP?e>-$`$D@bQk#jj@=P6gmPAhJ`pf3c>tG7SgG{rX#ub#rD4?|Bw|pxG#BmL#(i| zGKzhd*sC8Vi`M+i^bUDGZQx89@F4a97xeiI+KN_#ok7KS;bjkUfB#@{Hagtj2>rs} zsTnutTmnwRvp{>tR6#EOvZvha`5#X-Cz$YZAD@Zt+3>v^4^;g5*ISBE#@d-$Hd)ZQ z_wJ2&<=1lTa2^99!=_jldeOOhQZ;nv#1lOu1n3#!smF$z=3V^hrhcsC=UzkMUNdcR ztD7g;o=VH`Fpd~14?%IE5A+R(WfN(Z`}+J^L4d0d`ZTo8?^wYbSPrIXG`z%=ggZk{ zMdnSLH~up0@B#m0v*#Si!6y<`L1!-DZ0HqaZg<@f%Ll%KdJitmqT5CvrWETf)4Z%O zCi0r`I_;YGJLdp7K9);twP@A+`;YP-XwE~iZ{-olc9WDlY5|SR`F8nS9Bp$_F7;s( zP~XlleXJzuAX)`~!_?UE54L*9!QMn-)GOQLn}Q3nhV}i_>7`#;KGZ{dA7sB9#wh<- z@9E=OOwW465+AW7hDY2#puuGpH|P1$XR*bur+uPzvMI7t7WAS%6L@n9Vbtq1gDOjp zcozBwkIxvQqs!I40d$K&^pDn^zW!OhO&o%tHp}x6INUyGo;F0_fEv$dY3lKtBi+r7 zdf~vFjto+(8CrXvl>=%W-}Z*bbt;^%7D5}|i;7~q`!9;j{JB{cOfpQEZk~g7b;362 znzzbD9lKfFGL8!g=IH3GBx>OaOJb8d7lYYDL848t8jSFgQ! z_hz-krdZ30!#CIZZV2|K<0M|PdG|$LG_Ty9KKTj=WbyJ?Qr?AYu--e-sVPED zeVczqzG9zaMGP3~s!R6i7{E9ST|W&Jf`};>F`lOraQ*=_WmZ}YF*aO8SMH1YU{MonOV$ML6e>3hR(C-$?{8`I)bO_Upg6zrI$~ z6GIO6MQHn7q&t==m?iAAu-V!Z>WlL3>VmC-8T4VrOF?%$=HD7LVlqCc3OLissT{Wa z3NAh+JY|U9Mdz#cvZO~5z8w4NXr$qn|0~4AaV8<%DRIQ)IHR$M+?4dH*Q96sFEskR z8#-jyS&99R>qm#|&s&sr2>LpY&PtwK8&C|XP0gs#MAbk6p41HW0RGmqFLp-LlGHlL%!N0BQGX z$5iS5c5}_;P}WW(j9Q%SnT}(o_lvdg3}=4(=U(Jf8SHM6a)Aro{P(_c3mG6vL-t|W z`n!);-{A&e%OsJ4qO4GocW%*>2SDwd;&7{E-O{!AZoA(2x@YXtu<)w^hB%%3KepGC zX`U^GDU!)*D7~8DtQ#GJ((?D;KbVLMzRG}4+_v2eSRbfXkrAd1b)ct6Y*5(WDA(ET zHZ)(WbCl>G3_p>qI@;r~hcDG|;U#ehm(SQ$9!fm7SO_afl!tvzpU)vR zvDd!X{#=q?4%&}YP}0%8ocIqyrczK3y&3zs#K#L?qID!*QvAtv3FDNT&ZaP~fr1s) zSF^DBALF0o{Vu8C#<=#j(_lWP$M#L6$7&UkzO4sSt3@mRSgz%O+nlgZPr-Zesgi5X zg;e~_9Iyc=hnJCaaELEZbZ$l-(kMX<_^5f=Xf>HL> zudyapbl#;`u~de(SjVpHCg6#7&~a&;^&b?ZeB(Uj>h{`>gecZ4#?uCk?l_C>3@Rer zg3-M^ATZnQ3#Vzzo-qs&lFErq>c|MatF6)7<8I{#fmZy|-vr&4B#ubH3+QCX-tLfO zN&ZPsLl-B$<;phz!eJM7xWhkjTG@0>yH}wC7uQNZ0=<%kz@aDJm_))Si1y*Yk85Z%1iTrM5F zB!S;_qPjupi*OE|2BeAJ-b)6({YRv~))1ZFSf_Zx#Iz10M1 z&B4na**Z92X?4Dh;BxgReVpe`%}X|~T=v!>7k}Fuh470X^(P<8-gFjS3BDtW50CX4 zOxh4mp2l8!o#!4JAj@jau$M=sTT(n;h*oAr8W<0IUE^*{(@wjflNFBHiVsC(sZ^W@ zf`xu@OSoLXZS60`W**t6+TX{5=7#nyDqe;@(J{DXrjJe{2lW+F_#U=+9Ota#z%Jd z(d{YZ1Nn6pD`8rHz9`bM)@3AI9An*5d`R?a4v5>l{@%Q*Rv1%Q;AM2}C$4AEoOy~D z|M>W7%d^+ujE`|E9O(H*?aIRppM!2hV>Q@AGl!@Hy>+cbv$?7;76xE!v|ko7j&?5B zV|M-%g@FuDb$_?(g=}yJf*EycyMt|gi3g#deedWSd>--SqboQXosIpJMgUM-YgP*- zwhu(>bkM_qCb53(FP)N62j}X;RCj4|b{>1Zq4bGleMJW5iC4SV)lS6eH&yq!I1Z{a zmS9Gbn11Mw(l4|#c$>)5)(1!V;e)d`j!2}!ZMYh@My6R~k*6xXT9RG-cZ2`P=|j*s zMa6qbCNV45+w|j4aQQt43?WD%CTv|AE;cZldeT>CVs(B~v*IQNJQD`Pd+D(&as25H zX-LtN5XPeh`d}HD zvJEe%79nKtZ~y_oh>OYT7s^p2h%Y3mUfa%iR8%W&Wd6CJ`_~iL%l?Ha(m z((E#W9(4+<_=tv}Q95AT-I=-S)Pu}V@kX3PS~WR!_{XE)Z&^>huFMHdxQ z!?A!-3ep;Kp6bImCHseO7Kn0VVR087OLI5YfBJ z?0)U(S_Zxy`kg$5qJ{SdL_C{_F5-m*?nw}E!*Gsrm#=xE_tYDExegtEYf}et_pM|P zEK3YhFMpZLV(ITCUfw(DF)?*q6?}0oS8*FB@uOrrZb{*~BaKm&wsO*atOfoRU zR+#vU%UlrA_icsDoScal5-$W#DS>-ht77zIvPD|-sEVX}7k2(RvY=k_-sWuPuuLGT z@N^JoKjvVCvRw^53Az{I@+wl88oTMV1!n-#$(&nPnLMX0)0ImpBu(XAGshM8(02+% z4E;0k1`ZnoLg|yeSP3q-yyRT?rR?pekO{&*4E8$56`l=co zv*nz*owUp^o}DihV4$(M$=$pltA*9WGG1}qO};3B0gk=QGz`=czQ;jYyLNmwaq15& zmM*JgGX@shdE!Qz+R^&*s<*iwC143`3n~|f0QasGmUtItBVUYHtvq^VmKA@!+VPRB z?NurWNgNU=ctrGE>Q$DgU$V9%y8UnGx@2k(^b1kN<*_vI76}S=Pn=P^&hfo&e=hbV zWgOm#X>`d~4GdoHmTA)iJ^%G`zd<|_V@h@91u=?EjM-}e4ok)S%7VpYks7SR2=+{X z+OefqiVkSHY=2)%Cs>&0g-FhHkHONkxwiiloSntzu>3&PKv;5kL;U{S1yYwZY|Wsb ze1CfX(@m!h@LJ5J*$?8IrHDyY?0 z63~yioS{2!t-OEwzOeAnin;jt;Jg^THsKp0s5zWX7B!)py0WUc(|rURq1N7}iMD3-4*m%j}OyrWzuIajBSkYkZY@ zzvZ?)0%{H};uEZsJ?TuulV-4@4?-Tkyx1kkaofGi*4P=1+Ykz#?Y!Ldk=!c#Qdm?w zd6}R#wCI<)^LbcwQEE?DgGh#8V-`JS9#O{zph{BQlDv*MHV0ZsO^D_S#2bMMSDmc4K~J|vlQioN zHkez$R+lYU$K*r}YT;0_A$G#>Q~_Z#%1k(p6g`KJw4SvYKBM{6H1w^x_S2X?cJr1~ zK?80Dx4d-HpI}p!S5sUcW|WYXRx71fp5a7TFp`;S>-W4v?Dy}yuD<+E-NSqe`8^Wq zuys&E#L_h1g;;eTGM)$3I57>!HfA^m9KE&aVU)O|uz|l|;%RQ^1%;+Abl&6H>z-Y8 zGBPhe=>J@MOe=gVgi+BL6NlOC)>`3X7daxEj${g7BY9ao?JDJZ54nnzIHQa_X1&nvmwV<4M% zeD#{#Wc!$kQdS?wt!(=j0FPcKy5HD$MIIRSMJ* za*w~&2AeV9+x~D;OuyHe_4%jXaJKcD!q&;HWWVQ-2CfGOQ}=itTa8*?h4bPN_dL!> zoR6U5ng|42&GK%}uN^rd0#`&%?EBG`#lvCl-m$dRp7C_tKWW5%{MIuc3P&V_Xp0Th z&yVOhZgKK5Med)>=`~E*rh9RFKRuW@&5%d312E00u4DTVBQ(j*<5IW#Jol%EL<#E) zre*K8nDea8Z`to<&}p3?JepZ#9xM+=Xh@f z*dXztB_dw-Bzj}Ift$R>FvFO%l%&qN;p5HqHGKbO3F&22`o|;`wfxa5!AxILl*uUm zM>D#bLt4T(-WX_VlXh8+k+57x7KK!}m+9bC0u_0SZ+tf!V{q$G;8r)ot82y=l(s@> z9JJO+AQ><4t|s}i_y`?6rFY89fo~B3GRXS}pV3YD9x9UX)4NBoM~SkOpXO#PRG_z> zVeuk`Kbli;e{H+X4Nd@!ug>;<2R*WNml+@emA^sltDAC?%7V$qXU+f2eLKXAX9@9O zF%N$^4Yty0qwmjJl>gJ5Vd6W~jJd*3ra-al?Qlw3V81G|)Xd1~WsDg9rcQDk`_i0{ zRc~P`(faJbxfdVCQAVUXdA|&}f;p3#kb}>+HGXfEUSAamj4t+jOvO_uuf<8=Q*Rv4 zZe`J=fFjx{Gno{&D9B8p?UuzC@GzEAKFM!=c;MXgwHGQi*jDjp0lnDp+CTvAug0Bq z!r?hifW&ku{{_F&aLay4wng;_?ca#lv%wWsBJr|<&~YO3@TGy(*Q1;ndtY@NcDJKK zx60F-N03mXZF+idQoi-n=C6efa-|fF@8q1)Mek1bsVEF+wx72e2OmnI86`RyGzj4! z<<=OjSNuCz=-|TSq;1`g;I!W2#!Arg8QHdFRP*^Q%I2f3H>1gHrGQq(5OcsZ44K@( zPLa&`5fDmzn_n{?BTucdbjZEsXY5D@mXBkWJRV6hNW%;7ij#|%i%-tvPM-JEo>ueH z0dnaTCx1etx{g0-MX$YNy^I9HZg#^O}oVPT$@cqx{e+C4ip6X%|1D z;b%dV{e9O;13%ET7E2tZmp3wl#BHDK3J}s?FmN%Sz`s>+iSDpm*yp`Zvo9|>Fcy*E z2ABuAn<8~GQlKH9P8W-QkxuE(OtIaM_rqxR2<#s`f#;+3>O}#MNnlB0DKq!fzfXLN z(>(uLFFD3df{YXogP8Ef`By##-_eZ-YDs?&O1RZxRy4yChCgd1>z-^Y_|cs)qj9PW zhN1^$5f$({H{eIcg1$xxN~NiSYoEy1khuMwTAY(D)T&}&MqlBQ7}75z22a!PvL|~Y zy1~WPCSJU5lY@M5Rpz&%@RFDTH^~26IBOm2nl;cUkvhFPGBs@(m<+);HMsC{_=zJ; zRbH#O3^vA^tKqdJE$3ThD<2=LYr(eIEMvR;rW&j_(YNm7NpN$KD2D9|#zsN}oMD!%R6U!3dXTiXi0uP}T@~nq*0pRl~$u|AP z8~lP}(PSGOMz%YMXp-og*6>5 zwtI%{viXs~Bxe(QHSD{CPe8FiI@-`2(#y?+R%z88+UT``nzG4ko8hp#bsqqSC*nwY zLO3U!EDEoC$wN7dqIbf{Ko+hwe!+xD%hVp=r5o(Q+Ym8`1&kq*2z8`$>trxI-l4w? zH#51r_Uq{Ci|sZ($6AcrqN0BSUsG51*bL3ess?s24!J6~A9iCD%;*y6IUU~&WYZUl z$!foU5iuqyIw5|3%F`Qnb^T{d>BU+_15;V@K55nJ)9BsACk>m+ULWK(%V!Hq!I8Sa$7ci%7;y>M;{GAE=a zO>xV+n4|>po;)v+jQ5Mg_4A6wMV`=zZ{YfBg$-bRoQ`_OZX(biomwZ{qRUJwgWl37 zY4RPsFFYDfv-zA_l>5fcsbMF%2;j!6)<%e;!W%*R!tL1FdbrTbgRA(!b|TXq z-8Js?;imgAWczVqylmez1dmYNe-3J41JUN$ZP?$*^K;(%MJz|AGg=4OloJT>Jxv+@ zEp)lDJsWq?eIFJ?$gA+w;I4pIiZSY0F0Tx{>s3HnL(rzn*NG&iRqaN1}VLdecikQbujxBJJWAsxad-jnIh28er*Vz zsyAK^PgaBG9YJK;TjqJ_h(uz_AN@a z{N`x{(5THn?^|>0=golsxT6}P&oeBi%ih$qrsFl#qj`gu_IkJKw2z(jCqfPJM`US; z;%XXUusQm6@evP+NdUH}*9Czl45pC!8+?#J9QD9)(+}UWmjEnw|5xRBAs|z?|gJsF>g&= zp(&7$P)|l_A$)aJ0AJ6Wq0Vr&_&oeEqB0Uum1kOSBfOu#){L7v!2JP5G%4e zQ}v~86UoL@Hx4If{e>lNRhHB44MQ>z8h$_U3^cbscB+zHlpq3+3SQacfURkMtJD7w zr4)cymz-(6!4{ewX>yf`5oYO=7El904Ofm6DR47IHO{cootsq;$$7~)UpEIYAm@q}5p2uI9+@U-UB`jy_Hm_@mlP4$hW@G~@kBkX=OKSX< zBz}16N2?=J3*dPuIrWmTv)2YG70H%dW9qSj*B`&+L^g+=(v|Yx7uI>w2EjM|@5IH& zB*^!&aeG>}Oocu61oCaaI8^271&6C#Tz4mSb3~Sf&2|;^9K^F#TB$iB+a=!hCS)T0 zK-RWFiOL4q-5*HnsbS3KX=5tls@$ZY8RpjS-HW%`*)IhWdZ^XqrObXM>6Tah8?>^i3rwf~DZe2&jTKcQt$h)Y#cnHP4n9Mdz+x}x zf-0dyC#@>KFFKE8mexVEPj20PujqMI0A0@$`g|YW7(%m?HPS4=qt{0dA03l^tkPt6@Vk56 z!zMoFNa_e1JeQmDy#i&m+X*^Sg!uKK(J!(Vxy+MK zB{nLp-^;_j**>&VoR1EZ3$UY4f}<9S%W$g&c!^lTduu!jl~^`si?b*vT++}cRmBO zW4{VdH?+OZ0Vb z0%mnxi7usDOXCX(Rqj_$xch>a$VW*#G%1flg6^0RKDY?e^J=^(l@7&MEXW`=D-TJO z(9YFz&T`VvG}WPg{vSDwp=BR9yAkvyFMDT%mL=V8f9=>o>uY0<-krO5S^xO(Pmm>1B>n_ww1*`bW0O zx)Sv8xCqtHz9C3R9kfrC#y}_7)sUY=ASQWdddRHRI0~-wFBfYCbM4QXI8wOzjq>>-dlzyttfK^ zkOp;+P!|LJ zS-#GfWUbQgggL0dUb?xQ=*l#M;ww@ID@@La#&UQ$VLK#Ce1@)HF0~KCvQQ^?>33Ps zuH^lsrbRhzD--F`cc{|JI=SwiTg`|o1NE_{txUc_I@t<6o4bA}E%@`m%Q9>q7b1hE z7+IV%8W`#h1#wGou%Er3KuP@8;S?Wi_Z5CF;TXi)YMrdXlp)eZ93`|fM;m9W<2M5P zV~KIam{dY5paD0#9iX2)Gv&)?$b2i|Ne2rUAh-=?IH+GGrKLlfY4&R;evS02c5yVy zx$&UG)`#W7uaVoZDDy*5`<;pSw!{6n^6_*=Q|L$zMz~Y`C+F(WxUqe9@4LQjai3t1 zO`qJkz<6wGX=^G2t4B#uJ|Rvo<15(NHI35va*~D+oz=aSedD(5k#}*JsEY3j`&~E# z^S}U6yyssyc?Zr6SgM!OR!Tm9e?u$23z^l{0f;hf4FpNPfz<8QL)gt zL^yapPeu&;m=WQhY`v#j_%$W{x{aNV_sJvmrP3l~Ke?wAc(^3A5X!gSc734cWNXCp zTsym~HldASP>%~%y6#kVmW{qdYqNj=lD}zuP74#516e4U$qeovbZet(>abXs9G|B! z!3=sZRRexJYTQ>j!;)4Q*#fOvw>+EArzUSe&el>@*l8nBuR|U#R2=Rt#$22%;@et` zHfF)o{Bpg5yT}zD=XnoNz5C~~P4(?A0uud0T)r_S});ZGxz=@}#IU6X$BbfUa$*5C*&bKLCnAb-#ZO2+ZJ20GVVvZa&QD zB)QZZE&g(I}#W1L<$Z%Fw#ujFPP*>Ntr zy1qKTFbct?@nAHlW?LyGr`6yUH-Ol=@3|YY;%hwi%2&MmuYxJ=yiqEn<97V}Lx=u^ z*1z^EP__A{vii=y^{Nmry29C}PS?}Aon!k+PfH%jmwoL2`;Po3e&<_GiorE@$eciN zmET~hM~;NYW_bB3x@>tMoxd^wzsO|Y*XFn%NbDT=H;+;!5A}CLkCZJOTXXVIoSnxI zmr@>hX?Di7f_-vcREblvFsws@m#3T;Uk_0m?|4;sEt~VIj&*i?#y6SxgeiQdWj=Ak zKe+Co2ao-h6L$G=XT=oHjL z=Dod$pE(DiFT}DlhUJGk@vdysnYF|@F|uCDfjqSTwTQEC{**6dif_J&5=t-iY2A9iU!_78lv zx1W89UjTD=T=iv#fA(|R%k*gAq$Ed4`p1su;a+iimtpQW%9l2nR>;D2Fs0E?&EGLq z*GFwzFUM}#8z)1l8L|e!gIy4)5ouF+^v+xDuc7mYT!ezMaF#5tD5v>i_^i07*naR9__f zk@)cN8$$Qwl&>(tD@S;cw#nX?do}PEGNt?0^P|je;eruuB>5^Z(T!BV)y{m%;A8?I zzU)@Fk2rhM8`r1-+)&8-)_Zerehi8Mg0@}R=)klw%E-qVha_KBj9)IYaFSuWtsv`mt(m=Mn zXksh*YY(@mM~x2zRa^6!lV0X?4Q`8hd&&_Gk-j7)8WunO8)OV%53~gPaeEr-XA*t ztPFcD=Y^{0sQNpFLe)>Qsjm7RUpY-3`-MX~HMsGUOvYuafue&Eo;-z4TBl%lv%fpPe6b_W5G2lt z3$5gQDWd?DFI=`sC4INeqyEO+ID90xZR=;fP%3`)xi)_w%s4pdj46j5gE@#tZ7|pR z;bdId$mf3kZ65ecTn1;ffo405k~(t zfPLrSMX6?!aluB*Of%v>{3yO56EYP%`GprAxnlz_9nl}SnneB+03ML?3YU1`v{Iw` z5!cywWBJla%pDns>X`YMryxY%=pA4E$N6ylRQl7ol&sjGGE)~yvA&YV01r? z{f56R@P{LK=8vasJ%9W94~F7B{^IIz`}vt2pEy51|70bp6ZNHDUA|e9v3G}kaQm}C zw!d6>-tao{r+)tVOUA8ab;{8n zD3?0xUAaBRo(#mzCetM97t1;ih81(H$y6C^mvkbvBV>_^ZnZ@B2kGVX* z%>R+69nSpqg^cEMZ}7>ZBSFDFb%}`5=&|7XN_u584zD_xr)O6q4HgRTv3OK z_)p?1jvNXekJKY;k@yPhFJ!8d|G-AZFF7eM=jMH*K{CBD3aN%38SUX>{AS<&;8Zg= z2`7{I>>JsvHb}S3H>(w;?d5Mz0@vWn9=3Y{9LwnEb3rwzOV6M=KUELG7wt1244qeJ zK^ey{nUcLm9ls43-E&P`;tRtCv8sVx^JTpUS3YBRtR(!l&v^ceQS^xz19yJ)s^7ooauJyt$zj?jrSQ6n0 zs>0T|{!^zK>)*5Y^Ua3qMozPjYyRxJFcp{hYB1Ou3!}z9pXc*(qAQ87c-%^nuClJy zdgpzHhU$j@{@4psA|KYQ_|1;}%=Sid6prVW^9N3{Mj7hF_-3F&ChrD`X=?7|Ig1q*$3bo6<>9G;b@@R{9- z<1b`l=tOEWBpKbSwIEmzQUC6*|N5^dkvJ4m;A%=DuXp8DKdp{Y_UZk@J8*yE#J(d9 zb{Sn(8z|}aIQ`{f|mZ{c|*e^K&47|7HEe0ncCm)n5l%3dJIx-jJzj z|EhP^CBFPhy)gD${^I5f83^@oHck%RgkybWg-2%geLjE*`E0x({O#ZV?bFaBd5A-t ziCh}}L{Z%*Cw%0nSqJe%w)^mye%}@?b5Is$DVzM^l^bw)kwSt4BmA*zwAxx{b%i6o z9vh19$43*eu8ZO75@|<2`3|^Ypl3n@cy?eLe4AORSswu!{N>GB``ld1<>UIW&iGPL z=R-VF<_t*k>>W%x@F2$(VkUuMcXVRE@J=!gto7_YZLj*H1*zW}fiaCi_u1j*+J(SX=#Fu{i|BAN^gw zu^YfHdJhBkwu)1>yp|9tsQtcBn%*uk`rfQOciQr{E-D~7Oq5b^e*A@uvS_2kJtO(+ z!vCz@a%xP8Y2aBe);|d)EB?m=XUqW&A{yeHq-XgqLdAnXmk1 z2a`096Z0M1QqZ*=0wE7yM&HjXPLT9zDPlC#g_!Un+0Xmz<15T3Km2d|_675ljd%qX zTNBC*wnPV?RD$@a`tc>_WH^dI|qt7DQJh<5!}By2HWRXp5}RQ4D;}9e)iL# z9=_wU_GeBIUub9A>)-lH26mK<_e7~em1hNacN8EJb4HD)Ug)7gqDP`XGicuR02Agf z{_-!q(At00z#JEW;!6epNd1*1uu{GiRH4DBuO_T?`a#F0{>VA{Kw)Uum8#=)7@$Q+=IIszpekRdj9!;```bLFJyQy^M>m8 z{WRnSU3$2*9-qFDQFgL(;lzVi9J;XUbwho!$8(X7cV$MQa1`-IF6bdhkTX2gS2sH~pX8mFeG z4kucAa2nqGSIlC1>>n?TLp&)u&SYb!1DW-0)t5ZlUwN3L#`$XyrRn4(*q6bDtX%_R z?b_)}oQ{9QWSnP2`^#=#&7=X$8qjo6GikL39l-*lHFxE0Ejz>rdckv#8S4R`H` z%~_A-{l0mH__R*o6w~pSLR~I#s0PWgKhRiNw^DA+JI^AHi@n2_IMDrw#7|h5@ssVX zeoEASefGh>$;~PiaSg+y;w$Jb4Q1%_F87}di z`1qdu&-qbIA`6%6#7j=4g+!P=DfzcTnX4^m`rx-genx-20mF{8z6NKP{qA@F&7c2y zeIfG~j;iOJFDP;B@uS}Mh@t*3b6r6kTi%IaMz67A=nI+u^Z))oe)re^_FvAdR1+2U zBvfA85waz(e7^@@p6PD?w@}+IkHNIT8=?9X>g4_8n5W0LN1HFm(k!Kq(GLb%ZYSs*Y^jQx%{ht`M>^-FJxrbdEZ>~8!ob~tGd|_rQGqhzw}>UAP&!P zEFA0Zf0TOm+{+g-9h6-8Lgs(^=l@)^(_+_OxsyuCX71#CDlsnZz^eh+jo9`#%(i<@ zR@AM}4nD-+F}5!_VsG2-AOF^ta9qAwD}rBYG4&kZjrd!q{^r*B>GgfPeAI59XrFvw zBzu<{rY}IUe=18QaODoymT+y|Z|x_KcxB$Dsh`IeGEaGZ0O!FvPDk?iOp}QXbgUr)@jBPg4Gy4Lf+Y!o|^=o3is~`rFyx>Cae8Aj3=)Wg7=M&qO z68USUWv#q{{qE+>My^W z-n zXX@XI^WStX#OA^G=6k~k3l2YU$Tv)|<(UV%_agq0&z$ZU!o2Z=+qT_*#`rvfM~L*7 zrfSLSw~}D*FT^9a!vR;~`a?eRXa7#J>C7-L-@FrFQWw6++u3`%fIh#T$VPnf3g)&F zTh`auz5iq*uCSW6{_s8T_e{02;r^xq!IFpFR^0~i|nGYllbUC*ZM3=nd|A5Pb z0sl{a*l~K`%IM92vE*AZm-$O!H{Xt>{un7GZQJoSX6wgi=hNp$Hevsl4`0ZT z_S+XS!z+#g?wGvM0yEFQR`7)l{PMNG6xDd{{6U+o_q^8p{tcNs&Ue3%3BB(Vb4JfO zEa+>uQ+>6)DVfu@Q8V^4V=%<{qzCs^P9e_+iUN7rb-M{m@kLd5;=_MqZu|Fe;Nd~a z51ZsV_5MJtH==%a>dU0pOoqsrubt$b5_ID?h9!4wz5#$9_xVlCSLO{GNL`2oqc(@?!<~*n`OZa6yBf zF%k~OlF_ZX>iHvOgH^|0{Q+%zJu3h&lK99IXLF>#F<-4m1NYJFUaY$JTUno*9R2S{ zxVh!Ab>>~+CcQd3S|9R*Il8-dyl>05$Dg^|Qo=hy&z~F~*YVBo zEr*Sp9aT!t|B>3$`#A~_E+iNv`Yo|VmIWh*&f&RxPh!B2=DoM+;*;6Buk*9lT?;RV z<@6Sx*vJKcP2#rWx4|3pW*0V?%D(~pj}~Ov=Vx=t?-QSFG%%9*-%?m`_)Shc964|e zxa)^MQvSq&6B}d6KLsPd&6WPT_xiuRe&hEkE_TIG6rRJJpY#6j$?zPV{I-5`@BSwY zEWDQ$?vB5Go$||V$GXkO|7HK_8B`8_4oS9ia$pqYb42Ft@Evjc3Qo?q4gF22FRgTb z{{45pkon~Qg|FVkJg}Oy6nV@Do+F?Hoh0baSYOEer}_<AHW$Os#>=aLz9 z#CH+gR~s>9q5lc{^ad^M)8jdMdNCsZ+ZX%h7_36hYfvHdQ27PO{!y;(x}=$am*oK6d)*`BigseH|;__S^ku9zu4bjs_o((*qd!k*Ca`e4Y%? z5%qiX-h5P!_wQN%d?EAZOO(xl4hbjYZ}A*VdN=R4>LS~=%Xj=g)L&5sqd~XL6BU}E zZy&6Yxf`Qjt*h9-_}9y-Yvt4XIm#Twpi4IFNad3VlIdaNbYK@Y4P2179e-ir6eeHF zob?BH`zOZ9mGQ>ieR08ueDkwTHN;qld2>lUw#MA{aiw$NZhv5hC%w=0#Y1v=Iqzck z&)2;1`qB8pD~A&ShNpz@6An9G;E?DKZd;-~Hqc)sUhJu4AKaUN@a-ir=eE$}FdY*; zN2mI7-!}D|*?ze_HJ{fvxKH*Shq=mw^P|@vd{^*~UHHfji34^d`j2wzs2Uuv(>t}< zv9KfI-x%q({kFv?+Pemyu+rPs=jX29?e{4@IG>mr?DqlKlHZTriSs?*+s=uNKP=ym zd3fl3+c(zQfeoKORGBY@6_aefAD?JMzdhLN;O_MWe{dkregh*ZtFZY(=IdWs0(<8J zD^fUNe)-lHG6qygIdNszwW0*SA(Jm;N&#TK2!%(s@TB*=lkiS(Bd~AXC+x=jXdnK! zg}6C>BrbgT{)i9Q-y-ig^8Hg{z481OF8PNekIzpVU7X)){{gPn{%*=L;^m@n`S z8Xof3{Erp(Bf(Q`#_{=V52_y6KiADzwzL1{q_sy86P|n zJba*;e2&gRgF}Oa=a%rXdidYCw=vv5;&Ws0J#RtLyhS>?o9FC;_`k6{eR?FJs~l8A zp5|cTK`H}W)K|lXln#H@wdoh`%vRX6cu;?|pCr7*$serm4BkC&1pe>nqrLr(gf3Azunj`(kXVXlf~_K zEMd9rWVEA$K@uwrPoB-c`(QT*x~B|c$Cn+yH_mPUKJr8hS262{0DXSWiXQy8df&^7 zbrKHj`y5H+X+=v8;*jp&|IL5se?#UkXYKbNs0)MdGd4_-^H-Ko<9H{ow3vXfkpJuc z4VizV-;mjc0>c%|+e+4}l+^Gk^lc z8`Vb)hBqe&O9De~`0g*4*RK3G$D8lAzmj}MiT{QX=UcXo|A+L6LCl*A`_{is+jjebjm-T%*Z6?_79Uti@RIl-!GY2Ewt;y_9DWG=@ccO73z@&F=a$+X zVnXSaVH?+%BX;cDhHSl@pSFH}E~UC$@vXNXZ6^=?>+@=QW1=qC-~Dg-M}7ar|B!z} z23mvr1_wGY$lDGs5)|5Rqd`gj2=ZpfAKz2GW5I?-^2yVq5_}-XE~!f#@E}Wd@kn!U z>XQ#V(y1N3%_a+x)feuzN5Aho2KwW5df1TLa-zb zf;m|}k>C$5cJ!bnZ;W&s_hvtsvH~cldu4?CcEdTSucs&aC+azR^6$*yd?LO#+U&dk zm03(1p}PGD>RI2ne4ead@z}MK)K}C*er5oU7{Qb_Kkf7je{;>S!v*I(%dmwD=3vN2 z+Ob!xep^zy$)Fv-W;lL_`cY-YxG`c0Q!KiJK`y$*S6-x0K=K2-OsKv{Ft=gH1%t%q z)4r7V%)kMWq%YffS`a-R_@R}=Z?DmOpD@6$p0@^`ajDM6U`GnK?HXgd^S|pa>~8p> z`^1YKxp8nr=8k>G2M32_aAnC)E1t7}Yx}7WA9m!%=5Tlij7XWy*2i;!|!Ht_c zQ?oMW|FR+(7BTn*q-^>P8U0_D^o0zRn~Atv2M+m$0e|N&{XR!XxMT&TzJ^dIHuwddeY&(Q=P)pQ)0d<7o;S z8xnoy@^wZ#)7$#ak(x0LFvxFmRDH&dL(K-=s7U&f^4?hGe*0kGuACp+%3=O0Q-R?4 z9?xwT|68mb7yI!ojg&_*gNLd~@Q3rp+;(uEW$H%?RPW}MKit7GI!H9JpDBUBzH`BL z_=D2jz1knbIIjVx0}{2rr0!>6Y$pd|qCd025*d3K;gx0gHGc5a;9UbyxQ4m&8x zoj*kU2Jhw|M{KS z8=XA(4O*d}9`ZY^*pOj8JdER&(VgSQeC>Zlzwdyrp0{Q*(cgK~$0n)$Dm}Ixd*iSx zX8N&me!>;ojom!QA0BB=<2=L1ui||3{o8S`-k!lRki);LFJ%5n{a=>;@}Jc|>ML{^ zcrP-a8|w7~yyoKhps-iz{=PnWU}VDjFv34AL&ALNfxo1#fAgI$WIB!o{96a#o9zwf z!j_Zq%bEP}VF4Q1i;eY2D9{|^4Dt;@@3aNRyRmGk&P^g(`u zVm!>{+($L}OY@0Ce%La1jZ0HMcYfe(m_cyC=7Bt>wHa4d6L0I(C*JtqfbWdiC&2Fg zf&ND&`P}$BKX8igL&*#;AUOCg^UPm3Y)E{No9h!s*x*>fJTu@>BK5ug( zCP=ePAOP=@)J~GjzcTnn68Is3B8kQqGXJW-kjWZhEm;oQ`@Uus>J%vdMfU!94v*IU z%k#6Gm-{fU$nGj1pFQQE|CCR~R($!B@8ACI-}wuf|MBHFWQc>@F{sbh3Ago$EM;C6 z$$a?SnbD^D*x&pgL1Rz`X(vj0XMA!7D-Y~Q>16*AjIeiu#~C#d&&I*}U)5(Mj{*`@ zLzBP+?&Nwjd=25UYOxO3vrZk5ok5xZ{0Fa7hWk?f2Ys}RSygjXZVrWCD9?}v+XwrV=x-W zx9<}9U%;(xQRw-j%0wmJ;W<5GNWN*${1408g6)3lvH3d7B&|c}gIIroRDtR`%l3S8 zEe=75sImIG+EuDYzF=>hn1A&r`I2&$m!k4lNY%wm*IRf(s`$ z!1D2yFGSa9vi1o)(>r=+X26wJ*X48G*ar4gPrg^C~Lqp zJZ`#|o7Sniip4*U9*eD^6;8u$$Brfv%xy;xI`-@pHnHb=0r&jz-SUnj+pW7O$6e8d z<-MwL>j4Z8a`UKOl>8~X_(U{}O^^Zu>&sQ#%mv7hvk0lbGZy)%~I$(1i; z{#pHojO(rqm`zxy;@25kZ9b27AYNBKej!8BI3nfGC4Rn;(Z3<{44LR1@6{#*cJ<%< zrB@5C?&@qO8OxFUG*n{&HtQSz3suKx2As}{z2*szq+N6vok`#HP+H@Mt@PWrU?01; z{|1Ju?vNUMJWg+WUH!G0G!dvIRvyCs~(#!_Bb4S6SP00H#@d(Nw1y) zbz^^^`mMb9!H2wg4z7Cj^Y$>5@4OvA{z7Gu zv419=_QDrvayehfNTf( z=O)Zy*)#49`$Cj>&= z4?`q~j6&oXY)d#K5TrPeLZ@=3M%5T|&b{}!-Hz{();wp;8l&p_ zzN#_ioMWvu_u6ZBojiKv?)1SQ>XECn7Wq-fI&V}DYuCYVt2v6~@z9#D2XU3;cK5VG zvuSnhLDguDHNrI(CSpoFf-=4#V-ANxFBEwNHlR>iK68k>SM|`BT`{9K-YjK23S?BX<1dpuZJM!478^BfR&2Hc>F6&qbFF)V&(sy*@Ylva;rJ>=2x$+@wr*VRCC zaPkP8Wwas@^s&@Ht~n?I8*o3L1he+ohspCK{7`c)N2gyu#AT^ZmbU!!`WenNNhDHv z?!&oc0t z<~Sbfpti>Gc1umW?bi>(LI&t7GNw81>n}IX?DjqPjS? z#Iv@u*KZ*tIJomx<=3w)Xp@Zs~In){3d_NIMlt<@B=KFZOx>V9iYz8kn zCvgZRSpuW~*bSKhyENzsu-5=7G3|xc%UbH*vE*{-3J zQA}NKO<}1))E+|$okKhVoaA8OXBIjLg$37K**t;n5cd%n1a)xjoetX}zA@dTIjA*9{Zwj1RL=}rIJ+aP zKYRVp+OI5iy-vfQwYfZ+!x{k&s~|dV_uim}@vK(Azm8cI)}GXQE-Ca3rQ5xg2QL~} z{8$*o=|2_+N0rIfb3-)r+?f14M$wvLvk~-xi6FUU^$-dqmhm!w^Zic{62ti!Ty5yM zhsS}8Asj9@y;n?|ENw*^T;bdtALAGV;nE+C`643Gb)Fjo+gcT+Z+o?8K2+ItZLY;} z=i_&jG+kWFJ!97zj5!yI@errS)rX5UAS#`XMiN2Wh-nt3r!MfcW4%OUJ^mmiRQ0Ic zBfYJdwmyIEr#p8YPn)rV!cTvX<9{4yX6`js1{eZ)O6N6XwSJ_jUp{hUg(fc+`~(xQ zbYm2EhN1zV@-h_u^Q^~Du{9o!*o4KO4tXq)W;7E6uAiU8C&?lQRk$;}H^#cGlV6%Vfbsl{ z4)e|WG2;}}bfh7-xIE<32AsHjr@c=4ro?C>$_s*H+v9HQF)Kc#$?nGRGB-J|4GY z-<=l&L`&f-c+UL*p3VP-bj~|9%1WQBM{RNwlg@Pwinci9P|>;hzW?;=Za-N!WZwV& zT#)mIOo~~NH%3;u-p~9~e_&_eoC_QaTsLI&M&CEWWAyn`FQUis>^Ee{Rt$5>Is30(R*$3g6}b-y(bP99Ja?hk9hYz)V|VDg^Fc7CP&W? zJmqM1_$V@CB}dPXHdYwovi6TW;{f=Fo}f^*?K%~)*S{)PfA*I+N^r4?69c>2fzyt)Ch*U@^xiD z)=3C`h$9nSbgiRvPHQ#fs}#jX%d7fQLqp-ebA1;+3BnavjoD%nzf;)h+z~ zhEj1boex&E5Ldfz9n<}L`-%_lsXD@+X^Fr3M^9$Ee)8m*3Eh}oC5R0zSz_28<0w4) zcs$4X;;~JY(t$hNQ=jink&ezAtzVo!S$g!zCk-CE4t?3o6=P0b?D$SSFnOK|m%!Z3 z9AK`j(O1#cPu+S7WD}o1GRDjXJiq|!)qy4%G!`8X-<@rCHODsv9X$4`x zIXAtEyCv=7SL=wA=dpNcK8+8a>aO(hgM)|Fahb6=mxRxojtPjK{-;RppiMiu;ZYkD z7T`CJ8#3>H<=u%|ZuS6bd|cAhL)VY~)t|i{8F~Sbx}^|shrhDa!Jj}I{VuWp8UtT; zxnO!{w7VAdcNpFgNHXURtl!QO@x2}tVAUig{F$j&JJ^eJAm2~3c3BxaJ!V|pC6=sx>%x~dK zec$W9vee|VnVQ47*W7jB&AG=B*}%b9h&Hha9}(hfwQKLlEtRttnppT)arW5WKhy0z z;%Azgo357_IVN~Ekcm(L65|;B321c3oMU}MsT4xOT}6t_*ptQNj3+Stwnhp#Y>c*P=W3ufgGe7NM&ha|Kh!vk4vTQxSeAN0_92|TsygiN|OPoFyo%8s)7cMiVe(9LQESxET z6_5Gn`r%jh;R~bfOs7Y9Lv?ok^nB)2z4Wnbp6KBiB5T*qu1_3G411oZ)uZbmz+6A* zh=&H&v08sLXJ_LlXV>Vyvv$wB$4~PINdBy$5zxUBLc?c3O|zccwN)I3EjoJ~|B9|2 z_#O9{zZBN|-7k%ksV8 zj*cufVud0O=m*YhD67_!_n>Tjlq3(Gd^ND|%? z+sR|hxTG<4PN>i8LbvmF{Ty{tJ_(4Gf;eL^Px z%F?(YLoRe>t!E2g{*FbjMlOUfg#v>f5v&=Y+*`BYyYjBtSKW${fjG9*&nuDl42xdp zcN}qBdb^exL??fq$IeciToX4O9CvVv7v`*mlI_lymt$D$m1E4&ckYSG{J$3#P^<4b z7)RH73PyOn1RB#fRvdcHQ8D)H^(8uXI)aP?D<*1b z6&Q{)%oLD*;DiAuT0c+lhZ{*XAKf}nV6N0rR_5p!+7&MjH*ESR9Kl)pSPpS?JAzo} zc3j1~S7=8nUh~%VtG)7dUctrY#E;I3w#vjsx5sG{$5w8)cHC85{vEq(632?MW30b= zXO9;k;ed%27_4;W4Vm#*mQs7Zhbg35JcL0rub=458!~(x3_w58Q+(U8Fue9b|JiG9 z$iN3qDZ77<)0aKNqVr_joX}bn#68nNQ+b`cS8#G-LO8Qfw?w!_*+Nt0sTgq5*|mCX z3O}JC9$Z_WhL4>GyEGd3_xkW(S@9p@?E2Dq2!EC5F+X{(w27lLaoYecaS)pYJGEMj zCYYXP;t5WNSsjaxEb#7|RIVS*?r?&im`)sRV?fGzL?1m(s8ct351nZcKZ}u}{>st^ zvNFzZxY+r^0^$%QjvJc`DjiwPU;Lk+l$(pQL3@qTMMG^?9pQKwWB&ZJ*KtF}S`ih1 zE%MQoB_EcU;Jr;QWO{tZfjQG8h6O_(Tbh0u1dj0^arS^C&O7+JF?K*C6mR84!Jdq( zIgS@NtkBlLv-TX1*cuKFnT0rRhI& zzvm`imgk$nh(N{$QBgH{peu`pTC~rrDi-yCM}^llSH~ZoEL>UIy*{wYN?h%yY@I7S z{M}DXBL^ej*(U@P2h;1ch9kuHGfsG7Sn;Z!?yX40i?3)^dv>Hk3%Mb1hLR4O;m0?X z+?45+!s*s&qNk%>`cG%2!@mdgh;BN;@t9)=zQ4sm(@qZYbcF6}}%&e6rf8Ia{N%{}2ic zxv^I0+spqicEbD&@h45?pl@ZlEn$R{^e zSopnC3Z2Bcq)(khmf{a2RW5r6jj)p!n}5CJ_B_~P*})B9uuFK@`G({IR_uun;+i(p~h zkolp1_Wif7iciQ;hAahRiESzqH_4KQ)W9dMPMJt>Op5ybSDT1W!q!lod2SWR_ zxGB|mUHZa3A;hsP1QQSqVzShN1qWvL6PG>1$j74)UXU1n*LbLnKA4AU0VCUaSDcO! z=L)CTC$f>}N}qVokyrwQNEh5w^`458?!@+F(@Ex|1&$R3TEBjHb8=KTdaM=uE;?09 z|Hy2fh1>kmyb9q5%{b`G3iJN>D@*T)zq0g!@d+8uZTQ&dxgjHt)(p7588comdOt8@g$HYz}<8$`J17tx3sBF>h>0D)$1HZdj1|$+kNlz zc`uz&k4W=8uJLy*X=^jPMsZb_(Nzz;sW2QIw~E8GRSm?jAgB+E{~?RU6^A$)9j6%U zhTc#q#wY)F9DLa|<_z_QtmYf+UPITa*}!Y{oTEkZQV~ty^It#GHPxAP9M3DE zPTN`MSVy_;s2B9a;RtKY5!tasWx*@{SatBuv{XXZQ@>w7>gVr{WTPvr$6jIbRBOej zMBDW!<7we&GO|*v97cAHm0P{1UTt&!Z+j(ZxzK^T3Q0~6R z+C9%>SmI}S(CM+>o@e23YTfN|v}Mu4XWUq~XMCM;w6Ji|%4`uNzCVA1gGV-9oKW#k=KG5@UZA%<4b-0TeB)E3kxBE^^@J4l2I); zNk(^0D&HZ)CuG``Obp53(w{}O029T8f_=zBpLzkF6QXub$g0q>YeXXm#YJ>IId=Tg zUwBL8vLgyKkvb)Uq8G! z1@oBAV>rZ9?+LX$rg;_bV{qI1JZDcxx*_wjZpeJ#wPo)5IWHRXfJD!OPnPpl@qFm` zk2O>pIu81WK6Oofgu`JCJ0^~}lEo$+x%ic(FMaQO(ytpx3DqtW%RRFglY8$H8%^1T z68)96;@vyP5IeKvJH{NL4&uHc%`D&@W2|npR^`0><^X16jtLJejvxLh7Y?$6?P(`| zI9E)lrX$w!?c7HbEqJ9jxFH~oH(siUI1l|q-I_;5`SBW4$B*&j2j{G7Zeqd`!NSLO zEWX_ev**tnp`JX$E1a41(b;wJ?{??yadggb z#IRs&Jp#q_;D*Xe(cb3@e$PE$$PQ8X!tDIctM;l&&vNTAfCGlUEMsyk#*~fg>jw>Z z(tw-oaSokbe{h9kn>w6(+jR?Wx12xH?7I8lc&ez>v6ToKXG#A7h8vo zBF@Gik-2p}JMq-9!vksN3J3Xx>A3m2agcfp9V~iJ!N7T{MH4F?`K;lH_FxqEqmabR z>H9T8&&=Y(SmVkQp5Y{x*rXPp)`CH_S>f`S2$+jmc;bKY>bRl_-I%ezxPF+uor}h} za09j?DnANlaQ?9F2qz}j&(w{^nMTjjB3VeDhr;Y(Ng6Wxwb)VWDyJLvS5}ON63e{a=b#?fK%Ss?aO#I`!r_jKt z`tdxF$NxD~JzmUnAgZS4AZhjo@u}x8d2Cny^Nc_FOrA=IhkRQKtE?QYcor~Ze zv^fmzctIT^PZ&(E z|IlY9r#|B0aMOEmZE%?q9d5{c$+#f{PBr8en+ax?!}OCDD{;_R`Fgx#)b86uxE`Zj zIqcCsTX_!irGj%7W-9p8(JPD~>49F#+RBGpkQu%@Ktx?Y)xru7_$95G`Ffk z6T0I`bQw|@Hzb5n9QIxZTJ;iWMGz-fSx2l9gF_n@BFj2=4fte@@A5` z8{f_fO5-O7JYw#9Q)Yo*vIB@aGdjdzTV&8=YQArbe+e#-*4;Ak?<*NH#s7S zyJGsdYYxSOOt>4jRv|Z#zz4G}VqzdZsFG304L{(1i{F%;VrG9T!53(L(2F|=gU3zavCTOM*xA%dJF|MZ5zQMegxZp zGe@*R1;fm#dQLy#@4%`ewa#B*=VKFVRr2h9YJB;$y%|1^+e%lGRa4pwF6IUJJPwR_ z(R!OB#ovbSzGBzXffhY*J3R7zDv*7M;D$sg`7ok!$s|s_zyT`rrGI7U3MdafzMXgIo zf(L+kuOf26sHn-K)Li*TpgkbU3n<>otR?$Ig*7xBd z*1?)@v`-CJ=)CuL{2t;sMzbeDm2yGE++v~LggMXCj6ZD1A|}D9ToEn^*JGFMLYq|& zOJ_*1CyfH>g$9Q7MW>YeMwvKogYE-*NrzP z{$-%?VY?9w0j=&UW!?+@rgX!5i01#Y06;<`hMd#V&`5oS@@_H+Of{#;z4bSIFL7AF z9_WtT*L=0iy8<=LPnz3Pd}@Si8@V+Gv@2{LN9%VJy0v(`5ON-kVO zr6%i;vC@NN8f&43@IkX|)@-RbJ4OR>vTwDI*QS%uAmK`5O~g<*uW>H#0+-=)c`X`$ ziQHgl_JQh8pWBAtsv1_%HJ;q<-z2s;xDJMgco?4rZ$I)mbDH!b+$47{)571EyuSD^ z5D|P`_A5`^x}KYn?M_DhJ;^YaU>o>HtG=VT#3>wMmJjmVbzRTsC%)og0IbDnpr}jl zDQ00-GZIwCb0)hNgtI&+FBRFFr;*nJD)vV>N*z=)>Dw+doIB zMMOPB14W4hKGCI*&I^OL`4>o?&$HJbk3L-Ol0WhitKUybZV^SQdL|r*Pyy2!jd!Gg zDCG4y3^(Jyxp?y=GQ-qVle2WRav!L{maln%w+>4~X`R11CNBiNC zjCKAz4-Ivi+z<3^EUDgu-|_iKc!* z@)FjxG4A4oUiQ+nVgABh$NzxY@fG5wdT{ql`Okg~zthOFk>JO%8H&?By49gJ zA{I7Kx%Ctnv{?H&yAxUuC3IP3c^KBwSGaQ>Zi>uZ8eBfZBvXqXvA86x!C@jfC6?9k zQIB4SCLk8*9}?R4$`l-zwRQdEgX8S%&3ZZ;M&&|YnGko5L3$P0{o9AdJYH|r$7*Pf ziR)QfmPe4+&3TWhT+glIGfKJNc#ANYR#$rvn0@&|*!Kq@{Ev3!s7Fk_ zW;Li>9_-NW<(@7M-R;Kvktaj!`SJ^cV6LW6i$j{J86=_Pw#WPR;&`xVbrHCN_-)k> z4nyMmd}@-wg(7gCkC&@UF5{96qL|hnJ95QX-*^cOZ;n0fcdb@wmPMbZhfK&g`1oDQrDtk!37hRCU=5JdDz`mw zkK86u2@5lEHmAucP$jPuilRLX*y;*-aSzvKV|bJb6Rw* za1Pl<5*G>%{o*4{Tqmw2^4U?ufD8OSM?@+2k9+@6H-_b7}i;|EX_#AO!fjCpUci9+ndP1(cPkwNf z2A3MLH|MmdLXKn`LjEM3>0SJtLE$r1k4`*822u5wKfrie2;z{oW%0xhfoI`q+bqIr zLtenntZ=~Y}}YO?b_C55B5*4HYC_C{xp-Q-C+4Oao={sF!=%?u6lB- zrc1%npSeZKEewVQUtmMZFIXUgJpIr^jWb#)b8{Xi8i~Va?Wg?0Ja!xPTJ+CxJ zc(^O4y~Rx5t?9loE(Tt^WN#Cu@Jf13{?g_mfpPUk6Hi`xZ%CgC{$O6FpbF%cc41Z_^vNe_74m zIs*6bid}(q)W*-;dCVXjZkLjS#G{G=QPAh@Jqy`I)o&NqE z%C29gxi>@jC>Y+J%<}37)wMN#Q$%Ga*zEcT0*d#2wz{&ka63q>Lgz7+NUp`BU0M5# zxW;BwYpf!Gt`4eTn*b;+wh?hk5sIzm6@e zpx9P=iF-n?))nDQe2eJw)^DU^>AWD^Jsr{o6A_{T&4SZPwAuU199@P4@9OYzH z!tEb`EwkQ<&EJZqa}zg2x!P7&E-rk12}EglmfTX?7D>DuySn9k`I- zW=+!NfDKGZW!4AeeY6tZ#IP`wKDwRrY!-vbHUH6SlS;)^eZe1RcSoTlvC(Jn^zn?r zGPz)x>%IOug=yD4lhtY`2Y086T#k>1P{&GVpF7*dajHhS$zlokn!z#-^NR`-#v+&m zZXz=bGQc#iWwvY!&U|F-Z`ut_rIs*@*OcgXk6?;?uyB&~)@VU?CCvT@65oWR<4~Y=@gt3Jj6Hfv147z>9A`O!{53b#0uLY8#KZV z7=1VhzD!a*ep{{xoN(Ye!VY{&%SQQKK>9wH1Krl+ap$~AmJHq=`b{HK_?+M=a$fYB%~I85do>YbVBT%&Jl-CLqZx>#U0Z5{9f~=*us%=Euc~( z)y|NP=N5u4S`ehuWPG_E>3AlcR71j_GzKF>B;j_~{yye}zxrUMr8ZWqpX;m~4t`tB zZTp8<8h6A!S&p8u4HXKXtCD)T#j(f#)$bDzXX@5Z>4opYw_hjF8RbP(BPM(=%*IQM z@d15C$o2MT%HQu8uWiK>fT=xVXv=lwrq?mt##3=*@`oiQk)LRG>1^WQ>)(|h>bPnK z5Lc9c^c$A(A(UKVG3ffgY%d&mPLEzAS@{CdX1Kh(_XQX^iT7}7uCpji#|f*Yt^1#V z>@%_W$3MuNH0Av6Cc<8kB~Cc?3|igMc6$0Dn2a~Zi~d+`N)bzYnsCMMugF(INv-LM z)Aa$}G~RaJLQn24tq?Rkmm@m1u)2&k*G(SSY}!1Knt-mA z7P|!UIk10Up%M$Rx-e4pNLsR8#d-jWt!)p&Y@%PdUJnIhsrceh#~^#8T_Z0aqrEXB z*NrcRfvtcj{@v}Ylp4Gc9c_$U&{UvdN`e}a6dm1t8KDC!3b;@q7aOEg+KixMxoFW> zvdPeQ>H6J8cn)9{txO|5A%UMuAKz4IXre2N3paG(c{$#&+#Pgy)@~0D0qV_Xp}0oi znR=p`E1|NK3nV)fKf*0iWl9)nkDB;m!y4v@F9^;f=^`)wqnNw+CDwj;O|=uT{KW`*@#7(X1x%xo;K;z`p08w;N($ zPYy5#qQqM-UvPeq31sjw&QSLDRYw#4X~6VzL&ihHB18%EcgtCkdX{+8x|(^XxKCe3 z3_P#1h(s+?U4Knx3#^Tg3hUA>tVh?$j_58YpJ7JlUyRaS!(tb4Qu{_8t49u(d&N!O z`I(KL^|HYSxTXHBP)&zFR0uhcs})KvU@@6%w)|STa8=>{_^UQ{uzz>cb=`!KVo^_( zCw-CSId7e1X{s1QU!xKs3Du^(FN=rEwQX=+g2vUioUqm|-|gw$t)8p+PLgXgWM94i zCtLvb1a zGcccrsC=ZtD^h%4K%JF9uI)YSsn@+8yosCiCTOn(U?QiMhU4t6ze?w`(M@(1sAQS~gD{5NERM ztfT@MuF8!E9}o6MFiJU{&)1<^p(MJE?g5j_B+~Lp2r&oY8L(N9?iR8FBDT%INnVJD zOemL2{RlqH2RBo4G;fA=>EtX0NJj&z?x6B zP5F36nHEIK*vXZx$t|*!r51_i$ZrdtwRiQ~cD+Kouh-hGU=P~I^cpVc+5R{wIGGoi zJkWbC*hyFeQ)vH4t#&f*DfmAe%dk0^+dyRkU97M%eo4N@cU&ob9U=RIP=Hi;tiXPzhQa%UJo zhx{-}K5CUGgH0h*b(i`r{_B-0RAFcFpxUNtLR|f*Wzl`*N6grR`Glb`4)%$BS&+qs zdD-3jK56Sw9J!Y1ABkFXme~>#BYzk&Ux3T}G`0DAohkUDJFaIiU<<8xw|SWFU0>N2 zr9Wbl!WVDx21EauYrOfzQew+P42PUnhP&4<2>ivk_bXtVH1YJecDpaqQc3XIe# z#(yuVS7wWrk*548E2MnevP%+-_)hLl`ZA#6a!L%Mw2osNnr^4h$onK#zxvk$A~nUnLgM5;`BaXdfg&UC2Mf5TRU~bh?snsM_Jo93VExh zkgs{WLrW0@EI>=$kwd9->i7CgLC&k~&Hd7knUbPMsUAFuj84Ih??hy_m~Hhd->mz3 zUtAtoNg#Im5la2%)nI&httzo)VJ;W!UwwM=zjqHKPtF4cyPW$V&RyI>wos%qog)}& zej(tZ%p-qHRha%cmz2am6daV}U4H zky44oWTi=K?u_2)m>yrgk?kssQ?~vy_ias&qaq(AEd}_kH^+WY>rG4!4jpfH&1MgjKaObHZmxt~QLw3AExdWx(UM#!Evhm< z%d|QggM1OI%RICI=OMg79ec-Bbo7U_z5MsozvFA+$FldvDVneVIV zawYH*u8y0$dP^ePdZl+q4hk~o;Zf~g%$I6n?2&yee`BO!F$fG$xIXdvKch(U+3_zv zkkt;QI6RJMKNgF>ao&5nmuU8DWxgJ$za`0?WbK_@ql^jYbI)o`%XmX}C#)8_zJ(Xl zRQ!NN8xgbrIF%O}usc>T80y4T;5$B6YaL5S_h|lM;J#7$se_buLX%mp&Z)iI{yH|J zv`KlHMeP+0oai*fr@rn<#3UD#&93N` zpp`p2mRPUyKR=!kDH}ig8Uk?bjz7JZC*S0$qUe-LFg5{t>)Cfc2TiOKgp0T@fkMt>@={)Ed{L zq=-9CS3C1$bpZ?e53-c`2K|?K(L0`LiWbYBl@G$E+;7$cK4O8@c?7VfB*=k>)Z8NQ z@)U=8pHs)~Ft5X=J^{-?EF>~2N%*Tvavv(is`i#i>$`lF=~`R9r=?UxBXzV9t;-QM z)xeNo`H{V;zH$oeJ5zC&yOFo*M~Q5ou-4z#mEFj(9*AG433|#! zZ1A>c#ZbfV4f&D)Gw|@Y*k_;ogD03gmh5_A89idThyA-*6^$mB6~y@SYJv%WHCz^L zY$OOrcU4_4)!=&FqqK4~@Ed%#3Nwv=ko>MtLeui9K=lvp@wacH?ZN>;Ys6gjhAMQf zdQq}3h;UR#oq?p(oMc5Dm--dV?hnr*jdK~_U&);*_++tD<;q=1HHfeZi`hAy4>^_@ zgwb(pMQC~))Cub{z8+M$QTaDqozL|KS9%%Y4XadODWQ8(s zijCs5WdSP~YLKHz*`M~iul@FLx4g+xu^JIK+4PC+#fHO=2kC(UoU?9l_lwD;r9#Av zh_d^@Lhz+{Fg5_?(yydqXzID@+VEp$rnC25qu(ZFQhwbsRN0{TC450a6m($m?Up^k zWF5?-1;3ITSl%&QMOf+lAq2=He1MERq&#z~i|a_T{`>mKl-wTXmxuDVQajJiw!NTt z-h`12n@bDj>-lcX|5vB4;1ZwekbMVD3-|qT5Lc9HJ>`(R_&Iw4w0;tfYP2Y#h<33O zi`MOo16yrk#n<0{&G%7Qy_@H@lcvDcM_N*}e|%@_FrBTO)_LMS{R6jmMbGQp<19Nz z5_0yueV{XO(Mz`KzeIDGE}Q_Eg0yYN(-_pd>eKW6*)M5G`mI-Xo!!3=y<$pby7zn= zl)BtB8*>MGV^Z72;PEiwurq$hzR#5I@(3ab1;%l` z_&s4tq6A+vB?VGW%*MlC*bSd@Y}pZ|op5XHGlzy``KWAO6i2bBc(y-n6=bd>!vUus zCwDvUW9x5S%bgalz*7^BFY8e511ldM1niYMqXWObjp(9r2ri+C+cV(xvpkIts^?!& z_?XqK;tc6w!ygD4eZn)&TpWR^Y@K_Ra}uc6Lbo_2k_B%oi%Jb}n6HCf5+4SN20mTE zoTC?+v355?7Wh>MFIEa){-Gr4gw9s(uYm;hg)LuyuxQN`R$(?8|-FZtYGjGB$XXbqwP5OZ^qyHbH#JbP6`R&r`%|nssxe- zc0Ko+muAmdNvMZq+nWpDG5(P1%=D2Bzj004i+_MN2k-zig8h6pHVN_12(SgX;6Y8E ziHznAbZZsHojVpC8GRQeHPvvCow-oKKN~q;TTGuwF*^X7{Ij%n^1#38FCIzem zHT(eG+oh!l9a0tA98^b?&Y(hn3i@j&kx8(G>kGOAAVCbe_U&I5ipPitu;K0>g@116 zO#0>}%k*}~9ZTgdPAa45*&cP~sMe{!H5DEKO0j4TNhg4maGn~Qb|Q;U7zTc5!g4Ak zK0j1OPIMZ48qh)nJgQZNV*8g^AoA>(iMj1Rh%1P4{-)ITy_B3u7A_G!$J+eS=R~o${*w0`yKU^CAx}(!Aje8LhN>x zKucy3*mEPx>qYeBu8r4m$!4(?UL~Yi<*&T}LS7Lv2Wbo(yW}KpE)fD^n4}JjY?U@+ZBnaNrgPhjYORZ z1h4Bhm$eobn$xgZ&1XP{i|$$>mB_h}E{WD!P`#wI#Piid*0=XJ{$yd;;fTtNyZkNd z>0#{lOIW&o7myPrXEm+yi<%sXjlZq$asgf>2e4`7I0Um+Nj@Tu^BF-l+=-v7V88#| zq4dZIa*OunZ;9NlSBT4>-VH^a<@?$chu=->88peR2yGLt85|worY8{i~d?kDmC&snxOv~5I+-UO9{s8#O`Imvdv<#*ErtXv%o28}m zCD-9in!XDXgYJ!&m=Ho(E$q@v_5Z=xwWKYVmUALHB_rd%ZDHX~d5lgc?wmH0PX31v zk&cu}$emT6mZ~ztYHrJP?vhqh>SLz%7XBg4Yq25YbbG!Ixmc4=r&*HwOJ+`&+*iAN z4b4Py*%ZOn#&6zj^6D%VGw-gMpezopsLLrW&#pA26B2OL8*b0-Io~#M70CcG-!Vqs zx7u3gq@yi51b5ml&J=&$302{ViXJ_4Ho3;UlQHOOMq&MTKm@IE@*XEM%g*UJ{N{=<|PeIXk3&t+CjbD-DWM)(S}?y|!yAnni=*QPx@oAeJoo z>a3D-KIQp*6S*~;E@PE5)T2zs&PvRrxEjPZU; zv3uY`iM6Xn5C~G`OU=V+pZc#pDsIgX^;U6xWY5Ne zQSImBv0N^R=SZfJn?NiZbGZPDKw80a7;;NJ{rnW`r1}z#)4E6EXv3x}tDYs?URBOd zdUC`XNjv6Gmr#88$$*1v8t4BkcjTJD%vrN5UC))`BCjXt#Nf*pi{6*5JGW(y5Kxg@ zX|BizZ4>IX^_9=v2qG`5MSh&C+uBU-T5pX^-uLK=jC{(hQWX@(@w_>PN1_~TesPAd z>&0?N9*Ah4e~F`n+X=YFtFFIM*7(c+^L990p1?vPZ$+2WUN5lblT;cH8WCe+=ITug ziS-#-jR3L;Iba+5>~$idostu~Wy;wk6US`JIzhi*S<0K>r>+}9$vK|OA<4tKz+2@h z<+F7rv?H1>_daHib=9N0(kWxCr)$Q!9k2NQvn(7a{IZ*Hcia3jEzT68rg3POOjS67@<25XNDe)--P9Y~w z3oRFl#24k{SZP(0XwL8eZvC3(EBE1`*L0NOU!FMAIj{LJKk3OMj>3!#FX18%sB%MiX87n*hphYO`+1M{k|WXD{ar`?Qw~Z(0{Z# ziW`%`WQFC6mt&BF3xlr%a`l$6ztUP3Z|Tc86O>Oo{}LY7-u?Q{xu{rBp=O@8R?>u% zQ+)E4@>EV&K+E%Y?ZuzUMcGX~)&h0O z<#~MQ2iv2p7cX*lp9IsfW&nND=v1TPp45ICjO>9tXdy9Ly0iZ)%Y)lrDjxnJ9*0)i z-^e1+-DsYhC#8Y|0?rH{u|HJeM{wg~P*nRb|(i~dG9#s1)f z4eo!^;^gaM8>zI`tJR;;C`(GFE{@ObSsp6}>UXZae7tSTH4qr~-K1;UJ6ORpy|bdU zF*qPD(R_X6Azc}>K+RpHvB*Y?TVFOgx)(S2C#B}BhDiqmjlpOdvUdg6Do*vu zbq55#VhQPl6_BoTqHfO8B7SK{LjAW*D~lU6Z02nLPiI>{OxAG3047eRXLY5kG1Jt3Z#5;nkVk_`wt`w=CoNBj7{R?hzVHN zF4X?MeO1K0am&|)?D+I^d19$L;B;)6H|7cEK@g@{sf<YZ5>+4j! z&9OfW9zWrdT`3ZE)0t=$12WAtlmt$1;yd4+rQDKUW&%wgj`QE zzym3Agi6bo3q2&*^86e8d~rtvx7uQNRK2H+z8Lra^I^lv;yCU%xPB6R1MOVxj8ix! zNTM}bDD2y@^=jqPV&C`A^SRF`?z&C@>OeNMXM4&&bPI1@YMQjAU)ZO(WFHmog% zo-)=J`H|DZ%*Hn-H~61LoMMQs07bygqC;=0G0JSIfAi}!_2KiZvuSYW{B8OX-#3t11xh7v1bbusH!sAs zlLr%>bVi;bOVus#dQs||m6nqDnZiA?iqV5_WcmK5I$CEqu;L&u?(fc-PfN9i&AwE~ z{+N*tAy?J1J}>NA7;_Y>xuZX5mrkP;GcM!3J=*z@A+UK(LDENW(CF>z+46#vU5{rI z_EXs(>}%z*)w8eQto`ywjgegbARC6CYo<8lK|i{3$JgO&LvTuL)7Kwm@=N%=b&%kn zlt1WZqz|XByXjJoSP4N{6Qt6>)3=xCcXr|oVAR&C05bSW$Foegq1f>R~ zqiDcM%^LL$yPgts{aY1qvUHaYeWnGD>$^dfke;}EdNQx?Kd!kPV9NRJtHP`G&;Rb* zSXnGT0Z2m077Q=JO3O;H^YM2(l_LI{kfJ)1N|oA z@qOJP6h^O&Q#wL`4!&?)beCuC72FuL>-m%$acnVunnl*01GP?K=MhHB$D_IF zZoBd3PMDv4i(6*&tZWFm@7j*7GnoplQR+bFeQAF4UUCIjYVXuVQ(6{!17#d5B*aSYu zpl>ey#GHm{>ExjSLD2ep99H_IW~_9$`FD9e_bxL@lr(rPeokscfPqIfNE9P*I!NwB ztFWe$8`8P-4jZFqjh&#U77|Kh=Zl^W2*x)jaOJbg*;f9|DHLyKGR$%v|6L(GkEMXb zzp`bxg^W!F1#N2tE>(NAoQs&`w%ErH<-H|$tItsfuJLQ*eM-qe zsun2k`sVihj5Zy|euq*|p>IoG!RYJNkqPPTinOB~WSfnl9p6w{pOVk&9%G8;qd2IZ zyJM*I>34_W{)&X$KxwxlQP1*W+}nKZklKK3$%@WRp0hhqoGwy7+_j#>gMa77daI&P zczd{LK{RE?xOc<{_(qIziCte~aWEj?SYrmPtuhJuW9u%yXUv=&uBEdc8f!QY9*&IX zkhsRVTOI!(yYuc;_^k+?BIS@07V3b^nk%!A9&st309YuC3*}!^pvZQDC2r=Mm0xiC zChq_n-`<~Lg8Ti$-w#Yq3ptQvQdbWK0t-On9;eQ{JW6;Gd^dgOCCb_-!zoCKZkN9%zukZA zIm1`*bhlDD;tGJ&E_%}iZH-;991u3Hzx}?#O5J3-d9$NWE4OM2YiU#6c0U4mN`_Lw ztVie8NeLQL!7g>W3^z5JLg$fy-@p0gZdhT)qm1eNi(K*cz{|U`&5w0jcsDV~cSL^2_ z$U>BIM8V)7SnFAf5lxWqlTEvDPrA->w|fS@PBnJIA--C7f~q@|f6`4?`TWa0a@TPD z-3BK(=7l}QP9cvX;uwAlUC^_TF%Q$TJu$qeP#IXXN2Fmp;GPzw_?K`$0%A-3<+2xc z$mX}c2`BdOT=lN~-(^fIP7b^XAQ%t<0$slWWX%u{xFkJYr@kI`N}-z%_&{lQ78PN> zmwjuds_R?wFwh#)74|-%7H+)#-0lUDOLDIurS1J((MZrnnab9~z1_SYkCn#GQ_{hM zu$zQY1YxIN59m#2?f&;QJpx^)OZ$wX3MqZw+JCcP21;B6IjWaZni`XrssXpGkeg1K ztCNRyNKAg2-PoSrnA400-ly>h))%1T1kG<;C)dgjM&eje*E%zK&i5S@;e)9}s#)3=}#aHS8&8p$%zj-v@WgwX7bbn@YyMrSkbB|m7JWgL$84UAmiHd=->tij_i<&;+hB5>EKMwxGp9>}kmO6;#nr{$a1 zH&D8Mh%(tJuhAT{6KPU@%e_2UFY-jjzkj>mEY#o* zTTIaLtup;zAsc)=&CSZzM=giBCWh>OOFuj8dO3&*+S|}(;sN1HTryIHXb2a5d=wvT zT2b+5vtJ?vzoKX#*D!Awb;RuvDaWvykc{Di9f|>mowIOf7vL5xf+>TWAU%IxXO}$y z+-3=Y1ryi=Y=f}Z(S=Jl`USGC99+N}RRRf>K%vpPc_nlXbX#e88E+%Q9vDdnOH?n+ zjZil(KQv`Z5IyUYRgz4W<6!bW{$fmce5}0}D35lUORWEYk3CQ}nqR2kz$8Lbuqv?v z>abJ_GA$1P7W*WLpZ^gv8r%JsysX+Q#4zyJ_|eD9s_Z!`5G~8V?gHkx!o+vkqygXN zH6z{juI8Z@U%%GJhn}1x`S;0SW*SB^{HaQ_sb80#Rv*{Q{;6G8pNU_GV)o$BB9YOb~ znm6osJycGxR9?;++JT3*jlqzPHtVHRd;*{KbO&GZzhSgT&dSuY>FKJGH@arx|1ASB ze)UepoV-!-!M2tP(auk!J7iwLIsR(brV`CTMrhu_G!+Ul#-dVG$*KV9e%U5B{38gc z?;QCL27x69PF~<$S$k8;6C4FerGd-GOXC;jHMr#s=DAkM0-o*!%d}Alo8pQ#ip;%^ zGuirEQGUJIj!A8a@uyJm#EmPt``$IgZ5*^Ss{oK~IVW)WZ;k9qAIo?c<`NW>e)krs+KhBJ^TG>$ z{lyTNzGbl~P-SKjMU0ny;-40dIMB|n1(!cOPxRK^GMPo|HnnWr1KDzxa4$~g5$f*C z8q?jXuv(=c2E$z}l90gsJ zP=g-}t-na|X_0b40NFED0Ska6b9tZ@Fk74pnyUV>e%R!6DbzIa)mVpKU1 zDW^v!2XilUn1+3{iet4sRP{Yekp)l2ws#FK>X?I}^bR+_=$3jPv<7*P-I_T{*2+hp zISr&qsS%pUF4+yLho&c|9#}s;Q0`RhW)0w##|X76(pnypZ(2^NH)$u?QqGQkJOkT+y0@i|agiuB`>ql?Jr2 ztx1|_A#cADhui0-ezZ*1*Qjr@ragq+uC(1bB`h-9@1!Y7m`xXaJQnoceQ?_$ef?co zQGaqAOS3%vm>7~5d1vPMQsOj}4^q*Ewo7fb^JW!kw(t_Rk!wCZb^7zq zHsY_;Hb0RlE2;kn7u}Jyzg2)2+5JJuI*~DSO+(<%sTj^d^8YXE-^MRR(1EuCsx|+J zW_LxGilS2Sa=p(~M#$l3j)j2KOn3yhvF3 zkM!IAkMsjjsv-6<0MGf1Hu@2fSn`@;`xQnRWL zUCVh2RHEs|^HF=%pZ!EtF=Vxe`wiuhr(w~bT4{qeb%dI7*1jv7U>;v-;Rn&LJ(D;P z;6G=Qq7at3dqq-z*g0vcrjmiwzBlO-o^Z4I-hf$V!e_=>qDjzD&E@s0 z4e!U5cy=u=(vkSv>sSt%>5mE?pFiICmF?XT8q#Ag>+=&E=OsV5m$3+Mc#0PqF`gY z!E^3moOr_?^Z1(7?E!VKUhuGUo^5{%GU*P#Hi5ND05fdQ09$Qm=zpPk60|w;+2+=K zF$HNi6F9GzG0)A_j-x(*)v!^2tr|pgU;tENvCs*{$he-V_&+Kg>qbJJT4J(cS_1zq zuD8Z7eZBWZ6zSi@*2K(08v!IdJ15{jm~azV0HAA!cBP+J*1Rzyvr2OC0tjH_K29W;j#xYpSf{Ns7Psi3f;r3~ zU0dw&Vl)lUrAb1<@>0vc)WmCKKfb-AcMZR32TO428Q{5oGd?mQB=&C3Ns%AR$UCQg zHJ>$}1aXDRewlKfi*jLe*2ux~-h(4}Vya^`!_9nQ5s%B(kf_KJ8R1z!){ShNA@BDq zD6_-Ven`%zh@sHamopn7Lp1+Tr~AJR3eKM=NJQaW2cn{z!94}c%-dEoIyDIAjUPWnt8nCA`j zUWPtAwrh~Xry;mo`sa&emewZ)(Z3qWH}b#LfwU$^_(i$$h^`v6 zD9WelBS}Qt^TYpMVl}Dn(ra5010Ql~C3djtv_@6QV$=Yx{d`z*5{;<%o>pgJ=zWvQ zmQoY!UoYQ!?nj{M`_%R0m!Hk$xDV>>kz`K%c7DS`;_zsl?Q|>#MtFJB{%>&7pk@k5 z?QwUd5PBbR?*A(I0*EQe`1T)6mh@k6VK^Hrzl72tE5@tkqJT+j*f=h7 z!p*f_?-l$`6A?yXbCYG_JiJTn0kW@8i!;9c24iu}tA`8ol@l3loRN^4>l&f z%D0%_Hh*TuJo|l0VK@>$W#v%m^sQtCm&{<56Z90|Myr=MGQGM4b!#<=_KH=RNa3FQ z+*<<&hH#j2e-G$vpxU6(f!LEC)tY)a7>h)>hUmicAl7!4uQ9v7b6fZW{PpktR?BWp?SueNVv?| ztz4?tw66##MOmvyLx#VNiZ~=8Yj9ENXg~KxILMmFdF9i_f2}>+|F!nZK)XmvEm&26 zbWEeeZWpg9iyIfCiZ@E@wnbLF?lAfqSIr>S`XB@g(qs@ChOlBc50GIdt{2h%>m5Nt%xwiX|oMw5wz#64OO z)@>XWLl@_(UIjF@Sm>1_&j8E2-t;(hM!=Ct$nD1xCHFtcN=1H)<0ezO__D1=zh*v~ zc|wu<$hfS!Ryd39v(ls$`|NI^Uf?Q5x;P>Gf)lLYC4Cbi>M8;A*NEWwFevlL@usXJ z%g~>}_3OP4wpA$GhBtt;OkExq6yZjs!G$)2Pwz&rmoc*Z>C18+4@J-e-&?V&kb9bu za%iag8+%}qmWNh(InSe*>YCpnO@Uk()yf_*8CU&Dqq8M$_C7rex_?&a{5c6hrhGBl zmuYg|P}(CP#U~oba@`W~(BPNndY~+Fs6rNGGW}s+M4MbiB$J%IA?8S5{XJ}hK2w;k zOSoLgP51qO-mR0V`cr_UvK;`RT~8p~^T-^^Tc$n3)Xi#tyEV}9E&FUDAky-^nj#Et zNr4qM9%p0S&}u~fioEe3zAsxG z9v>^(Z*;8Dr`$8)?uzP0Dlv4HS_|?4yDu0SFL@SM zy=>Bkje2S=Ttw%eXT{UOmDm2^6emtQ%St6a217)C(QDIbzlZjNBF@r50!OfxrohCtsXr!2xfavwh{<+B@1l%cE}1Gw#dlJJtDG!Li?QdC=f7Ko>#7b_Ha* zmf;tkyuiz6@MXMj#lYw@5PXy`_Y+Wd5k9N3Jh zxaLq(pAf<27lfe`&kk?nSD)gB%8W~28nMaBUysmH4Xuu>e%z1|M_a!kqdwYHu`l$( zXanb7)j@xcHAPz<6h-?cD-&(ZP_GJ`O05=H6OHKk%f9?^+@Rv>{{U9$@aMVk6{>lBHQ}4 z>qv_Y%`y7Oc|^+t4qn+ka^x4b#H*e46fv z`wYAlXFw~?CwfDM^@Q>5>xyzDCIgOY|NgK1+S|*Y`mUS4)K_utsH?5`yb;3k8_qpf zz=%f7dHf`vn%-$=NTEiY=pT)b5^v^KzgWKcwzmdQBnyB%Hrf}7gjy8GF zQ_Jmo1>G6oWcAS7hCR-6QiLOB9bt^?=$=<-ZO*{+XWfu-l&V%0$IPa}vuvo_x_;^^ zYn{?mYvzL#J~A~|;#9kXVf6!%wo$)aonL*PyH{S5*qW%=P49g4MMWKaMcP=qGumfZ zyo&XB|CQ%W=Fsgt+SDg7ZDO+cu$eqMU`R6K!$BNN94na4U;pY)pf0W-_`-K?RnU*L zP~PCl>akouBiOGWf;wxPW# z;^~lC8EF-ox9>CXLd}5I&Nq8Q=4?&S;Lp|% zPfA6ymFr%tks&nK;KqM`eP=n_Uug+TRXYYhdOc1|Yp|aN!ZTh6gJb;`A0Jje)^FkP zV;$Fith1w~cw!=5=1xCO>m)`6YGNPQX8xho@y=uZl?r@Dx8qcTHmurUuzS2?k~n~9 z$ikJ)Imlq_+IWqrdF_1V@MdqJ@Y|9HZ;f{xlQ{6=I&GoJg$G+Y=laC4=+O6QYd-T! zRU>(RFn7*MN4tHMUr};^!QwBqbFDKCIA~ze#@Zh@WP+f%&AE&=akb#0NlcdGadrIQ z%92aG)Ch;>Fa7+gTg+v~4PMSCTx%V{S0Enw8Z>Um^oN9j8Lx_I{MS1M<@810AK zuPg~GKM(QXVujJ>=ZO~AcgP@Wi1#~j_BU|dkon3*yx&>hD6_bp@44C`c)ua@Tr+kD z-)G=0GXq+2xFNHDsqbDSYsXxF1^|D@cT7GRDwxiCdYd<7KJZ#n>qbXg6cr)C5T%=> z*vT!QoUJ=2$CaLCXV$FcopZ$gm*EyCEu_o+( z;#U?Pt4j9A`kAYB2+9_2Roe4C1$Tzob;xptCAM2>bbLN1q{HiX&!Lz9VCCnLnhJ!2 zc>2a})hLXgeS#*EKUR>8lp4=Jt4>$yk!n+aYWtl@tJBY=XEjRUarLUbGOj$dv1nr9 z5tm(KQF%Cf{o)A2D*^CmDE`m}e#c~va7rhix5Fw&K5d6ZABzt5sAtbryyLCu3bGx8 z*0}YWVZR|m{?6ZXfWZ>Sic75Ps10V1;}?b)w)50x(}?`Z zUsf^V9KXZ$Gj7Pt@9={#t_QkZS8dgi=RpA82vL1^Ae&lwZr}*}kWPcm=8i9QL-zP2 zUNqo$p60|$@p1jCBfScuBX8I0sE)y(^FIiAGem)l8zOLad!`G9T6PTn%s;{5&DZUL zPC~J0!GnUIKYOjevZPI~*M-$X7#`8g4IL~x`5iKVo?`bm5>L4yBV7`!O>(v0)D~j* z8!~Td=I*5X4E%)8fS$1S8!{^DY+bloH@g3fTMKQes>OH6yf$vgkQBX%-W+t5OB*=l z(UX^7j181uP5AeZc>;$J%xng9x|(1rNN~Kk_6?cP53pjqpw}m4^y>s~=7x;YvGp1< z_Xu=9Sm$@I$9r3t*WZZmkh%YrC8q6e?=$d1&4AW2ZpifSkm=Vs17gh}MxQvAIQ_!8 zmHS-xVqNsR$y_PHVY^Om6Gy+t@PU!t<9i=0Tx|Elk;QL2o_(UDp{$$Vqzk7IkHie8 zj2_ycW91`eJ9=UA`zthuv2i1R3Zyq9v(+1`em#4FA37VvUPa*HVWyb{LblGPmLQl&|!T23{@qLT|9HKb;S!`_z{1*$o-R;0UwlS8R{RIXG5uB2vlC4({*~-i7sD z419--2FDGXd_q8KQFNyl*J0%8KD8HofbKVB_8GeO?=$dX%z&00ZpeJ>^}jcZ-q#id z`g*Xe0h4vz5O?Ry8#1wOJ+#$Ay(+A|P>3Xn%|fHa3u0x~6MC4({=(9s4rztX%q0=& zgnQiUx%E%T9Qe&KH6)pS?}6~7w|X7zTzQ1iSivd|Uv2#Hsmr(_^Ky*++3lbH>t`VIoq!o#_xN_Ks~*>T z)xw4jc4dNEEf_ep`jN{ctho8J&Irso2(gYs8%t(kSZBp0juj3ab4D+T!PIj{empaB zCuvq=c;k0X+E(is93d_dqFDKeV|z?-Z}x&IR*!3zeQcI(Ay9U~__Mb(W z^O-dp=SXAreO}2BGO05!u0g_wm0R;FhFPAU0Uvdr^^Asto-96Wu|cieiqnz>8!fF} zQqBLns_S1F>#-5TYMepGxcJ&y*6Ls62aBru%7yxeKT>~XNn(7WU2SdL7g1YWe!$A> zd20!$832`yu?u&K#+97{;hrQQVL#Yz$ozloeFvZ&BoSC7 ziUOi2L4pX9N`E}y=8a(*zcd- zovwSUPMxa0Gks^eUI$Yry0(H0>>kXJ#^4>_JttnwJP{My7Bc6e4g8PuAHZS$B(#Gg0*AhA1mkg1;{9vb*ew0ANhH)VVI{yobj}i5~f(%BcxAaC}+#CVSVJyge zq;pJjY3~29afamn&w0ZA>YHM^mRRp$oizFELTG&UA-35op7GaVoUbAxM9AtUZ&o)N zXJR@bWeT)i)VSCn%vD^!FRNDAMm%%JrI??lqC~vRqClQ zyRKqZ1zTE-NgORM*`t+cm&}3M7~gauKA9jhE;;6cN9q%sx+XefxwXdDxX4l3!AqOn zKcn3H)ArL%gRn`?Ixg&S|ACz^`cRUKg^v8Q%YWD;PokZNK9n&&+hv?xLw#W?+X0?7 zJFs<(^1CqE%&~ZvR`CS~dDsGu*!T(S#1VyTd?^@e3KukPC(8Y67FsFu3|bV8{fCh_ zu7Kf1_D=pllySt9jk%)lc)b6{rK`4aL3}d9i@$$yhRnqJM@U?J-n=CCw<(Bp=gMO}LZna(llX?i0t zE{s4d$nYLA$h)zRhYcw+MN*m~XuCNE4mMGcnM|LuR2O7OK~{dDL+`fsd5V6!`hMROvTY_}-ODrN7yEt<`zO1DnLw@E&3AU11;wD(TNRvN*f;zt+#)EB3gg9mb z4&Cx{BlLKGC+M9aqa)EyCrfre`hG8T|A*<^rp6;>sBXQ(=!c+x$&T~iEzf62im@HwDJ1~cyJ*9zPMoP znB!YK$PZpFc?PMg5_VRw=emd?54CWs!pM?E~zLaJ+6yo_iK${)!gF_W@|6yF>l6!|Bjt1hX z=le~4{QMLQz4<=T0zEw)!-5%lK-nunT*L^DYO)apTXQM6(n9>00k;R97CTXJ_(xoF zsZ5j#GB9!e28X@)<2m`E8+kGgkG)36tO@Mhl$;r%@kZ_@EDJIR9K3hZbQfiFb@NDX zgf(lEqrWl00IUtPN6;)Uz$;<)f^Zw0a%b(ua7fiA{W0< zNMjSQe%ssq8W6_7{<9evbm?zWx66FB14%x7Z55oiaE45;AftoSTY4if4vs);K_>0- zzVG|~ZwJKsYd-G(Hb>|-bNpDLl^aF^awL z7JFBbxR+Ay@!*5CLsSK{IC?WiGYV6x7k_M#nnM(Ga& z9P@&|ahNh`yIdO58H+c1O%e4w#`)6c${bksiJzxBL*~<|Z)!ZhqKg?7ZD-5<;-@`r zX@;VYeGCh(Bq+?K)tP0qnXCJkrnSoF)baG9-eLpU5+9j(R*3E%@Fz#ilsF3-R>TIC zH5os3{Ao7Y;qQw8#zn4*i3J(hb0G!Wr=U;5Q=HnK;8eKEco?}l<1@%h8$F4VL1}@Z94bE|Iy zb&wv-_3l%~KjPp|PUoVvskIDfO8nO4{A)bX$H8=A^oa0y&8TrpBbM7kAMU@H|AF(z zUQdzFkOAJtS8YH5!yJr5709jRXIdHbp?bx@P$E$s;-OURRtdC?h90 z23C;4WB)uN|FXjQ4-*tDIiD1O@w(}AL503_NdC3xkB0@B_1E=Fmb&4|YhMtIo#$3n zuqc!Ai9bYQia1r)V(1F*pSAT!3mA@#;dL^{!OHkk^zI?UvG2*=2#iZ3fc?9^WGT)c zY!AZRp*wv(5by~==LVl2Fh}q&h@2xh4mN+Vlcc$)Htj{HoKTt}wUr#*cu-n%l)~ z=)};HC@l>~5nB+RV>i89v33uv{rIX_;^)^RD$0P0QDPq+kKioDmn~TPvcbyz$G*S0 zVK0)ZF`5!w-jANrl=bUJ_ zadGzk*GdRZ<8^ZG9=L)|rz`7t~b5HpXKvu^ zwmqlk9XEIs4n0`tb_{@w!0m%YK3>rm!!a-c5aN zqldX@^q>CGFXuTV7@(ABKzxqXK{X{b@R~kzYOVKR9v@ zMVx|mLi^ord~SoCFPkX+DA&P#tV_7mLJlqXnvR8SV#D!oi+8Q6g47$=fOi@h6EIpS z8?JqHB1a7aS8XRh-#@(H5tH>niKcbwto>ZvtAf(s@QK6R&_*M1B$IXvX=yd{CN_p* z9OwdVW~+T-9Qf3B=zMXDt!t15ia8od!r14^jE<;X$xd1FcRbeFQ=Pg0aNhwRVmO8@ zE`hap$sRHFb~imcd&~=b%ps$cxO4v~(JY4yZqK!#k+XYX9rb}C4>7lcILbOEeQ9UE zP0}~Ur(d+uPMiNmObC5bi1iq$2f?N~S~C&&7;=l6nQ=_KQeABP9AKcBhg+My6f$F? zl{WYxPV560vCbXWZjBiFL$ZyQaYdyn%&s!}o05*{gH?|T%`p>i)6!rpnedAxIx5u8 zkkNw7+H0rBUdInP^5YkT(t|nn8&n^Kaz9exfe1DLj@TaiplATt^BbsbEC&ofK;C`- zbr~KmS>h%+!q?zxTZql}ydg=G4Z&NTJ_`kzS6+BSo_X{o`P?*<%iIgi91`p`KdU1! z^1Tn_!8@Lk@wBSB=(1n@tfu+Dp4U_VctM_i#dEI9=1^MX%vzeQJ_1b#eTfEr|?*BUSUOX_Oc8Lz0g&+CYr` z+~X@2Mgna9Z0^&*8z2x9UguA}`;5gjca?aT z=>|1l4JGs+Ye6P|iK>FvuKH!EP9xqwqr?m%;N3g}(g3fo-rC4JjxK+dPnoxfnm3bU~XO=qI&Yns4emY6Tf^ zU?H#f3NQOicc3^hnh(BA^wlfKa3p)OHv;3<2w<+`88R%$aNgm(5Vy!}y8pvyE7ha} z3i?HSlcVaPo1HkT1{Fd?GWJvnHwzDYbwEQ($|gsXAMUhM(hoK+uwmANE!%@b{1FbH zRHG_bDAN+$sLB>>6JteV{Ep*DktgEN&iPe_8eijTd*!a(CW7P7gQ1$Q_HTOE*6#ht z3l95kP7-4QJIACu3oOkHn%~=C;qi~hFpm_R{M^>a<~jMeGgnIBxRmJHGAFDQnLfY7 z(YN4fsWjR(jX5ww$C4v@rf+Q%D;vv25YuF7#fOY6q zvn6a$(~YX^sUJwQrB$;a!_Nu4+#~yusEh0xPw}-46y%LIZ~NNpXyS#ADd<)F)nbc# zu~#HL+U&FeO*>`Q=y=(a;kvy4SV9vjZMvw7hTohy`Oa&R(Sw8_7IKJp_7<5v+ay)=~j45?R zXO-9(b229P!hkkD(26*40GBL*XCRBcVIB%w)NpC^6w@onG=`_w8Q=)?3Niy5sQ-3B zhP!>8U1PsbMq)c`GyvCObmv@<89K}r6^;~?D46h3gK_?IyFZeYe|>erG{-70Fr;VU zg%fY)PfoOF4m#4FB>ixP%*gcEYlqT8h9&`f5rsDO@QQ7li&f``8%-N= zX1h8Q2OWMtMoh+OBkIwnA0@EFP|^lT9VA}Ruld*W0R(a3p2Q>X)wc|;Z^dXM%sYl;F zckoV1VCiWp2ROefA{Xq$f(r}m%!al^ZFXXE#~RM+2V;u1|Fu_6v5|xG;6OU$*3ajU z+dvsVzZSKNKXaxZ`%%&svyW}s;lp-}4?PZOvCwWET}2-1Y7F(n^1YzGQ}SIJy?Ed) zF`H4Sr?xX^*HB+Fa*zJ~>)5=Hj(ht4;26fB^ayGGOa%=LKkv)PoGIILNo+~*{e#ay z#=n!>iKiT5!Q6Ps()#+8C5JKrdiWO#X((YOk`g-D3#QD%j&Td^l=OF5 zeB|Sgdj%N`YH#U{z&JPp&4SEZIxldK@7x&90hm?LW}h<#Jt=7?6ZPh)M$C`Clpn-Y zLDC-*FX-Iv2c+6f3TjQ$*n(~x?&0`|$&`t@wz=K4#)0N&98ktQ!0pU&&25IP(eNmX zdzF6Os_kHFV{F5biDSO6&g*HgG@yD2g@oe4h2)DIkxV^=z@Vc%rvqj1HxTw#zhR; zec8zaS#aLM8ZYEka{`a0_;?@dh{Y$BJdR+2(sA^sbbs&pO>+}rOHm@mfnVWB{#2y* zuV83^CN4E&+7^~2%-o3~Lt!o3477W6_oIK2A%Amo-@;#zu|@lsE6DJsF)2#C$(37G&zDES2f2`N2wI@*5fXVdDnIHZTpGf_eoRj8JbG)e-0wWCk{3|CUphSg;Bf|Q{vxCI+qqPm4Dm1Yh*IMVo9DoFUpUyxD}Dc`PhaF~?g!2s z$Fq(B1pN6Ka63Pd+Qa!a>O?#$ucCAk96I*4xkc#M8f-gZGF*FY?ldl6LoMt}T8d zg0O;SaI6c5j7bS-*m`-H@5@bF z0~k7gIO{g~%QX+mQ~!8T7Fuc!S?QaL%2F%OC+8l2qv~FejkjAV-=O$Um)$G3T={^; zolVx?YPrDT%^NSdM{d3PL0NQ}xn-@*zS?2G{kn(c#!K&&NA7!0#vgAyy@~GJvizD0 z$--Zos{?!SuWykD?|fQ@Z2mQwchOno;G_BdG?8a%iI5XPgYoa5m`?C z+m@q#dcNR>y=#B>tFp+_bGJcE*Iz;Ak$az$^H07>9{c-q@^8Iq^DJ{tFJIDe`oW%S zG~SyS05^9Y@vB?p{@b6FNhg~~mRaLVn(M-v&xdlwnYSs&bh7%!OGcMI+s{7slAL|q z4K{AC|NBk7Y4l|Bg}J8J&(6~Fl~v|%_&)#CEApGeua+5Roko7R*Ei(;+n$sw&$&Y$ zyz6P1c#;WafhA^>4fXQ@|F-46zdSBCUH*5&PdmfsWU0aP*=PN=XUFlSn9DhO63RAaIr z13tvJ?PNTizinIc50)~E8!qKTIMYLRIH2nlWXg#2TD=h%mqs8KWIlKsXUJeF`+wY) zxc_^bns$8jb=F$reX`KXIj7ds4hc+~oxa2pL+QTS)BnWsbbVl`Gf5}yuo*GV#YjaG zODYKWw)JzP&Kw|8LgUw<)TEJmP&FYJa%#|yCe!HaJp zvT%$u`MNoF2}6;%G7kCH*M* zzOo<)7TPGC8SCZ}?XWNgWo>VIdk+}mqrLX+#3T(3)%cUE=Aph#lB%|mBij2b$Pi0k z{@yC1TeiE#B0l|SgN|*tu?GwAZQ8=)sIsOvrE|EQvC*TQk~UsePfQ*UeIJ0qnpk2W z+G9SMGp=>$$D7o+U$1R0PCY?~;d-nsTBO7kDlI(Tj zcl5L28Feej{QIA;%O2nPmE7}}N6dG^i6)Q_lp70t*W6@j8TQNX%J>tGZ$*UFX5Lla zdh=a5X2kZg;*doRht>JN_>&y6<5||<>?f@_gixJE&pibz_TP)8nUo|lem~#+LOJ7@ zt7R?So&!(cBHKUy=p#91_lxA56Ry>(7x*r^8BaIbX1{OBR{O4*@o4O};VE+6C3nf* z$8V-ZjK9jgw>@U|ZT4M9o_y$ex#q$ zdFfxTM72ya?dRnCyM0}b+4Fo^Y6ZRN^K}Ee>GO3L{!RA!?l0v-eZR1@eFpbFX_Fia z1(olwaG1=y&}_2t_AAK|dXwq+nFVmBS*Mdz^`_HxQReq2{Yj48_1qkbq;t*p1^Lof z=8(&NcguhEroQ$`N%IExG3G?s{517HchC-t&J}Nydr_N zf#wS)UYmL;pnYugpY!8&=ri6|;P(p_#VqC*@)54znm${yLde!{U_lJR3NAic5`;wz^F$87P6HjgmG>*Qf7KMR)#=;TC&T z`cWc+a}*7b&0ha`ID5gd-^LYf+%D+YW^790xvlKS{Raux2lW}phDyc=s*WZxb$ka_ zStKytetdfAOGyq&2BV$*ZjyclUs^dBoU8aXA9%`~%{ct@6%DMu5Q83l(}^qwfSs?1A(sA>H3m4hX-ofGk;&gkK+_Vvi7)w-Ruc#3 zPEA3Row5x}Ta?_kx*!uVXm{|@F^wd;cn#89$Yc{7u}}dLzRVD8#GoMaPD??CwA?Xj zfA%38$y!#?+R|3c*eJ-Hu+PQv>%;#bbId!l?5%~DFYB2fALxPcjhEajNB;Ep^4cr^ zk%NBwLs@fEKO3Z5L1v5P4!45BoZ$JzCbGy^=ae^Id0WpGxlfMXmHCzf4rLPa&&p3&7R->r4=GD0t+oUmz?pJJ@a7Pa^?N<<8_Xc`4*X7 z&be!!tnb!1de;l&%ww;SDL*%)8Ed-Qz9CtDibtrum;KvmY9V?r^UB{*>!wiRI^)Uta&y zhOO>&E6Ds+&l=ipt)JT$=nwZ?%gzXze#WWfetkst$-^#_+irMJrk`;dIsd->?J?)z zef7ov=o6cU$s4b|C0|)-K^dZFuq>cwG5!7ar{ovv^YRO?T0v@{1(|o>9w}d+b$2Vy z;j{3=z1NVbrkz3_z3-oPrp@nuezk4R?{3*k3np_~n*|vbZWdc+Uiq<(D~eKg-}0Co zvGaMB-%4vQCP!bit@&Ve?fG}f9-EwEeH7Hz-*Pz{SG-539(0-I0ONnzf6sT0hJR?6lC5{H}yrhM2lt)1sqc*IOgK?c5&ha8mA#yFJSFlD?HX*uXBAB1!=Mh!X5E#*c#WgVaWY)?P`$>jxl z{bPUu%DYHA{UI|)P!iXdjTq-hJ&N=PJQT#h!S5d)aN5|YK5{q@)>zyU&oFU)b4~xg z@}!Njus6BTBZkp^Gd|IAq=h~_g!VqIl=hbp?bP{ad!k|fl+hmBw3p~?wabWSn|ze? zbbCTU*b6KD-01tqf=oh~8#u@l^=9m{B94Ad7_F*B(>~9Nw9ij-tfL>ff_F4`ZPSkc z+kXQIem`R8@m;vYC#3eT`^1sIKf7Dyw+vkrM_|lI*6``{! zQzdc0@C6<9l&Z3YT$JWz9nGR;|DhjbCl^_omS(Ro5KS3zv0XgcTNmn6mL{6qFIj3s zbZ{}Cj!>F{(UL(QvT~v4ed&;6P|=@M zg0WGM`Tnwp+1W9B9=oZ0Yr7R2S+F><`UXqMq33R8wzh%{&T!c6+b7HCr<+QCf6spM zdA$rIE?1s?yX^D5U&&;8Db)FQ50#nbm{!g@<~liQ_wzc6E^E!Tx8NWh1NP24@5zlX z9V3(IB{5i7^!VOS*N~qawtfuJ(SH5qH|3l2?S`OKN4 zFW62mWBRh$xy_?;hRi?p@}i9v8?I*reI&oRad%lri&1gm_D?(g)N|!0e@DLkOw@BySUi80vV zOT*>q$DWtnj{3HIXUCOI&w>m(t9)|_IsE*s8U`;_PJ1|{hw*%+J7EqK3IKw zv4iB1`=61GwjOL}ePx3-SnPxDJKL`+yZ!uI(LCC1jQ0`~ZDTy|e*B`(2b`4>J163L zt{9czZSx8XGPs9~aWgKYz93kTVGeCE>%7p9I>hG7{G%}vhWMC}@s|zzGPG}WpEt<^ z$t7O01YNYD&v2BqQx*)OCcNg7Z|VjBe$-T7h|QTCpw0z9O2(gJa`S!f)>}FOJxy-} zdL!_e9RbYGeC&0(WQpm<8N_v+8e+Rk&P&}9P0K)HL{83W@S`s!nH`_J^vL296k~v| z3NofMi?1S2j2R$}X2e8a2PG>6z6GcR<}KY8Z)8#dZ0=}By@M?Bil^7g*lc_YMLW3hebb1c-$xNc9%7;mZR&wTyS78mC1U&1!A73i$MyO> zZVf-6Y+}4u`!_L+-d)lk5-<4o#nG3ojuN?Hqc0_N1(RAT?2HC|BW?dta4g6G1^>ta z9xUXFHngcXP}fyLLEuA#7xo_uRu~g1$7jEdws;jQ@f|jeXEt5gF|So6^fPO~f~G$h zi`Mz;eu>)Iiac(04K|x2MhtlQ`)!~FP5MHzZ9JgYg}Ks3Nj)X7v{BM#vQn}OKX`I` zm_O~D4anvA{@2C7iKU(r*2o>(P0wa;#Km5aS$mx!W*{H^Dd|Of-d1d|zB&prwPWrV z26CDl)~z6u-i~eqNV?45VC|=)&SRq>v)NY>f&=T7rrX&L)Q6w8h2Aq|X?gDPmt>v! z_Z3_s^QV6wBNOZWW>DA|GT%P3%xYddx%Jw=%W;?NnD3#o=~9QtL-#ysMUO?7{bEO) zsQ9(s59EaXM#zGT%_(QzzE?C^n?(wAMz&a}I{ z{@R;z_&M9isv9n07#3u{y!5>C>zj7%@H=GtbL9L}Z;-mUaqq2<%N8phW(6)>x)txi z(?OyNZY$oyTV6ljAS`O&ER!o9JGA2r9Dq(4ez}}-(4{i_7iW@-9vW_bD^9nQ6}jSB zMP^7V7HsOABPO;{#ChnhXJn??r|r035O6DOkbjgs7y@S)9rHR(1&p+8<&h=>J6V&NA#(8of1w6RT_ zBiq&q0b#!(wb4?LsXfU{2|Hs_(kEgf&h35gKjfh7##P58zLvE=M-Tdr{YT%7PTo!p zVo*zdO2((1vbNVa^ogOQFC~2<)^+MpIhC$^w{b+i*rspP4YXayiCob(Dg_yGxJ;iP ze0H(_@RB7~&~lK}pffjsIQF|0WN3%XoKZQzHsd!qR5>;ZGG`wBN4um5so@NlWxuhI zd}ZbNW!W_rl&PnivLoZRg3P+}?Jv(f@j~O$pW5-eUprJDxa$eoVz)Kr#|QgmJL`XG zm^}5!zvRTrce0!Q;>N;1+u?Wed#Lp*F1Mv zu7cKa*dO|X)9rpB>;GUy+4q!9sm>BR@=7Q!=w(HtTaejd&Eo~dBb-UJ#&?#=F=FGC z1Fo>MQMTM`ZQ1UCb2Gh9VO#ySYrzV>H(&90yKD+Yvs>RdPA^w_Q&yjS zccVp7qrN{23?ojxSq|F%cO69}bLwY>_xQfQK53Vg{ZnuJe7D{l8U+$8xa6!Z$e>Vo z+uJ8~u(GfR!)Nl6>Jv!fux zecw#3_9hLDNugs7r&~kT1sR{ulo0=QKfpiys<47k$bq3Kb8<<+lyWdku)5TRCF00q z5?K6C@$r|X7#|qo?MH|9%Oy+YvOkN5l`gg&*2JjVy)$Gw$Ec_2jlgGl1TYWr3>h!T^goBxi0<=5ZK*wduB%aHyqcD_Lk>p} z*U1kqYMWS}gJIRFE1=@MM_zJtX3*qhB=FJ4+l5J6=-5lY0C7N>3PJ_rK25fmS7pa{ zO4^2+1IWn3?+ObKKeg7EEWyWlh=vA8XChyT#a7eqUd=Tb;epQ{7JPS*6D_itQCH{} zyK9MoWE&P6N|laxOR?1M1}pUm1y$hNSPZVV!t_FZ4puSfB2Kb8GRkm&=3>#k(YKL(T=$2k2aQK_jiW= zeD~r#WPs)S2RpHmhXq^cj9s0um3mWO{7qJ!N<_yfixYbnYa=#v(V-wy-$SNYN3T_< zl-blU+4wWqqejki%8_mWUM_wzxJNjC^$?fZz2{((Kf6>|PJ{#-ly*9Yi1sRcQEqC4guxvYY-Hw~-691jW50ZcU{h3iJ$ndhM zcvIi?78q)m)sVL@84vbZkcs!!seM_r+UvMYWut9ZH1pbD+(-WP^uOiE^R|_h*Ka-r ztM*5~tsujD2d%i)VsgyId&toXP?E0NBhrzxL2N?B3!dV&_Pl;M zAqz4*iwS3f9e4Q-4Lm!i9(aYEe844R{S2A6^$eLfVX@$VA7u6Z&-sTrnrt{XEIxr% z_V$7d{B#P{fO#UtFLHfx;sgy}7KO=G{Iq2r_2QQoTxvJE^G3SaFU2PwyoOC*xI9B< zy>;=}YqD7Mw4!I5t>Cg%dFcU(@VlDb1yc(=G1dF=%i4!d7@4 z14L)^h+~ExC9b&*uG5|eD8w!EdNU(;fq$*HFEdke#A4*OcG-Sdy@=7dW!!$|&aGp7 z>WPaC#B__!&4(RWaz{M*iHVZ7cAmtSzCZb$?A$TS5{-OCLv+z}!_@Cgw{7q4?=fgE z{?wElV)mWZX3l(H_`3#H_?Pi^JaK#%Tw8jKQzToCX?yR;Os*o!)|Hm>fwKG%De%xL zQN5Mk-sD!;N~<`-qxhF?Gi?5xxF>Psp#+=rlo5jt`%ipdpu<8GV}mE+(Dzt%M0l4R zy;s)YB`36{2wC=j+L?PT3mbDN8e+>9L|d^IZ6^n8Kyu3>wr1cswQX_4wsA2IB<)3d z3moK$;}4&J<;hznxXx@=d&*eb{%1};y!O&{5eM>^xN)sN`CMfkL!@9+8z z+3}F|ES0_rGH2huU+ZNjaNK06gXQ6So@(5$rjOMRS31g`ptSkUUzeQ@-=Gz_#_W69lY&O& z3>g+t*8I-Wa?lxD81Dvprq7dlrVlPz!r4WAEek9@t1PzsJO&|GUj><8482T#dGKX+ z1KpFa-6>zLgZt53_4Fb3)T+MX!PZwnX1O&MmE$hkzLnyJi|>(LHaf{}Ec}-@POwK@ zZ?Te|siS8pUH{@ytvWMBN!tn!t?^UHjT%_4IxFq6!$k2U}4 z1}DqSSKcc-9Jszch8s9uvNYWc)7aTk(8Z;#Aj3l1R|hXBzqn?n2+wUkK`R!`vT-yU z_`%P6$nXpq?C)(0=YVXdg0n+EU)VKgU67&O_%$!kTZj~pKV0Jcmvf@Y4WHirGWH(cZpBzj?FUD|hpYw@r z;#|w^L63IGUO~pkr?>8nz_>U9eHCQ5J#pWsu9n0=a`xdo!Fi4oN4Fov@?<$U#h#xoEw`jGOv) zZD;&?`{&Tyd&bP0lJSYAK8__~(N27~`1EHE5l5Tzxh3<#s2yS`X}8Z`QqqRc zMzOQyynO$F#S8g25ss_Z5yLnU6YbEqVIscROUvxqC71tHV6Piz)1SX<7zoFb-*qfzM9$hCKYwKoCA!F%GqiErX``eUF_f@lWIAxF0H)r5$P*>Kohw@W z`)9-@m)cEz*BO%TA(MXp6UpsIKybwE2gb)hKV=_&{E_Z=4yTg`^-VBG%@_{EfdC6| z!&5;9H}&NJm@%w?!_uGKQNAXYzR>2jZtUda&uE2;F@1(i%z;>*Sl?TV96#>bq^U9lExH z%%!LQMTTwp8+&y0xp%i;!gK5%Bjog>uacQ(pI$D1>}T@YOK-|5U)aU?|NQE4GTD@q zbi^9I74ZvR&KIzxvv1^7(0|)^>ADYaD#r7T|gXnYNMX>3SnDZj3-I$b49xA;Wo~Z7yru zx`OkT`D@co%vJ~7uZuLDWmHt{+x9^O6p&OprKP(|lunWE1{u0@q?@5TML?vxyM|6_ z7`lfXa)@{C|9ajJe4e#f`?~gd9`V~rYOaIMjmuPe2P2P1B&NzhP=b|`z=|IlWNy6n zT!dpG`*_nldZL6-`|TLxWL1>X@0hdfgqEGuL%A%vD>J{X%-Z~4o%4+Y9QsD0=3$|I z4Xdd!Dl3x#&*BP@*r~qo?CQFA4wor{7B2FDrDq3lv9U}bwL6A(Pc5}{mYA}_1vr`< zAjHrCiF`%8jA>`xxbrdWV)4w1v9hs`V=u}3{C+l=X9f8EFt2AzS$u1_S70EoIE7*s z*>tJ57MRe}qWF4Y!I|?1UdDwjkjqckd+Tlg!Gtc4tkzy~(#eL>`vOMh0Avd>< zm;Q8{t2GnW8=(oJ6%uX@IfQbf?;_9kVJCviq+tP01++orfXbP9dHjeaK4W~k2n<05 zIr6kQ=6v!;Q0&0C6B`?U(jsi@P*h{yg>+^msd8E&WU`u5&uf|DBxUhPg9mURg|&YB z<0ILndx!O#!JSqh0#NnYo&|g&xbx-yO7nK)?=c&5_)F)N%P+5_e^+NDpIpZT%qg)6 zdBXfWb=&qwC&C@*AZwO|LrJdS|a^2ZFGga8&XBp7C8ius9A2?5`|1{2)UNcXh8q%Osw!A_kMf zagjie>vyB*BS0gdeq4uzudc;m3)?B;&>{r@m-?>dbwIjY#{v(p3xMKJGd~>w04cYB zTJhd@r`DE9RpfuZHcGiXcl23I+*^MXRW!Q!kXPGqiPg5Ks|e4#6w|#$AGKm7ZOnO~ zE3|HK;*ZaI*s_{bRBZ_s&pDF5A4m}y)z&u5@3{>mJV%>sj&3g$NOfqlu>C#}zp2aj zX3B$MLBz3>(QdM?O5EX5!n}hzxP&!!q?bb>3)ii-+FOd=szlJ!B==mXuc02=wJ>RM zDPN2yYWWs(|HHB~8ipwJVz#wvC8(v!^4abhopEn`<2bVD@v|iF5-O2PwRVwPc4Y-} zDQdJ&y;(K1Los#hAm;wY_Y9OQLg@!SW}_GZm3YqegS>{Q#2oKOTLtO$Ai9T3-x3mb^fxE76oD@;4sSUS1z6TiY?MJFK`5Vj(G@gtXKEoAPdl9! zw#ae+gT?(4Zhe;Hb9gt4&gKW)daC*Q&VlyhAyc=qK&`aSDk=GQm8 zvJ1z;9l-nz_Lf(a#csC?_kBQFZU82SylU@B~E;a^thyd(Q*c=1Fu$&qpK)U!HV zDvRPlV8SbgDsNg`cPJ}P=>0`$1rl(3e`wvR=|h9?T^=)=$35_0qnDrg0Tiq zC#J>LJ!b!xo~}DQ?bzkI?=fNfa-d5tn|)h2NL%deftA}E!nnJEHy~@U0eZH?T?c$n zg?pc^;q2T3$vzny++kV;JZETIR_OIh2!RY!tG#Sa*=7ZA01(m{MIG*rN% z@GEw82@}+(f3S8~rEEusNtnDHD&*0k>8FQ>gu(`P8ne!EMf1KHCL&W68wq@{ssCVs z1o61eTe$hQCA{MzVz)=6 zeVU%<6yp09+PYclvc2m`=0J@~g5LN8?nc zl+;oxnauLZ2f=OXXa*U!$fP-$eq53hbh>`Y9A!5funLRo20!typv|9XibYznz6%#3 zO@pgsEc<4#%of+}f*wo=2VZ6jku>@*dJk#7eC(#A4Pk{EUrf6r>d5_z4}))MyFoTP z^z-iPv_$qdY9q5*k z<8I@J(^$F8^9j(rOb%7wilrti9)-Mp+ifv+pI=oq%s^ge4W_<3lr^9oI`t1+W3-5k z?fQhtF5c!>o`(AbodgXd;(Hd>!TldmqEQe{EXwtGqOT;PPu9-^Rw>#>cc4{|r@qTY zlu12x^jpc&$$pb*RC0T3(et+|)!SBGtnv8}h|5jQ!e~NUZdo2ks@2eS1fOM$mO~Vt z`3T*UwcKetvrTal6|tiyv~c3cp}9J-9w1$1vAo#bZ@4{Q?-Hk}YKOIMP||tzd$NM5 zL9TIaMsbU~28wUf^4T=(kH_2Rv^|26H9xt^G! zvIsiXL4<3^pY5!3pGns+xIyc;*?`heUQyd0mG{;OMcT${#W|)5-sck~*QM&WD;dmW zFE*A-+jo~tuGtN>!`j#hLsWh$;sVz@mqqmczKD1rGubG8YSv^FrD8=sYTI$U$%1x3 z`w}~biScxeAtEoili-`1;(KkYc-95)9IHELbs18|8TN}FV?Ta!61t!FU6L_-|EL9|J<;c~`2H+xXJS_?{jlLO+#bdS%q&ID4j_ zm0~}c1#`6P>t2k|j}f!q=+smA1CCEW*ATDsC$u$Dh|Nmc-eb5t!D?(*>gR%ck(`Q@ zG5UFa5|qBLDLYc4>5t5);GUPIo$`m1)H`ntV?qnuxn4$xXCzxI3NQSLI=dsPg*uBp zLz{vST$V9J7$)PNmXXlDR2S9^nrpxBA`7@bf#eEmBxN-QvIR}cA_9tPu-zd4T#h5* zymxsg3NrvfR0;Hw;L#gKd3qMwiO>kb*_VSXbrcZNRo+fN{ z=0=y=npDC>3X~>%0TPo+X-b$jBfoYrcCTK0#o{l$0UFRN{nn47fHbVBN4TCNJRe$R z`w_DtM(=pds6I+j&4;)b2~|^vjOnjCnZk`E^(g6Z@DiHc>3d*&qu0L9xD3UB0ORc5 zv^~}A7L+a9*_2)3=Tk{tb=u9}twp-isWhcb@21<<0{IoI61F|P0Eq`LYEmb1?Q`Ci zc3Jj~kHz<=?^VvtdLEOy7`bZ%7K>izI-89qb23eJX~q1b*~2y4dB>WOLimw4jXVo( zt|f?l8{b31E$xLv0sv)hkz){bf+w~qZbr?kO63#D#+p}2xRu^@_Xov z&s4AFvQ)ZuQj+Yy1hUYNw5wgdj`yLM7!tDCO(|GUn`eT4i;?PWUC&3eGi@E zkin!RCxW$#Tm4ZLu|Z|K;~=c;Dk|qD^2=@4a;K!>`as{&6EAQpg=#*jjTD+eJ1Z7y z6$_=9PH|R?KdZSxBf`lt0dDo|TdMlMB8;+f&akMBw2<|GylVoA3jVE#t;>yr9GkW5 zJ&Nq^5*gQ-tpi*q>9UvzOL{4qp(Ha4voRlOFht=F&tz@dxYx}wKb|=zO9hmlJN+Iv zhiZ>yO28T*@w~H_-cisphS!EoTPa72dh*hfe#vr%%v=F+{lZ1oNcb7h0%QoW2>)lX zV9x)`J4Wt=s=bs0gZug!L~V~sazLiTG&5sw#O#GQ!{(?NktL~5w=1eI0PfsepFc^B z!sgs)=$!TjG15!wa)HP_GmoK<-n7Qs$~S|enOIKJrb7(YSNrt8a!(1yZAPmQAy9 z9WT*es1a$=_zgqbfX1Mb$k5}^4}m-sbxKF>t~*|w;LW%3X^HLqHITfV=I7u3pYgqE z!%%HxY!^`w8O#0_7~UmB`I(Hp%`E=#ckJre&lzg#>FR&=)%ymaHTA0?!P-Se<&;Gb zEAtcemRxGrP{i@0A0dNXAc~3M0NW=f?uOClE2YOCid8>#iYp6%f(dareyfdao?^e+ z0u9&8AAGx*EY)!5*9v1+tk+1x_@V7S5$*1fFu zX8jcG7T5pVthNT>xDe&}j!(pRon8P}0yoRAlsKaO+}Yb4mE=kx45xLMQ#q51fzM;E zenx~kE*=@q^OLv70{qCd*Gib#*B2)Qh&&^A-O{drSCtpfRx5R%fsxmZ=2~irW#(bY z9OKK_50GzhMb?*^tQ20X4u%U(2IZ}mCYG?;_v}cXS#H?hCVH48C za-iJ!qu3NxxVHIRRS`9c@q-4p>d3cN_3BLu-IL0pOyS01d3h0fCqz_+Xj`5fFz(F8 zzX8_+D^_kT-3r*EQ#!tiyW_?4BJ_=%(3yJgYb!PRr_z4eWPdC55+1sm-0glvsi|aa zkfZ)lqz=k%=f-0V<@=zLoZ~_ZR5_BfKkc64Iek$!?j-oC$gZ$|u}-b^<@!n16j^Tz zH=3h%nsD1!WxTn6x@t!`R`F=M5QpBaUSnR+#B>|egEgU#FEsjzVM46=Ljy3z@wpX( zv~8VTrukB5@=oM}PqlGrC){X+@5!Th@UgJ&lR}!VaBCIc6?GVS3w51GYpmSXz8{XL z-~DH*PCu?JDW8EvSB6(e{#H#|q>(mm{=yU0JQfO4#dm`oJN4t;a-}T?MVsa;yj-pB#r;H-oeQpTErh?cy=1$RPp=90lHS{H&@&*8387#4rzqmU>qI5tjeuTaAhNb zwnZ$HD+gBsPowD>tT;_E4#yd`FNmj%c$)iPNsG~4vug&w$jfJYj%NBOIL%N&lyCUZ zLOV$z`bbAxI+&LJmoPECfyr#QuM#&jE6st5hs4RmZYI&_qEnl2GE8T8@*|3W21Ec` zpv-;9ndWK7AF*kkXWK_9H_JlWN&eioj{ys#O zl$pSfc_oMz&74~KTU+&zQ6aVK&s>@$(N(r3i3(~W`ogU5mQ~bOwz7?Djt0zN1K(%E z+98G#@G9om$9+f>5I*>rR^Pn34c}(;4EUsf5pg zPh)>%TwX?V5L4ahc3}!7LyTnqeQ>si2IPBu-~>C%H&N$DsUdNCW|}3268Trt(=7xm z2dE@ZW}N6RLHUksYlaICRE@3q6o+`fwkf&QibGcq<^!f>+P5?1{7AMU6)%jq@u*fj z2FD!QbWPD%jp8$U8#g=j9V4fVe6eCB5te3>Y)rhrim3d$&pt&o|HT)AVel=a>4ueO zf~jlSlZvFhA3s*fp>v;d$?t&kenF9g4>qg96MNFSRO~~t{kAsf!N9lo!pizlsPi0E z$fYC?+;qFDAwCDMxM1j8SPy6`e@e=+2sp%IAvnh~@vN#L_1d?rh6h1&n1$9XM2wLH z~R$zUJJXmVS@$mIO`t^6q2`$cUF zUrU&)|5vG|Ki7%JbRN*st5@&cs@l?CjGfU6Eda>wFv&UoYdpYt4aIvGbL<}yaq&nl zkKiwU-J5HJx%VT7H!bUTm>SxdF+!b*w_2j zrE@C{2*b5}22>f`>GL8gw?DWI3mqLbx_*_pyB>%VjW{FI9wL21x|(y6liU4;{MNSL z;<=s&lo{$VvaBL8;&78#LwkJ32OKwlM*adEgsRS8xEAp~%7#iH(8a?^u`LIlcr%lmEOJ zc)X-nj19u_xN(yVL_z5y3erf3M`!mM!N=r&6OE)3akc)ZYFJucWm^;Rfb$%gFQ*Mz zcTm^uZWO{g#i_sqIAV4JFneEoE#nmdet_4n`5QYbYk@~Y$+N54=5;0#&x?QqN7Ydj zPcd5>oX2q-i-y=tbP03XH0Q{=3TmF>J@I36!ZQP)=exD*7mq$!M#9n<8CP` zKm^Y*_;uok^hViPT>VY0mYla^L%m~uNLigXt_)Y`k?XnM?_Q5(ut8(e zwRx)uV1P{NfG4sHx2?#XT-S+&rH1M`{lSSh2%+=Hn|6Z5X2HV>jjX2o+)~f=WKTY9 zOO;aR7tyli%`am(&DWg>`%Cpidb9j8=#)?~fv$Phn;Y^_KEC7)26W5%p14r{vwNKl zyVvGH-rZ1CP1mVcx`PRN*7i3Fl`-;%XHY~HbhXX6vxBmDbQ4J!KPC+qYx2?x8| z7%mE_ceice)g^}3*_le)(O#iUrI__O!U=3j$Oh7%rsDL)A1-haGb7G;)mwAF#jP`A z6D%USmb*@F;MP%k|EvmvIafp7r}vd$)&ks^Z&{_8PkUoK_&Q`GE+F~PZnsKR(-A$B6W{W zXO6HaF7NLOw1gH>jSCN5+^?9%9uJGNPKE$ySQ#}}m5m=Ps*U}3;rd{IkmyH=H9%dQ zV(7X*@OOFjHNOi)f--@vvn4LO+`7SJ>884GXMxE0Bj%(?)#_!LhUM1VQL*2-{uf_$ z0kh*f${jH__X9*}61iSM_nhLw(PIKtxk9c%N)=xP+4G92c`k!*En79Ul5QpLBu^z? zL^G^Yb>*$`TOc>LU86N^&`%P2z^AZ^1rpc)>gRVW(Rg=lSWkB=SpH^_1u^_)`P-ij zvfdFdGZ(l5=3eKM+}BYa)PBKfBuT5yhdW##1^r<#w)R^Nlx@=&-Lfh9O=V$_=^>H- z#R0)5bo=s6`;uv8yV=9dTLMFR$BC^ma7B(%f!dEu%xwNjZY1v4h!A?chRf|4e(ld}{unbI21);C@N~!MWWusJsr@;FI`3E%!`yxA7lV&i*1hD`hwbZu zB`Xx#`>DNG0CI_g$Sbe4MGSVA8nI7_7cDuk>)?aai^5Q$K7qz*#yI@e=K4)GMn1{Q z?#$*1G2z+*!}x=DyzJFF5scAxqnRE*1OrZn&bhQNr}O;xMn5y)JUMIFo=RlslQl?e z5wXkqyrc2n4T_O9Q`sq?Ltpw?$TrB){65+=@nlkkeQPnid>api#2z3$32pKP+eleK z8y5J>$J-Wfd}wUBYcPqw2aC@IVeLzPZvb!>Aas_scc>(S+uuK)u985I~RbU=a2OWX3Xdr9z z963ts+jDE?mlrGcj-W9hVqk{XXie3TIkevIA!GwqY%$D_7g!lxt1q6qwq^D(*2Ku& zX~gCLubX2gcUrc+C{n=Id`T6m7lvd&&}wV(o~_;#|*hKVB^3y~|Au z4=^=n8H}}GF2ilnUnz7%D~j-l&qo4t8O9&7up^cee4jnlw-hI-cJ)`qe~=o_@=hAMhmD`o6z)ptw_Fi4fwRiwqU)BhOp%c`3+Qkr{5TuQMJc z#qrI;aY{`fr9tRdKRSB{w4{=?e3f?axek(0q0L{i&A0Rt?jD22pJm-FA|ILYQHpON z*q13GKe*Mu<(>J<`CKiWx8fkW4OG9!g8RY9M2*8kleB8}38ab5f26Y(pHE=4*omD9 z==Nq)d-ydDw=vE*8E>IjBnB$6lT~|F94;1J3hM5bQ|O}7gJ_ooB&v8`hp|ZQ)F|Kl zmuBTylU3!)|GXm;S6Uxe)iqm2Npj~#x%!qYe1qWJP*zY@3%trJ#X~IWqBpHop{}p# z82J?Lq|T~Bz9Wk;^!p&cN@G>#ALix$&e8BX=t>7Y^)xcIr<8$m?~Yq!@7qMl5x9zQ zT?4K2+!>IlcJvDC-eB8fhs502j@wN$!)iV8pdW&nW>p5V~10DKAl4ZT=^ zihgA3KMD06E*I5dNH`<3SpF8*P7XpD* zab+pA*T>M#@JFd=Z=JnNcdfS4^9}m6_>^nSAxhCo^#3m!v7 z&!xU^KQ12&F0Y`zIusNs{GvQQyvk%-^1IVnvP2#pnEmUVYKW3c&Ta)y9Cid^?`&s{ zuiy^9UHxJ-sLHzMI3z*MOS!*ARdH{xQYlP-`AyZJq9i9#@P-+FMO!Jclmnq%Tt-(f z9YOf8jslhZfR}stGYWYWm(F>oCkr@zP3>8{*ox4--JxY(5Fg2rg!v~N$DaLIw@7DL zeu>TFf~kU^)f?k4fX5!3VE;01duFtZx~!8EE55Yye(rBx3;)wn;p@zcH6AGNZxtrD zCE02g`5X^7e2wlY!n$h^XKtN}{jzaj>(20xHLz*+y=2;#1hPjw zk{`EbdokmT`DEhA0`870VJ)Bz=7Co`QP!y#aJ_kDVhZu~R(2gmVSpaep*m!%ryEuj zuso@D2?rpOGblv9w&;n7f{GftywaYk%nj}QibKs|mxZPA0Z8oP_gItKiz-*$=dsg= z`l1wc_&{5$I^s6nmjlTwBZ|O~B-K<}6^XvnE|=-e@P;toOXHWsdx6C%0O;6#5+rt; z?dK1tIHBaMGPsq6)2NF)OGOJF+v1f#u`;~ERA9$d)P22b{u+>57;8EdN8&U4#}e19Y~*Y z+drzGPOIgiYSQV!ys$_3ms1=gjiSFg^t05LPd4iDX#UX3HJ~G>ak>?pIPslYeK%t& zOc7O=CAW{AbR?Cq&;dRl%2swhLX#Y^jc|(`*>Vb3BNlF`GzqgC7v%%Xrp<*-smA_R z#zxIJt>*HRA=P`*Lx-kv7BUblWI4plwM4H-47{Eq+V0IGPp%)B^1&f#kG3uOaz&dn z*BuI6x53wDMPvlNI|#L;k4Y{M7M3AichrUd`d~5}bpnWO@Jhu!42!WW-5uzPeiAp` z@Qg0|$VJq0QnP^AES6XE#ICD849_*+w;bJ}5@0g_G?TdwQtF|@AX=UJgm}%rRWoAv zG23vX9`6mAh}RXEeI=VWb18`>>(nDvDE0{^(j74WLo%Dfb$)4osz_Ro#tJ0Xj$5b| zjRW*oUKEioHg6yN{i`ZyjC7QhHjUif!me2nwk{4?HdoDX3VOS9P3Bmg^@P3tAbaWU z{|LL%MtsC$|2HN_at=uJM-0MZOB#L5sA+llPGy=AY{m-%;v&CQ?jQw6krhKT0>4AaSi5ER?IGbwAl8_;WFt%h$_J(7~Ajd6B1J&ZsML>h6H z>n*XX8|(xLS*37g{b%cJ6j{_72W^)XK?#4H2Q$;QiJFAX{;P6CHE!d4eFf<4h_Iex z+RTGKEDdn`NRQLV;yKvr!?oG?T$zL~F%nReI0Wtgi|xVj_8c zh6G`Mo}O-td@QRGz8$OguM{0>C(Cz|hhMODGL)d0m{*cyiv8X5ww}s84Z!E9j-nJ}MXc1255Rw(L%t&u}J<5E8B9_YYYWZppwWv=7+Cwb;b?6ulRLp%0VP!4}FGb<`I9Nuf}LSbDVwbro>c5 z)#HS&STpSRGIrHFz8pfTy4dUd%$55(U7n~cqvRQn&sQEcf_Gt~>@VL@GY(5~v~p*+ zOgm1z=B8anP<qOYY{_uX|7WwsVADzAd_G+;P5Zf`CPQTFKns61AB=8+|&+x6`JQ^$5k@fu@!m+*24+mDSj*k*K(Bd~f&6w&;t~*c5 zeuMC(iaT`>28cLe7D~B-FX;O)7k8K2Kw^TE+EteB`iGS_il1r-f>E*%J~V?vi0Qy@ zRwgYatg+PO;7D4WB7FiGss99$glR{KYY@>+L)vv+G|=Gd}|SlPo@R_3CtM}y}h=OG^eVSNNc5*76pbUWTN6uNTibmjtk~= zIaLTDFeV3jFRn&og4cm-ToyxgCmct#VM6M?(JAiR+I^Nx-~-*)WaZI8Sa&&*;<@&b ziXzqh`Qroav)Neiy$A5-!wKbe++rD^tK;$B(Q_27to)-jm0SLj^-R!MnO^$+z`E}+P8gQ$ zT{{mVMCw;J4#i4QNA$w)H^Yac7mR2%l*PS+JxRpiN0sQ5fcF+EZLn`k@ZZMZ+GzGq zwAdo`fwKGMWtE9k2uSYUKj;Ho$l7z!H;Rkr6%--b9@;)nRi0gaYUrU#v0Qgm9%RbI zN(MJBDxOxmb`}w)1(?x@yWj@DdDM-ZG$hctuD3IYjK88)=Mtx`zNOHYdZY3R)x45@ zZEMoc)|@JlYpR$6|GwMGLmO$UL`ARAD^+Wg6O5!ioyQ)LM>L4e8F>Fp1o#*~giM8JdrV*`!FfdvEY z_L_DsgGbscz071>el%rzFzfl937BYBU|E`xBe-{??SnvlIgPmglEJO3U}Qfv+WIb(aMNr#S>>E~1+uS+lA0oE4n z-Zi42Leso;I3kn%0b6Oa_lh#`95$>S?>q+cOLvS2{NEt}cI-(lA)qRw|JNR0_}?KQ zL(KpgO;P0d`eaA|wxZPAlv(i#p@NgiMwusb>ss-FIL7(gqB4j4c|_3Bo=#==({I5N z?s3DVb1&JW8$+Q62Rwvy*U9~|#t^23<}*gbZiIxI;dn%^c!UKkKOC~V!gS2|8X4%; z_0d`FVlhb-+`f52Y>-*sl#5sQFv6n$zWgKlL#XVK{A4Y8mv($lbxcDjD(Hcd-57Gg z_&-roL6M!Ah{`=giPWJndNhT9#Z%(`yb!cA0m-2;&a`76bv!6787b(xugKxM3O5H0 zgR-6vwn3J(Vl2BD*L~cGZt}@C6RiDakj$lDMIwPhV0}@rdMhb+D$W{QozJ^OM=;v4 zpG{jgE=Wu5y>#54v~$2SnM}h{$wYKusfZYfVc zlwD+8x*KIs6?HYA!dS${+*f;YzX~f2lm3ilEM4LK@7s*K z%y6uj$;nUG-Lh1dlC{c8{XGz_IPNTFx6nz9N_&kWkSPp?^-Se!{Hl88dqvm$tx^`!->rc6LT|8fubCA>Fykp^PG!yC7jYUUI?yK@Nr9)R=0ELNF^%co@Jhxqzv*T?9Lm&1&S_F2~BM-i@R8m-;B4g5+lyzMc z2hap*en4I6cT%~H&=G99{$cl%8bo04#}`Q5q|kNt?byy2pc={8{wU6^T0LN|QBAKf zrrlm!7=>l^2!3nC9;$I`tCo)D<)=oAmMHp|+vE}WKE^pcT+E4VQ1lsGp>E~491(yN zi90)i)vk*Chba10PO_TmOv}cN34Xc?7)+HjxX)r?Gtk)mtx@95lB1!gg#goNmfUF?(~`AcHldH z-`kARxr7&~%$Ll!B!@iDCRgydi=Gi(4J=yEY7n308&=W$Zv9lNw>a0RWPW#jKZOZR z?gJ?RQAzFKasuuxu=Z+?>JazFz|!M){bsdjpE;6nOF2|qnkruQ%lf;L9FI&x8rmk- zi=mk~Z_c=f;Q8zyhZel8Blv%d6?`vp*#!n)zyRAkCxo;-V|Y0bug^?4dvLzrf4Zx8+)1Z(V#%C zcQ<=VMx{gcGtJ}b@0aaHHN~q`Jt07#@XIC33>95?#tu}iOAH2_y}V>7-Mlci(W3Ply@NQk zTLS1qQxOHgP*lOI;FgI*Blh8Utf>iGWWDn?(d>(-f+SlXYc5vC32>r0bavhcR za-HZD;--3tpeK1ncRguKxZ1y|m5%#<09Oke20wPB&ACk@rg*q{Pb=W1J=!9P_#ypY zY(+yC@pGqcThHL)i&9D9Z7K(Q2pWA75NV*GPRDV`ka&u{-G<=G< z6VX>w%_k9rMCFVAm)%HQ>?1L1v5cnY3oZ-43TDRh2Zz zPG%PreMftAlkpj=`l%zS{JFI<--?2Q=)aS)-+UvSb^2Ur6`OGv zfEauoN?|l!34p^gl=(|xe>R{pZBJ*nGZ$TO!`8DvbXY-`qt{x{ArI9<{3&DwXr5P} zDk^uI{8+t_u$4rf-Q2rZF7c$>DbXms?Y^QvVw}(j@hYrewaW5jusB!Bg6@1+DarL^ zw~gv|b-Yt96H8W)D zlSOqE#Q$STZ&~3+Z3o4hGwm%2ELMm`ky#;@{M(744M?0x!%?LLjRua^HA+e)ELLjr zEaZy+e(M|Oh!vD;oa?3kM_$b7p~WCAxp@p#%XsLqjp>R(B4vVl+<^`QyMuJYEYd4= zr&|1fwQs@%vlRb)Bm?(NinT#+j>YxoGm^I(0X?Rp?wZSQYd4zJuPk2zes2 zL&&WdBN}#pf-car7XLG2P#wyb3(dk6^@@|Y_lgD|Tkb3vPcW3`heggJ4&$lI1&>HxkXSR)kAxcFeW6zwwklIDESy9)qrO(3dqW!m|8EUvueqN&zBTph#moU_Llst5PYT>b_;*wApTj-&Cp$&!_)zS@r32=;FdIxHtRmXpq z{Ntvvdq{53P_p*F7hwvpGl#QJ_YGWgba+k$w2=_Ib4ELji$Rs~#jn?STFBKn zC_a6ExR#7)H1v;2$)<-a8?)P+4=426?3DdcMH?6`*dL+Shs|eCG;99HD2Gedd5V}F z&g=CjBaU)e{toZ_<;=1$u)RRy2e*f{S1*|koi~oQ#oFznZ0$RKwjC8(DrY)&rG&Y0 zLOa=RSMET?V?d{wo?8a$%$Z6%>(gd^+3Yq*G(`*E&22l5)4|s9&`?(b6uWj4eh}$O z3vXCYq-;M+s9($?QaXe9g~A#=+2=PywkEpP(^T#+@@9?MyLiMNrU(v|Sy+>`ycWz3 z4awRqO{i}0VtHwTQR+2-k0oyR_X0&(mG}BS*LGH_qZOB7u-tcPv*Ad$7h<<#u7@F1 z*=d#W;S=IF=Nh&He1xiJdS~OdQilsvXTajU#$EQa<7Ok5)7ZoQz6nLoH9w%wWeu=M zXg`Ck<6<21cWjd(8(Vw-BZcUDoYgBiVV5qR+3Pe&!fD5L5AQ;Im6D@gKmqK+a6!dY zu~J7x^u+J5P-NzF^kf?Uiu1dF4J;0YMcR5gjoTu6u8vPf5|Bg{(Z@%(z7)@*Axa)K zfKSvf)?&i;_~{i+RQ*P02rifH;?R@*v(FD%kA7|fP81;_SVNQ)%=FmALeE_6bkB!D zDEMy`vX1~?LcgWu0^Z34Z>a4IwFBN^1mF^MZ3-*dIat9{+OPKx8FS~tAC@<$fS+^F z>BBEb+&K_(FKJ}IlrC$op0V0x7i!^a@q4Ie4uvPtU?{z z%I?O?@?FF)4%|T-opRffa%gb_gzWLT8-=#|!1P-J*MPe0L6FVpBFFz{0fdIrs9lCS zEmP~F{mmv8@L9z8h8}P=^E{TsCHa#oqu$wMd&b%v+bW6lJvDj~1FE9Thb*pCNV^jG zHcG-jv{ZL1=IhEeT>F>Vf&mQ)!ll7t93j`8Rlm94d}c7pBH28wlLEvL1pNNhaj@6{ zH%<(_?fRWq43+4rHe5)_j>tn^=yk3NzZneObrf%dG~Gji=|ds>-5V3UaV9v|c(M}bg=3Cy9_W?< zuIWVn83ay`lIXsWkQh^o3e=^WmY2=K?KGH#gJzJ#{3lD4SM1`KI5~ecAM-Jg$doE1 zcTkOL)ncu|F}oNfTjMLFJ+)UfownB@Os~vzIW=5A@BN&lN(%@8>ynzT!nR?)59LZs zCb|wO1G56Eiwxu93*~x4N6&3kD#j1zn}<;nYkyivjKuwY_71HZV%jCDmqcdz`x@o8 zt%ys6%Nvc4^I`jSRmv-;H31LMw`qtwp`&h5E#3NAXtu!mG@T*fgz@kw`dD$?m}@lV z@TYm>u@Wr%W>%sE#Y31u?7_)@>UF1QMVrCH;H1R52s4sz8&)QwR311qinFQrEwdOS^%IODe*eM07^-#jZFI_aeJ&!?X0#9nx@ z*QB5R`O9hM*?f`(IUKax3F*nl|Ec(w&p1PRTH<1ve}TEv#3Qy!E6BuET(0`l&1vV2 ze~=#h`$JJJZ&sQ$ZL;mV(&67Yt@ok6le=%+b@SiT$5#Jldf+~bGtR1F2q_S^3xjQLx=V&ri}mAzG#aaHX&nd_|6&5+KcTwoRG1;nHJXn^OdipU!MJY zeXh%IzI|Yo^3i9Ukv_KP*Yp;qYmYr9EjaJIm3>(G@9P14e>ow82Zv+fc+r+QCUE?~ zCHWW+#yTMbKI0%0G*~<$E!0pN&ep_G7cnI_F%qZq2^p;8e6W!N(s+b{j*+<7M<0L5 zNgCE6l~Mx0X1Pj6qGVpmE=KhgedMro3`sp@;bsiwht~g44}aru8T}QEh+U;S6b3^V zA1$yIZX*>WV`~5?&=1!5V8zKy_NUB_JPvRkavBi7=3JohT3xo@M|eZU{-=MGOe5;* zG7%oJtFatRpSS-Q&uO}(l??Y4mS9jzi(sO$qStNXM=4o zXz2@SX!@GumZQ~tuqMtm!jZo}`bF*8e~4{;WB#;J!Y8(aW!fntju_}@NBdERt?b9S z4kcSkyqH97wYK#UV>qm_NDMKSNSX+s(?G(4Z zJO)q5AdX}tPH{p;wbDr(zfea+eTIxR&^D!RRWazpanP%b4i79jPpUZ~!-x7B$k%`q z>)JvSRf@}5RGnFYP!YnZH6eo&`t9C-V8RLLq{BX%R$l8(TE6Du+tOa!|6C_d*57Ki zG~w_qN*Y_Qx?j5T!kZFKx_3Ktv-D=UqT-PUpGfDNa#i})myXpF+4gNkoRGmu_}uf( zo<9Ax_0yZ*zOY`|@{P|No&NE_BMB2bub6qpwB^ogrFX8gOuGHX`_d8noSy!2#cgSc zw=9%?e(`Q4hx2}Qb=q-*@93n(=AV6E+TdfWq*-R2Fw&b~%8R8V z&fTT>j8qsjA+zN>_DR=Va!Xodi3QS5`)`z%UTL9p$BlnaC;#YzbixtmYR(6q@aeSj z+HWfKTfBXrbnT_Lq=lq^HvZH*_3D_rZ+jpevG;G%b(hIi9&*rz{*Dtee?IH_wEYJT z)CrTF4%jrUwdwL{oJ`tWb>Xe)$bHXDe?0T5gbASE-Ftu@@F5sp)4k!+6Lb>g9q(N_ zZ7o-sEGvC_%k}rBZ+!abbk#++jZV0BV#3uQYrg)A>8U6FmG=4Br_!3jAD7?BB-t1K z=X>duv(J*wxMSbWgiXYv%>GAt^Bt`JU%SZM>EhqtkoNxc&(nQ(KbY3vYE{{9wge>c z_9FaHe*DYpqzym0YI^m2uS(d@zBlpIbm8x>llmO#xU2R^uawtVW4Z6Hhth|ZotPeb z^r^J;@{6QTes!I+h+Ktr-6glDgLgYA-F^Gt71zFHiMe4CYpnam$5%ffU4FrhY4vrM zOWREvuUEw2wc5XstF;c>`!wl~8PW+?PD=AHG?&JFF{>$`kjYV-VJQzuB_)hE?U(}$ zJ~|;Y;~WWB_F=(*Xm}gYF=#4Qwv+x81iLzLB~HL$Z894QzRm$LfQuh^Fi%Qyl^^cH zXCotdesv08do8yt1s<(5nkGYs1i$KXe$m?ri)DkDh|bumm5NzyjSB2ohPNzPKTIp@ z3$DH@jsM(E`oQ0}EU`pd?JaN7ZFu<&H>6ARtwukTTa`AJS7i^E|Nb6`6EaUf@whbD zMv(kX&)<9N2Y%;mbnr1i|3FwY6w(ei+}7)qgxE7VXHUnwvT>LLawRQ3{500($qeMh<5Cadl)88bHyJ6FhIGS(lclL0`2o@M}qfBn{m|YCA7@uRH z1CDysW0_pgW8n#Zu9R4{ms)j&Y{|zQX>GYppS2=T>YS*6kB*Q+u2r0)gr(GSe_9g$ z_Al6pp)^|WU(gwjh!7wbnM&6JO=~qCY9lMljIMup5e9yLXlER9uuj=-AGUH3-um>+SFz;WC5`M|F%`2Glp{zYFH+y)He#p4k7ajeG`k4UZtx6;G7jS1G6Kiw&T zA_FCKw3Ewt919GV8kdhNMPK$~fMoxTI8|X)wBYsV#41Eq(ZcpCMh#}QwRvhek~H5R z^0NP}?hJ56%`~-@@l`8;n71KgtEj1^?f;%&-2T*3G6$2s{e9BUGawex=+DUXhk?Ev zz34|7{mBCxo*^^i%<`5cJk(bfKd;cLPR0PoZC~Pq?XPw@cvfDlxDnulro<#zVGDViFPjkxcF>(3L&o4?}+3N5#^Q<$a<1U|2PGC79Gn1Uy9&_1*wBVxi6t}&$ z`*}Ly@N=|2Q6`1rl?Z2_czN1o(?gZ-4-bA@Z=ZTzUR(O1Wxt+ox#8~g*}XPOAN}H5 zg$o5-?XdZZd+CY%PcPU_Z;7b@J(a!*8C+$t$H#t{=6mhjI#JNxQnTkLf0|A@^1QV2 z`<6+E{CWpvJ>wS_r(L%=Ot0WL^87EQ*DvOOQN?XcA6e-e>4vNCkXyWFNEbZ%-IB?7 zzj|^yV&7k-bw9Fl+Ve*rEqWC0zUBV3;W88SYMJ9@Qecs#?AE2dwmnKGh2F5_0(vzH zCV9Br`j>msCNi0U$shXc^ua^ZIj3Eo-n0JO(*DP9L#WCx%N0*&9)D3ZrW$ZqtZO{&6!TWb07Jkcp+`J{NCw?t8PzQ z?Ywr{ai0w9`;KUax>T^87EP>1D!>3nrsB zT6$u7_@T$rl5bf!9d+@pYMPRL16K#FI_Hkszb9Y!HNCBjaUeMXx$D83r%kq7J({Md zjQ-#iGEd1ZOB_>18W-9?^r1fb!iKTTFgA|S&WHL=Y_5C?sJ9$<1R}TGNx?G+)!{y~O%#apaXrZ+0R$C4y zW7sMGA4rVPyh6q%WbnNoG=jtq;b;L}Kb{DTF(WO+5u-y5DQVr{+QT4U>L12BfOyoiCMzhSIaZkx&pMhsNZKSNXyN1e8U zN^xTExSb`FvHj4{)ylNi?Ke+bM?7SK_FsQl#H(sC`CaVrA3l&3>5%}6K$%Q z5v1ozt?`megTPS~c8KNU4CuMIH#E+wjs#^Hu|)Q5KeaVUb3BOZ$Y*7;B6_rXBJ_8S zmEaj|D1*VjtBg>4)=?sdTfD?rH3w=DgV-?_+9-jKcGQ0KF1|yg6Fc}_N`fGpIOzMSozR4Yosq9@;?!(Wt@z>Ywpjd#~*vD zy7h&AYsh540nKE<(FdQM_TTvzI_Yx$LkCx!&&sV(E6%dL>Nv4*?x|O%&u{qMG}k`L@@k`4+4n`iNU+ z)>`ljHo^4AUDKOZTv&NmpKC|e?Xv%-@~ofkUvCfm!U^fN8}3b;@A#gy;(L}zzdioa z^u-T<->!nWbH6m>Ofx7C`gqL+zNC{I-#O*8om;TlMqDY+#o2oGeN}zj!eq<}^7`xHLv*K|Z8xncrYl2VQl;W zrhl~Yaf0|kd~rM(gsJtdhi4EiEOVT1Gj;pO>}(05(8r~ewDN);I~r4ei=!4Ueqv(z z(+=tUh6C@EPRPKwTJ$WyG#aP;Zd9mHv257O%nbl-gDUFwvn>$|JoWT7>HUg&nUFD~ z#xO_sW=T)WWcCgF@e6X5wJj#GjvD4d=~3A-^QJwMRl;m>EoOUcVEb7SgIxSrnhxu9^ZtVY{j14W8P4ahdtBU z4P(yWqMed<*l4GXk{C!XC(48j9_qU;CSEv)jN4ncM{=q%=2c z2!dcj2G5Z3Tb3l0fJ*}EkZV+uWJ;y*Ppt_VOe$=%*0=OFBV1vz;=7hg?|9!*a_iEP zoht>9?goqRk?y$V-gMxJpH8cfU$(aeyk%$WFMJ^Fu($mMn-djVetxa=x&1b({7#Vv z=1%;?kK{qZ^QRN9n^f82$?MPl^p&*K@~=-nl-rmN+5N<{=$jTy+so}Geao50U7Aim z=EC&0)s{$yoxM{Zq|)_G$gDBX&N@l4{lpE@qRpfOoUxs8H4r9De(>8}^kK6bEFt}K z^S$y`jxEwg+pJc_Km`w|U3anFI#2bAQ~dLSIxJV@Hm@GA3Tue*La`d)R(QZy=FVdunuonkjD)@4G5w=4lH3%Ky%&mMjNA}8&@Mm_3SV8=om2VT>PorbI^ zcKro6J2-SANwBy z5rZ;bdp*2DrtYs{ZP)|TN)N;dnP(rzElYXPp`(BNAcP0f`7ENSS%_l=b$zmqQjak; zzGl?nRGiJ9qu5CPHQVIYm}iqk?uK|mW6Yw_0{u{7S;tNpO(B^%Y_tQ*IF!X66exx& z+$s)LgjQ{j;W?6a~5{=gH%NR%y? zVTsPUXOLZ)Fer;|4C69qXY~9s?LfgC&DXXcqWFXWU5T_hYmC|UAMHRBLng|IX>qNM zKF;MO4S*_~)X}`DqVMQZJi>&GanY;JsIH-n%%^|b(Mk!{$bm9AY#bQ9|CyCbGDlXo zc1y&IMT)3t8>IbUz+1culNPK-uVTx@vHehW?HMQn9JEuyN^VNpD1pIZc3Yo#(#d*- z%)0BYt2W(lo5>a!$U$xeL!TAXVL}E<2iRc3Zh)R2;IM#HKVr};WS*7@nZ0%soUvf7 z0R&_T*c!eiBdtu~N&-x+2^nN?!Rgn^!*P#IxBc~=Dj}Sh;#ngf{lc31a8}$7vhu4w zs}Ev5=F&a%?LU=6uI#$m;pvRyE=uus7EZ|QwC{(~M|OH|<;Dq_#g=|u`o-m6t!(k+ zl@l@(KKA2u>QDbz;RaRW$@(A+^zDQU-dwci>%K6OOJ)1k@!O}>*IPEN@~Y41Lv9a~ zhw1jc846c^tT6L-oe3HIRsEcku1v>&{}1V+bFR}{jTivi;qUi6sJxtzx%liGWFqaG zI$;0d{%@yQUiAvXN0gI3@so7&kIzqQZnQ$$=NF$;EVtLD+pdwm^quZiPc0#Duj<=g zEf)1dzH(Cf;eNl?=d&Dm(sq3iy)A3};$KnQZ*JXJC!}x{)a-M;GMz2Y&FNcsrB2+& zQ@U^93Yfj+nMr5KZCiXuF=FD@J3NbMt1qsp6K;KZOs+2O(_@{GK@1&pnqW5KG{cH2 z^Jt_)kB>i`keMJ8GSJeW5*kBu1hjx|w>lZAhr|bZ{m@|Wl3-CwK{;&r)q_x6s%sN6 z_E2^G>nCwJApNKIgbdTvUt*1A0U58@2NWW1I3Y8jlZNVH4@?6+fbp5PEMY<>4?UWB zyhQ|xXtSo-NZ_NWX}ON$1a*UCm%BS&`w6tL2)T{=GIp!8GkB?vHr9Mv9Qi28YB)_q zk`akSDv)PPH6?s`3=JphDG|f?8O@~HX^;ww5fwS0cT6r(p}$9h4u8lX45+In)>RjmD?mP{^qi3S`9-^@C6-&TH{@n3?5z*tjn50=giPP*(0mmP9jy|M#nH3jJE53VieSqy7<#o$=RrP!4|AJoG z!e`>}A;5Sm*6;86X1S%Tjn~#M{lksvlkYho%|6F$>CC(Awj;b<>EVAomdM@uwDp5` z{bf4xfZwFWmw8<}`m%Zgt;M3w37J*bUN#+k>Zki4>GvmHk#^qXf3^L8lDAJ?aOQRT z&}O{h=nwxmsMiM*iJuvNP<6}Fi`_Ty@Z)n%y((RF_FvK!f4n)}DOZDG!2_ocICi_V znmla!#V%u+kUC#M>~m>7SjP(fO7lci>2@TW0uKZmmIPyLl5p3?>mo{y+r7CQ-f`P+j5R< zJ@C*@m~_OgeHwmUvMQ28=>n~d59tcb15qWpJE$ix`9V&KUA z&PdT`ciucG|#B}S8e@pxSV%zkd zjo#XU{rad2(^o%wL^UCU$)EQwu!|08d;aiaY27W}Uislxqt)dZCwSHgCr#olOM_ya ze)L6h%hQ88iLjVV>~LA{jk~A2Zo4miQzjTN;9(*JZ_L8mn8vd37NkLwK^T~Sa==*$ zuTI}|`_Xe%%|Tx}G5z>kr>EDv{?+NUoA%NNGq1OVP4=91=l*Ffc_UN1 z9RIyPq`i0eSv4UuwfBviulbw4!uY^THER5IU+Pa~+7O!RllBaDodE8D8leE`bueE-_ zw>n4HHsdxl))+VO6~b*cSVp@_hjExACB_fsQx!F`^x1%c=YQ zfxk@}WREP^2H3~4#CX7KOG{(Xf5zB183R1q&$yIk!+Oe=HZQV7e&~pEy_GN_Gs7%1 z%QIxgJ|P27@aZQLPWN2y0UCS@8-g>sbdE`(j4NdP88U*(#Fja`4@g^RW}OXWy-!9T zl(Az>tqB=SApGP0N7Esv?NA;T8*|_U!`F}ec>2KRE2Ynl|86?}^efYQKe&9_@7VgS zOz^wp+#A!U-g`j8$@+0uOwg-4pr4u(GQ2VZyn~*ff;S3X{>Q(jEp}QneN0{lJ=)SY zA@k8y_LWyh-#qX+G}m5qYuaJmL*y0KGp3Von4}MU-C>I!PgY zlR=mW+HK3j<)O>7>s2^6Tya<0Y{jq1#MyM|xGN{7g_oGWL;uM&zLhRL?=R(6D%`^L z9+?n(=gVufPfW|L^2YS7Uv_V)`qj@a zOuKJ=#AHocwYkPRh8(_S2@^6{#xf2>j#!T41NEG7Zi)4^*K&mnZdt-o;%1U~5UGH? zDmW^epzy0zcIv_r5H@~D0Z#qkhg{0MWeH>MSSMsQ%+HXC`B@&ewph{1=v_oxK9^gTg{aSF2)&{Ei}YY z5=9JTYh#@lzjdYC4r0hlNe-(UihCh3kTni{;*9xpNq)ag*7hZqVC1wJby5jX;;##Y-2(>THHbQidZW|dK7A4 zb}edV!P@pa>-B0v2FC0}o}i)$WXi_<7j>|~wT=l3BU^c-b?eI<=o9U*VLxMj+=p5W zeIcU_b@O&h*A5$(oQ(+G|DeXCR&lZdtB~2FNVr5h>R5=gTA#*RM;tla(=@a@Pb|Hl zuf3i@UVDvOmUMgNWB76&Lb%X@iyx>Fp?sc|5u{XLEs@!!Vg(>CED|=UnvemA7DEzG ze_B~@?bNrj@{`KcnvmfH%lccbmL?p&MHLoTZfyMKN$DZEO62$}CZsza zja!LETlyws_@G(55$W)=zo-usCJ){|^_dTRM{^jz#VTpi54Td&CFjVqUDo`T_RqdY zf2w-$@sIaEGaa$UA>F;UB^}bu4?-M6v$T!bx z=92tApB8w7ze=3Xis2O}8Z@tNx$d5{@v;;2`7GG~9)0AAblP9{PV+7>M}&=7=6-|Q z*}f~U{GRa@_R4MS16#g*pY+#j?@XBR+54!EE6;Xoemh$IZnu+KDqk_-f)EpBRtgl zgsb;V^S`cs%hf?&maBKZd3t*HdT&kNJYidecu@-uTBhcN43gpa0Ld|hGA8?CCuA6h z0U3i6Ij|ds2TnwozWC=nVZ{f10-HWVMx-9l$)K)RB#NQ1NC7xLL#A&+2JSw37f+~j zZd%k;w1O9(xWFDUu(cTIA_hOIn2QU`-gv=8*KIK-#?>@itl(bFW zgbawu;A2ZWei6YKkkpwJ4=CkR4rnu?#(4}x?(#r8_67b}7z1U~)s|=nhD(j2)+B9F zOP*+pW!jlVompgR%f>=3@bE;eAK7Z_NZfwZdS103<3Zv>oW<|QMlJEwIiHpEX}43vAwF|q&Xm-f^zV;4sFDb81$`+U z;TX?s^b&cxNLnm^)`*L7sf)73M_XHtSm&kIHLV}&v~ug$W1r=&7|>-fiY+^NzzLgA z$T&yD0EdMX&%v04{megHW8RCkmP|}oSHMJ&WaFxeLSKm)hhH*>AO7C4#UW+_) zx>DZ0gy(47clU$oUE|-HzICFXi0-!e5$W_}FVsl~oTTDeG7sMSXu44FH_3_ktLK{| zeebN#RaZhx?FpG{FTPDDKJc6yoRF{hp2hX8Ki6D%YdZgotF)&Fy{@`%GxSZ!07pF@ z?)vWa-zv{_nLpik=R@+WovYG4cRir53;+4WyH?L|;e%yymC!OPyg{xMnN22cZqvS8 zVf7``#phnHC;u17vr@3&0C3B9?3=E=^cI~YS@r!(r`Ik#ce?6Nx1?(?xi!r>&#UzH zk++-mgKtL{pl zkcqs<9(^h;vcv-E9cwO?=9+i5gy$AraMm^YyrTA@!g$^ErptX@-&%#+)8O;EH_n@G zxZ+NE&dX(K`FAd<4?u1wWTs@_-~`hg@(|&5KJs?icjZCK@+Pax&iiY+WL5Xo83|ETOajo&;gbc3i)1`6Z4*`Er$_%&t z$*2sb9NS_H=3vr@F`;5u8{!#`euZCf9CH;T0P~|X1^ow2$k0w-Yh-ub;#h~YvaQQ4 zswEraA|ACi*6<1$>!aa%*aOpI54009Y(w97zWwe;4E-Q^fJ-|h$3ErA<14VR@q`36 zi(yBgl|R6;lpN%Dg?3jO1}NZYr;PrEM{yBBZ*n=0Yq}*eke4zdTMTV%3GDbF`{<7{ zeH~-ohBK@Ca0?iQYqx`sjI_F}qkC*e)Ftwgk@Xs7HRg{R%D}eSGCui$?c=1KJP{Xd z&|@J7rD?l;!T7L~r!CXsTwWq?D@R9|N4sr!;>xd0H@}!A+G&xfg=B0>%xKbHj5*;x zf3~Ig2Ttt^k|`jf*i&RMNjSC9BbH$&w$;;@EddT6<_~+HPnS<72!hh43*hZb%Y&G7 zv6vo<@uM9~;OO#L&cymLn8&7#n#f_cxJC>^L>ztlB>5p*AKK_gJ0=02piU>T9F zJ)59Jl)ecWfXfe}R1iFNDC{_Kfy9K&Kbr}eHd%qa_yxi8f&+$`r=L$}p{B+|eQ`wz zt^&CK-iH`MC2lcUbCVU*Zijrhns7k;?@u~S-<*Tzgv4d(etR5)l>DB zB@*NL9$(qwhv~*^?x0HL8z z+W$A{8JPg#f~zUUe`J-ktzhSn372*`@dxM0RTXEYyKcQtCtPr)%4+XlCVk}d?@4%@ zQ2X|vyKeq_`qKX#k*>eu)HJ?`o%a9yvHD=&^|o9!O*~>t#bMH7 z;>UlI&N|`Z>dj1eh4>D8e{f(Q@ah_n?DgYMr1x*OVu?G}0$$<$#Z3=O*In*8!xxiI zUpVkXY0+g~TT$W4oG;58xG-rK7qs)U@|>Rk+2P$$t>s7l@p$^uribgqOEhlq(be?1 zKI<>})ztSo?aAFYa5dQ1w>v8R{*=pN7%gMl?X=&)575F=#?2e8r7czGpU; z$a<7Pg}#&^pbI5*lQkg|_V2Zc~ z8V6qS_R-VwLw(yTWLi4L0d-u6qn&j~E87}&V)z?CEcs|>eK;Y*o*2qu4@`SKFf}J+ z%Ge-dsg4_>fYh24s>}NG*(K~e{zUJA17i=QazGiD5<_+Esel8)Kz~YD=}&34h%gS0 zf2h$)Nnb8%|R!#)5WSlTFS9?{Bwa_SFA@F{vaWy#(nXTRE>*hSF^O?T^UXyRJD;^gx+7@@V?#XO5Nu^XRkH{B@F^k*4kNVTYe*2+>Ujfd*#K}m3n zjdrv3e}CW~vE=K>a_pBaVQhkC5C?mY`PDeDMZ0S)7HqV~SP>WP5reYX>h{~Djxmgd zr{xDPm)1r;$`~I$j7mFnVG}JcB$=qu`pbk#PdWlh z+aviw?wPNU*>msREW*EQoio?nOv+PpLIx>6@#sI()pAwL9XH*d9+#^a7LY467MCku zUccD916%a)gO8`{FS#S(MEf-h&6$>5{`I52jc6d*6dEvLf|KuSF1jr}^RH*qf{V_R z-oECNvIEpR&=k@aH%ts%bK!02w!hq$W`Fgp>8qb3 zkHM>~Pr6~xfRuwUY0G8j-IQ*-{@%3E67!~I-?4~3mxiDxANyyz>d&{z6-f)E+2^h& zfpq)H)j^Lv{G?vpvB;a|AMs(weB098^Ujfeeal{HdU=b`lr9ha?P1Mph8d?%Z&+&n z^cvZB1}$jgE!W;7SN`0do|fC2aCOyU@_d^aXPRMW2))EiN3FwQVG z2L@YckQ(~O#<{GNw)6YUVfeIDPfJff@kGLeOr1LkYZ)=HFkIAAm!l&QC5b0ZjNkTt z@bsq zF^tRgsP|!@=K{O-j*#G_o3Y4GNhZdkG=8Z;As_2gDVG8x^-JT){h!Bw_}U4b`=Cz^ z>$+I7JWSJvi*}%DjP`&iO{?3lb;vzQb?xt&&>mw%EbLuOcmLx)G!hS^Mn3AsDw$h{ zBY%G^e(oQVR@(=jZYNJ){&erkp8gjQxwubu{iu;v=znibclR^4!b#UG-P07rp+;MV zRsuN9KF3$A1cyFypcd=oEdDxB35uXxAdW$4W*rjofx|ee_8i}ASYz?{k%Rn{v{Ay7 zHcHwkV|?Jz2-s*>T~jk%*Aio}3>3@cq%@mm~_EIbXWbr(jCsD z4?a8Xzw7*e5wCS*3s5B1GZq^t{N4x>7jY0FTilKsr8$0JgN5B;PI z0>5c#BbR0_eK-9qv+57ta6-m=W>^~bz_ice2N=+pyf6mq+JovW8i8569S>YxGf(5O9?&w0 z!O{3|dk|s_zK;CFKt?^#ZJqtEKC&wIQecdZLJjn1ei6ky$Uxa*zyT~i=yTY~Cq>m$ zM&DS5eR7vU`DsKN<+~)0Xw$@Sh=K)g^|2haCM))W_g~B(DAJ)^FYZea+eL|B+;3}N z)V%MCzu=2+aWGA*A1N>8%MQR9Hn!d3Xa<=`h4{2_+WshMu>}kOUv)NyOdH`%*$}!W z7#tZz6*SaQ0^)Y3jdo&yBeskSYV`i69f62xStAePaGdh&+(YT>YN^E>kwKo%QoHfi zj3u|i;C&QSrHzrF_R%q6EfNdbS{BQL22rmO~}lE37Py1 z8SwM_iT+0n`0#up_69k1k2W_lM) zQvfIqCuF>9UP`57zk5#F=W{<#xbozr>nEj|XPc>{hpRlcd9OWpg>S7Y#{Xd9-aG%1 zHhs&a^vpBQre9w7^~rje?|(4O|JqO^PsrE^zyXT`AIAks>M6^BR1{73$)Ew>J$BnW z%{=4mX?IS@Q!qZK@eh ziA2c>7+|$Yc2t0>E_{Z}XcIC3tG~JRtSy7mPN`~c9j98syshCL1A4sndWKnN(>4qz zWO}-Ms2cXbOSlL6CS(Ss>*GK_B2Er~7ZWi!p~M)2Ar0d$Mi%>E%@Kvn{3TFRS<%|! zsc%m}A`3AjV^khuVvHxR<%Rs5f&P>ku5E}v4zOcfhg1w5l->Fu7quTyCYO29>K59g z1!YpZoeZvNed@X~kSE$qXYD5&C2_4^w9`gOAH&FrQ8rZeNT3cKMQ*cudC5*deTy-o z_Y*admHv?4dv&|vM;tjIBTv*5JDNTE#B#|)laiW9<;01k6#Ny-$ibqhy<};+(c&YH z`FV8s(2v}1txNQdC*nYi=1+Ug7W+771;L^?X)!qW9=}zjqqV@mIfMtxEnq0OSLBW- zfDQ%LQK+TW(w%OD?B35dH3l5MM;ybtGcewM#(?B@XB_3p!QF$|xFiSt++y3td7Y)j zk^{cQMrBs8N`B&~r;HpC!#vcVogO6`b%$@hvfcU^XPlC)|BuLeI~C5`X*$+po3@rnqxwO0~Xm~;wV5H zWye4#N*O=kiwT)m=!DE3GN?(69H;$3r<*##k>g$oXb6mps>&GRiodyf;!+2cI4z?t zTJ#HkQPFbED-Y^wZdOxcLdH_g8HgzDKrB6@@y2FUm4)eSO(bf=37M*!hLvFtOjA8D z)(M%xV?rKFwAF&jKpo$qlflANHpbiJiIriFyc}1ck4wZ@*~WJ}Bfd`&+rO{>`#8X^-V)UQ zkS{3xZ|Ffj@EgfyZOAQw(A-3blWKGO!NFWd;zK+xnY9z2dP?$9QV*V3wr?_0wBt7; zrWd2S?6X8ZB3m3WoRC4>f`JhqC{$T2j>!-a3g!NVI+k)gE3W8M#&agajkKmui|OMa zb|hOL5B(tVfj>DY4HK@`$@>rO+QL$?La8=E8INk>sK&|FX+x$g~qO;EGEO4*lfXjVpVJn0egrmZjLgFQqab zcDvu_j!l0&^J;y26>`G~<^l`PD-TJ2Tl)0Z*Z)6BH@+PUw@mH%qphkdc>Yh4`u{NK zXcICzVC5g%wtGe!H(GqDtK$o4cU&Pe6R(iTSO4XqF^*A;V+TTorHp^ns1L!SAWN_6nKIL|Bp0pK&S8CN=qAaJm19ArbAZ zKDGw0y`Fy7;Vnz-v!NXJz_io@aYE)!Ye(%|i`2CM%+H0RKf*RxH=~Oj}D3x0x zbHq_^jCR4du-bo&q?v~}lXZOXiAMj!LVxpxhnH%9t99+rGX_t3l!HnD3TY|03ihx> zx1jD6$;6ll5`Af>quX*xPg?N%15fuQj{cNg9%@4KL`8o}plXh2achl%-np#Cb=Q9+ z-;r!!iuPD;qab>;qfQJZ?U2*~*DkHyu~s4v{Ae>B_i|^9+uyZ=a|NSvhV5^DF{(3l zv^aJVGk*K=z$H6;aFmyfgZYzjWP3)_sRf5E&@nMUmZWIZohpyt(6i*Ej2tZUn-V8v z;OX{Gh)O|RrIZR|Qa=didjyfSXI<-}i-rMuj=x(DL?XP6I!I3(Jh0OFQs5C-l$ zApr&-VEQIxXcseSi-Zmf<58}F>YQxX_*>tSz^qgoVq>m@B@#f+i$o} zUg`XFT55&YrB}^q5B8n9QO3^pFL{XW{dYg44_=+7 zLe2fHHu4&jY(IGzre&tRWpsRPp}zJrEp3QV+ns|LJsE&n6pEDV!_eI zq%HDzxwV>i%RxNRw3qFd0d4NU(T5ykk>n?q9FVk8(gv(w!BFFfp#+xxl+gyeE&W<} z)2~Jv9_xtc$HFu7Lzt5%)`@0O0w6y?!5$}MGLd3n$5H}YwXDg*7%E9XWkELu z!3`&5nm!p;hdnTj^ZkvRE;!$Tg4j-`$Onj)X+3InQSGbT1 zK3K>@+1fdv0t*|5RlKDRK8zCmlq)LW$vXZ*k%Lt@sQ|)TmN+5fj$)LB@vPQeGYqBq zbaAaFr&OsU2MlQY@GVPK7Y!@J9++l&0N>x4R$fIXgrAWsWcZz@qoOP0)Ubvr^3}fqBD}Ff#2u{iXIofz=Y;3^j9uXORsb@UYnIk3eY$0*R z;a$jjpJWW!$nU3q<`FR#H?F%F!}Bz(+h36n-mxvHvpvMIU0oc|BQMTjxRzsNZ?lqP zEZBYzw!j(4Ny)q*WBbi+R;>9U6YTuBP5M3%`<9@W_qo9WIL9DTa}hHTexZZqLrlNS zPNW8i_iS~xc+Ft!>;>4m{YMYdQ<~mto~4)Q2kBbD!^In@u$ogyOrwPpbBfLWmkuOf#DR@0C+J(2W|r4p zkC*SUXr<$T)%`#Ud~h)FVA|>d3^}7s$e=|W8%Am!#~Vsw=}SpHCuAnzp}w*- zqn!Q-gR74ZT;z*(!{?tAHAq?72&_)X=#R5`(vLiyaSwyJ7&(9wR}z9vcrhXKul(9; zsfr7kQNz-<%(PrbG|MFxK|OL1h8%D~+lLb}J)JjH4SQf3=mC6xCwoFBk9Rr@VT=kV zN*EhBZcyr3k5;-P7e+V?gOEJ2afZ4B3m?_u2v@9?Lh=A+dGtnTU(t5leqc;#qel-{K?r zN55DmFJ&JGZRVHHe+o%#0THz%CnUEgPhgon=QD@}1LE=nz1sNw0RT&LLLdEMW<17G z7p!E_ZLf_R{rh7ckwv^UIVG?>G1kasZEG9r?CCuON{eaiLbq0cf35*7rEfpJy~@8)$g+G58`ZE=OJIw zKU*FmBOqvn3?LiGrIbDBpb*&eXVS!p=_z^35}qMrHX^Mcz%Kfp*IWM{^Ykqa;&fyiQI^!&%KymK`z7m7Ha~76UNEY3LBy@nkSb zJ!UIr4tvqFa`?x>~Of||w^8T+sswWg_AYTHP) zK3;NB!oLp(J+>clijj&+S;hQ`F(2pS{!cqkV1a`_xX5dTc01>93FDGTXL88y@xY6P zFsrp|X_Er{`$GiV)cR|V$}1KajX(lxM=*6U5$QR&22sddYwMb$V*kS%3$TW7z0_Q- zS_Gq0Mmx&XQ$jXD$H3pe9u^exyx zOunGfBC&()kvIAhPe~r)sb~JhR}6}k75j?Vf{_CGqCar>z-D%$SQkTQ5sw@o3qz>u zVE+mmIN_oDq|(YN|9+5(Z3-?LzMYW4ZzvYM-3{L%F0o9VNQ)~nPRL+q!sRvy$_W`> zwgunV2n`_$osfB2CS>-)gp7Buo`~c?@KxuKSLA}(CS-zkK;poHYd9gJJuzGjdtjRD zfsrO;(3mg^RBa2xN2MoTN1t4Y2ci)?mI)c)K|@~D2mhrjqf|%+&&l{HiQ?gQ$oFM0 z6Hif;aWs!>r7aIAT;1ClwSqA+w=O$Z!G9eh4srL#r#Ai;P;*7ULKTkpnK| zfkX~7$o;g~2E*50kFi6Cuwf6pM0x<qLLSq}Ye}&<-p55XG%K9Kikmv%uiQKiB`kWAlM)!70uvW)0k}?|+}W?pnNN z&5QFrXYYOX*`G>Myz-ts)2p9?x~uhjH-f&@QBh4gnAAK)zk9}mUtjt-vtM~m3F)!) zxkGyAx%>0n@7Z5qMCBv?*0%i=gy#t{oiK5&NK@Lg&xD_Wc^NOE5s&iv#+ZJtWX^u0 zgaM$|I{_9HVWHZ=bX!8bex<-RSY629pE&taBE*SMT0&uK#H2dvJO;dY)Ry$z@xbau za`;E%H1Pm!b0WJ>H0r1R^H+NxyJ`{963MqI`s-xUqPGH8RZ@rYSUm5dv6_P35c}8K zt5pljGQ@0+6~GIa;%QQ5T@fKh_CBj`nI9(=Y0c(7XDP5Z=#DS+Ysk*K3T>A+zrf2AhLe(yYU!A-7l z)nlZ0m=5FgXC&4so91d813JdH`v?M(kFD&%NON7vIt5v{IQOSJl);};&Fll=CT}Cn zk(ih$aWr}h!XQG+cip|vi@bjg(jMYRiT@CbqnNEk^*gu#o~IxMR)uCr=dzl+afeGw zWF=Da)AQl&G*E4Cg@paHfu}L63|T7~Ypa8^?p0e+X(0^9R6PQQHCcUo%v1)7%(#NmZlHDAL_v&B-}v&NXH!@<;x za*M34eZBWgg-DmxRdg4eq?&k$vGx^L3MC*}P|2i57Wde4(w4OD6>K9wdkw4TTspwq zP4uqH)XtihO8Pl>-DQl7#UiUuIVs_r-mtWyIP=~5!lC(*Baii83?XC$Uv0Mx{$Q~M z?tD~XXy*VS^|#pRh!36ZS`6hiu;sUbi4hvlG*b1zdr$yk`K&rIBQLF9EA-Q#z7A5LVAbKiUMz92Hj$0kwO}w#VyIW94FO^t}wA>_GD1lB>)B)P> zy$9R2gI_NxVQdlj*xT_WFMZxpDi$T5b++ zEeIq9a|?6(QsN)G;l8F+@D7xva>9Rag(=AxHV@-aetbn4I*Anul?owmWneM}ln@i> ze9W2ltK|rvOr)GjlDVtFGTIjBW=8V$smsu>@zgx5GqjB=Bw*rrWuiEeA7Ux{B)%8i zbfn2>X*x~$TKr$bYZuIBU-gPfH_w_4(R9EYTI6KgBEFy8m#)Fl@=N`JnyIjdrHT+p)>bBHpyE>Ci|SHoQ~ z%fu0-{Yl6oYbNHRrWd+;prOv0mAE|kA#)~DOjlDM9R)PdV@$S|z*`o7WIwqtXF<2e z{o`5xyOYG>iyN$DKutzTrTWEAVCvJ`zhWjef(OzEkRrzm=90%Ut0j0&<^iF5_<$$Fu`uVig?Ktha^9RmZ-J;Maisx^iNBJ0qVny|x~Ic= z&r+uF^pxPz-8+*I80GH4s?%&S&ez361vtEM1;FCBtr73klaDZIgqtgikqP*VO^gUK zq(e2`%#Ucbg8(GFTwwh)3u#RFb=@1?KI5viP;_#Odw>uF6R>kX8Z$pMo0yfvTaAY* zIZ8IACIu-yhsO%?tq3?BMh#ZZu-EG|sf8nR1eA49kxf9!5f|rw26&Vm-0DPc-mY(d zZfGYtqIU-ARlc`hJ~1BBMR-VK(%rtu0_;W($TL)ueSXT&vVjl)_VaeEK1+_MgeAih z*$=SaLu7VxrhoqXeyz4qa)V5BoaJYX-Efu;H;R=qIv&38(6dn#HOn9K=+6D-)g2yy zT#n>@2p_G%D(;}`en{6ILJCDvl{L-`74FazK`+;n@cifbSuO7_#av2PvuYR(S|`VmLN z+J&47A}|!Ey+pD?urLcjXdSy-^m|Tk$6gw-gV)&ROnOQu+gdytY%t~6ZSslz+gT5l z@AR?#=fPc_lqZ*b{AB-LvCWx%6k2~hQCKU1tna7{`_S@EIvKS&uOLrK_@ByYOi?mM z!rZ-M$S3aogB2;-n=;62nyYnCRTMLm6Nqc+ccLsZ(4YWQIVw7r`$F(bk38;UwDmWzVs zB9~IJYn>1S+Ugr4hC&kFNDGGLF7#i5m)H%4fmiW04v}$hy-<-Wi~mY5&Zw0p5cu8?35FPC8_TbTVnrge;DT3``|5X&X6&7Bu+XDx*C z*zthwfucdeJw2{Wa-x(ek->(u9{wHSs{MlO%IH{!@zPGb z;FPkKrLI9Fr+(QbviHG}hQgsRV=D0vd5kK^a2_u7mhg_Hdo)7X8Lf+NyLqW9m$u?j zmCWKeMysE9rPTF;?X) zXJ&1mL~_5ER!`S*3`X}U42-<0UD282i$OQ{FmHBW>$)xZD);BmlN5hsx;{aPaU|VJ z>u_U#GQ*C`31V?vKHIME2Oln?%7Y60%-;(I^@#r11opMUn8< zAiqmoe-d5}0!Cy=!y<+Y+HY%tm!(=dc0CohN$Tc7aj~#K<9I z-65+odyPdd0@)4$xZD3|YXibZ1RUnKWc;U*FCHNMV0!;MG0pK>?L6`Jo4W9|CIk~v zljz*=5U5d8`uPP;B$X1Ge_MWA*kS2RXSM>}RQ^@_EaK8gk<;>4FEFGST$me=*q%z4 zS?h(<Rm680JB;c6( z#gE*wA^XfsOsHP(tIbnMP~Qu4uX&s)Qq5uO{ul?9-+9N_q3TqzzKYKO*l~A4#iCsC zI>b+`>+32{ykM0!JSMCg3WAojJa1<_>%4w5rIrZk2hCB0IJZ9Z$%$Ux{^q)DD2!!Z zR1Q z_>UvFE-o>Nu;4AlpC8P(|vJi;ueUW((fb-4xIjRHf1h7|DrHU~aC zkc)(O1{uk=Xec9$wDp!@ZHJD{@G_Xk`2XeM?5{-?1W?+ohE|#Mmwq$u5!;rQQ1IJS zl29Th92ME6$E`&k-9`wexWC)cuR3@cZg8?aU_vACC6T?{5o@{^|nF-aN#@q1n3ts?&$*LW-CN>Bn!uQgM_)XUQQg z%@1fhq=`jF^J57b;!1`4)ox{$|hVv}0WN$@cEY34%MJAN$-#wjpXXy6hYTpx?+8<>hdZk&H z^Rp%z_1E)MX74z13cqq`rkX&*V?#0zY*8Y>iJy&HDyhQflOJ z@0=0+os_H=DXgMIzGg$Ea2f?sMK09FP((|nCxzWXbgAC z75eXGCgs5t+q21DKkrC1i52ZLH)gc7yg2jym3%p+Do}xQET*paiz0ViMPGa^R|4qC z@k|A zQ=S?3ZTS#Y@fWj2bR^sTW+;Sa-LyG@=}7gZb~9#y+HNJ;5Yt1v!F(?2zvfQI#|yjh zT-%=+sg1sODr>$Z%=ftk6y~A0FY|Y6v7oTq0dxA3tWQa|b7TxW?qo@Ljh@Y2Ls+0Q z)VIA45su?7S$uhA3D1t@jl)xY^Q$DOrlfpG_ zJ6(*^ubYKAN+tqv8p~4T2j=*)UtTCuGrK8eGv7k8yj(h^E!VCzNL1Vseg0UwEDocI zojxI_7xy7O{m{QZ|B~j@TT$^CSS{{%`7BcUv|fb)%L>^u=WAD2>Hw+=4evLRiPUdt zMfeGPH<_2wAUxKr7D94PG@FpV>5%&-#r|fGel>l;K&%0zdn}q{Cmv1?a-IUPqrOeA z!o}k(qT}wwC>422RuqI8=AD<^1tWBUny*Tc@aC}=_kXqdohw(a71_FGs zJS;I zg?VPKEg{#ScFfyX1+Qu3Qy&<0)*J-%f2i6nC0z}XTpgl1NG#&*Bpp)wFZ1&3J}6>N zCb2SOcaw-640rjK4T09=b*nm4!0dwRp?#(42ug*4mWtKnK&d(SKeA|FpQ$I|EJbnq7sB%!7>;}Q>!A%XS2jKZ+@40 z`i$oHSrqKZ#I2G2ecdOZg3)0&fA;ez7R_3Fom^;|nB7F5tKb(jr~0%uC?>`5s89Eb zd_t1n&l+(}$MnXH$JAyShgV)a0OkSD2xAAicaEeo#(k<{iQZ%`w-sNanN^Y_PAfLP*Cr~TX(S;X6gpS!c`5q6BuP%cE9r@6q9zlkUs7uRV^jzp z-e8I)K?V<0Hp^!v^v?Ok^uiaR*ZkQ<#NHu=kEWn-N0gz!)j$CfSR3Jvf- zAY}9(5W443e#$2Jw4I?YN1#pdUKCd(k1HVa$NB!e98Qqvz^N_~?!6<&(_f}6H>{3N z#0?5KxCRY}QEyWCWeeL{dX?!CuHv$5nZqxGlB2FZR7xxn`AVRwRrD#x^H}D|)Owmq zP^{^1NJK{)=OvbX?3&T5pZX z(f>`f{LTAV%aWhmR^wnie(Xdr&#iMsDt4GG$Q(7zOTjGfuFgC%FOK*8?=BjU%92BDqcjc;LQcMeZZ zk)B(sp#hIBvrohuAE-pHV{0F4x(K?Wt31O&|075O#o<{Di z@5(QjyQF$vg%8b0Xl$&$)0VuvNNlJGkOTPQNgEAFFFPS(Hp2B;@zbksStBFISU{Ga zu2St%qY8pgHHt_~-`_wiv0MCxfO)xCuz|#09ZhQ=Rbce(QSf~|JG0DBHl%`W)$;An zt$GhWlKN914O)#@IY0FAt&G0S@jI-x+>V(ieE)g%7Z2lP9TNa_`#QzY4@Td->han# z$866YR_?!JiaM!37UCeX6Y>a6C#?TV?|TUc-me2ax;V;Dd^`EfoGpLjf~-H1P@D6x z1d>iERgm(*tVXhp36np6icxdKvwZO6TyZ?1pNIt`#1_ zq2Owdjh{X(sYbtus%U{GsDgGh^lUsQ+Py;D__{ZJ2WWQjL*UYOScVb`v`j|UnY$MN zG#S;`2V;H^kGQE8TF@Ia_xpJ}%;!ZzYBQKjO&#D%Z~mhPI>7ZSOg-~2Rww(TJB?U5 z>bI-s-CZ6ko1eQ36miq;wN#}=S$2I<2 zutmpnA75NN@v-7V`_*;oq<_)D-lHP8tYc<#bryTjSeI(XhLS}?|7)8NMW@Vum><1N z5X|J_6>0>Y{mD{#gEt8fTrcRlju_DZQa#-JV;YI_<$X*C=u&Mk?2td{gf*bhV7lG% zD_r|KyXKt~*KPYqy}DW!no zTH_l~n(x69X&VeT@t$6{LjyBQsc2kUjvm=tXaW~rGVI+yP0;&J%TVkhzKy$+LK zT*F;=dZ-S0=!|m5rAqSpcQ!P?SGc9i0U>t*}o{DvufZKPgY3`G`=4IBeP$yw>OjZG_xtnnQmsBYN|8OPzjac5RM@=97+c!-JWt1i@}*5q*XyhhZ(1kKPK zyQqrCm8tj1T?a@{rTD2$V~@hUgN`8f9$e`(U0|#?cYHc`0#>ld2Dx8nnt$|2wSE|j zH6~~fZ7x?c*QO%)>deA0!IEXj<7e$p4;T_#01ROl)Y>&|eeA9Vj9Px+{uE?i3jn+! zbVe$Nv@`s4Lx-M28T~EXRI1~ern5=WOgVFhK`LNEPurkpv`pu>=W)-t?ZuXoY!h2S zv2vT_&=&vCF_a64xgeGm(TE6=c%Q^9{etN~vTiglH@Sw_-oXj&tOrInT7S>qki1gF}{Q(dkGmxLPiXibhI0fpmq?LIfR zGa5d&6uFm+&jF$1N8?gdaUo={#S^D5kXIAHx)g!SPt1QKcx8xClaN6o9mSHaAG-@~ z3!5?%oDUzT*X7(#^nPg#tHX=C3mTJY7tRsk1iYNzxRU0)?jiq~%n~ac_N{-Fd(S;C zwvegBZT=$hs@WlvK5oYaGKej1CgW&xYd!A!Kt|)Z_1>e^@9yu0|6?Mop#B2(Wy9sM z@K=N!t?&O|COMRl8^F`l9uUsA;qSd5y5ap;4%+LI5S!f`0he9tT#WRUspU`@Z|>K= z;@i0%ug#Kkis!H}`FF->nDCIji5`i?TM>|2Vm_sYxqcsg;*Guk^bWMq_W6(g=9|+ zB(7tFa^_`$Sxiqd040hf#h4_nr<<$Mc4_4DQLoLdQ{?bA4&@EuSQPad3VJ6|l$B7N zw|84Ic_94PpMzujBYrwAgnM2>ywK!xax;E|m7G-n}k{_XUq z&K|m}qABVhe$hlAZ2rS%Gw3_yi zaif$2jVPbY(0oDkeJQJF$l2h^=X8hcFNJrk@_P$no<-vdJ>v|O##=9KDuGr67CQQ0 ze>uv@{>p$h^oZ{VsbGWYvah~MfXC1%9o{nGFGZG}Kxlcd7{Y(TJR4UGiL-UWi2mF) zi#4c~3V%W8dMn0PRO8s5IWTT%f39@wfNEbnHoaQaf1pH8r*a=mINDza#0uas zY4myHp?6Q!|K`K<&ytfYfs+=p+b!AG{5mbPiqyFC3uG6&UeOGAf7GhoPT4cCfE!Nm7-a9|O(_w!cWp|KWhAtYqhpPRLU>wNG_2cRKg} z?{;=|(_|%vg)N>9##d9W9T^C<}#nmu~?mKR8&YrP*-WS9bc2IY+ zb9KW*Z?^s0e0e*N5%B<7%nb|ECElCT}_QN3UT9PGr`L?j&^g z`DX{A;C|SS;`yqZ|+jJA%hsw3_Iyh`?($XT^Y4JvD7v%dsPhFG-kOU z0Jkd0h3eM#owWcDYJdrO^6p?0IMD!Eh`SY#^5SSmxQjW{bn|N5=pdk?pB4 zZ~I6)BNxoCMM2DK-V>$pvAk=r2MSSz_U~;#qpMr|;MIt`bi_#?;C*k0du|Z&7nNCU z8v8$sasqEu5l4HDh)TrPwIj4@V_bu`5^(dgXSraGiLnCEjPEu-lPgFz;{k`)m#18} z-G|D3?G}-s8+vPvzOGaLtE!fr=KM^?ZzT!vwkXDjz zwHYQ!djhZsYZeoa*fwc1ws(vCi;*%>RF2!}+`5aAHfN@aPBozA85pOMDqnZM9i=sWADHkbDy&sh4P$l%6493Ml{4-5H8!xq0MhUTw#~r z@+yT-q=!(p%x$G|7jnU3iBkIa>|F4q<{yG$PM2Rj8y`@17A-#@ihc`?AgMi&YJGf>~(tLuz1^453AQ|^CPj8#rRz= zSk}AV(>2<{UTIB6;1gAXqZghxhNi$9k3?}D)m=6s^w-J@t=B6*@wGiB?D5wKRw;$J zs~7?t4AC}zW1z6+4Y;%ig2#u>MRn0GH8G(ST;1)NM_l_Ya@cc{u-F#{O($W^pkwRX z-+^3Vl=U&5r&Z$={-0GO(+Jcy{~Q|t8F}yy{BH3D52|xnO-fb$uSbuI79*AWPz8hi z=*>+xz5<1GBfhb#xcs=lB9eZO62XSkKdC27)(PqqczAkt`#tCGcRuruqngb5k#A<# z1$2rD`Z^n2={@L}X#YcxZBcAK7OK_ADl{qm`#8>fGo<}CYzt}*GlN=ul!sjAFEZ7H zia7A`o>+5l>$sR-nfv3CrTD+=l2NLSTzJF+l&6@x8_-4G-qoiHPSE;=yu{Y3d+<=9YNY$CgWjiuxX4uQq3c3R8eH?aG}(^mY} z9nV;Yw~qAy_2{ha$>b7RX3wuZxr8jXcz0)%dUXU-B$NHC)cCcA*;+$?tC*mM{o`*8 zF3LUmmM5M=?S|*ryDDi-kK}P}0z$T9{kypUIlPc(wA~hPo3T8zJQJ*^>Z5YjIe6P7p>YVS zI42QHR2So>T_RQ)H~8`?;-NfL6I5LYosQPoeTY__pUlS8+aCBp(f*oXo`d_Gq!^2Q zG_nR#3t1Cn9UmE`Nwpft!;p66h~2r)IRP&Q{xx5F%=h0__VuBkQPhs#)?*WBGxDf1 zHNtP__xo%n-E^R8;?rc!r;?krhem6?e``lmgj#q?^#fBRn&{!fYuzph!r&z0$FQ@2 z0h@a4yJn=pj{u`(O$(3Dn8L~I3#(RJKU2hb9*qal(bIz-mKQ&b$V2ejL%qYVLj2_h z?RCuLD%qb=1Y7%B<3>7PBebd;mxZ+oAH5fdU)EO{|0PP97fG1A$VfBekNR?gZ4n{_ zCtcW)z#or@w^0cN^mWf(rO%h`Jc&T)6aq*tPB|9P(OJV&IeB#+Bh8sC#f85{9ABWH zs2D~q3L>fWnSH>mm&mj83#a4=-L2z_gN8(368YOJ$q{?+DSC5t{-N}TM+OdIjY4@hN#KY*Xbx85JnP@ekYxO*9C6Rcs;7K@Wel)-ORnlwWStV=Ui|Y_wvH0}ad#Tcc}UAo4ySv`{nB2scN(s1 z8(jk}c2)RjxLEFlgUK;Mu#(1dPLc8e@ZAZPU*%2MDb=1pc(&gTG$cdX{oZ=t?8lrk zRl5(nn!tfLxe&MMxN=^IA;|enEijQN^MF+(klW%;GQ6`iXOL#+spd!EEN3N&F+R)O zM-@i)S7fTI_d&5KYNS^qn08?XMU+QyNz3e7Uj&lo#!)VcC#vu1;S9P@_9mW;Q2~T7 zJgEH!sB{P{)-RVF?~UcwWx8t%gzzHn<}B7i%CUGq1++VR@sM|4akL#$EV^r`is;#* zah8_`H6;FZ()VRz9N8-bz!vw?1I`aPjqfHrbqG2ePeG|h|30x_>^cKaNTbCa|CW{= zu0-qZn1r3wUJMzZKUc%{H~b!r5x=_3zEvu5U))QeJy1eYe+EsqL$>!~HeeTq@E+ik zeENmgmOp%rrz}h$(w1ta+H4s~ENZWBIyLMc8{3G$m>b#yu^&TQrU$8HI3FWNG4+k? zWqsHfPv|=Ix?%ure1$2as<1tzNW+w1w~+UVrk%a^d}WJq0{3-Hn%zN=+jjYhDXUO)oApIq60!7F1ck zP)hKcT#W{w@%Yg;H;D#Bf6I3^I`I=`ESh|JSH&&uzFUQvVAVYdbd~*(aakLEr9o#e z)AXkRnTl#3n0P6oFnEQO4Dj$N{>6 zTS%Oow48hIagh+k5*$9)W{$>3qa4!KPhhukvCyIEs@?U^jmCSo1a3<)Kv(|+EPqP{Ic1Ku6l#dVp z@~)`dZV>5PAK%mUzsDP8L_$+HtLJ>fzaRy|Zr@0Z-1&{$LCR**^XjoGGB40Q(5v`; zXd6Vknwimxn-ee0(ZSEg15>>o;#*gLKLw><phSOPClO@7Sc6nm39T} z`D=WT2 z(uu6VRjW;HFHHFDga5|fU&OX?TRZ%&x(m9v0ii|~NWemAq-|R@Z6gc1T|`err6#l9 z;XAqvJ>$qYM$BTQ{d2eHl_IV(uD(s!nM9*DyWWD6NsVF22~qk$g67A5Chd>&tA@t* zSN-8!EcGjzkufB{yxZ!7+&}z|3DcZ+VF60)%tjDdY?&b$gsUl$KPt@d{;D zDoZBp%_u$*=wh^00e`>M#i~A!C$0=bw zUm!4L=e@-O()NSVG7r`<`Kw+sB`TpF@A$We{5t$$9B{>VFB!g9%J`-?B#u4p4K3FL zT4s_JZaLNLZ%qVVHA4H;!pr5+x9j!*$H{T+f9mNIrB@mj@83Jq3LN2U+YDO*3t4$( zoR;I+-am8dF-_|!>raw$2$?*Q!O=*g^pmeS?7}ytfi3Peon~MJqg9YtqW8kBq?`?^ z$w&DvMta6=0bLTD2{3?g1N;LPA5DMb9y@jgfGUHouHB6hc<1fFl&IF$3nZ{$6Gsz^ zRi_1QVFhenH-at+d=w2Pb*fx3<_4=J&f=4dsd*&+gisYt^j%}&0+6jh&jAAfbwy-Asl0KX>K{EB9uu}w& z_Z3TiNm1Zc*WOGeSITn-ZEPuh|IJur%U0!kv50BPPxE2Ba$4uyox|~CsL<2hl6>Aj zPohL>d>L{OU2Cbvs^DnPSc1$BK!LnKvr6-nOaQ z^J^ny{0Cpe`y2t>6X-POI?6HiOhq-uPOMv;=caL}$F`A9GDdyBcEduqtu!lpTB5%^ z6{W(qwAHmnR#YHIyR%36DQ%BH?P`~6VrgfiW7|T`faYjF7&vKm0h80G_Q4Omf~3{y zjmAk#)Q$3vW_SJApzYka|;?Co`j^UE~y8G7?9h2SFv$)ng2kB4nc35grr?^L)&H_b)he+>n z&=~cdsKxrT*i=nip1?kz7Sjh>L~Wq~c4MI21ikg7vUK#({y_EqzufBk7NYDeQU3nO zN+6H6QAe;fI^w4zIRK^u$ItXcZWd^oRa~)U&r$(2TIg}c9|#fNmi2Al<9_9%)6n4F zifj;?GjWasksSf_IOzDqlF;!#Wp1+>cC;5RVf`@Zk9p2_`D!9Cz-kN-fM_-jIE(-- z)DwkX``37QPejd+SDu$dbImn?^_CJwADtuX%JJ%_ z8oGKFs%BdUVM8$fXS%AV*ZgwaiVz4Umy9b1UjX@uyR6vBLT=eL;^kYi%0~ws;WjOA z^Urd2xSWJ1GL(BL$`TEiGoYbhd91(JTB|(g$UU|O%p*X7Z9Q$#&P(219`ooZ^_+ox znrZ|4P1D0%F=vIJ^^ixGnVrdDb<>S?uJb$uAB*L>-@uvp;SOWf2k!Q%3Se`uqrzYH zh5G3Vu9$b#`F%`pTz5{xWmc+!QQ!jNkrBDq+UxDm_yx;gu*)XVSjenSxKK)d!DsB# zRT=Z;sWZG{^0#$K-X|SnAulCS=2S9U+f;Ej}I|uX+Lt8k^&xoxN=jbpyfr?kLbUqr8>k*=-yjIrdQcfm$WWoHNU`U#B^C$1-K z@Q|^rV*Q-JV(7weB}%kY#?qVq3j&K@%gNedSX8*=GyU=bDpxYKPpn?(2>&$r6g?UF zaANu>!E%Bev=#^l`^VqL1Z+IZcb#7gIC_t=J74^tX19oYSpBZ!0t+59iZJc!)&>^e zqHQ2xJEG<-My(UI&18M<>sn2}f7bOcpkpV<-n8X&GgSBul~&sK>olx9z1EQ^!_4u2 zCt0Q4s#qd&|vPO7X;Ve#h)gG&2VIB?q~3pzl6S{v-dh z?p>*5syJvjKC;=Km3EQhZ(} zFW!a}b#nO~G#5Oo%mX7cUJyhQ#z=jn^|5wC(DIIr{clSofJT7aIaxO}t+v$)3UyW* zi^8{pR9ODG9DQAB&8hrXN7qE#m8Jbte515pxbXY)8v+$JY$2X`@rrX|DT4m)qHww) zXaRggLn2YV(fSo{{UKKC&i=Gc7X8bR?dUa(*oyw$k~MvqLE`@e+=O&Tc$(vt@huyo z>#oSy=h6Ja=K$8D^6B4yI&Qi@<}Lj`5HS_O1-y?^ZMjcJf5CT;(< zr7bAjw!-CKGq9d7qF^#ni=IX=-N{TY&EEvU$t|@8dsff935s%(Ey2;HBR4$ zNvrH=Q+J81S7o!sd(CLqpAasa19nm!-aZgu9zL|s6%*fB<1!iY%YEoAc*InGq+zOJ zZ2ID`9WvTbH+a}y z_6!dWRIpz8uq8mI`$myuDTJSr+M6FS4X22 zAI6nge064pe*V(8s848XIwBZUi3Qr!PFtjx{^QfJyrYiQ{GJy9T{<~94T|NK*m%NC zB9CXMN?MY_jBPjf=ga$o{*ZP@=vw+mIWoPU(R8r{skXlT+`H_8g8ZUhxPs_+Zk#rd z->`rXnx54}fJM|+1r7jiJHPOHo}f~LI_WM57jd(=0AxL;{ocuzid#JcJqd0aUTf{f zD0Hy7B-O!j=bYbWz)HACL>5!oE}rD-gvMX~S-FztT|iKnY5_3A}sv;192Uc2D0xFdMr(c61?w9}!QE&-UpX*N?d%@0)5^ zWBNiz{Kcg6qQB>V8vuh$)aTL6bP&8A==%r+J`Rtn!f%nmZ4(G+mjOIYJoGUWbgi0o zw?7G69sS_H^TDX|P8B}hV z>72(Fe=5$Ya``Mlc<4d9+#h;sf9IKTDKAT4A#oDt*dB|;E9e1>*?DI!#L0Q&{UW<$ zeyHg-hhoa~isMNtIT2Lu|4XeXZnh^q+$h1HJ|tY-ak+h-^gCnMApG?1KdOY3{#W=e zaYio^b>;hfY7Ma_x*;EaUrFvi9{7v@2xzyQoC4yqB?gXMGg^(Pj1Uw`%0d%!vXrQgi2_1%%3 zwpG1S`r7}S1+crNcg}FxUjuGl5Bl29y;FCQ{Dm4eyd+{Rpe@Lx;jBd$Hm)DktzY1R z(R9GFzpk0W+P5toa~T^!Y4#Gu0wF$cI=-T zsabyss%nmT>3a_)i!1fez&{i&kCJ~>vQ>SzaZ>Q!{&jOv+jpIO#CL58I7`g7 zaq8K%y{$68sV3q>!6+y?TkwsKxL#g0VZ2cQx!?hVs#5`llkwPo&>)| zq+MliWxJaFXq9BpIY6Z{BsI^G%pGOE&4-Rj3L#0K$TF(}Jr5Dd4>LuHzBwkah}0Ll zF0Hkrtq*N7+!MBAaRJf0W~BbzK}ve9lwKbJLyLBowA>X;+JGVJ`{kzvZ61g}kp;(( z>>Z-34lG6y?vZ-CiY~pl>6uTO5AMdRGA?hBtM-6947`YJIQ+yrh(N*H$ni@3Y*yh*c2&+uzdd%*+xlXHwM2}uLfF^vtaikYUxc{k2 zXA^>&H2fnZX$4Wdpbj{jY_6)6apxH5xd|#bj{4AV4N=OlIDbhbli#q*N&pbsBS}tw zvPO!M`h0{UpiZ{o!MgfUapnX?xk4Y}r?J3g zTl~uA#BaGj!x@o)MSvMBkWXm5q&V^!hDM7rxsXiv9<@7I9MnjD zEoCw}1+<;|tX^5gP%!v31^r>N*UQSurUF(3Wvg=6^-pqRUg6Qe92We+IM?r9 z8{3{_flWj&_Us;ZBC2d8eCbIje7{%?JTG3inX{jGUAZFmutg_=S2qWPZOfwbLo|&V z`qh6eqY-dXeve0b5@m3G%o!wnX)92CvOFSq|c z0QNu$zso&B&X+gxgVVLFH-B*{6q>jg+D0Vrc^=Q}ClC+shekrlvHBGzh6RK5xv-$( zl(C*qa5G(+?hQ3G!F2?_uo2y5Q7i+s6axF&FHpqs| z7<^dejd-xthOZbnG~@%L_AMJ4J1n}rcHye}U=^-vT6KldX-j=Ej>3mh#4F#L>6lL* zw7Ldu;`CkHitswJ$eu-hZP*$|k}RhG%p&gK+=1<}XvcoU0hl9vM7mnFBWfglHSRb8 zPE>$~QufR|^c^mhd2U^=a>lAherz9yqlaKLMt8%N8gY)7ba}a+?YJbtMtr`0*7pz8 z$FeFd@p-AQ4P5kUPpyTg`h_BbMGGs8{>qYmWa(}89WugoY{z-qg8lmOK_p>$ zAfo}sq^cl<)&wfRZ`@}40~w0&xG-ug8<<^ef!^FjvlDju9k+{%zdPVvSM6JL;DFub zyB5>`m}?7O9g9_${?K1U>6R9Zc^u`n)>u)jZNkUw6Z+UO{>bZvF^VHpo`KEq7|R|$ zc{}$pyh+45=%B%e;y3_!IUdD^SDrlv97a(pl{5!m6-oJBKRoK`AvqPd% zdGn}Q;5xpTw5&pkbn5(u3^goJtW=0$Y z$=P$@1A`5|^?X4x8^6p#N75tWz!HbVimeGv3oyYNTz>uD-jwiXXaL{qx zyne=iEaj7po>M$&%sNyud&zNTJDMC|drUs!SmIc(z=Ceu^;2<_ON-d*DSvRt+3Qey z%r1aszRwBYIG#{i^~44t+vB|r27JWK zdvg5<*ZI928Yr$2MQc+OWZ2j{OU*q0xqj!YYp--U`}_STq7T;-II0Rfy~?m}t?E18 z83yc#YwYf{ovmLHkEr$oXYGiERaSZ7}MqO-`)*+7#n`sQT*%XZg?~pVs+Rr{mOS>o|0X$r6`#)S+X6KDKJ;+`;F( zxAlu0vS?#_EOf%-aW&D&`(I@bG<$UDE2~o%J{6E3JbWxwulb0>d&sVN&hUF(^ZJ*~ zf8GbEA06q##VWSePmJ?QJZi%V!{=SK;KxdbKYRUL z{>ajbopV`_^CRb%=PA|76@q59dypkLB^@T$DsDU^& z;HQu6xE-d)#o1xdx@NEGD5i19O8-%FiEtK1SGBCQ_L$do$Yn+&@W=`?5V-9xJAkRY< ze#fEZF&%$%?5^IUdsy z4)Oh8#v`$N{T+qg(U{J-j?}e;OG@+h%@lS0=s4KsTwQ;I6pih7$T%Z;Mu?U>$8N}^ zXNOS;+*!P1Qfe5V6DQUhf{sQ{8v?mx#nr|vfd!)&zOdjGv5lY;Di?fghdM8JJ>LD3 zYzt5?E^z8I24dmsy`|h1LPz$jrUoC)6^o7lXR(e6A@lf`?b_NezQ(lbh*KK00!I%_ z=c?W3Pguv>x&Or8d3oyT+?Pry3>^}fBbuS8>&G?h4YlVPggPfm4b@)HARNX^O7Yo1 z?_$Cb2fyNaTqDrs?<2ACD^~Gpb&aEUf1H%R^LQgHziU{~ZSKIT>n;IvBSv3##q`*& z9Y?FxwB+A$Z` z%|XYUeb55~dYvWWs%5S5=I6sEV(UU^(U!uA@3I~xNUSn`mAzHG-3 z$M$%~$=@*@*JEH-uGsqEt+8ngii2THaHXe>xS#v(;7CR-#If-E{CP})&GiGXWm1;I zINgJW0D1k$6&j47xK)cV8nd>-dG}-o_auC9*ElM4)G&{KSuJ_}fKgS~G??{{f?FAx z3#Eyu_PnI;Z=GAFord&wwaCA2XZ$(}!O)?XXy_i*Gd$!$eDK2`@gG^zcgSeta|Lno zV~0RC!1)6i7a-sjsi1_UK?q{bkP9tNg1Ag$EG{{AovSu= zVh8YnM%Mv~1~K}?uUT&D@!te2Gw1a^O%C$I^*PIqeuvY`y;YtX$CSg4JD9WUm$-OO zQ(@;%xnCwd)ubFe&`^>6`dJ^y90tKPVDs-fsV7}t)*SORKF}CgjF;@MOpML$D&v8Q zIi1f+JwQt%k3rO*H$A|t5iMICB~Mr2gG&!4zxH54hPo z*P)o_aLbg5Uw`)+9;>x5Miwc>q)W$~|d6mY`sYALlD)BUtNpEfZv;ioSP z27AOj{6Gf1qegjXft4i}F^-?dB^LqoA~{n)oTKY4xCFO6P**Q|CZt<|FsF7l`s5$(OD`L`^3 zj@2~+C46k_7;cRV7a7oLOl!vd@cjDC^at~y<$+8d#DOD=EbS4rhQ{MB{SKKaF!d5; z=EUuPWyybs%y;~!1=3{dWPTYpABI_>4MwE;K_nhd&MfI z@yII1iMiq|^x#IGGcIvoh@{p}hjBA5cC{k);I~SI8K;#K;-Xe_kyf047WF$*Tt8w-OA364%)7=f^({>@U@Jn+-4pExcn;ytVb-U znd##Vd?>AYYrnMiEpqn{l>PzSKggS!zHF|+J*__3cl^+f)SDUjt406J zQvX>Ty*2(2Z^h4joquusav*-rMXr7}@6Ul=6LrYyr5JIyI@gsvcVh~Lb8zJ217)#A zstrwfT+#GoIsaVV;eqV zEPia)p?zk>@i{dyS)G5g&fF(w*PHRjF?Q^TDqsAZsL!45AJTcE-wtXGt{<|mdFe-P z>*-q*G;2B#;qID zs_lVD9($qZZKpLO=H4y%TQE}XxU>4|gGOAyXSlh_xpzyRQBN*J%Htmf^G2rcj$u=) zdg1LkIxap?iZCNp z5xwhs>2xjCxT~|{h-3YD8Gtu#I&Nn=s;1{tUmIG{yT0G?AFFXK1Lxm5k9>#Bd-H+J zSo|;KVzN3a!ty{yze5HMS;dBywfhFH z`)I?Vzhc+8aKh|y<|S|-9vq%>^+NZc?atzL6E%#kA>Gh5$NF049<6J3&D*iP25_hO z^Nn%gyreOXuB9B{$oEkE42OT!!+&ObK0l{Tz%?>JJkGvQ+2sI79xzz>!MM$JFN|dd z*gnwcFybyY8D=X@>i0+{>-==h#OxKWGOhTIL8N1dV~Jtm5LcgA_~HWzIK_xF|DD_U zaEN1z4}v-M*$QXBesccd7prRNK5)uITk+mrYlv9n_jVq9U*jA+-Y$EpmA~q%55@_( zlQu8K!C<3~BZfFy9fuDLR(@tw`0iJX+()9)B@@ng{g zM<3g9n$zP4=Ro%3w`!rW`gb|-ar`uI{{BG$@92Hk`rb?9stU-16*X!w56&#Ay2Te> z8?orcp0V5#Kf`prU7H-o{N~{R2xKoXR1X(j;`Ff(pKJ)~&4h&h^!tly;ZiwR>T5?#hu6gU{^)q202E6r-%`@H+ z`3iX$>{191WZv_@1DSDi$m2kji=4YzLfx{QYX0EnT%w`U`b&Q4@=||jdcw;3IwPvb z1DSWd^BsbZW3M)7>>OueI1)d(8fltuk(BNp;J&EBwJ8&__0+@pqRai;}5&~OEKd6 zYAojGv00x3Di6-wLno9gU(H&_SG;)6RKSDU*{E!yQRv77#Y0FJj_YI4ovr6NE+!?X zM-O(5nyBA)49SlyiN<#W1%@itGdQ9=kl~m5&Ya3OR452X61ERyVsIo5Bqt(#?DRlJ zPDW;n5!;#@KeoqVVRsmOvJ=jPM%68U$LDeK|8>44uedwhq}e)Pr@A(ATxU4q*DSe& zS#^l5{+%9Jk1fY*OibjNET!v+v1G0>d>F`T0N`(v3>MMJqO2a zih3aPhR=TUr9Y4vnq6YK4hXGS%Fw#!?c|Mz0c{@r{+hES|xhEdSx@)W`v{>IgZqC4ssMi)cL@L(x&)Vc=9gorG*f%8bRHEtkjJ9K3 zg^|GA3(2`gA$opAXGE~N5^tOJBOksw32*w`7;7e)cgB$myRf~xiz(2h)$cgPG%hfT zMVy%ugDp_JDe5{xpb?tEK`*ctQSf9PLk}Gk(L+lWf?e~{C*JKm%BYV$IQrdIJ>dmg z{!mt5v3P+hrNWmlTD!=R&wNA%_N&&4syN4))fb~bl#xC!;1<+k*px7t#l)@;^>28R^T7wWSA|u+1q>T%qUh_tJOJF zBi}B=juklSpVhASf=QjBy~{AGWIon^zu1n~PkCx2{fbha$W`@6E3(n<^?>IsizD5+ z8GXJ_>F*nrQE)V|#RE^%Ost;JOYq&UMa5ch!OK z`4Et}y)<_#es~BNj$X zl%1Qo)R4!Lp-~2g*kf(iVg~0~yU=&}?*A9C&WCkeXq@FFE(->`YCY8NTEb2`+-}k zKOQ_Z1ZkV=RI6jo+B>~|{LXkg|6?D>XpM+@n9nn6!bUFZr6UeYKT89_oC?-o`1L-H zw&J|Mlh0qN-yg_Wh_q_@)-~i1?1*%YGX*e;uXXoW@MjG9YW=7RyVk6eJu|H9#Op_> z$riTeAXK#v4ovVBmuDX}FM;Jj1mofQ;`XO+F%L?NFR~mn==vgtrLvA5>CM@3fq%hm@%{|7p#>v~U*V^!VW%#63dexHN z+*6&fA{)KJRVxs&-+Pba28V8+OT{#oV5AH#vam0&(DcH_9u4(~2RrnME6Ta9c>iw8 zz{BdjK1-q0^+-tW`22%TjZYkusF7&t^}Op?9)JAMpH+{#YkciT?ebDNh_lKISx0Q< zbS0_ml)zkzp(?xL&iMHI`PliT6Rqm^Y#Jj<>8G0nt&eQ9u<`>3BirNZgOOD}#ldiV z!3f{C!1XHepgZf0H^#aCQ-9?&56R3)oSdOfFD!E`+WOrwy)s8PZS_C+k&lkQvZN4r z@-Tk-2H0-f2Qmo4k;u#YK*p5g%a0hvs={r-!?_*9?M?=i)8$>$-sU$|CY7q5rHql< zt7Y(6y4}nltBu|@9+-84I%mc7Sm%K|i+AiR(I$Ub+uN7dQaew3Rv-MR&3byL3V3-l zB@Cg!$9~qh4RfZ|b1Us6;#=C+^N+f*yZXD2Hg#iN-#Qg1(yOdE`S39R@OR@Er`IoKviNvKPDe!| z6S~nGFzXTtBE8|4!=08v*47`cALStbzymi+v@qt(t@7Zhnj%(5Q2i(Vqmjazngm^R zr!36VY->fcHocCbPGw`PYi4+>|E%Bj!E}u^-ebZO-H)U5Y-v9bvS$fl(F8U6cD9`tq z%T)XeHR}!`hm76H0~t$i0~UuH?i0J~m7D@GMTZ|*O6ik)^F#rn*p`4dELx44dlXIs z3HmE`jZX)Pph!y|AHBr|(J}V{kG=Y_Awhr5kAL8FB8`qkJ-1xJgpY;)Slj1s$L;!* z(|tS6E8!zI+Sh7uEEl3WfZfswrMcIq+=$|a7zej(7P9i?c1l{}*rW~9=l={dz&A9= z?dR|47%|2DD(Q*x`puJfWayCLw8loVhil3}8o?J86AaDKaNZa{Dsd6fqwLXts6l;P zm*72j?QjnpwT z9?0ZhSz2=ok9OGXxv?Gpb?)Di)5jJ{=q1av_WZFn8ez1v*y4r*%?w{L#)Ldbqv5={ zSMYPu7=g>Oh9sDKoVuEcagGUWiPmu?%?)LC-@ctl89%=oYWxyL`7|;E`B!pO)iU6I z#KZzg8#vZ;DrlRJkK=EB-c&?vFx2&Gx^<`RdOm#Mq~GN8M`fyFxL9~yXV;!93j`Xw zm=MI#1K0g(;}@>C!D&p6(~jRWAaGtkYHF)dt~#Nl{a9O^VK_UutVF3VM;|BDIR5soP1^SA4~15(Ykl%`jB@$-rr>&JbjI!NN%9675r92q^zG|#5oj+l*O|no`oH=55=1bVE+1tS@P?lX3V3;n)$uVVDgIV!eJyJELkm#AR2Ed>U`OVy63pg^|Az4|Q(lSGs*`^zI&Q@^kzK-W#Jg@M^4H z9?0lNEq#+{-;AKz{UyKEtYd)I&p7bNuIW1)as{fg^M(fc7`6z?xNIqwwb4InyL3pro5Z#~}qXKm^v z4j(`7+Z0z!=bvArd0m4WXTLf6HSA+CA~?+&Ik3`(?>N8;Xft5PhATWkWmgO_;n~Jj zmjlaTj5!Bac2$iC-pY|J8JvJb>Fos*HFQAa&>pFyC-B6XKjwa@382n!q*4;ySZodN zJYuf6HNOg`hRo!1%Q=lkoduxgv62T{sO4Pg0c#)evuExVnQ`Db!x6`>Jp5S2!AO(y zua7_C9Kj$=a~MRIth<+YeWx&N*UE%v8o}I;kXnW52-;Jg1v3palB|f~JvJ<}lJ^1{ zi0f3Y31)>wZG9ia371;TKdjP=#6D-%WV}wJgH5e*zcBw2GDLVnH3(p^@)BoW<><#v z*G<26LP^gWLIo{!h~uZ=P}K)T?HUoB#?dv77o6OML8tA)-r%p9vL0It;f{EdVL za4yzxyZcY*4+SUnW8l^1g9~cm`lSZ^-6}?08qv-?06V}F)h{gzgtFf@+np1>g6=`>y^DnIN=%FS5CPwBFAOEI&}WRfjfg z^i*%WM$GDTOxEAya1`&8iC>oSKlDMm*QPdgVbSb3`8?j8=1{j>(b551wa|R54MiJS zCP6o>?Qyg^nroYmdco0O!NgZT{mzSa?5~a=Qi{_>Q+Ca}Vn&>RIigvPf$7552GQLE z%OtCUbPnBx=;1t1(E>8`2s8@@mt%)>oWUbMIN4QKv_Vxo>tE0R0}ic;EG=``*L3!_ zq#bS&r(2^;m@~fc5d$$*Ff>jd$h3q~s0T7B%(_zYp{e8NUzv}1($)zdX}}Ff$erzh z%j(4gnRmV8?Ve)DBv55>YnN-p#ZHWzH2V=NY*Zsm;i9!-99f`D7%1FuWbx^Ip)J3! z%jw|KP`7_@RxZcVkZmJ8_8gFT$QHiCzJ(+Id>;bOSQv?Dyb2Sj!U_`!x-VTpQ&_Wu)FqH z)r7LGlfo1GHPKS+N&}AV(j7Z0!f6CGd5I?t)z2LWsA9CqgH^0=%!_=^f!ILK9Rg%) zfY#j5TlDFh>+cp)Ih52Zs{GZOCnH!bI&^w<%nLbYjdgwpYDl18k?^QL-nt20fZz(& z$lf#zu>Dwr3dVbu1a3rABekia|86AVq=61r6saXOFujwE`QG_bTwCBIZkseG7fC!C z2wdR0Rg5^c$C2^thd{`B-tLmSUq9*-M_azW5ij5DQIEOxS_xaHzgH@Lji>vdb>A)R zUEQHo$FFO%{;fPm18l**DmM328u{cLm_=*u_OrTZJrDN#e~vTnEgE z7%{fhBMOqqL+@wn|6J-OmKie4O_QBM1%zR!7yaHZ*j zc|)A9MURf1^40?x%cE4B9{z#M&(tU&KA=Qf3Tg@S=m#<)pKUBwiko(23xXHCaD@Tq z6KNW5wYM)~*ob+klnA<3bk+}~1&&suOd<4fuDCTWoG^v6mVLc04=%7Jgw)59`kHMK2yX*P8Uf0rf;{A5~ z0vq%1F*%a7o@eNGP3psfp+B(8^`m%8+7Q{mHbE&+^~=K*p4#R@2v6hLum-87`vi1LgCt*>{muDX{XQsaCbE26gOJ{D}9Pp;V;d z$9SX|4TZ;8ymMt&xw8&GF!M3Zl3b-{@wV{R08@Hd--ERTV|qg&VS8zOUGK^;4k52C zy*mx!efEU|8zlq^tmKHNw91T4f$XTsX()!={@17c=V+{FU%D=M;Nfv|%8mqzt8W|D z>F(bxLQ56y`ary@0EIz6bfCGs@+<3Bg94oM{O`E)6 zp4MvgXrdwQcvY4i2D~hBtbD~*Z@&o>oZPqd`!=JDjEu3#UjuzK%CjnxW1 zTEtUz7~(jz(tGJye>{+x+1%stN~6!e{A2yH4(n^+B9mh>Y3Nuqi%Bzp>zI-Td|3RU zSF?^Dc8!b2oYwuqoma0PbjTYzz1M4q?ApiL6tTG9+6X0PuYA`&#zYP*V<3zA#PG>> zT#xtmj1PwPoy+=n#+x;I@Ah`r<~Xo^^Zu)2q}*$aOp7NEr0$=Zp`k0To!hQw81(Cn z>ze8M{F1d;=8@BKc$SgBbiiUAJ}OkbhBc?hZO8d9oHi`{urb09e)uDo&$923`O3FF z=luAcyC98h&B*6c+nwT1UWy-f8fA^7F=H;Slv*bubt}v;@3$XW(qCD6;XA)|o8-(% z9^ziQPiA^ddxvR5bl?Hg9SQC(kyee&IORLTfW>z09f!EA;yaAA6bGX|Z_pv2bB!+^ z+lmW5D(ELFO+`_udUr9Q_IZN8#!7YHT5~*>k19Hk<@@e}05O*FnF?hx`}SDtFqOPgb?<90)sa6$1seype)dSmRH@D+VDAMyi49?0-J zWcHQBoJca{tYu&O2QtF6q)Q!i*F0zb6^Akw9=}88UE_DiWCTDSf9c5uAB(?RVqu4n zAmbeI&aw++?)|14Z*(*-s6ZhH{fbx1=X$C?wIP|Eb*fh_^vcudqX|marnhyYGxhZY zrMwXdDyrCIhlVQQ$#V3+ez-Wfzc6p1`SWA_4WX68gLcr?G-xf?T;Uv8N1h=G4yUb5163-fijN(CAcls$!*h11b>WN zdtxB4ip49iFlFJaHNXu_FoJCn1@r!M`vaNK5ZQH1BlK)<*xi+EHp;_doDXArD$mPZ(3SjL8q`>jJa z)^Td@w&3=?_+8=1TBALGpz(wfy;TXFF-Lj+v3?U@c}Kn3L;c|mb+x7EnsmHh^Iv&_ ziH`i?DGN4gjHs@EzY^S8?7+;&MGRhQ3B!Ao^|Kkd=bTl`2R`y)e;}h@>MNb`N01<9 z0d&#nR~)}$h;*x3V_j_FC&CWa-4Tc1z^#ThZM6Q&8GAs3hf$ztFJ zrfH~2T!EjtI69i(ba%Y(OyGO_&&bEOGkaRzLA) zv1m)w;x+r`a2wl&_h$ygYU z2$9kUGxJ9ptHUB0jIo9mJggXK^z%>gcohkD(H8@NITuCB$>Ut7sE;~d+~6|DHV zzpGye>Lji#RHvA7fDv9>RuUcCcLdy{PDc1v5xW8D90nh`LXv#M!W%u0dDB`6oN;Fw zGpg4djQxIQjg$^1hLuw#Wm%5VPI2a!nEr^2Z)PGsy12t(d*b8~lb1l@Cn`^D@m?wN z5S7)d^Anyr^lBWTK%R=LMGP-_j(#E_R|NjquBcF%wfeD~BV8+*GA0_jo!PN-JDIc7 z6r@2uj;}zyW-@>)k@BpViUd0J!aK)ccHAz5>f{|W^>x+a&|(bad+Y-l_~@Zy|1lSD zUF8@~YeT>i2g3k%4`c!xNKFYEYjU)RyIq}-ULi}mE>At(_blz6<4jwaYcVvyV5y6B zoPKMp57cq|%ty!db}5Gfw+H5uC->-&?~r+C{8C@^iKC4rj$Lt6enGqw2L4E?O0zGZb$2Kh#4{Rxj!7Glh4TC%7opJg>Py$d3+|7*^E~3tJy% zx-J-e#jhM|49v;}7Z#N1sF_{;Pq#Lzlg6q8{OtNEWtZ~ZPD25(XK@smopttu&+67| z4aGD4&?TzgS^=5E?xMG_JEkj2j~KS+=zd!sbUGVloW6A4O+z{e4W>gN8(48^c^y~=7sB6aPZdm^O}2M9`!&*VhFSNy>Ig)OTvr~#9r0IC+8@h z1jC`MuiQpDSyEzq)*kQv-bPElRfE{-7xr3=_=;I|Sg?fG#*H9)l|wpku`5RXHMU|7 z@wGq*iv=e4L1V%8Zs&FXq>+Q;+|$I4#b^T7e4w?2al0k=Ef|O^77gOED~6b3^E<^x z{a5htWW_zx?YLXoYt3i6D^A)wEVx~a!mna(Df$7-?tzRb+|ZR@_4OON;@YS!Z0uRt z)FUpeH2OGccZ_hpxwCFU?rU$zoNE{~6z74=SLFj4=d|56kHx#jjL9Cxe%HG`O!BRp zjI;BV4h`(}8*LSMrqeY+cWkvIBOi1-)@Rbfevh(w6cU{HXke)wMQLfg#MK9*cHcUt z`_I~EIx8L=`q+*G6Ta-F;keD1VxzalALF07%y8tGXRq?AC0?EFdnDS_BVKP4mp``f zj5j)Y+!a#W7;i)ntY3<&OcAa zNN3H)$^f1jU*4Io9x-em8*O5+HD3{{JYW>-*H6cR>3PW^A3wU9Z(K=tYP;SkNV=OK zTv@$@0fzBPG4?EKmB#goP^wrUH*3mzD(;{Ey&uD^dH4?5zl?^8NPe6+E|6=Ngj z{d~U;vCaz6$8jh!X%uujH){=W!{#}sjsg96=oV!6ACKoct`z8CyG|WPysfBgt`T&w zYy7AnPoUR%3U7=>wdnp6dbg>b=~4qV(%-3e3~g{&_{8-<=2z!mSrUeqaA2?=qcenT zv=$hD_F8c;h{+CT68TQD?Ao8bet!BLGHCFjIL-qm71kAtfG59#I|)Hw!)J0UeuwLr zLtMdhs>un>|FExhto^R7*s4z)tN4l$FLFmKFf$6iED@|&Y(0+eszo@({J2|tcf^7A z9J?B>75%nq@li%{G7EK$VV+#4B&MOwEG@vlLxNy)ApEk+%EaSJfyEUSaQkN+v;lz9B9L+ zN!LzRJ~H7#16vwu%^MiFJ*pUfJ_yL?9D5V|lK=8z>+AZ>2Qu*c2L*i6;-xD-a@Wx# zCL6I}_BLAG5*Kf`oyGl#Iw`InxSOAPDvT1RnS($P80Sk;HG_PP;66N`V< z6(5Y0iGx=xUV*|7j(FDrBOhC^>CIypu{hBFVOmIk`7}EFlrCBJW>A`hb;oa6t2%|XHiE~ zu|rYc%<9@`%(myRIzk1_59R}zx7l~dXybi4pBwr&OrRGSQ_xomPY+}wMeHH@*}eu^ zU*G!Qf2l78`UeZc#ryf)^fZx|u}S__a-B%xD}IN&ic!!=GJYW>cwLCex7w@Pt~oOf z^`(FgDEWo)4|I+CP@V>Ts%Ge01!`CUy9ZL#fb}|`DqGC=f$GUQ7G8#*e}495T`Q;8 zvc}XW-eanJ-B#QVvvRKWcMoKQe->S8ltl;IYq=HIIoH~^=mjRuAFnAF;%mA+Mh<+t zaVf?F8UK+bk*0G-SCE|grLK0?Xz@znxW1w*?K6Fsw752Kj!m5dG44E$*D-(b#TV^= zHSFrq8&S)T9$M5h{>swHcgT27Qzfk@*zOv_n*K0GIDMeCzSohwsA0dBmrArcN*-R4 z8e_?}d~BtICoJQcy=UVy6@A$o`zNmtCO1UYg%-AJ#_^Y@a6Nw3KC64h?{W(B5P#)? zuX;0+pU+mi6RwY;G=h7ULs4Q_`GQ~Hp6@uz??3-A( zB~Z_go${x;@@m7X-F17rV}xVt82P}p>;cua!^HWg`eH_ls!&ea=!P<=$`gQ68-&xw zW3c3rn8p-KKpe}iLtSS{nU59M^~FVV^((gez+#2bw&KED+8qNH4Xg&*-y6li)v1|nj9c<^L(_=lZcHh9ki=!O~#eD6z%=URk{q1%Wl!7AHR#EP9G*FC9Kg^1g^ zYYrW|!$#smU;#h>!u&u+c!!|sLZLQkqBA*bLKZjU8|AEV0?1;)VZlAdq9uzCRyy+g zNt7MMbTjQ3g@iz6~b=4Ai?KmbWZK~z6ExtaHzi~~%MJxhBUzX|#HZNJMy4dh!f z#8*G~opny0Z}4>?Xp>PTkI{2t+o9Q9&!pFN&f4Ru^unw9YMn9vKla`yiX8%pK_gSGR?9+!;6Wh7v4siP2xHpBRq9SGFvd0s0VQ!r z2Fbw0xYepI84nzA2*H0K5(yePb_PZ!i6@p~bvJ2sS9v|_*}uKk-shZq@B3A?V!?O5 z_nx))TF>*WwfFgZ&v)PVp8FXCRw!wm>2(Y^*}i}|<9D3eV7nh4u{@EEPS&NX$qN^> z?lI>BnRkEO>g+dR^IwSM$2{}+Ev0^9&H93!QIbYKo8iO-E)#VpSOt_{v9Nj=qvh;@ zO!_nG6%S;7-}siL7RqMUwmYLw85J5Hl`;NjGCkV^nHtN>jwSNR(k`v@{ZF-ZY}HyV z;^lEm+PN>~R0c}>k(_gJXK~F~<;C?Z4t%%0#$9Lccb?jC!O`DwW3+>iBah(*D}P{b zQ_%FuRmjSqoO&rn91F@v4e0|N^R9=PnLjLbW5q)QTUxZr?ieDaH(D|}hU29NGW*w7 zi}zToFjrXe%Q~~?Jlo36n@It53~4ULqSo29XpCUawcy1fPuV@LK1=7}yJvfDwasT7 z;i3L1?8fX4adh@v&*I>9ot+Ct_Fk;xx*qM7Yj142nEyifgBizj9SAI1Smh=rOIvTy zv~-;Mx=o0cFu7$ z=3I~Xo~LUu58b!p)dyEG&j9)5S6(0Qy#i`1Cb?vfy2gPi){Y&_FscNddjI1Z(4iu5 z=n|)o1&5WtV`=yJjzO#2d-xnh0Mf?3L=N-!zP6JcdL7A>*gdM{(%ufy`IqYp=hseeHF%J z4z`6(C!)ux2U+^qGc2)Z?5^AO6w3*fP;?a2NvUHLViEo-2IfUJ1as$Yr2RtBLLX~W zW6?6lAMG4}tk1>>K1l;hTeio~_#NBZbALl)lBcl78y)GypMGuWDJIB*5==bvV z`p$5~v0$)!9XqbaiQ&7_ATE2x!Dk!m;gH_9YX3bvkkOv2)OiQx_Pqx(j45k885kGB zj?4BKTaTy9cWx?RJ`l-sNBe{fT0gpk8FHb4l|BnO_ZWU~WgW@G zX#T20u=8$fS1DF+b!`~fjyr4Tz7~Nw{%Gez8S*zFH*1@q*Yo3RG^U!+UE6tgfg?z6 ztlaq+Ee90sA;@{_SiObdV|yL6iH+1B$)Je{+PhzIVp#bKXRp@tnj6uN>w(OgtLLSz z%AXganD@N)MS)xcvo;Bhh1#)lTV>S;-{alqHhXZ5U)xyQy>DyuvEugpM_qd!VaT4k z1lfG=n zoW+gx3@EOatbRKnPyb<`(GaHVs|^nuu4O6wXoYe1p_N3lwMK`Eif>usOOoj9xIKQw z+i`pUOc$*j2LPN?fDzaMY=&C{Di~SXiV??(N?h%}fFX{>fA&BoF#`Y|O}XV`pUG=a z?8x(h48U1YFie<{Y)B|`j0`-G`TTewlQAKwU^wWe2pV)6yt%T89(=6)#CFaA3_^adzvi;KxweP~GcEKWY*?#01jts83+T;vCTWuYtH^LPJQ%yFL* z#M9_IUz}%_dwCx5>xL?jnY=2f)+3e2accZHG%vL-18aT2R`0s)hzSc`wZCfpyN>if z4y~*%#Iv?`Gba6yYncyZ;#V!Tv_@ftCnua#~vKT$m6&eqrUF2+v_nA?2G-T zT7XWgcC_+Gt9(1{{82+;;M!9UWIo%ox3+QnQP}kaR=JEmYv4_-@2G~}7}V(9GgrvU zTP^8OAKu6OHT#Z7XG~}gkQ|*^xb9us9aDL_S9nJ%ZVgaal-DhKZ~QQ=Os{lkjqNu8 zekc>1q?zlrU2|Ixy7xABz|DOz9E!8h!piTo^=nJ|HJAP?OVo{pi$(iri$MzuwlqAd zjL+j&s6xP@;_9Da6m>TC9de(cnNwkO+$Uza{uYZCdR51>tzgEFbBJTbUGGsI;!+S7 z{`4~r0k-o{y0jV#+^(aLw5Hy~8eM*7*Z{N!{KV{tQtIG5&IFCxwZOu{lHIqTs=j_8 zV};UL>(69OlNlgi#c5Uk^zbW7@s3W)G>213KfqJc$` zIDN&LE5G62x6pY#@NJJ-_B_Io=V~A99AW+bCwOuHpnIPEfecwo*VXi`?7a3fl0Cjg z*Iwor=YJpey-H(`9{Z^eMe^6ys>rd`#vgN`nzen-YG?dtbPP4vj_b&&%39sj4B~8$ z^Pb0BqplVe*peIXj;naJgzfdim0eL4LHA)iw`1^<0IPI*YuD;%+FGB16ZV}3I}gR_y4wO2V66tBm7o49PT_7SX6_s-Xhq32w0t>L{S z2dZ#oYZTQ2XAeDCg~JnDaDdVzz^D~lO0=jxaX8r{V~56t4i>*E==%?G9m8O-Gd9Y` z&nSq`!3+)II##%e&Dg9UrF_H1$kLUoiIUh9n2iF?Y-h1X9Sn4-5!SJSbsR0>!)9u- zkwK@k<{9@ROMSvobLW&|wR&7_G~mn9CJsh@EzFbJ#LnhOaV=eM@hE^mG{8+)c-;1y zxHW1Gp1QcvKHGn+(bq2FtbX+N*j@*HMxf1`HPxz$?L!`3_*llHnR6TEv$`F)~CfX`U>7@I9Uv`Td! zUDpz-7DtHhy5K4ko}RO}>-Z58UA50Ngb)T?AHO(S92SD-Tlls<;PxFJ4BWFih@DyZI}blr|C6X?<{t#&Xs27w zi}->Yon7M$gAaSfA>QpiwjM+09>yEHAvZ9QuDiE4N`)a`0t&jfpmzV1As7xf%P}#H zNE8bO+n~$7e-^*Oc7JbQYEVpq=>)nFnzu`O(19cFvxE2FMsM+=E8Z~`Icbq#BjBD4 zj-PQcoBjt49>~1gJ|R=uZt2{XHHYP+`ug~^qKt*|%&2+6UvDUh&B$Rcq~RAQ@!RdK zKqZ3&ftby}=eI2VvLDFwiweVE(ehUQV~-kp1~FEScp$JHp*DVD`;xQQg1Pxbm;$pg zXbqg6dSd|+OCA{59`9}9*d9|GzjSG9J{6nT2q?zxcQ#j@Ka~(fdkU6Dqto%uc4V0K6qtWk*m9<)l2_jgIpYl|z*Pu2Wsd0UTul(SJ11pPP z71QQFM8JW{xK;oI@XYb!tslHt(8Sv7*>AFe+Yyx(4z?rFz&hq0_g&OK|JKGKm+@#& zHF>b)2Y2NsE_}z(CXU4qzP8Mmg8(LG0MF)%+GIOF$F4-;4nF(G=!V;s&=-)}q^G4~ z?vumP$EZ;QJH+~Fpi8|;@%W}Xle`*Ofpn3jCA*+=_wR(QH zYu#&*=(sJ;N$C|lQX`I42DH--;kq8XhQiQdXdW?oss#j=SdFGyVDNM-?HvOayT*g% zpA6S|Il2Mt*Y{Jt?uUk;^7k6#{;8jS@AhruYp)##;oqLtr-s*hoCVG7MUFSV@j-q@ za)3<|(GO(CN3n9RQQZ}1K$j0>ltDF(D)tkS+`YgmU&J+LkJrY%HyXsGx} zo{eC*65!UTnmSUks|7w*xXz{B<6W!bD+KPD9S>kK(FnLe;r zUO=l7Ua_chM}(hH+RtJRaY%aDUVybp1j63Ml@%Asy~)} zO?=%BRTX(q#o|vZXEbEGj`Gd(T(sfJ-xsvxqaNZ{#}Dz&?fh9s96!S3h$h(ksO#~% z>YSiVLCveT>SU(CsealPi2PB^{1(R)vYK(V$K?Ik6UUF(WRV8>2IS8Ol+6Jxn&Jz!upDU1g{8rjat{w;#QV>otqx!J34NT%;Wg?AUu0;u~w|{9A)lw}KsEfgelKxIl@Q~mj>f@imS(F>I8?uwXrkdlyZQMYIVg7knpnY2l z>)c}zp9yCJ**jKABFrR7f1_)j5+Ybr(sar*zpW^EItEX#?+}MJdQy;UxYZ55!QOms z=Np0q#cC4Nh`1qBg#2<(=xB$@1qv8Bg=Oh+`G{e`$sV!xcuI*5TsyZ|HRAa*M!m$H z2{5AR(Abp;Ktq;>w%GK(s8wQ~H3jQrHw$Eh~3)N8AbYwp>6#5=}^Ck)#s zJ(+!iQ+34Fhz{q!9P!we+D^=m2q~WhS&cl@m7(4l?IW)8b6rTq7)O67yW|h8xr=38 z7Vp}O5i7(T>$v{#-qmc@oq-P<3L}XwU0&RnUEFspb=uBZjRNf{t=IyyqejFb5C2C! zevs?s&XmI`OvkNpM^t(nNizHVj-w}acn_z+NvrGG)6VgC6FSe zG{}CN+L#>AiR$qff|u<}-2bj`POA=xzS*Zr)iQ6n87$+%clPaL4?@nRN!L<9cpkdkn5@&m#~jJ73s? zMu2AituEnTvh;jzr}!$->dRUZIG7{<=pAbvO2pMYjOP_GyOwjr%(P^gtJObJ9p=Bb z0EnY)vKhP@$C3HiliDhLBuO(8c#79-Yj4sbQE4b^;i45kL*C=hID4MZwFp<-^6xG5 z%D?7Me+pR*P4?eC%R8U@Q@2n4`tQ2E`|i6Ve=y>QJjsuF$_H;srI;XNKYVil)mKxm zym<~Q?wi5Qa|=7kxFhdxdYxmCvorFf%a>lw+QbdN^LhXKnfKq%zajI(pZP&29y$Td zWt3@+^sIp111lKq9`9}9DH*FC>-^p}#!RMV(FuSs#%Vpa9-tAIW-Z74YGg1=3N6im9rVQc<1Ke&GZHl`%e&(BbM56HZ+*oR~sEf)N8}XEy9W~esYTS$# zjR{XQ#mE}5<4e(yx!;HqVs-E6{BmnyySRtCd#-^@AY6SO6RlWuO5N;Y(Hf-2Rpgij zd|6XAVuTkz;X%u8l%i^uioi|hY$TH&upze?FpG0^u6S^c@!@HvwprD$dD4-dyK!I> z6Qbx+3-d`HS+}?A_w>UoyKBN10)4gF_k3tYPj3Vj(9(_{JIqvkVxldLsxe~GVILNb zajJUqiJ<*1b;j}R8X2o&d!C@cax>Rv_Wc^3^?^D+dD!oX;#c4NK2t_f|H-KcnHU;u zyQn2+>X;!Rh^JSTIw@{2$1Il%xGqz2Mmi}px!1fiyxbUu>wPX zwT^1dyeck@cB+ql?LQM2pgq^sd@@VKcOEtFe$}D{{+e;H!f^db&zM{8?X5s&ddy2H zGg1Bc=?`Rt)B_pS=r0G6mUXFLmNBFqfj7Pp6}oN0JOIO+fytsVgoeQWKt{y~DYas# zpo^tdX31WSIr8LmIFxneT#_L6h!L7eT*K^U!#eiOIK-hv7bYp_6PG216(;LUu)RH& z?!?ft-7(})l-PXQ7``=kAOKlzt~+iz2fz0xJrj@Y{V40;(H^MF2wVReXKIgU9er<{ zg`K3?T+-1C3bw-zxq9%)cGnQ)EYFit2bi_VzziN>M+cOM%>8fMe*O6A0*2Kns?|}R z+wMsmc{kF9JL$v@TEmh1dFnkk{;s!`k^4jC31s}d^Cy1d_K9Ejows-2c{jIRo{UA@ z=1V6TY3B_@#8pk|hm$-RkF&CDF433|WI~U*)y)nVxSY(3Y8}+(w>$+v+{MmRQ)8g5Q)x3_+YKhoV2VUv3(B3h`W$(q{cf8v6uhG(ROY1Y-{fwnP$OFEs z3+JV(s8(`E_lWiZ1cB*qCgrP)WZG@Zw}fi^a?iY*V%L|Fa9vURRbpaA zH9gMkE~!&zy@v8&Wa2^uoz!m~_VbBNJpg#C@@myMCpsF$yDo8JsLDF#PZi+T=Ii4Z z1{@e^p;vK7IexkZyl&ykXZ^CoupNU&kKsGR5o2qUOMYpn&aMRxUX8RoYilSX9AaIu zxhKs&BH>nZbaUIqiaPv8Eba}nA&F=U6c4P6Rv8(KEE#OQCN3o}y9d_p2ag3e`gDUv zHMopeds$qjaGr|#K6uFzLE=K2F|2p?gc}1A1y{3sOEGj5)GLH}*gzI?p@uEJ(ZZ1g zvv8T?9(w8#$A>lkUau=2bsRlkuc}83H9d+{{hnXxT08b!1!tn!DAy3ThWB(RK7X(W z)!3+Kswqiy#M1&_ZL#;`Coyph<@rq_+Hc~=hm{;&>ip$C`iF0sA&|#v)uEQvj$;so zI@hblnhPdXITZ5Twr=iuHkvH-n%W=0<^fYAHer@CEFCD$RHV z*kMHw4y<_AP^*?bJ|hZ<5DS>mwX>}i-!WkAXAbNo)+Ao3xoxJYk+b~5B-D>c(x-DY zJh?y)7C6>`=*3U!jUAo%S&51fCjo!VaeZn@^xAeUa$!4*d}pza`&PD9!&&}i8!sn{ z2i1&0*^Exxe$zifQ!R>k_Be%lR`=9<&_Dx=7S{fX;(s7}H;h{DY0;?%GVgfonkOY{ z^Cdy?c&VQd*%EX-9MctzH{#>mmXoet*5Q3f#v~}LY%JzWC;sJ0E*@;!&u+0qQAX>T0bHw!6-lq25Pv2vrSGN1nz!qw)~9i_EELDp`XkU;D48D4z!O3I0aW#@=0r7I`ZE9_xi6i$TR0*9zVzN zgO%1So9!Du#r0BbKYqfMOV^NN9m|R#hE?<)|4|$+*4e6to`=|#-8HC8=bxPS5M%!D zYivq%vly+sh0~qeCgv85W^e3h$|`r#HVYiPW3K9xAI>$V5ZQ`V>c^^&zfe}926Xp0 zekxF$>pNWZozLDY9XP~gJ0q`41eI$$|EV4{)NW99$5v9Kwm*y=+jSaqYAWv6id3Gn z1%2!@tVbBM_I#c93pFh=`PaX06!8gu zytkj4(GU6h~OMi>uYyaOb z-ah{meVc2VLZd@m zdWtP^oMAnl7FX=Sa&hn^K^R)r%+!%rQnHN;a1W9k& z(UD6FTJCwB7v|jl28Qf7Uvm1ke1)0Z8y$2+#SAMx(Rm4i68 zYpY%7IwLxc8nMF9Eyx3zAUtypLZk9YO^fHx@w)Eh#x$S?F-Jq7Ec)^=25h0uTX1FJ zV8!Th>L2fKNA)X49QauIv~(@?6$gX%1PKreaUDl2=+M8i#Q+gmATPnC19-8bRgFG= zR1!x3vc|1rlRA8G66aTy_v{@b+@rWKecAEEc3u2C{&}2{%mI4XqmDfZ1?xJk>_jpM8)gAy+&o}+8v9AY{#e%uKUqGiy2qr zmj+AMkPkkVcv>AiBam+yRj ztTm+O1wR%Y?2c3aEGE8Q#ALIrdzWYsmzS910n{=u?$kOl)m>UO`o*0c^VV&2Z;ZsS z;-|*A_@sXnqjuj8!#%f!;^@%w`{rOJ7)F-rJZ|P1zBPQ00fW4yUqmo?z+mMEgY7YR z9oKzl?VXP<*gZbyV^)HvEE=}4l|v*SI>x6anLjYsc^qLMp$_J6Y9lZ#I31^8H%NBD z?J?>iuCKjb9>@?F#C~ij1fTmg4L<$+#!{Zd1DHAytT)!-i`kJsVv3V`GEQvIycrVbOy^#%Jx1PL=L!e@TVeMalw+?$7`^agg;jtP z%f0zK%Y$#xYk7ve%(CqGCHQ2A+`z9icbxDYOI#K|7VTHHX!AE@-q8b@Sgz(-csRmw zzT>z)cBZ3UeJA2&l2G5`S73ROybaA_XZ-et0Fyg zJCZgS`d9W$W9Ncl#)vax6?c}3uUWx$FIf4|KgNwZ<9F;?yX!N*=Q?YS&I8};%THhg zXrF_<26U`p1@!*^dLTo7YQs`Har)=+nhnR930TZ_^{0mfK@_WXJ2>0KWR+35svfr- z6Wr*-d9e)N*4?|Ql0JhgaZeq5BKZ2c09yGA$I_tIrOD;Irvp7C5vO9mr5V!Rj4EwUYycDar zGmKN1IwJ_vF0HLxmI(U?nx1{4ooSzRj6LSKW>thd)YCs9quAUJ24y`q>`a`n@zQgX ztZ6=wiLM;^1|u8e0y_?P-c%&XdMWgU?SYI!9W$WQDE4S!upPJa&M+PS()P3Z;I8r$ zmlfs_PIXQU6eRS?&UPbqw-c^iRu)-<#Reo@f;^f_7li1m295v#c8%`l-Rf9IMqcI=I z#NUvau~x{Qtz&4f7`zsl`J3k6gdy86h}JzboL09AGSZkeMzlmG*(7{?KdhwHO zu3IG9EP3H$ff1*Fv@q7Xb{r`DyKcp+B`m6|*DBW<$2fdhy~Kr&6^reswifcM-8CHR z-mW5lEkxi~q*`F)0jFAMD{f3j79X5FymeaZS8T-%O`IW36H7L1*Eqve#jd2!pFP)W z>+OF7$C-QnUc>Is|6vK_Kw?ijtnD;S1nB?A^e$B+8xXg+qFbm2G_ zz2Ug}j|ytpEjqjZ`MwpXvG0Fp+J&QJ?*aI0w60*+07rf7KNA;?#7v;Klj68`i-t9c zlHui63bUTA&0cdg&a>=Y>sU=yZ|sO*8Fuv0wx@HKgmwISmsUbo$S}}W_i=6iL^^Wg-xLs)< zXV-evMo;#OKm8?t%ToLencQdMLCTVQbl~v7W@4hD*ck#W_0CdS>Yw&vG8=qKTF6t7@Y`&NH1$Ob?t95gk@=V4d1w#Jlfa zTRMJOQGD%MPwP;~(}a7~91pyQvdaGuwC419?SqQ#$a_QU8d>!K`~A4p;CIclJUxEZ zKGPERRrRj&Jn~ZqmKt8Qj;@1NuiBZny~}$ zFAppM10+5ga%mBlUlpB4+@4^NzC8Y@@9%4W)Hg-O>VlS0@~#-X7MS_F=+=fI+iwVR z969QZkhr7I9;;ca^gMT(o3$cxi>o$cnKyPZ?>k1=!l`%M2KGL)w)$GtGrSc;jm%%w z=N9pmRpY=94*u>}f5(8epQxFAT#Bqw<;R0cVenvkZra4K_+|a;d-l`=?T%FJYH?k? zej$(59-UyiGFf|!HZfVYAKX1Z{xd5*@zE7#|6f`8dR%?-tIY!$YmGS!w{(Qly5q03 zstTjQb*|d@qxs_HTKRb?^2@sIwY078@q$CyF?wYm;r4hjX${0Mydm47bi<-pN0K~M zKjSD0jPkf84l!BwB`epj&-Af_KWarp!SSGMk6-n#^maVBUW+j!GR&j->wHma%*@;U z{ara%4JV$Tj2I6L_5R08ZKvUwd=T>Q#}8QS7+!Wek2rRZb)GSJ)^WnWvat3z^C!P@ zOSRXq^Lh*|Fjz2H{KC^Fma`j9{8#&*@Wy~d!Bw1jJA$~;`#xZAJ&^e@&!1-xwBa9Y zd(gb+>eybB{P(cLc23uu3IYdk-Fhps_t4I3uBl_YuL|Jc_F5F{`%hv{jvx8E4jNd; zLD<`4J74{)W#{cUj#EZS)*W|G2aariAftyZTz93HxJ`%$GBaPqg^LF=`AhfN(Zd-1 zNtrA*&P;l`C1Je~)hA`NBsuIyc_3q@lzY)CYZ;7J)gWQll22o|CYuW6^A^ps+K6?v zj?0H(3XXa1vCjEcwpC;8u+JadSrY>sh11>ZeaX1c=p1}qS3CLKe)B@w5$A1bm7ZJL z=lfD|d)H|3bKV+QW%NCX(9Yj;-HUhZ$JM5;UGGsHJ8#baGU}S2XZ6l-TBt7d%WJUSGdl(YFq30KTB7?awB}t*V~Wk+T&Hz7NHyB znrphUdMPJ)u^l5_{8;(4+`|zQu0N2eH$>>Js+}h;S=dkZ_{uh;ufc<(4~9N*^38Yv zxb#qiV(`Gh&+El-3Zs`|j+esDAC$mFL_b3?KON&+I^DcQoHyjfpe0Kk)0YRHBpvB_ zJC=5jpXv=^aL@9jfve^>wI@6 zZ5UYc6Q_?Aj#$~ziek8rNxfP@pIq|QY~q>FY@BbrJ3JKy<|DhCuE8rQdbM zpx*f)WyrAojjsPb ztf4!eXzzKTl-Al}T;^S|kM$pQ?Lnck)Eo%G{KMBoCpL)t98Sy&8fW=M$G=OtPOiy1 z3%ZKJS8L~HUnhWAc%Ph|JF6=-tK1=u5c_}Xh@@9^d<{Cx?qS^GL+jb@9Y54;O9f7y zb&olaUcY`@ANE;K8mSOL<5sjeT7xQ&TUYoUcg4rQ>+F5$8S6E_vh?<+Kb@85e#4Ex z#ClYzMWRs{*Y7tz{HFh^uN<0;Z!*;JK<2HG_bzE>?r4d1|0NG(#GsC|0b237H9DYQ zRd3aa9*&sISJkAZ-6Gxn|M+bb8i%TpS+AynFFLh8&ZJBe-h%JXv!j9It35MsFy2 zAoG^yAxQZ^MgV4DB=CNdfh5mMXz)mlO-giRc_8yc;}bGN;T4S4=M8Plp=M5pk1Wel z1J1Bx#Se&y9de=aA^5?9rD8;*{x1|Ap_oI&mM<4v|zlwub z+#GKH24kDQ!hx@G9JPkVFg;G>SSMR>AVz*ombO25tWfA;iDB_$_uOYVe8Q?C?*`)y zWeD*ga@=Vmd9XE_J-eZJdpE(F=A zLt7~4DjbS&-L~5K@k2}&o$@&Y4`j-}wvN0tey_(Eb_|+VapA$JO{MePl}>r%AyfeS z{;N=5x=#0B<=*jyTU&kmuhlxswN9_1n5BLmmlq5+J<;*et6A{kYmIyOMrfMaAFmJ- zdbiQpRo5A%qS+cd^GmKj;-R8H3aWcQw!bpY*l4RZ>eDAuGae3EJ}Iz2kcm((XK%F8 zavrkPyo-p}Dz%1-XUAQ|!R>K$cDv)?_sY`O=D%iXSF1?r95M2c7aSIU&o8{+jr#nz z&Y+^#OX(dk6+bo7p;h$s zM8#e;=CbD!-#P;_w*S@fBNQ68W3Hm(^jdpSyuS}CZ)pr@8Z+Tr-Rq8eSsS%ySYTzd z_C`9Z!+5X1_$B{Q-?u;gcCs~Y#x6wHOfAzgLpH;im=EHCjJ(<`IvJ^>ZK4yRwyB>G zgAPXol8~ymoBpWpSL0ijz7XHCq$Oj5W$)!H_%N?@gww0Xd;3`(Qy zah@U8YgGZ7`9(_`G_VWqULg-+Xg{mRjh*qa9>kaGs4O_Ndkww)UF+E$LvJ8 z+JR%i1^FzWFgg1JEDU^jvR!M@i2z%V-6-QJq^VobUGmT-$McMX#Nt42aYpA%{|tME z+qFC~W~0IU(T8)T*Kse#1exO}bwWUME}dPA2QvD0xLkVTaPWX+@>*O?#hWX?!gXRt z{O$vpv0rBHg_xeXqK?ctyn;jFfehcW1d({Jk(exTwT^hG3btp?b6OD0?<3}S;_ki~{`KeIz zMJ%p-C26h4y*j#`?!rBR*WmHj_J6LXUu2kuQu{!9($EtOa474 z*d13`GqmRXr@Y=xkF2A|8OYn?_^1KfG4g?vbu3z*kf>vLCTTVf^WJsHC0mT{J#SxO z(S{|QeYIx(l+maCE48X=-~KudG-mP7w!gCJCFN8f=e+37hcVUwGgKyNJtoDwxAYN6 zqAxur{~iZ>h7*>OgxfK2ciVB!H?oGwxzv7hasJu0uKGO(n2}|vSuxF~&s`&b5<($< zmQQyfG%R{&xjGbF_o=<}z{x-7LDi1)C--AJ6CL<_9{R*(;b4WK4GtcD`m$hT+5hu_ zkuO4d&9vaUucC#umerm;e+u3CSRdo4O#fu&IgBY$2^L|!_JGrNWAYz;JV#&D7M$p z@oK|aaN*3Ixq->Q;9C5p&C|U1M0A$SWmFIb5463u9={iZFT3N2N$V;mOwW}Z&g$xj zIQCO@yVjK^+aE0RFMeOhyXz#MAcO#2=>X}r(|7%jd$!&6cFvXl&N;(@nXz>E{E>Nd z5#v0uC7zt-M^CP^c!SREzTiTDoo(q%KI}?OdbVCqzPIzL8WrerGAS2gIz@oy8S?GAE?1c3cb1<{1~> zSxm$|E?mbd4n}w_isQr1`A@PHa^DSr_u!;XZF0cJ;=jscm0A~moqr*E{=%6{zEW~) z#}V)NGfa<*V@#|Ycw(aQ=z#(>tTZQIov|nmMsYCMj-!q3n7RETsQ;bUqI2K9H-w2v zq*kK}AAorr=^>N~(srOm;Oh7M@a5+kHjg1GG_cQY&2i8|ZO=jWip#Sq8MD{BV~As~ zbmY%5g+*hx##m{de_|f<+7n=|L1@Wp5d$std%M@A7)!_ClV$rSFN&}>^L}II2z=kVg=5ziucAbCYv_4zR zmkifGQ0zzaZP}{j9SY@Wt+mz`h-Qxw*|qk*aGgi{42R}DOwMqS^+4v^KJgv5x8IIm zS$Z>`nzRQp=p#@z2@7N9%iR!?p8u|_?=ww_N;VtM|b=YLcr ze&YPpx=iX;E7KXo$j{PA<9CEwHousGzrUVQ#?{O}-o(DXn?-?GFf zWcVnm!Wse>UeeBZI27mnD7-NF>qzRb3>?4unGa(EDQxNwoZ87BeYIJ@$+A@3-g+SO z*{M?)hy(7la zalf4iVZdSi&P1BiBe|`YuH1f7%~#wM>%8wbl){@&O;}WA+5Qs9U%40$WTd|H<Uy=RDY_^C$bev=rwh zgxci5riOxJDGZ1nq1~-<%x{nEZRT1G-$OBfLRw+Z>WJ3Nx!e58%YOVBXG~w`IAgBR z=lZUFhdHQLXRdAd*y8Ru5cYR%MR;`<&xp`eY*jIuxMQp<1M$wcFUCI;s}bHvDyCJu zkC=zYIdy&<46$-S^Q<=@6RSS6FCDhOw_nu`Z_SRk>cew|JI)+c!yGmygb2T;v}+QF z+2h2p6|0tc>lle@PRw27XSwE>f}9UUV)8ikp&}l&>}}>>8`ACK@7bx3I&6nmhVzb3 zfCJVu+lA*`Z^ftwj#}YUA8|9iKJzJ3+>YKkX7Bo4vvfWj4`elOYd~CA`Bfk% z%seTb3fqa+4r%E;;Ki|{*S<8#uk+R!l?O6>LS{a2NpA3i#Q4OY@!{u&O`wbm(=!cR zuOlFdkgjfwWm){d#&LX#^HL5iX_IK>d?54HuiW$r8O91$c=E5xd#~3rsv1TbPTEz| zd$9_lxbQ`b*Vzq0R|HPIIp5(-OltUobs}xdsHSvkj6IKF#(*5eG<8wL3wKB3#|pn= ztYNRV%~`zjtzi^c$Co`Tb(kk{&;SLuE3t35rm7Wg`8x}3ssL4%TwMnrw#Sal@=(7# zaA^_4FH2kcKIuHCpGJIl^z$A184NQ--GH);JW__Ph1+b(ZR}3OgJr$)i{Z(LU^*Y#i@BvZ;jdGHCp(NSymi5 z;7xoB^qIp=1@z7DYv-AI+boceI zL@?vwEPu7Ga?8K#g6(W$sMH9>sSu5;ZFSTlORv{HvgrB?4=V(@P{_R`ocVsyMMbDH z+(Q?hb&LaTONQDv#Q_5!D_t>Q`MzjH6NB?=_DrQ}tc={bRE+w-V5y@k6T=p!qwK9M zRO4r!i5+!N#138LCt)@KE|w_y<^~|p_Mc89P5Fk01DF&4*y7-)Y!KDPUpj7eoIK#z zPZTdqwK{h7DzYwj?LBB>Lxg#jUj64UO2Laa zkmSmzli8W^*_acO$&^rySoAAPd_u;a`|B(uAx^6u4lS64sXRLd>^s>c7V$FPMYw2!LDSIE9IT zyRnX+-1-aHaVUhalR(8sBekR5tzy3MSaG7D1Hb=`B*or)|I4?pf8{GTeeJa|+nx~hal)4#8PG%D zG1lQ7fr+CrM`ERWd?@$ww~X1e-na>hPHK!d69Oo%m;C5s&7Gwm;z0?nQDEhhi_%`P;PV&LA{iTSOSj@%5jcCUh=N!g) z4My9aV*vHi>pZt>Qs|e3U!H2AshY{Hs5liXJQ%DZJB}DuCq0ctII!Yz-%YNH8zsJ9 zQ%SU`v3fH!R*{cH0b=~PI#O-vu+1FXl~*R(efy&$yK41_+O@1%s^;{^imX@lSNvR) zcqs)=P6A^Cu;r--D^=i76!%q&+p?ecP3lv%lX!p)M%(*l` zW92FBbB)di0iXHDe`euV40g?rV&z@i_v%Ain2wW{UsfeAd&jc?06+jqL_t&zp14p( zsDj4}{6ZGL=dZXjbk?5DtURZQq;3^AI(9j{XDPjE3F*aX=O_N?`U$t&o)w**Vb%|6 zfH;2SUU>kI+pS~Ci$pV<`@c(bUL3p z(67D6bmFVn?lt?|yE1!^oZUxl(&~Dg$ByIcC@HO~Yp-V-Z7vX#>hn)m?HXVF>G-3* zpZLV>?H~Ao6vziM(VF}4sG9m2kLERX#)asO_=sQTkkiRX>2wGvOOGh3*GnGAz*jM} zt+Q@!VnVr!HLhBoSdKX-pEMGEEy-^RUnGmFo;g4b+sFk;k#v88+h9o%4wn zcE<^rvoC!G6L3Hs>ZG&)#JpnfxS(48lhsO+q`JxTBPgpJ$Xd%k{RwmK@8;$ti0Df0gB6B|u z*pZ(y%l}l$Lw@jCL-=Lo%U3$J@}gSmfsB7;Ns*p^K4(XGRHSYxRU24db^P{9x&~EE&R|840Ex9ZlQ-G2@QZ{-Q83^pGNd*%1K%VJZ|- zEnu{3|6g%c!R{{&X%&AaPJ&{N&*3uc8CQdPU}F_gjQ;@5U#*H@ev&t=aNMmqmzCt# zOAB-8-uasRxql~T;KntAIJUHnW524bZE*;pwSAoX z0A=mCy^3QEYZcs!jT?kH-=N@Qsgm=&bGr&K>cOX7F(UUoVx2F=s<;(vovl3idHabY zA9HDDc;Y9=FF*>PsZX>UtcSk3-bHHP1JZ3ikon{%+gGEP^!e61AHOsyCtJ~({5Gh-=sdD$uVh=4U=JfVYk*3A<_Untq1ZcjJ|QD44`kGqCVE+SbUZK77*{O)5kMWAHBYWr+^h+K z=+MLxPpgEnaMLS{{P|L6Y;fh*zb}9J{oB{S7N3yOAN5uKfM`)`)~Y^%iAY$ks6!oK zvTXYV5lH3q%vF{p{O|~BczC#?bGJDnQ*LnJV&&d31G<=wfh&t2E0X-4HEQ*niT=k= z=Us6s1Vcakk=FJ%J*20lX54+{s3r6n|0sm>H#q9UimMne`H9KqHYGmxg^Wj@{>StB z_78Z8gc`ZKn;eOal`-R!I*~{59P0p5AHt5CeXg}CD?uMia%v}aE-0D1b)wnFpvazc zMOTlUDWGGBIn3Olvh1hzNhNaZ+q;y{S~*AuW`S$Fk2S`5lcO$&@1r766Au7K>jL z+LI|qI;k)E+U`+2e6@0STgi%E@c#iZm{MbpdH7LIE zPp#@J=2xNMD}J_s?6}?*2MiXS9S7F4=V2Y{i^-EtYQ?rpYzS%p@urQ4e&SX7i6-r= z$MKOZI%C&}c;V5&R%Fx=s>HB!_Tze^hcUH;a=Sh%Pao@o8y)hYT^L#s$rPcK;LwB= z5H*ztSJ$CUOp0pH03|`%zM2w%egxw9Po+ti4N*^XbpM>FK_h>X7rrCDr6?5VPbFeo zt5zZOs%p+!J>IvO^?4YO%&{Yo8H*~`eqaC8mfx?M?{MXU1{MwC_w0_nr{UOGr-}I) zyI}mO!C0cUDX508J6xwSp6B_RBaiC5!kqt6kpGrZDkONgC(+sBSC--vGT)XDWX30C z0->0nZA3%&95VA?kE{Mb>*pKe&Lqh}7*|pQQb5i=5ch@Ga`r%`7Y)v6$-A(ub~u*@ zG61u1cdQZXP3$aB`Dh(?B&iZ5-6oBD!v#wxq~V>)Dy{a25fy}m{q!jng~F0j4VKGSsgDmKuEO=8%>jO`f#wTO#l`B)FIs!GHT z8o6=TaL&fT0~z#3A|;H)yT21Uzy0=*q4$O_4EyhVGZ|ElHK|lJUDNGXT{CA{#W;R@ zb@~I+_0Ca|dLW~ov0hWdI!Z8#Mz$L63@q#s_bI!MAGD=mrAd$OqOkO+1=dx)U7@$h zl}Wc1xOEN-G~bj1j1cuyBb>Fv4RUSiiH``!>AaB*AM+$)$Ewdl`W~k}XIPHdIKDOv zhUyyUZfnUOz#`jyxh8W5n2&@u))95|Si!8C9eFkTY24C1=$Ow`@dS+=*1b}fOfD2R zfOHAVqS3d#wvYVUK6BfLhkATHD7D(`R#QZ;)CjAk@~CI7jv@S5u8l?rtKDj!DS3~_ zZ%4t$ia`$YgYOvO_N91^fR`mEJGNlT=lPSMZ~;~qS85;>el59)15j?zj@No-N*5@` zBWGz8ix&8fSrp@s?$bx}xd$>eK61%&T~wRgc?BEySh>v!42-*E7qiB6sBJ_wo_uj~ z4t->M{pC&K#JKl(-RG^)uF5sUTDeDZ8Af7Xjy|cvQzI?(@nc0me`2Cx{ghiCYK`g5 z<3~|+%&p^xm}&%LwLa(eI6kujTkNhu@QQ_hW|O0wrb~1e4vdbgJdcdH-$#>iCs>yK z2NdV8yXK`%INO@bqvZsBmroosg&x zYkaK3Aq{khW4n~v#(6ze2U)v;nz>UBpLG0+^_8fEUvU{}W?l3p$3-aHJ`+8aWzpCe ziZM=&pCKuiBPL9Z9CqOxEsSteqX)P4)RTW~=l&*RgTJWTwW(!HXz7fAv0k5#O0#f( z>8A_wVec~??fRY9AH*gmR*L&Iki#gB4jASYf7--*q=UMY2)4)cK<1O3(`+h(@#~dnScrgPER^2>~!ZCHy3DaI;^5Q{5S^5Ij@fw>j zaKUum-rn_hj^eziD>0ooNOJjnQCTt+J4J_A)LyN7e7}$r!-`LxSowO~n3$L0)C?Ws zQCIP+g_6bshh0~?T5>;kQ@z-+MI;*CKvpDPh`q?@6)PwXq4VtG&w!koX+&jT#T=%r3{YG;tb$G<3_b7pm!fZVdP z&L>k-zu>9Ucsa)C3#08%Ty|beMm$UkbX@wzs^=Nd9zJjD_!*Ixj%yc02R~z>e`a(1 zgQ)yFXV<*akRSH0amMLBa_qdmZNTf|ocrGfMx*1jRdXBqpz}N@&y{WxNkyl}cp$^q zUW3tzGxf*$FSnm^MUlc(EM23ti01VzMy=fj}63f|#Uev(2hG)L9-ZNA+H3#G*hLsk1 zJjU3j8DuYXwyQO|#JjG1#IeM&_PYYF{d@hHUBLlVbJxH_47ey%uQt4B9v13QV8`Va z7|uCID8wZ3;Aa*|EUa~dC4Rg#w>$_DY&}PSR>TW+Ve#)Z%0KJOWWmV7#GJ*~8)Ax% z`b*ob>O0$>>rvdbn5(tsOd7sXwVE{Y6$f3~SaJN4Pkt>}_vUEmPOCqbp-wR?Q^YUgEI4~(Q}{0I~c4o3ajm+eOdl|g>!D7a5z$V z;bWyuPAoB5M#k4xD@o1+tlTjI6@K`+pHWX%r7T`1X7tIm`@y5yuRr^o78Z1Q?&%XN z^pTI}!}tVB9=G%SX%-rL{mh$CJtlmwAHLNJyZeb_(NFEz5tkh8{M~bjTe#7ZVMkUo z3#{z$%TpR?5yL7E3zZ$V->5m$k9#?BQX`7cG-vzHxsn{R)Y5(T+x2M^YIN>#{LI13 zM#`-{XCSvMaV*;TrhjaC&FOs5cyavXZkBo|U5~L4XPBr;g=ri6q398fens$Djq8u%?33q8#P`+|6H#{Tz~vLx7y`A2;< zk@;^7C{On7fa`mrhL8tTSUw^1Q69)JB{`|1v)@e20cs>C3mQH2h|{spwAtw7VuL-E zCrL1wzp)V>B!_h`k1$qwAJuRb zx9++H`Zj_`e)?I*bfV&I%=SC0by9{9nE62F-5;O(iXp+!_8T`uzS%ZwR-Na@$U4O% zE=a$akCEil-OwDVH>H~9HBsw^8kB`WGnSz$O` zX)y8!=0NYHQDn{XaD)L`p6gb}k`XJGYT`x<6~)2a8~-!4r4Dl}>z%h{s0ztR9k%72 zEgcNmspCgjl)z!<{6Bag1Aon5J@ocW@G5Q&e&u&tYaSGO_6l>%BRzPsU`xZ(P=0Id zb(FGOE8e};H>Nk>eH)))7+EWBDWdbKV{+zRMr5H*CwLx6@P@+kQ}t7y{LI05{we)b zL%O9`6f*bYCv%TNe4R5>E_B#y-{6hH5FfOefSh^l4R5}22@Zc7!fzOZ)rmo(YyNQt zn9n@|Zx%2cj`TP5>z*-zGk1BarFw?aa8hAmdW(8gZ?<*ZYmW@tUVG;q+j~~$J>u-S zA6;3@rM$}GOx|A@9eTueg-Id+A^WHhR1TCD;M_+GNj+w0`)S_#{3+7whWpIJh@PE_ zx)yP)w0ql}8qJIkv3~qemu-wg7y-sIC^;6q*yZ>Mj{G?f$%SbldT@;%NrtJ4jRFf7 zH2&OT3g&eyycsxCHuFjcKlyis%0~t0jct7A_(A(zgDAS^D56+UU;L6kA@k|a)ZdT+ zqspXS))76B$p?aC8w)eP_L`f0oe*=$jat)DZ$f+k=sYXr7uwfeQ=IriC8f0LvFRKQ zu(tFn_apwU_pR(x9Xale3(T~T$K1((k1zPWIJ(H%`t8v=d$m2f=9M;jXK`ZKeGF$D z>Rc=WgFts;!jYj(u;Z4{2;}}70mi>zLj00H6F+XCv1?1*hhAumlj38wBLaOQhdcx} zD<1h7{|1@egCx?M8cii42`I{O2e4+sQW=Qv-` zwfQ12PLvXy^EyO*v6S!>5Nt_G4Ok)PYv^#ldbr^oSR}PygOSEx`Eq>i^)G)VJ|Xj2 zL2?&|n#V+J=3Lh5u@VODR<@J8u%i`B!Wa@Gpu0DJTB z_Tan{LGz8k$f3IOLOIVOQx4fVBF4b}AuYYAoj5i^ue6}QlgTrNn_=fKq;nh5aYk6- z=r0dBO7YmDKCxhmSL4RJb33MDps_?s&mQ8kiV?>OWsEC4;*3N`crWEStB`2r3H{+r z6M>Zn1AS~M$U~3&v3>uAg`IdtInux*4xTEo_`#(e14&Feh*o{@(Bb%zUueJC@eF{$ zg6Wq1M?T^D+)1xWD3 z)-129I`&rvQl9AK(gmW2l!uqViuq*}G5yb?YjRulrwofj8fL^Y+jD#@~4RXMf9Yx%~^j z?YG^&=Xd_j+xP#T-*fw2-}imjL&mc6=wcA z%P6tulgl|ruo-Z;$am!jx1PJC57P+vSjM{}iKC0k?q}qC9qDOAwdS>GAN{3dBOBlT1cYa|&u6)NG(>${O>?qji4My8uE2HCGn?0a< zoHn&%$%Pe;7{??1ia6V0%}x>nB%iF?z7I{UA*eb#jyzcDvkjfg5#n$iGg|Aj@n;o{ z;BBzxH&^Q#=~;dwo>{kq0jEd(XLSlg+!5 zc-$#w3kLZ01DQ4-ubZ!D;MeF3eC>n(=f#!p=Zdp)#C*;iL7mLOoIQ}C0AqM`0fXpf z+XYpT`9B`Og04Dw;g*#$>clx?O2_f4IWZ81(EHCkkl~IdxfYO?7UZz_U9sXsj3T zvT`{Gtlsk;d5CtdW1K-bw^4x^ov$|IT~Wt6$L{PMG*n4WtqDtP!IKhvvB6n4Bx^-ah`G^I4oZ z&acs2<=^#OH`OO|rstU?9L9>ZfAnOs88s*{=8>(OuBqJSd-xcCZPf;r+}eG*N3B?N zWbJci;`;;DJ<(O2@WGRZe#PC|$7%2N{ll&!yk6wtxRwW>IM!8*#U4<^u|mPe%GcxS z*AkHyto^PKYYT8yj3?(dJ9Yn5)vPMLHbtJ@E)=_u-|C?f`SYAG&R^P(t3IR1jdlF0 z0&}!RM{T%^Hvm&*o`;NtE322hXvo4VZ#ZKC<6f=N!}AQTbHMXJhHqJ7`x8eWTc3bk z5%X7;G&&{XflS?Q9|tYb#D-n|hD=sErbKP!%6wbWaDSvFwDmwngZ^K?{$GA1sn_J! zGw^x_esyO+C!_EE_CJ_OU;02MPcd7$>)FLY<;YXeIRAj*fy_JO6Ea9~)1@nsDKJ0` z-O=6a0F8wuT*va@M!1ZP2jGc`l`evc>DO)Osi5Sbtq(wI`bz6;ie&%Vi|81sAfKf5lh`rfQ6;-PyCrbbNh3D{?Fh3(J%g@ zz_;)IO~2{(Z+*}A-2RP!`Cq#IE5H4>-~PGp{_fj%{f6Ig)BpItUS7|@>lt`%20r-U zgWEs)$N%{DfBu7iaQnHx^LK9l%m4Pj-~Okc{n^|9^$-7H3Vhdhe&_AS{^%dQeeRF_ zvFLNu4(;gX-;q)z|0k}_LOzD8KNy<=dPkh_6@_hk$(Fb?rM@7S#Mypv^ zv8!dJU5B{+KL2am@WzDv=X*tslkJHIUHE%m{Ht|-ZG=5Py5PJ1tUZF-PFl3?7}`}w zwQ|E|cjO<(PYR6{<41>Bsn9ZF$AQO!rhkP8hb% z58S(dAO}{8rGvLTYkbYG9t>>9uPVhYY_#G>xb$hIP7<%!y3+TxKvdEwx= z1dBG-=UkzuC#m}i{_fxVU8QqqGaLNM(%YZ@^eEhw!{;13^+%v|9GQRA7sA;C89}G; zP((=IR`Rv@mZi_@Tb6tj4I2!Rk_g-6wKoJY2;Tz*=Jf*^ChWC+Jp;eCX5gg{WH=?9 zjq>U=!4qhme|$+XyFZZWdeZSZkZeipkZco`rL92n6ERMCqunuy+Z=Vi)oC*>H+tz` zdmxh(B^ZmEUv3X%Hpz&FXL}%1|AXz>Ku#9EtS*J}Wes6xFr1Ft8Y4iB7oGfdvFx-5 zG6Z8>eBs)~AHV(0zxB70|8IWp_ul@4ANuU=N5232v;TVadInz4z{fuW|HEJV zYq$UEr@nOipX1?-mVf78rFWg7>58z}wUv0ME8rgfw<^?VK zyY2dWEc;`7p|gJ}!YQXYcYW*BCxo7TAXAc7VB6uRKKeZ`v9i&MpI3PxQ(K@aILk#H znWz%`{G*^w;Au)n{r*Wq_0h(ro^T^lexYk=Da#qHGko{KE0R6+K<4etoyUC~!{X*^ zCx)}T$ngeWdks;j4x~Z|RfutX?e*vX)aM-P;@BjUAT>A*-I94sZq0Dv`$I+ghdA@V+ zdW|M*`eLX}UvV&4V#2Zz6I1Vrbh?}8{5XH^4`glch$oU^`}Mk2jPes_y(%)A2w?m5GmrEc#q;q&2S<>ck=) z$>uz?Hzkf*(>QzfHTk-qyxphvjy2BtRYBi>&D|NLNaQRRu|hK!BV(FaZ4>h-|Cw&j zLHn$xt60~hea3f;xfNM@5`AEdc?5sh|}*D z?2f@dv9Y5kCRRHC_(67{^5=k_G&Op(x25RA@_hdVyVusm!Qk7mS8?N<|LTDZnzC0q zVD`BDYc0`}Bb`3~@bFsSwARm`soCy{vmdKH$Et5s#}J=sPc(Mz&Q}Z#T3`Greaq4( zZg0OGpODG(XE1c{9gEhyATmaDh`;fT*o){S8H7440$eQmgv?j-6EdG0J5c~bW7KkT zLc9=qfm=K4Ctm-`(hFwnHT3li{L?c7x^VsKJ&dh)NEEVSTu9dac` z0qLZ=EnMewfOy_e4Q#(z|IdE%zqtL8AN#S4{5!w*_ul^7KlvYhORf(i^=tEb242s= z#SDBiey!=>`;Y&V+h6>ve>D;R>7V_xw?F)c{@^^u<;;h|2Yx^D9XxvKTydaFH(s18 z6j!_V`#FO+mRwlfbi(IcstkUnFj0%Xv?`)r<);qf*p5lUSnYAw;z|^oV^JU+{Ld{K zosW+i=s$x2duBWTs2v_T)5IGh)u;*n=ca7O?yXyO% zeYP+790&OzDkp#fQ6vaa>_)*v0lNu7ZA0-sHnEcrp)E9m7DcW4r37LN&c((^CCaDf z3z3pm1-YRJUpP(!l}L+;X#;|)MWscM3#ouosJZxx?Q?d=e~j^)W6ZhMdf#{30rp$3 zwfD2eoMZgQe~dZTv&LG_e4mSTXd)+^0@;QQxrgaEcWOKEV$%GHr?NJy{|^*cV!w_zq96qDUL=Yk9~ zbINlDZr2R#Ey!pD>p3r7KkxVK!e|3`6-Kztf((u_YDVUE&KWpo;P+<+zRlhp^EdzN@A2S& z_wW3j%h&z8U+e98F{Yp8`+0UmuM@o;n6435XnBFJUdQPU5)a9+l@x<26Su4VBLn^J z6OO>o{@5gmu@?v29*1<;V=U2&8*SL|JeF{*2pCp7df~e*t|J;}AhhrHnLjNiw3x01 z87(Ntd%PfO4np&gdc_1A83++scqU==0{}-;dA3Jm5>wwguIq`9jya%DJoRaJuE;Yj zy#B#CWyfozLc2#k5&QYG#uo)~lkNe;>e(~tG=bZT-BHobV%5_%}$T>+j)DkqF}&IOruw$A=J1Gi%a zem56nGy}bEtTS@mf(#2RQ@u$NcwGWWCEXFP-I!rRg6VuL%J@5FWbhk75aHo!ne5RQ zWMqZAJnV)>%Ij_@9aH|>7e+Oo#85nK6)XDKLvllP$A8ZHHGlrk+dau&ExDGV1j9zf!hgg9W{B#hCdRhVyqjORZpMj`!i5u5S}=2y5z~k z>mP_yf-Qf0cTvO09*>SJ05Q?5{XH*XYI!m|1Gj2rgORf!@JIVGUkt*C`D0RNTm8wC zXpv)}uyr4B=vGYGqqSy_T@qiUIJVyO492yl49)%EBkY0C~qk~p&+M6ic{ zA+REa!Wtfz%RBRDukYz2OWOP`nsr{v7ysa&Ik5Ye`T{^1(JExTNZFDLuLT*t)E6G* zGq}{bW~)eBpm?-(j9zG*-ysw8a_-L=xNS4Aw;;oL-pA6tN(S8MH%NHP!e);gqkn)2itIPt%?IIz&cJx5^vW-ZZ?X_Y9tG;S0V1YRU(i+ed=_+uyH%6gR@8?yVVjXq@T z!7zTsopCi(uz>-kp0;ic4T!R!3HyAHm?#r7DzNO~I>KNFkKE#CjmAG~!t;pln0McA z%e`tMQr8k2$vIE#KIergt6J#M!jHtx=+V2ho$P@DX~)N#b2gek5EDoGzvJESzP$L+ zM=wv@dm`pJ!Cb$Fe>zLACS0l)av8s}gu!&N@qz;4kxmUlroJE}M|jdLxNfk=Z-sWA zfw5(@M_=#Vr2)8`Omc~@^MLMe}HoTz96G9O^Y&?LA)S{xS51t)2}SK;))$PAqAas zZ~ekltEgwM-aA6S&3?t@+n;&nqA&3Mng8bPUbBnuY@IW3&cMSk1Nu&xSN@f+z5IfG z(eHP><(bQyzF9wsy{!{_=lN0o9m@@|-ypgHt5uQsZ?Y;1b+8Im+4kaLt(tum>L%l| z{76w$9$<{3(!!MCx=X4Cw%gPuXC!t>#vvPaTHw$liHCp3D$ODG*M>9+8Xjgyq^TAiIJZT~2Q#`c7+A%Y(BHgS7 zZwBM_nevMNCMz2A2RFyRv0z%{RfyuRR=9VsXy=TjL8_IzM=MiSl2BzCOpX?gH z?2a)-fBxd%ebg2f8}nB&s)Zle*^*j}2@Z_y6TFzAd|)}puKasTq{KgpyYi68P4QdY zs)-0)OH8Cn^Z0NzQcJyO`KNWn>;q;}!dH(bT8SyEG&L3hCLlXNtwIK7@ zW40h;?~pO2+gOn^UguzSCjF!oPFutxyp7oH0;4?KUT!#M*F!3?P4@TI;V$zNGo)6#i93X>I!Du*kw`g=8ndIRdo zH@)rh*k?TMbxS(zNXJ>=sETNnfF}Ws{RKIq5fkL>pLq*p{=$HBRkFd6U61YYEswz2 zKI2o<-}CHyE+5P9kioA@;&h$btG>sk?KReuP4qJM;EW@(@~IEcZ-3wemoNCjFTDKL zZ~fNgd%xqY=dUTn#GU(d1|G^8(62B3```ZL;{vKb9z1BS3CVJ z&NQhUUmluZzsIxyDl@7cqy9wKQPlq}ubY>hGgv&(=FI>S3E$$FDK>w**VM1xQPg0R zqEq&^IKiQ(zh_%>khk!<2KcY;QGX$G{^R)aW+6i^;THIwM~~a6BYGqd5aKt7IeHo3 zM|R#q?YF@C=#79EO*(k%A1*X1Vj%HHqFdKBmX}CifF)!w|a#)+QPwlbNh&|n8UqVLp9mE_PD*5*B&#tVbifl+Mb_O)4jY+ zlI+w^*~Ot_2v_mMd`hncnHSmh@^w$xJ7g5g<0aBKM+}_v3CZ&jZeNg5P;io-ItMcy z`_u{_Nj}(u3^DMRywNrsHBigX9*><1GB=vK6LikN!!`pdR#z>^#OWUAy60}irpn>| zXeel_b(D@Yj0Kseo^lQEToI0%rUIiJM^SWxC(%@uk<2|#_=d&{xdHr!&SfMbA8kmr z!z0I?@nv_6E)>?FfU_X;M*C9VzV$r1zPG_3M+^s7Ty5?JaM){Ect^<|#*= z*L9wUX(iQMB@uu^Pa0+$0KPBeAAM zc7^Ywne{htsj|+#|0axc%D3^Vo7+B1VokI^vU&|BepOkRszhrHyr8Isj5#xS@W$%n zPh-b$914#InvmURw#+?nups@~Yf#z2=m3myNa^5MkQtbymo~V+-%={Ir zp4S~qFnG<-15~ehID72S-t-+ZGj~s{ksdYOIiKiB^w{~4rK8z8_2&#c)H86jAQPuH zm4XVhtrZ~pAPe2KQN`fZ#rerMeW$%c=4F?spM1)ReL=>YbsHxR9@B!1DD$qO7izFQ zBDwJv$tvT48uspF?faITY?NcIs5`%uYS>N}Q{unJer4(7`3@O38v<@*cxj&$Cb3CI zJNDf0x>70+ZgOlyDK44v@;2P0!}XP~`qIn){#ygm3tn!e3MhoE!egvVf2*>%Kj(pavdAI;pEi&ksJN zm~d0OZz~2|aEdoHyl~E^%UHB4T+g85jd#&lP{o2LrVIAMD>rgNN~ew7=H7}5>%KN@ zecw50ci-Obe$_TF{dea$fAk8``zDzc@z9Pqu}60lCs2x3UTT3y7Q3*Z3pbvx5T;{w zizyi9Jy!5#4UO;9=vaW((XnP3ZuL_?xa+x47;K>tWj%tC`s4^1gDht&{Iqv=v-jsO zTwxDK?z56+5D2?&M;LfzXH;sB*dj`2ICvh&)MIhnzzw4-h88-x`_?(6C?s|eWvNzR zvoW?k^Bp z!SuKUkd(q6YVxoqYBk4(Te=W@*!LRFY02qdEXdef94$A$8EAxRYC%Sl1sT0V#`8_R ziIL;T!E|7-BNembJit(NFL}&7>dQZ*i?RDBKk<{7SAWG$S50ai1TNu8Ox=kjcMT z`0>~EJrhEzyT_NXfd_;BNWy!B$1Z=`9oyR-Lp+VZpVH>6W8?2GGpFFHXHB*6tZ_=S z#wk>^{JE(Z{?FLVp>uYB#xZlavywDNcixQ(&&qM6XQYnzi7@)%9C^mzf&mkJ$I>*JfH7bc_U2olDPjb7<7M#+d{w=ZIMWM%<3&8Ll}&@6rkC z2(EmEjn=SZNPof)`vc@F=tKuulh~W8=DRDiz2eura4B50;;EBuKwcz9XA`$`_ck7q z#9=|^wO?&Nd##`B_H&@TXdzb)KkqEb@sXttnA$`{nT)7;hs<>gGIGafFDdB3(C=6; z8cFo%xgf(7o${Q4TQvhWS&-qotF9UtwW8OE6%Qu#_s4=vyhFw=XXebumc+xu@!dIt zNQF98qmW~h)P*v*Ksn>?F8tzF=sCY~v;`UG3OC_CnFX1Vfe5sjf@UbRPmP2-46lM< znlyOMjBop{@47r`A6GhmS!tQ9vwP0KLp}pvXXowr*)J}=?XC8)rT=XH*b=F))nPvS zL4_PID$R)(JCqsm=w*W>9FpAYX;E5R?7itbu#?7=;intoump3V)WR1TPf}{ozx%GX|WaY4olvF-{d?jO=1^8CK{eBp{S(gTEVmXPT~T z=a0&$OStj_}%lavQv>_d1C>=O4ZMa$kiye`RT%t+Rj5z{5QQdkZqk zIL`mr->i&XKh9oLR7d<*mR@XM>TBO2<9~(A3lF%p){MW#qyZd}J6^VU1WLBBcNCr> zv8W*H#R?x;(t;Fkmn{tZ^$wXg9K1t@DDuTWC%7q?bD`&v?Oo7?8eHnMP5JljjrNNz z@3JpC``drxTQC3N@;x^lcec+NIA`GDnt|_r=Q}U|>399j%NxGtYcAja_P6gQx7Gp9 zek4xRAL@hbV8Rwm+M^Fl^mzA7pn?^gdZo7rX>gH-5cAape`H{~a}ZMliGhqPVh=~x z?YC!ST~}sSP{`R@*!vUAbi~#i`$II->Gj{r5D%Ro*)O)eGy&!-C8``^b{8@v7z%ng#_NUwDngR?q^iWrRTxK zaKU}nzC%X8vNXQbH{)T~yk( zxAgcT%cAIuhjm>uz@ss8=>E~}95LfvBTTW(-fMi0*Ni&u^*CzxIKAC*oej)tN8hn( zuehyd_N3SSdK-UmjG+2+2D7icLxZ0A!yj3>7gD;}ZnoKM_CTPXv%x>Jz};Nn#lPZj zLKqfDB{au_XN_C=R7;q>c(X6~Bjy=RZ}q~P`5(9hj@!33e{cxPN07p=TVM{$+^e|@ ztM=Vj{ncaET!rp7wP8Yv9Zc6r&xW18WXEDJ{?PbBvjJJq0oVP-)Ahm8H65*XZ9NBR z@#6TEJzBB{e{e|a77G2&^VGSxkc|Uee98wxK%T7 z)q;!$vetl`46fG@@O!mG3o?(|&t5$aNez&Rr_#gU1 z{&UyA^20xL`K-_UOsUT8oPl!&Zov%v`fvW`<#Yd?ufBZNs~KBT6_oXN7>ibe2S%GdRw&DMOp3gSvmH2u4fLOJ^s*@CE``oYAJGTg{EID_;ap>tsG{E?Bk$X#CHf?zuToV;<+M+Vo-Joi9`#J1O4pMNt}9%!~B2)+m1 z!oo}_v z2IFc&zvnM3cBF7{AV+l`ia^tq5B`$qkw^aW6Q=u0$A&-i4z4^Z7B-jt_(GP_pZiX1 zJDp$lUkgmclD~5_84EIxUG6<$?~oCz6+h2G0^KyMymJ|Ina0io=Wy%oEA5 zJ%N=@3o?AEZ;Vk{2o)g+wEEFk%xTMs;zB&x=$O;FAj4#w@|=O&H3Qcz$n?>y_2H!fBsQcl-L5`1q;z*5 zd(7wIqd)#WzbHv}Zs!b~Gw|@wz;n+(e|hxPuesd4d-wA34}4%Gyno)KHJCL=&x=94 zDCu=!kX_MG4dh;uI?g-)RBaT4IQ#*V7}QU%!-m6O_W0{A5SxnBa zu8~E)bFG&jaqvQ-N1{h|Uog@!UbTIbL%$|1F9hO<1hn&nruaGzTbGqrwPf!-anSFv zdb|6q*sE~Zke#O%1?wFQdpP;b8Lz$Y>b~LA_hcPZ-b;r*hbu>URZFZKIvuupw)B?t z$6K(q^CyU!LuAsU&bcK+Cc^75$HKhO}HqR$_$nAN8}d;vAN z+PY7)$snCf$@j6iCeNRuoEdws^O5_oXJG1Bc8xY#jsqjDE5cj*Xq7Dg$P9b*Y7Y)O z{X%U0!65NP$^#p-=Qb%Zcwye2|8M~7{-b^>fC2YvF#duB2MjqkrgTj1Do)K^=MaxJ zaI!h07ZKCi7xZeYni5AAj=*0o@342sywL7>wIJgsv7J8}&sdNViGdlDy4v6&fjhn+ zBS$mpg+vU`dj3QVxc1Q!MQrrPuPnVyzp@nG8XPCG&|uYGH(|92(_hGRe+TrPIh_kK zeL~LmIRm$620oPq8I5wUoKUe0Yk6&Q3GX{ z&s31DuYwGR+Y>jG;xoghaN}fnZCa4|9sk+uSdhVA>+Akgex9)J!ol{U=rmLOftoQ= z&n(3wE;u?)Ioy5eV?K5*$ei0b1Lq9fo*7UL`;0GtwV%JwS&d<%b|9lJSkL~e9i>nm z>$x!yv4Rj414*Lk9^tCIo^oWkJ!@K zEIWYNJ|1&r13_e^#O1A?HsVEZ@^D^fdC3*g&v{q!)bf zppE(4vIcs)FMQg(h{XI=dxc*MzujsBkOesF`s9}E&bQh034p4+6%<46628wtn8-xQjI zSOc)Rv*$nez{28(6rc1QKk413`(L$O;hK@Fe6)o&+Tz%5rp@QTx**e!tNg?zj=mtH zDTwEhoqzhprHts;uqX_2^=;&miQKWjhBz+XD3e+dXk{p!aY8~S$*(Lu^R_p+8U}p9 z?)l0#@O~hrunG0JijVgq&FNf_S?1*Io-=SuX5c{K`rwT(=1o!)>qRCl(;&q`96}2>u-4`yOL~Wl$IRUw zyO6LTGcE*4;=I}Je4W>65+l)VsNn$%7ua(_hM79$IRm$E1|D=lh8GiYGi_C&El5xk z`Ycd4x~{kVCR;RC+2UTXTZ54oH2W{O`7W-wId>1|GoHMsU2|CTTeHjC9zM(#gL+WK$X|##FvVJne1EIg5^YWd6|~ z+TP8!`DakB{{02DK*UE%mr<+b9OANvJ%8dQPC+s>cKx#c8+ zv=J{VfV26o*QY&<*Ek{NX!XM>m-->I!t^xYlC5tY)AhZLFS7g4#)cny`em=!*&rp>5)f$-{{Qf20i3_v5GddXD=M5-^@bf{r}Av+RN=l

0 z@o1OtkSUkQpyweQA6a_s6ZRc4=I8TZJj%t#W0<(7pZ!;!JI~vKj0ZDeZExOr#AZV) zJ8eL#V>q>6S^6XvWZ>|i8oV}>5!WKFK?C8Fb^gjyPR1$D8MtLLpknnUFZw3W`XkT( z>W#9WYK3{Mr9qq>qZ+be6ln7f8FO+R&Nv~_XO=3eQlaLOqxfEk6DOez_@yLQJE!fE zqe1;bLSNj^J2o_G@{mS$zSMUt$haZ0Qq1@tB%!UG1=SYTlG1@yTN{|@#a@_>1A{yl zWcqBK?Q;fh*9@qZ@eY|!*gIs#d7Z=H+z)-+1V=%L8&Q8jx$}aSS->BOzGPX5UAC=8 zwyr$AbyQT}`#r8wDka?^sE9O3mw+%JAc#oA&>$U>1B`UT03uz|-Q78K$I#u~Lk++C zdVfFb_cyaRYu$V9Is4grKl_=A{Y@tGfbq=^=EP<5V5VIgLlOOjJXFPvv85@UfjW-j{F&Lk+`Aq}tXVfS!g0_jC<&D+rQxG=4 zXCEVu#)0gO{yIWFcfB=`fbU}0$sT_0e(fuXhs;{Hzh%S2cR>W;Fb zFMs`_el8mncpz3)-tL>~HJh28TfFplYTFu`70c}eOD*X&+;XIor2d@a z@-mkGhAQ}xC|DfFL_xwk5dMvI8Hff$(QqF!%5$5?l8YO=cF{hToD(sSl)vZE1c8i1 z$&7n`C5HoVaSA7ISY*d%P!V&dngtu%-|ljrwU?aUM$c^HL3Jr3I>)V%KC{CjO0*=a z;#K}$8mY1Ivf3we9$nD$Y%dWXCXFsGfD#OjWNp?P8!$gzgj2gONUF+zrO8BD#hONl zc3VuRrfq#%2Id6Yc^UUH?JfW^q^0SnZCgOzSW8}pTpFn2QVgkAb5NhmUAwprzm(lr z`E{q2-LHjwqw;botT|EJmJ?3|vD$jsqY{xHrP0GdBFJ#+=;~bd+85-87tXk?7be9k z$a8}1aGjohE&}DF=f2%p5B;MS<_o{`Hlb^}82>K09+!)nKTmhWbf4yItaKh$@5+m2vu`$zrxEJ}s|Ao0BA7>qpidh|7&(fN= zXf;Xaw!xv?s25Qy8`K9a&1}S(v0=^!yyq-@8kWcKNTlKR_>F#QkDcX z?D2xsv*UX_G8vnSO(}Qlj2S)cTaeiSKmt(D4%&_&8Dr8F+=K9GE)=b z-MqiW7vzaqv>8K*lX{SRx?t23BEiPF0Zwc)fQQ-{QdwJe-Gi*@+>>qSyGhHZgDQEZ z2~zY;HO$x*o`itviun$@=^hB+4G8z-wczPgT!ov^`S^*Ij`O${o;*6Z^`_n78VBoA zR|h_=ob})n!J=)Y^pa(DY6gGIhf5sqJhzrPB9^fbI$R#*zYo*SD8Y`dPyKYxbu~4g z4n#w15k1fCeNM>!-4R^NUfneI;{$b+Ha_!VxwvDuPJV}qprLQcnZkfELWZG#no!}S zw8kS-H=T`kF zY9;YP>AS{vY_`;p*BD-1LMTH9mL@&e3hRv;#Na$?`DQ4TqgMl*bO&-wY*WP@vzPiA zUczLMwT|=O)20yq?`Z?@Ger|P@C8ew!p>o%SNB# zk!=zM9WfStLOKvQLccq3MbU(1&3{ikDW1+-f4MZ|$Flmr!m@*wXr|9Bun33@ywGnXT{*{T>D{y*@w1_v9QouSc1SsG`52 zyv)b#kcM1w(7m?U_kzURaBwihnRXxAzwq?S{A-j}St@YyGEhkqVwv{zF(S;l%plHX zvm)>>-bZJVf00hpKDyZ%hpk4|GvN4_^$zGRL!U;}krjDy2_fRafkjOSZF3FpUMjtNYax1qFV}|7+m6iIdq0OPjQomNJI3M3S)VlB z7ui|KY&BH7Bft}_xWm{U6fpzexQPEejm}~VyKn!;#OF|hRZzBxuSZB%Eh>a(leet^ z$56XNl*z3vX{Wbk&%Z)W?ie*aTbZyPU-xYm>BoHMs->jZaPyUvp}n@Jlf7gecU)4I zDA|+libPSt;XP^fRpjy@0S-wed4tP#+CyLUy&6Pw>u<~Cwe%IJcV<`ExGI|w+J{UOmAQh~a4AN|g>b>om&W)C2atyyfU3c#Gz0%29t}6{ZZY~;%@)Y-Uy%}Y?+FJiywI>^roPQ8a0vI z8XahZob_mzS zn4(R#I(<$I)K@ z^llubo} zArbOyo9DvD4*g&4huex1yp>EO-dSK(&#)xSJhgbqU9dX!UGhP4E%b9M{FzLRs7`Gu zYEtOlPj~-u51+5DX!}uWzh#$R@#9?!_w&fxf8R2MxaWA^-FCk!UeT;pzYYZ^VIIGM zjnnQUg^9}F{LD@2oQyDg7Kf|M#aH(rWENG%u*rehI-TfX7f$aTRC0yqP?{`aiS_PJ zXkSBD^agUkNQYsfb-im`^XK~VVc55ZmUbraT&MW4#0HD-{TO-kSN`fyj!6*hN<(uk z)A+%PZ}GaISjV3hEo2cP`nuUS#cKjYKT|4GR~!58>Cg%vjxf#|>}BUL$N1e3Ci51I`u*=8J{T|hBHkl;7fd=-!IFb&gKuhI#ETV;#?y+?O>#Y=Xy zCD%kzmoZMy~d!MeQXB@MTA zD09@a5!0p|IodDp&lk&*vAJgy)_#hat-Mb0rsC*zML9mCA6;a8V;sG_jUv_)_oKq* zRuVmg8ep@geh4D)KK5DVa4nFAP|aRaC-ab46b940Aghu(ep#5>--){O`9p;MOa$;( z4<5t3l?ZXTt!{E0zyAY$5X6sqV?VZcx`*+MA1x_O4Y$sIK772MK8D9F1OTwhIRC*{ zk&dDd#ly;AzX5J8txQbZ=+V3qLn&aVMB<=!zQtwlJ9o56&}m*5SAw_E<^?~<)VxuC z8NZ_9|IaZf4%zf+{gPVX-ap)$qw=Nhd#%@i^j020X4T0_iMK0S2mAbtQ82vlsmZ>8 zQ-=;5Oi5~_!F;wKruemqHdydE`(d~@xg5I(pJ?+E<_F)igNMY&lN~;$ei{!~Unz0W zajGK~6t(pFaV<}uas*W)6Ok_<8H$EqZ8=>!TzBV}cEtsK-8^5hVEP1ZC1I&EJIEVA zPkq;~TCgUY`_G>7ssyj#P@pz`7<@iuzjjToV!DP}_?WFK`m+AL7%zlMfXo0WCWz#j{RHu^(*K%BDj$ooZf#YGSd8ND|GvM$U)#V6ROQB?Gi7j68{u_IDn~s0S zWQp{B4^NEK++ev6f%rGHcd^-#vr6MWDW1%X1HLGa|h+95IO#2jmV3WM{J;dT$CH?X959 zM2)n2Yc9Olz3qK1qdaXF>=o8L?=F1*f@=gFrO`+y)VJ148nVhsX@vf%;ZfiV`M^-; z<*AD}|E-O<*a!=uf9maLa%4G~ZB?6Kq*L4La>fM3yP>S${Qpw{VfZ zvPz71INu(}=lvY|Q_(~MQp8CLwMeS>KhV!5j2+BB*__luW>ShEnbOSyU%UXxPVBXY zsptXJOIR%HKwZ3l~_fV z#TRQ0%MpXumcBEM+9ziI4GGQlwe*a>kbZG!&B*dQOWx-)n}alQiC#>skZHjm!VaI^ z+nhi?g<>fYgFXWRDHsVTyaZaZ1_@exyfq#)LIFoF!s>Ob)*{qLf<|JO*6*fDg0KK`2#~o(7#cOk@=s=}hWYjoe0Wj&&hct0JF_~abfp%eWp zl6-3kUj<%358#raA9t$U6efNN3q6H23j2Tz8}VAr@bRdU!diI3Xc*^r8GIO8y}VT>NEq>tfx;4gV3typSK($DZabxt)IGL zf!+(41vEE>a%j1uJ{w0M1z4x78w(|X$WJSfI|gPB^Su$8IQ$ZR+EZ`%RJP#ThZ>)gB=YOV- z0wUI459AH_(yIBZa)?4^(s;{IbH(6gmwWz+;aqZ?XaZnU0ZJ6dr0%QRO~Lru?wL;8 z`(Esy5l1msb?BH$>~(W_k$_rMPM;>-_$=Yo7er zeDu&q=0^`!#90b&BHU3T$8`HY-*pl|cyM5IrvVa$`8rN6rYvaQT903WDSyuT(szz$ zH5!|Szo#rXG|*6WqgijqXW+_0!{eIbjm6|NWHrtvOy=fJ+{oT;;gM9TdgWzmUbGi3 z*CS|rXC~v*>JRA+*z*nm|o(D0UB!WP|xib>{2(-3_-7o zqTYV%8ox(Z)YhfC6-0Xm)M&|IadQ1YOC%(0D8nWGXk5rz_`|487^{`Fm-=pi+blV= z?9KhGh);MxU?QaCOX}HmGA?4|K8nDtCtZrr*Q+apd?!eH z`9}Ar@V>HCx%?IN$&5WV|6wMIv!n{=rN_!s{wMgOzYF2|IP94)T$z*iz25h>D&nhc z{B;WZk0#Xw(kDMl{5s?I@u&;Vs(d3KhD-qufx9}gSqHYutI#A86iNN;;z^FVf9^5iNWqu);{ z;v@cSTKt;DR=>v8Sr_#oFe3OX&>KRc(5;SGau{CJb(hqC`qA>2i*BNW@Vd#uuvd6% z55b5y{`+jvpm3)zB4t zPxf%T1s4WS?|a8r50V1_r~DoTP!Q8qxb%i4a=(-$HRE{pW3VUPZ{Qi1P5{?_+~r~o zt4taFxhfx~<&o1oew%ksI(P--!>_X1}REpZDw4P8;S#sE!YSydW<7_@PaT%0&@M}1K z@3k`@mzH!Cg?>%;mj61f6w=iR=4fM>sefh!T4M^v%Wo==`=MoR5{#OHKqX?jPR>C= z`ibYn^KEZ0@Bh6&(RaTX?bgd{R=B=zGl`2)%+9R#u%wNf`X(bu-cn0P@0saRp`~e@6Uo>29SK+Iw?(AM`;W_TC?FQ%BP+JJGOiawLTRU)q=5u>X+ z??s1k;O&^I9%6KTWCQq@z3g8$-+{%rF-omazcpNFb55vu<1l5zJRmEO3H^bBv;7iS&~53Jax1SEr7$M<*JvBl~myLExIh!N7ehk(s&Z`!X2+*ee$xzz=n>>0W z%q+u)$ZGY&v>vud&5ribQr$5&p+O@>W`uklW(?`as$w@xvqQAvZ)1s~Hl(6R5vU%{O&Vp(#q)~t6w^s9^ckZ*tb zI$9OR+KjY-pSv#}49m2Q=(Vm|>hZgJ8_Pv{#Gp~dkvYJOiDY<~n5m5WE4U@bSI?imUYHCN1aeniMp>+N9mifiP*Bsu+3Q|xNaW-rd8uNFgb)pNqe>RX& zPisxKUp;B-Q?a|m-d@y?bcW-jQhShjjAdxL#g^V3qpl#-n&p|}wulXWa?Rh1_NFYqg z*w|Ey%s3goun?CV$3(KnE{8*NGn$s=4e+=B{{i%?TwDh>_P)Z)Rq;ep*~GON8|2h! zThJAY`gjxjR;=O;R21DYuOz7EkgZp?^tNt`{-K4t&IC`b+K`HxLTRp!Ak6B5hVQ{N zr#(5nMd}@yahP(d(LzK;Ro#=(kh@GeI%D26)f0lX`0_K7!( zqj!+Wb;@6?p#ZyTtKbtpOUur#{B24unzh-bWzS&q5!qo-x;r!N9@wqeV^RL_?D~rL zG}OiQmctc&2!-qUIZPBqz`Iu1oMz&-rk+@j0fuKt1q``gfe-6?5Jf)+ZCGugTp6!& zI=~erq^5+is0xl|okMA~b!C3UC~;)M9O;3-Mb0#i>UUPf1A-U?Mu%Ra`HND5ll#74 zb8J3NHI$uyx4TN+xN~B!16QXAccAo+UAbqQnT{l$^!6!1yyW`mIrOD%Glll<3o_8i zhPSvCOCVvW0GG?le;91TtzSHbb1tf)>$~BE755Ol@jVHYG0&Y%b&3?ikYj|z*CfJh zZcvle9uaCC$@r_%q#5e?4<~PCWjN(wmM8X0N6n>xoc8SC;jp9MJ~@8bi5!*;{I^!c z^?fI}VmnI=UV0}Freb7Kv|VlGCJLn9T@MwzP22Msd`o7$Mk6VPMcQU*PCT6#Nm|75 zoF!M-`TU?>0b@p0YxT5!*RKZdDj%+{$ccd%x{7 zn9Jv%do7683I-4^VBG~8!i`mb%N+J*`2}~pktSIsi8~1=C#xwEN0!r{Uwl^*ZtbTy z_0Pa_KNx-~G4n!+E>Lx}Tf~_1B=f6#3*Q#v-&VnqicNW2CPqRe^h7F-zexT4cF;*?D-i_id&G<>9SJh$p=}Pl-n5 zej5#kWm-0Po2>JZ&XOHM&C;XeIp1k0 zwHJS}y*$W!eUe~;426S%i9jP>9AxDK94NXpXt;xg-X>kb2=fzN|RGQ)`G`RUG$Vx*KYEr zQVeS7{OMSJ=-GK661!doXP{*|Enw5VQ!R1?l+(}4Q0#d56%sfY3e?0(JHp1SkFJ3V zsp8JK6w6FKVHCL3(HpkB9|OdS@Bn?e*A)O!^yj47Fp4BlKU3Tdk%lwIJ7k-Qwl_R#3gowXX*Cux+5{sr*4%U^2-gpv(TH;p8{X z@XwPqj;|!uu4%>#==nzl&WV|ID>1|~;$^@V0I0f_Z`j;F{RRfPEHxXa;80sJkPYsP z)0vqs)(br51;DRq^BBG1xdBf=DI0A~|;1R&nqDTQ4{} z5Tx*)yed*nRzT8p#Q*TDtvhaYkCHTMKwN8ATFaNN+aiuOf&|UE+IRl^u%^0reI^ZB zvHVM-$Fb}jUfMUkdaH=@`1Q4r;*v3re&W3OaLu0ews9fFo6OY_{o`WuVQ4xD^T&=x z{(xA0M`DA9xis-Vr+z-Pa^%Kl7>tV+@PLD>r6Z5#A%(uQxbYCeeY@@_;w@C6Pm%F| z%WQ0~Z8!N_Q)x9ko+RoR85SVmBL%_?ev7C##*L&R>}%H23T)~*Zs3fdarDAI zPfI%qU)O!p=2pGLnF$iHdEaIt*>0v`Sr^FLv=kvcTlWO3#PHDt@Sw5vHnjty|MS^u z`|q5OCf;LMImeM~G(Qdrn0_D`v(@mn<8G&uQ;GquegbFb-b7++{R)o)>TQEIDmd&G zVXQNWaSL*O^m>Z+g3e=|ltevl&1($ak||5gSkWF2gDadWzz#_Utb3ctqgdRc!I%qw2% z_tIu|szBD2ILB&yNE;NGZP{O`r9H`I;yT;Vh75#+NM8xJ91n-2t`zf7DgN=Gxu~Akdb4ghxXe6B6 z`~3nJW@PlnOxpIt?f2e(8oF6`?T3CzQR@`Vhm%)di_bVl3eT_}BV4wHjgLfExthJ= zxioy9dU+71gqyphLQ!to{6ArD>?J`=32q-JHEyO#S$E|<`LVgTg&bd9-XHSY4c z7TL4+F8rGQx&XAE&*1F*b6ONOM*T1r+4GP3<|vkrLGjiI{l40AQ`a)_;E$)p{*n)l z$V#)VBaz4MPyQMMb;2Ltfvy@YSiY+rx87Qy%gqE9sC6J#p6?e;uJiWTDPSv_rVzRV zoC6UQqc_?>S$yW0dJc)D9Me;L0uO308~xY9d8#pX*1>26d7cqbI|~;0D7(whqE+43 zlD}-SbONJNQNGU)TO_@bK_t?EQXvgn+-JLWl-A~E+M1zE|7J6;{{%0dg9ZE5-=B{@ z{<^p%cR{$5xNsIdKZthk#lxgXowwI9TXqkET!kR1+bf(^M+ih!>Q{>k1vPeOFcNHH znjQ06=R(z|5xsU--8@y5xFgZ&wTZ=MGgiQ)b?o|#=3pG9cu19-*C%=jhB20R)>g6? zG?~sc)rBBsl)dpzr??sG);O_@*13{du-1sqw~lGoNlfoISav&6z452PlhkkyfG1#K0-&a6;-vvaUu)qvP-Sk(P7mJ1hr@EvFj#OvUB2ELqlf za!g*dd)L;3zQDB^|M-t`7YBB*8~#-VA0VkN^d)52SEi;hl`d!(j=W^t*?T#C|BLr< z9%}0>qj$W>Zkc4_$W=4zkqjw;U4bvd0!?*YY4-+lirL>I{0oZhHl^gb%&KlSNhrku zn?U$Y<&5DG7vuQpCVFbykEU^TY<9V`jxThyyG|Zp{j}UE-p&8~>N>NR0gdw7DaD#;KP6I%8=k;b1{_M;v{ zSR&U+b!1_7ud}fjXD>DShk5esXz(L|A+N2O^1`(>tR`s>+j{BK+fCfM_Mbl)0vQ7J zUFb4nZBgLbj3oO?(6Rq9fmB(s+I^H{ar{a0wSEJgwN)jSsPeS>cYHydT=TG!geqQ+ zW4c+murYv^sEs0yENSFKv*2{QSwGe9hM7{K!$8oBe?DGfj5p~h?ap)hn`xG%*NiS6 z)9A(%0iy1)C#i};+vU7jaZM}ux44i0G1c~>|7$)1x}1^CTCg!yWpJK&lT;4i;||nS z5ktl4H0}p?6HN5CEovJoD0uA=l`cQQbeU8;peb~=4wfZo2_X_(85z;L#-OlQ)e&KV zhkSGUhlUF^BYbz@kACzw!sCZEk}+YwS? zenNv^Vp&;r7TUP)!(&KDMxnz)s|#-{pL1t5oEGNeCQvi}r4*e7fs;cG>HF3t4gjmR z-?0#_9eH-kkfqkV4jo*TxP#(+lxYpE zS1T3!{~QgsQ-YA#XiV{pc)PXnlQ>ZcTcY-l=0CR&51U0G$u`Gc7RR$E(nkg%Ov+w5 zMQ)ERp7MXvgttf@T{7ErMI6}?z{{QkD(fb%FYlfLbK7*UB1NxX+{FqDtGdU>`Z{mW z8}>?H{k0@D_7j{FisS@wHMT-y8U=x%4^OrKYal4m0^D*%)hpns9iWyL)zh^={{2NF zJoVXgzG2UWq_x*>BLR2wm>;@u^6lGVeLc{GxJ*cal2HRcGw;Rpx)MFjZn?!vXyJnD z<~ln%O@!<_cVBbxTJd{UoZI>Mm!AXO3S3g#C;&_e{MSO+EM~6o;L$ouJ|QLP5b4TaMy@Z$J-F8C0QYK4H_AfS@E@?tAA7< zh9B24yh{waKqaSR`?T7n>Qy%;17K?dT+iiF5kv!)b(ua4F;#jmv0Y78HgNPU;0Lx; z#cfjDU;aS(;f2Nh_h#bG{Lljq{a%X@W6BU2+jEyxKMvA5A}u& z&TXO2AyOVN0JrF?KPcXvbWCJhA)?kWzH{AE7-O;qL_A_}NpG76T;223KBWxDqdDv= zsAH@04KYXNJq_dYJXK77c3{Qw?tc3=10riZH}g!wZCo-MjoYKx?6EpMWA^g(*;Nm9 z(miis^rAY-s!RD%f^!kSq)p9!AH9IZ5*mv5(@TLQ5WT%jHh46`D3pA5%+1_Mixy@Yk5^$u);jq>n1(X<1g zv|0*FZ2-t&6a{io1GeiV$y25-pVew35fB=V>Y)Ex>i-VFyXX|=j)Ehu1mbLP3g z{$_4K`8`_yyq5C82d=j_0|QK&f^4~fZt-{U6By=Z;}(b`g(##m=#=DrVGWho<^5=& zN!rZW9qM1eGWEd}(T5ghjQHo3fBPkHv3Svq{GUXvRFF4%CW`j!RS9{3ssQZ4w!!7fqy&`3%o`(&S{3RdH=Hh_e=p|Q z2v}bEtd@Pt!hlbSoflw5Z)|waBzty7ARW=fe`1wZH?}~2OZcp7PHbzFU)=kHb&&3` zH#=@daX5HxLxSEfGjq~QkG6cE&hQG1I@`b==(Ia|sP_Ngo@calMULL-AWr>j-q7oEgWqp@J>G0{ zaCq#}o!oi=V1nm=VB)`BGY0xQt@0^2XiU|1;UH0J;kyIX0!!47X$Oh`FSKQ|U__nl zo_Ff=z5VI2Lvmc+1A&r_;KM1!(Ii{k#l$H6&rCE~b;2JFvoWQxgwxFg+&;sv7X2Fr zn*c}7X92ObbR)E$Y~8@N72iKJR5Qgk^QJlr9sV3RtNFC2QbdqC(<-aVcQaARXlcWh zYnzJ{JxI3Tqh+oX+aV`>7r$VHy3oNM;jUjHNsC+8odQAS zmELkREK^cXbZa3ZsH2Z}VWy}*lXWZOF@MMYC1cBuLENMih&=i1DsDL^sGF3v{1q3u zpL~o!mG?z??z1i29?_EJ)tRU+33WI#uYe%usLJY0VZue5Wx&f&8s-R{v|;@-d`zHL z8g1b}T@uxXow_F4pKBcL2WA?vl0u_o-QY`!$MClA-v?;FZgM;i;{*{JgtAYOd0jPd z^!{S4I}1~T%dQKk=e#94sB1Urk6ZW;Z3~M2cc%wXxQ=2Z=%IaV`G)0ChLGw(#c0FI zYu(*>t6t{a#_=d9X#C|wV{w)5cn|CHi5Pt%=l=FV|Pw(z@wf_)?PzibAeV3z{Ie;epKXCtyC zRcS(3m&EhvCJF9KlafV6S`0!{TjOQ9XX?JqN|RjNt5UUVoDVZ%_CN7D4;wRno#aqU z(Mtf!55z6hl_^#ckUV)le-u{Y+N>HOGr&Q8u5Z*~T?S80adLJh}V3yq&5 zQKY@u-i!9s7IDYpT0_RJC2}m#Rj@4bxRLLm%2li5MZ@{w@e9H0Bxv``)>x*hc~P(E z#Xx6-1cmI-L4nuRy)fk3mYr6r#wHF7FFJ2K0I=K*oX&2F+mE12_=r&u>GZ3|Q3Pzc zP)M1P z2Y{j{X!WQ?sx`OX7TkdHi31X03hD*5|A#!4ac8w*tH~gX&jr|eRD-#;SKWR~(v8#j z6y>Y6ftpT*ksdTLr-7a!X|-FOK@-$B@5)bh9K^+f^bZ$A!zB~cFzEs_D!qIUQD+k3 zt&DFFepA(csaisf45L&9x=?nu$-PhG2OvWT#Z( zNnNwkg5CqOS3$NfsDNCWGKs#F%ZSfwWS;sB@ZEt#lzl*Y{$x1^_74FdsH%77y5F(; z+*u5+m~pLdlramW+ISt|?_9I09c`YG0Ey*5+5{%7b0p0Vt#FSR-T3%}2-#DQ7askn zGIwaZE6>_V%!@KotRWszPfz6(C=PCh57v0X|G1&b>E*N7#8`JMRb6#x$iz3dZrm2u z!d!au=tR)SV7nSYmX-8Ic%Ydlk+q$SQ>n$1nNR=gFPEpe6q97 zrL1Upmrsvex`mC;ZYWMtt*w%uUVP($y(AB~>)WwC%V2 zy0CU=tgjMg_ssMk;ScctkmHa4Ax9uZOc@`z&`-Z%P1S79p_V>>T4RnFoO16Ct%Ro1|>_kPZd)qW|U1*7O= zgXfA z3aP&xFdN{?+$q*MJEW@P%6z|IA6^mOI5JhuZyCAH;o9>qc9{M@|5Hb=OpolYZAJKc zoc@s4b}FM&PtY4yuBdd=)D9;|Y8$ptg z4<3JXDRhq6!n##8B)1=u1gbJjn{>B-dj~CjQ*eETp*GI?v#6W=;9SHRZ%#z}p3%j@ z^0JWg#Q!s0^Bj-Nf$W7pH8cD^A%kzygWx?n&j-DX+lEKCRr$yjI5OoGdT&o>Vhb*g$wv}BHQXW&wCS3yRr>`(VGsrAWIu(w{M4jl!*>~9OuMz>Gv2B5 z*ssC5Sz+Rnr@!}eZl{zUwt)^cfYORj_&>S2IutPK0Xzo!o9buz-wd@uIGBxDv^#DR zYY%g{Vs6#BpN3IK zY%vZwEejI2kG52-hY;-un7#4l4Q_@HeTcjhCyO6ay%M8bT!89Ie?X%Ba z7kM;SciS<}wo?*qjAgoyZ&-aHMy}D&M5&(Yy}vW;Q0ZK5eQH=zzm|=>LO3ZQuPQnM zHrzH4zgF&<70O)19A4tkiiY;vZ|*Yg(VleH+#19wMdE+GYHNFSut-ZukRJQpHDlM_ z541e54_nNs;j=rCA z(6(A?&K^WQbOv1AVgKXWdC>~EfQo`2$XD?hW5uC1848vCK?rny0w>O^8^|ldgQF>qO zJ~NKq7>^Z&|0qSJMqY>x*VAWG2-X6$?>0^zN~fB>G9tvWbq2F~jByhSlR0Ic&9qll zqj^Ce-A21M^Qb&)z7yTK^=k?0gtpJqYs$_I5q~D8mFY)C_vRMnsCO6Ni>cj<5DrZI z``e+;k!(7IZV3c&JbPp)XFG={>2%hmESKl~@#^D_Ktsu%{`US(D9mSE>%Hq`afj32 z&eW!iw*V^DGHvD7n2o#@l`=5wQ9ImIyiZy?Szc3&=_j}0EUn9Q-BI_tFK4_Hp)pWn zzqK2Mw`Ns0IYRFx_IF&Z>h(3*J|`u7;&Iyp?dYuN3@>-kolVnv^)o#!@fshy4| zo~J>Ksi@C(u9gDA%@+f{I`n8Ng@`~G;9D6+?vSl6Zp&bQ(m%#k+ZZNW*b7mu7y2tQ z&hvXpbl}txBq6kW47@3P02uD^5|{u%$J3WC1$nu0>AhFTxpL1a|K@cro@%S@3O#HR zkn0Z3O4Dk zP6vwOe2y3rB9>+z9T%_mGGnqmj>&~E{jPQNfp)E(uv6`BltP<2rJa-`LE{pK zZPEm4N#q3Zk3DiU&X2wHMD~-&Sxe8ru$oaB4|SX&Jr2XbO;ZTce>g)SV$qAeSK+&( z^Z#;N(&s8`=`$sYw#BwZZ`n=O$C#cd?EKs2olN{Nug1XfAnzUQ{fCY^b2t%CxMCg1 zl*WA=)oX>9GOm6imATaJZJ{ShUD#QlxcL`u;`J1rpHN)aGDmUoIjx1_*2YJtfc??L zQl1Wi<>CFhHk^$h1x$nB)Sh5kX2PbZH)VlQxT1vOZ`@t8t_hGtokau4brW#<>ZdQ- z=d;y3sqxYNk>&Or>E3!yMZ%EnP+7;Oh{G%MBp1TE--C}Gj@8?L3nTD9%ErUax@D-S z2%9$=oWiW^3g*gs)emwc%)r_Hj8qyVE$d<&0F-{xc6VO~nsoc!%z&_{TCg zql*PDsmNE5n{{vWrrR&eT!<(HtivdihHs+g9Mo`|=b18dwyU#8wmw~9L9D%gD7bTP zpL{qsD0tqm?`7fQHAlk;y&#!f!_VH#M9wmKZSNco!+i$#=X?f3oIJi4^zFZxty*-I z$k{wFvgc-uUw&=G1jEbn_T*fY8!JKdey?b}yt3mT1(a6Pbt}LqR%p870+Eo3Y4?#K zjwxX9FqUh@3#^%;)nzaQ@_&}T@OIBP)=a?tFeAZXmq(;Vt6TaoJkUl)rdnBw{%vXk==z)jR z5NQ5i2m!!>2~y%-ep11oLbG^Qap&Vw<)ljYAabVt+sC#$G+u%{FB`W43u&^u-cT)|6 z55eLpvI2RT29-M{oGjaqZ7S^_h8KLwLHS(2tmute0^)|!K%Ce648hpqbDyQ1h0DvU z(RfKskg-3!-hR~(P-7OGy+mlkqG-|tb65@5t3R2C4Y0Q$abC&vi8AcIHk8=fJ@$wJ zosbwh8mjm=_wl_oz+*gGI}n%`bw?dsa`U7uEkaKnAFnt0QEB)FMabF2_zUgVbnqjv^P1`o#YlClM(O!^+&?+_904#TW_B>nK1N;$z zWp_MPx>U~j zUwfrq|LllB4?NJQZg(3HDyJuEwcl>NvBHQVd)rYcX5otu-OIINkrA@9u0N54)`Y4XFdmJChIyv)-hM@SExyoU>-oGa1((hwn0iheL#y(s^(y{bbfk#H zC)5UuoHlg|R@x6|oEH5C+(iORQNkvUm$O7hwi2d8cHy;~K9>$R^=Xg(1EmkQrMfLE z_)yfQBQR)iA)N(vGUlPVpz&Nj;YfelmD@<_3l`oiKoQwLwx17WA5konH$85rA%h3* zmLE|AaV>j4;-H6>;#4ki=`eDL*p3>Gds!iks0xx~!pXbv+{{)Y>#lR&Y2!_B^FFE_ z{{gmrGWB=`KAD){st`k4hG&ZEo&V@Y=s|?2=xhL#|+PJkF2Y4{rLq z9P6P5(@+yWN>l=dE}2#$##p|@q>p+aCsYxTY2FKNnY*g%P`QhUx*~&~l5spvC;`FJ zU=;ZA>@02>m>NLlfubvM&p&5MYYmasLHvIJ+CU}0I@d4#@~>R}`7i#9%j5P3#6SMW z{+MI2BUgX%>>dCA;l1y@eCWd;y1eE!uep5YOaFjBFV?ZjcaKl8v>@|heProgeq<>V z08c(E0qZCOXHOp5inrox!Y7mgErL|~OY=wUBTLWZf{ZK8wmy-C9Z6Yx-FE0#i1!_5 zyK_NicX~c;m4E#&efQ-@-u+XTZ~C^^UB2n5d!M%Ra2p1$lG7Jtlr!i5RW;&4#zt9N z>k!`|(-&kEyT|W2jCKN8NEx}?}6oZ1=9^;hjW?(4qs8!zAc_P1aD+@Jlkmp6UOxA?+LxA$0`6U_7W zm&q4AcK+;jpPS#W_H*_H%OCjghc7R%g_4)P#wT`NA^*z1W*l?v>S5jCo$H=7Wj}or{2a6Pq(4#a>aKW?ar~{WmEmm{}g?94L>S4 zY$4(4+#P`(UEy-h%;49Jd}YPY=K7)xP9_z7<&wGhk_V6(x-A%dc+LuJ<#6N9u5)!? zpxus65^T4lLxT4aN@))tzd02q7&;Grl7$Ox$9?Tz{i~P1|9#(g`HrWbxxD#X{(55l zXa`tNUKV8ZV*Unm-D47)n8>Rzp0OP5q4U;1>Su!vU*zTTm0$H$mmjqkoj>@FcU)d; z?Y*tX!%j~0_b}M@*TIJjy9SeN-C(L3QP_d0w?G4s`;NQ&Xj}(BV($ii(TnZ6 zHo{qhm5DrlThikwSo}OBpefNuDlr1gQ<6I!Eu_8rE573L_{&~)`G0@;mnX!^F~^B1 z6h;r(up>nV2g44WX;Eb11;_k%4)L&oqrc*w{fGbX^6!29*I%^Ir0rRIK7RFA{e?`{ zX3hv$kRjs_+BN7CpZNIYF;R{~qe&YSl|JL8Syy@G&{qn?XUwe7i_kX|Z*%5cm zM|O_?hwb=(h8=&!{GwOB^70G+^q*e-$@|`S`BQ)LPaZDFz`x4t7G%KtYXv$cxwj|J z@eUaj1FI};TVK$WJz7sc6ThgKVkopQ^X6Qb(apxZ^~|HTEBCv0 z@zgcoqd)$>8GiPDPiNo+#b{%bCsZ7E}-7{8q+txkDnj<%| z>)m9J20s!T@+dF#_|Xr!YJ+X%?Dd-*!5}*?tYw=u^tzAQ!c&ulA#PoJgeg|}XglKj z|Jb|Q7_E*fd`htkkwR^2pxaWx!nW+%lCpdjTlt0prGdZ-g`z?bMPrRAq143G_`|eD z<%dnA0z#D%APWmL@(otYH_B3W3+1Z_#U%(R5u|`?JkOkY?#z4l?k<(qw&{D5y)$#p zbDlXf_ujepo}D|dX^oQ@+Sg?vI;4$bjFsH*j(OTL1uJ!|!J#QsAkVKzq$5^F$OjP7jktmKw`5q0D9!; zGiK&;O9C?G|LkUl!AC25EDISaj)UV<9j)0H8Sq3z3~hI4K;|JI zkU8T5GStw77FLc+L^3s1$e?bHFUgpc8viIKj%ngI@l0_;cSdGO`*cO*N%WqOH+0-Efguk6umHBPfJ~YX zw8lXuNNnRxlVzlwB@@fsj2V3p9oWBLA_iM*k9u)@^8uNpi$2uM2N{qV{me{R_1bGv zwREx6{<}6Gkoi!L^!ryiu3rXZuIhg$Ibz`2&V9UPo$(0ZWqusw&o`3;+Vt3Dy7&dg zIMS?Ur#Ob&NQhzzS*(vS$YYE3$k*3?U}Ii5$yq++pq?pguu*R%&%;Lg z=Wk+|ra8a@9rfS>27Oo;3$1N2o-(9$k4CC}F$psS^Z{B4yM_@g%?Vv zcVJq?HPJd(ms>JCk+C&Bi(59{&ye|%24ueP12VK>f04`Tu{~wLHgx`w%i*HqPkW^J zV;h(PkNRUhbUgCN_w_SmR%t+HC}+s1!Z>ViJlNY7q@g#cV;W;1qp$Vr4Uy|>+Ka%% zfB_`+)dj9tXL%kG_5v8pzc?UslY1ez{)3)8OoJrO0cl&hy+KY*786k+4X7YP?q8QK z>8ElBrHOvB^gs#zNk-lnmut;Zj6~k#cKsweC!~dO5{DA|pLimf8piB$_h!M~r+vGk zB02HSJF;A7v^=geTf(NtOF_iN&&~|U%<};m*x)(%^y$CJUu$ZlRm+yrzJ2>Zr8#$< zfh=SEWI*Pkn<0aMjLwj`|9%&c*{L&RN_B>ebr!vM5Zm#(q{wx|4apiZ!=pug31OR9 zX5TTT9I0E>7a1B6gl`j_Oa(}JV1nL!jsr5}0iHO>vbRuUCN`*@Ig2dh12WmY`bm}M z12P|mdUde1r}U)hKglb~-I%b9&3V zJ-_D$2Ai|-g{*oEWqaCCMsC_61%`j=HDuEEq1jzBi zJ(v1Ixbis)s0$g-kHFDB?WB!IwvYJ&jw2ShgA}>=;hh?ecZ70$WBrs9AE(4nX4sSk ziwU@4uK`b>QJyrciz)9VuB?W9{*Li6pV@_9_d^+NCWfiu!an8;(#CW3@zJs_uSxLF zRO(#niRdYLi2_C zGFX~h3LKV8%I*SF&P0$K%Apu3+sXP-182t&$AHXA%#dL~##~9RAM%(uw}kdk!j#-h zVMm>(c8Syb_7DRc+Xj8z{%U~Bu?9WsjQt2&1M~bssYx_Lf&m#r@hY=Y?iFjY+&YL$ z*B0bix%xP=uxuO8#>AG2M37`04NHXOco>9H6U~Bl^o3BS)H9`=DeO$Aed&d9V7g7) zwsPj|S^2}pjncbUujCpsC|eoM2gpM}=2tpH<_WrH%K&&sczlB&mZJ|i$*E&NE-N6j zYs*&sWT~^l*+N#Q)+A6-ayt9*XW5FC?q{zi0WfxUmk@#s^#Knj}OB*oo6CtsT}SXzGO(+OXkIa709 z=D)B`_U}0^r;eYM+gi7jlCJG!^2{M}d*LlHwBCF}0h#mXE;#uJRvq;}-|73CGW!XJ&tpOQc_jShwKYl>rl!nN``Krl5PZ|rzfCFO_ZVj-- z*asOO7+8nXLGS+zqc}gX@djc31)RbuLN?jRO96PXF||^#eJ=qSo(-eNgIAQY=Gt z5#JsqCDPC*OBK2v!Ix(|Bj1}aL4N$LX^jSC_8vMUvzITIs>4U5M(g=}VWD&_?j$oN zPL!7Vd0WfBkGs}DLmtKZaxxSPqKUOi>FF1q*b<;${A7iMg)9AyIrWSQn=*U#(`y zZeXO1amIV6PJBo>$OVCJ^oglSIR+b_lOe11yP^I@)(VYT#`ckwGv$sL>X}k*clmM?bX$5WW&Y{a^&dS(yg>qh71}cJ-c?3rOTJOnFM3UjFG~^ zI}}OJoSECj60$E_Q6XElZI}HA4!DI*`#ka`88&#Z^dC0V){wyr8GN#Y`u+8T^*rXx zpD%kVE7gyqQe0dty&vi&BS(yoRyW_A)P)h?&x)y7wMvR~ro;HMGFiQDovhcF>uuY& z%ia32-A6xfo~rkkCQa;3l|H)u)G80@Ivop_ESAbQ-VpqvBij7%7kkTyp+j})DF1e= zHKIKmHgDGcSuGFi{Rp3Cm+NOC8#ir|efz51qRORR9+Yo?n4Ny_m>`;#sL}oWCa$s9X@Y8Uz$5t-=hj-W*h|6U-QCt`0b_bFW*w}q=79E+ALmHh|blAq6-&%?gei)+XG5E)ojgjuUIQ8o~n+3<9NM#fBQ>M_ndi?dAUZkIB@6zL6qsPC8 zUQ>ex^mpe012W;0r7%Ngw>~R1ZQ4}kY9H;|y<2~z;70j^J`2I$2X&p9WJ`@d_A5&* zWSE~JW8()~<Ee<%SD`jGud#eV?+0RR6E{RN2t06+jqL_t*SeFwM|MYeUJVgPg2 zF@TXVj5_9wIRWOZn6r*Lh7k;aIcFVn&Wd7o6m!ms*-Zr}fs|2B9X(!$z|!aro)g&te}+TezkmPm z(}-`vfTR18Wmx9KG$be*9TBeG5m1fPp(ImyIw3sC_%nnK;5|F?TvyWCwz~&D z|LB{r|HkKrm!5vRBr_iN^+Bg?8BW~)+A!Yu9l~SZo>}k@*y!AF@6FGKzGrL|ZolG* z@Zyti8E=v?^`(0Qc{r6YgS|pGFO<#pdG! z0Xp~X*dP-GVd};YDHTTb!}=T*I!`=R=!bDDPts4N9HTHUV$1^q%9)LatyMFHERHG! z-Nz0fW~9PTVkK&NP_IiRSpZ6dGwR~sdfTmGooC1VffD@OwnZM^Aq>oCv3aUI$mF4#W3<0AM0eP>nyW`*RQ|Ms-J!R zRp_zre&NM8-Vobzj=@Loxku=8hL8W>J@i1BsN;ks%r*lK57*pvS9tuAi^E^%oj1Jx z?z>^3&9?}hCz&LCb>F=u>^V2y7!E$`?C`_z;YOcm!U@BOkt3}<%ZxLI3l2FrthDH& zC7Fj`cp>!e*FSvx*=HsCG*e9#_TOqt^KAvl+;ip`C4Q^&pZ5UA*ol`}*3REwF~)e} zz=5;dG9MdJ^f40WJjV_WB|QIm|EO_mloOq9+c<_%mquoIViP4*wOqEuQ|=(t zBB9)_l8TZZamso`ZOmoAk<&hk>{RuLPAw`s#%q3D#QhUZalDb9R7&zuGC8J$tDvO) z-(0?N|H?n*R7XGTNoSNRA03b1(V%X*-FD&HYp)GQ3>Xml;(8~aKl=ES(0j)n!u=0C zU_uji#64ylciFb>x7{|Jed?)Uf(a%tK3o6uh!J7m0}c%5UwEM#%C_5YvrTyKgZIO| z_uU^JymxR|Zt11Ll{E*JYa#sh+izi)z4i(hU2;jL$%~tAym9#OqYuNQk3Aj+4ICKO zS!)gJT&e=x9=q*k__O}`&jz_QR$o2bc+E9o;)y3Nc*8#bEUdrrCgI7ao+@Z8a9^B! z{PF1b1B^cW>#xH^6HQcLuDb5Ju;<=;x8dJ;VG>;5)c41obW-SZXxsj+S>xY(@4Xkc zz_IlB6Hl0J`lV+cmj8L?nPF$}d0ak$HZ0q{dzfm3;%ge=)S`8){c#~+%nuWXpmGC`R#*l4r{Kyn$>BGf7H*v zBaS{M^gr?ll*;Ks94Eb@@A#8X4hJ84Sm?R-I$_WaeosTm-?RTGj{AQe9P4kr{dVYg z@FC%pGfofRefM3-DEsQTqmK@|@51L6?-lxJ3H$8En{N&i;{K~g&t71IZ(|fj4TExg z0viAAnZkvNt?6-i8?wP4D` zohGae$jmy&jN!wf!@>${FC6AuZ1%9!s@=jgGfZv)83uHC9Fn)`?kid#-UcgQC>K#~&9a z!b8EIyLL6(i}v0oybR1t(@!4;?7UN0cAvoZT?`a0xcQdh8?>wYVvB{|>#Q9XLD1*PS6&H+pMOF46u~I^&3Zh(C*|)kJh)ll*2$}T(^yhkVJ4B0DrIVN6BMV1*H94;>M9~E% zN?SQPFIRQ9Rj>0o;hpPG5l(eGqJvgZp#4X5-fLx8Sw81czS-;j8h^{}w)cPxe$45M>l_p9thDBu;lYO<3Jc<)?u=7T2}>YQ{1qPd??AwM@BQ`* zKmGJmI1)1a(SnJ#4m$LZaN^0Q*n{@ofBRclW3|=rF#B`3?20Qbs50JoahUcDt9=@H7fY5zm#u;V^&p+{Gm|^7)$n_0Se7kKNy2IUS!pKQGag7=el4v#^1ANV^I z0sWP6-*C_Ew?$Vw1O;Rm1YzJ{*PV9`yX~}7m=FQ|>u($w4(&g{u%{s~vS$q1kgz1K zjn6#$Y?uWEv+t;K5OU2zfx}u6ynY>)_f%|Nen~*cv}B zOgibL;n63b2nY4;oB4OU^&OQcAXD;h$IL%G8#Cqpu`gCwb+zyaeyEy%zWKu8I4)L3 zz~$?2zX``4e?mC#{PPi%nj{Q?|6RLuX$Z(rp8Ni;yYC4-*01#EWRoJu^JK~2!MEQ| z35bkFaN|e|ax8!!*-l5FEw#iF;g?@VhTHD?SJ)eYoiD!pGF*AtrD3y;HiCqoi9A;G zWcEQAeEX>7~o&Wk*_|wcYhapct6Q-PMiZJxO z_w9PV8qO6SZ`b3Va;~m(ggNGzBTNdvnP`iDw86h&pAHMdh7Akb?y_@uambKx1o~>7 zHP#5T&NiE!H^*5(=1>pF@JA*HQGUtgR|wCd|7PX>hs@GTVgJAXp#^a69()fFo_@*# zatg?hvoOwyqp|Onz`e&?_)+rM6Hg3-5O5iP{PDxnk3SX`oIm|YY5jrCjR-7Fh~v3O z&z`X%3dY3t#6e226sJZWjmD1d@GE{Ujb$Si0hzc%gdGKm0tX@;;osqhA0CEdhRoq{ zh79rTNz!EviULlnJfC0zc*L4iRarJ=8X5;k6_ zzXgYGcyU0OW4@Whs?FtJK11f%J+BGZoqJzcWSM!w`S<@F!I$w$vWM?Yj!j3HwnZnn3;uR&AbX6U&q zt_ZUrD5Jz+$BMh`8lHOjWg~t2;Qe9Z3DYl$_dWAWSZlApg`M%M<{A6%XPCNx4A1i= zw%sladGpQi_pP=2=HFwo~AZ@d|{Tenv@=b(d(zT0ufg^O^5v*0}Qghwwp zuQWU71Kgl3vgKCcdjunw|Lb4F;Iq=hMoD?AGS)p%ACQ4MyEV)1(s1*}n(o+Wu$Unj zZ|oW$%;UXW6dLV6A$o9!^6}4fp(fqQj#J84@)|!fJ<`QIPtu*!^7~QJ8!?f>tY~8+ zj{f;04Yi~seeJd*Q3)z1J?pGGTg^ms+bZX3rCRcNp5wfnQ&K;;-a44W%NRFjn|-$M!G|A(`4?HlW)@s|DIS0~<^!-N+F=mqEX*!o zdH?~0<(FB;%G^k=2=x9_Jh;w1|NLP%0zcOy7{39=QpbAx`}XS>jyvH5qun&{hR|cJ zwM@_Mf7?5pcfkc=T|7kIcU8Wvr0sqp;| zKZGZ7y;=<8IvwN#cVEnqVLA(T7>rS3KW(ttX5r>rZn4=l*IjJ0qgXu>EASRw> zG8I0eYylaZ1#!h`_72~F_fy#agiRY> z8=}l92Mn}e&X#+x5|&t{8%a%5T|j2VY4-`=e>*&!aLuk^l?^e-R}x=m+J2?uje7U{ zCxmHdnj(yC|5Q}Bs%`W@DrVaABAWH)^lM zhl1&G&RP{5`YkZI_eKZH;hkdfm2K1{c2p2wQqKLTYyM++xe z<++?51ZjtS#|Y{7fy5`Cm-RSJp6sAlJWwPJq&B8phTI&V^H>&7Iq|Ze`%n7s4aiW2 zDYuzZcn{7WOUntK_>&XJ?p5xau#!@qV^kJTGVRG-CP}7}dlMf?+EgX`ZGtLH*-`H7 zSj&z-+Am|&OSy#hU^D^&^9h^$`Gsa>PPk$8~qhX?8@V7no2&dq+BJKnQWcZN$CW1Nn z52}@b80vzlG%*ij%51etKa)n}2)mY5uW)6_BBSufspJOZ)zaV~!2|j&T15NB?@l zHfN7)j2|y5mQ!JrVfI12SJAAj8C&S3S|6GtB^h(Vz6! zd(qgo$^AVQ{Yn1vc$Mt2#~(*fp;x#Tfi=!FVj}OCUw^Rx4rd5WirLM0aR5mz$xHqk z6@x^ajrH!EZ&)z0u#WbL0p$J@Z7SM|ba2Ox`Khtcpw;7P3RVM0?^<^+~iUFC6 z5s>M<0hW*d;s0hfw$eNhtbPs}<`^Nn*9TL(^DAwx&GH0HCTG$n{!d)&&Uys+W zH^Yxgd`<6@_unsk(}-OJWbjHDnd4z>V8P-fGR{1zEcz?wk&SSS_my@jYA%Cq6lO%m z6{Kl6uBSs7aQFb5A)_^9BvP>>?FG3GXIA8C49FA$c>$PKKql{y_LtY|c361-?N7tO zXKo!f+-3O&K?PTu1!T7T+v;JzlTt*V3M9kw4KMT$bId<8i=}k=N%w?P4!YiE&|Le% zp|(!T8eI-F%z1@bS&DM}r49tc>hwAi9yW&Gu{ zSY)zUK&I=u>)HA&S08;;*Z?;?)$|qKp`HoT-kGWanYnTEIs!qV_ink__AxPa0hvoM zU3!-zj|z+Y`Oo2riyGHTpxo^U{A_^057(9X`u_XEFubnRwTJ)0`#OHj?TTs7)pS0l z5ATVGkXAsZ+B1JlJrBtI64wWyW{n%lNlRl#EM-r0dLR@omm^JylDfkr#&HRqY>1a~ z>Ttd^i06;fifFSp0OEJ&B?xgOD!Cu}Me&JYxZH47=%m6r+4 zG_YzZ$JFtKYhPZ~;M#K=)Kia2782h}<-OR5%kfsWYc!-_xS4FylKYc1IY0N@d7L40 z&9w~3_zW517C^9q>#y8`2ex(aD|$-O*8#7ntcvwp9t(%y)t3Q>91?cHL;VF8T@-rb z0q+v5^TDK}*B{3^gl>QFU+f>m!{u_>44I@q`EG%tYp)G|!^{%~h1BzE0XsD2V=j~oa+i(pnzKeRu`N}l*j5ujKNL4&(6tAu;+_&5V))s41$3aO`py-GISpJp*tKYoQ7 zG6*W1blU0W-(RDDWXea`y76ZCcNzS{zWf(vvK?XK4||-R!1+_kkJeo1!PV}e{#Rj-~Q-NCi!UNJj{*r$N*5KI( zv#*#KSXd8%neKSKXH~5IH|@03m<}C(3drPbGC^!E$1JREcG#f^D7tNIstd?ejZv3UQwL=yHc0q9C!|wrQQy2v+EwPRUUrH-F{SL6a6T5x&DTwl z0=~Aic&FW(zOt0R5+oS~WX7Lh{P6gm|ob)tJ+4d>K z=uu&URzPM<-8NSIpD_z&54>wWaPYlYyKO|FW71BY!ZHgl6!ynrcPqwOD`eIMWH^&% zs_rXeKaCDrl($fn6{f|z?tC>U56C?I+G}CC?RN-UuDw>c2-Ag?>H;!{oO4b%;nK@& zy_=hkPrn{ZhWSwZ>ep>U5AeT@o8GrR{4lJ68+k4c{N=ra?fQ`8S=Q{GRzRlOGk;8- z$AuR49r@!AQ0gOw#xPpZ7?Q?~Sc;k7zUj8QxnGj!_?qia7Si}Ao0dy35{vOUnktXV zNryaU1TnYi{g+wow{ny?I`huj?VMA#)8L^tdhsUs>WSCX?Y6 zB`#$80@lJ>5U=N?lsw!PuVr42*Fv?f%9?m+8+`A*VL-pb!lD20*E9?JQEWx5Df0+| zIL!hw($0JZ1O&K1>YuZEioW--(AD< z2l=OX~sp0pHRwRseQdHSz_$A<9z`YW$2#MO5BqJRuv zm6;TQ2996PVkXEUSQAay{P>(=`WbAt5Cbw3O^gqF!~j`gpp63kdkn7{G3EYUm-$B? zk30ph=pO?zeD!5Y_{a5dZn*lI;?*+Zi~Y|ElJ;L+K<4-Nr~6C2r=5AGt%>qkwpc!2 zFI#HKCG7ZQkdVpe2+0d0*@pe;YtbAXO8bR3j7Yj)9{TQkVJ^&$VR{#VyIE%TB`0Q# z0ht%=d?DY@QI|G&+$74fE3ND^WDt9Veg6j;)sw%$yH_zdSJ@8R$C zO|5iF5s+b3Oy(++PG$C&5p@9dM?gl4`pPI}*eRA`U`3+BD0fiWR&j}wN>tqn$fW-G zJ*5?<-aGv8{ZHYF#}Bqed8@MTy*VtbKmVa)6_A;v^F-m{uTCpXEtLMdu6`=??R`O* z@2_)&E1o9I?gB{Q~k=4#xu%NrUR}l zCX0>qm2=e`Cp*f?)gOwk)AX{26oJwr`wG}_W|!oZON^on$ne+eG((2m#w7jX$UoJU z%f9L~-WyXluf(q*%@SwC^!>*_!fLo7^yJp~j`0Lo5U=&hQnQ}>&*|S{-`|FrGF;E) z(HDn=p$KF!@e3c<9D7VyC;qBh7m#64XR4J}v3IKPKkuBd4BnrvCVSYxSDN_VbsmuU z$C+n^Q*rZIeVwSSfXtfk<>r&>$Kac>*3c$+DCvk_YQMpBdoK37($2evsquQ#CwJaa zb<65!*6Vwe&vo4mo+ADL6z#fv@Pp!dL1oEG%L9!8~;s#YpL3oZuNB@kfCPsg;7qr z^yD8i`_`K#`HuF`;oG)5cuP{DCh5?xY zmaYr5*cE{@@ z=U}l~E|$tKNY(i#`}{J~CU~8PNe|q7y+>=X$R7FSfXuxX=$LHU>Fk>IsLga(qH=s- zoff>BGs%=w8Abt_J>cKDnSa#X%*W}bfqxrC|L{r{zp~`7A6-VVBJp^oe_U^;2*@CS zH~G|4+pB1t!Lwd5gQpN{DAWaH*q`!u2lOXX-Y-sY3TMK&zo|v+gZm$N2n+Mx7ao4} zQCka!*LkLcF>B{=`=OUjdni5o}uqK{qD(#PXL|_hvfy&h*W&&%)$!Az~w7lP~H^d$~AEU+T=`@;WZMhJDj-8g2EH^PbnWKR(A=>LkGV6Pb=XZ|jg6NG$6U2z>6N8s zYrHUs^U=^R!}9Aa80J~Xry1rt^KvyH^To&Cgw<#FSE3&N^7PPoa-W)(%RK(CSHm7X zPBqePZyp_H?mBH4+y2S*w#seyKwUs4Kldr!=K0^Y`ZmXr+JKB1_wh`A=rh-HM0Dm} z7R4`I@>jXI%0*YeMZYA7FRkKJD>39Jj5nOR+8dB@iMF?{KeK}3TkpJW3!T+THU?sH z6#B6hkjXpbf28~hz6P*KpT6NO+{AZZeDUxvybH}#7my)+F1&X10luho;bDh{t+1v* zHSzP>^WnXIraU0?mo2sopJA%+`?uX1CdAE_Qe8miQoORXGhSI*0AGT77GH0uO9!2G zRyYM;7MTsNCjA?)D1DAEFwC;n+6MjdzQJKqd+tbke4@lBt!jrf zr5q=k^0wrJ@jN19#$Yy6RbIBL-U=r^W%F_!pBN_RZ^hVuK3({avst5R|KnII#lFDQZ9aUSd`Wcm(YA(+F(r)(iyki{+?5a`C9c46hhzF4XUMb_kV%qx$y65` znHNE}R4?TKw*oSy9%*-FuXWB0kKR2btiAct;mFIjYmhr|<8#B{LC=nLK<1FMdxs5n zTCPFwW{Vsd-hBD}u=Bw^!>;}P-M-al`A7I-*w_X4r>chxO+>#9ptu z`sqXM6WS#}rBV&ZkiE)``-iW;{4N}H>K0*>J-SyULOA=dTfzm$+!kh@ZMtyFzmKpB z#n|>wgYp)kX%Dm=knxeAeEDdqs)3P@Bl#F2IzM~eY2v9ue~ep9h)5SBCo#5oIRq^r z6K{%%lYF(zy4e9Bx5r8@Q{Ye_J^eX~vJA%19}CLZOS~SGp{%1E!+P<`65i8g65lG= zlzb7f2G<}7MZ6$xgU>fsXUA9|25XqCj(3igs&-Af6_8Q){Gs`Ucz3<;+5fc9Q{Q;p zu?@Pf#A1Kj4>&x`)1^yz>FTSD-z*@rCO$=dKi*GoHhZN%elg@{w3!r;dE@=}!e2Js zJgmD?_i$b1SJk?J%=5TmUuv6eZCri+$}2)Q_*G4-U=hbh@TtrVv39^!M;>A9nGlfq z3W1q__U#k4ue=979G`~$>sGyOMo{aOrD|{fG4(tk^DCxy=VO`1dBocPkKx^OvNW$~0thJ5{zCq8reE52knGed0KVG(x2~>sxOX7 zNR#I#CUI0(UZ*F=im!mo5qQ|-D@(*|gfF`Alek;pi$7Py*K({)U^0Ig!H$*iwIaS2 z^3wCqg?ZQW4y;_irEhe z$Ph6Ufhc}8ib>ClpM3gh=!T#OzxebJUdvhzL7%(vdhMFM{GR^-d|rA5d?6@L^mS7_ z1m6pM9|RioMJGLQrv-4+c}AIhh71ETr1iqE;2!gcd7TqBwE z$_o8kEAwx8_$N67VzWu~k28X~U@-agkBc6c0x~iLz{Qtd9(LT#XUHfZ!*&B zUrdgM&m}l9f|$lFYbS?i=;JV6p+~#0n-Z+I8~BQ z`mDTW1&Ocx#u$*{8Zvp3UDBj0FI!td&=0gt12VZHDUd?)^j8~@0karrO~@rBPkni+ zvz7B3$`EJ;WEy&>Yc-iJvGfTgiBAjJDk?{2Gi4QQQx-K zkfGo-(@hyJee^(^Z6b#*IsRYa^n(Y6De?Ny)z2IpW|(CvW69q1GfxvPxNqMu%be3o zBn;T`vT*aIkA#_LpDx_?)=>q$S)ncS+RVAR05&2d7%5H?D;P0?6V9cx5aK!&d@DF7mk z9JBmua=7i|FLV5hOvwvd%{8;7phrQ;Ne}s?r9VfIZ^0M@Gt4T3LH0)p$1|6mTjQd> z>&Ji$dDWELBu~DSab+s^&D(Fk6XwODf=WIk25ZFN)uhxT&MixmhF)w1WYl$kXs!SU zzoPUaex;fXUxmn1pR>*mCu6OGp7cmTy zG?rB-Pk-boiLYEXU6)h!K*_O-<38izYC|(Ur02SXV=h}cjb-6kCWif0!lLhK*N6Tw z2V|(HtyIztVAX!fQ*($LYrOS?bW2YzU*)mh3@17<(!o4$yVeL|`* z28fud2RTMQC>Kw8S&wQduM5avEExG|#7}`=54sPpL2QOk zV_$jkMewUWLs zIcDozdNIBZ1^=1yfDG4>xd)5ccAT)I1!P8!92vI3`eHX=f#YQnhsi zJ%;^oIUdj%pi;W}h8x0`co62p@k_Wa&C#WcvFZXcWN3d~g!Rc3@K8YJp@$y{tKizi zS7oloBEcKtwHhY2_oY`}39IAbUF)ah59Dq6cMbg8qQ*ZhI;?<y01`+awp!v!0t+(}0hQ9*b+?ZXG2V}IUFWX~l48{Ju3_qT5=?B{9v3D)} z-3mX1us>hMr`Wr8?E+2+Gh)^i_ve+E<+M4z8l^-;fm7>HNF z4>S)v_+U8d@WVr&{r4xtl4d9=AVd3)Jo<h#4Sg@;wwwK(Kjxcl86LV zWxy4L(4K%ydDqH9%B52j*;2hBhd?VJ)6gsJCiL6>(s1kLkJ?GR;No*);oHf=%g?+U zhJEy9SbW7lho>KWElk{L!tmI)XBIl^0v=*&QD4>>kYQSQnK`lI`HbPECx?co9(dJg z{mxH(P@{WHB)+o588X#?41KbH^ug}NVL$Vs@Fn`# z;GF1kpb!ut<%FicCHa32xN}?`^Is-BzxP}ZaI91bxw_(1(JH|A<1zVd< z*EU*8aVzdtT#CCFcMtBa#VIbu9fG@CumVMk6?bMC@4`p1oBOD(WjwDQLbss_g<uU6Pm#HVC(7=hI8)xq@pplEa%a|8430{NFE%}(53ft00c^C zjw!M!9`UEEr%8Gft36oneKX5;pYrg3QfHF~uV8|QN1>DFuKT1v9=kehdJ|%$&WK;U z`)iZzE3${*f4QFICH#l0gRG@27!mk2>pUOtbVM}5>cFQ!VdbvaeKeO^Xb05H!)a!2zc-MQ zg(JQj*}2zw7fKi+R4wui}(0sDtVLGiY!GVi9^ zmr*XBu(x&ze}o(0T$O<9aX*aAspz_hEJV?jNk(Bp8%qH5uqe5vT5@eb0W; zb5CL8gV8Dz(l!DOrZ8!&SIVR2)_5KV-Ws+HSlGta_S_y}Qa>^Qc`;4T256CWcgA!# zu$=kaDR0(OwWbh@3)CZ1kDETOwI5`-){P? zxsHmrA(Jy)Aym5eU7QyhwF18WLK3(H!x5KwaaylUd!P>vK~?uC)9{jyO?|D$kYsy{ z-qc{I9&PIe&IRp56se|3_%nCf-_TVQ8B=K=UMk$N?}QyJE{>#hDUeX`dGSSfzAN_8 zQ4JJDk#4QwAO4y(8~Vl^>S)!`>uR23?V0zP+pf;Drj|;chP%izonG%#EZs8O!_W=T z@R`02t)jZuEllm=Kr#raz|BSg7mHZrHJu*fie4k3L8kcJ~9xF~#<`k2tcUM}B0K zThKAnn+X5YaScou?0??u(oV1vA_BRsgumP3KZ}goNQB@s-le?fVxfNbBk;nxzX$oB zM~Y6lAm;=`T14dD!2k1x7F5g^0jpD(O79BY#u*#~S3NL`CEDCFX{W?ZB>*4MY@GG8 z5Uj*YC53LQNf8_MfQxN3+Tb`r{lJ5~BG%J6z%Kt9u13YpSTd;pWrMnXd9s%R@OC+_ zx$`itNs+ue7Po-Zpl$s1^xU=Z5RQ0#|I{oBCQ1y7<7OK|#I$9U443({Ljx%m+PE^P6;K@X^l zJ+_!E+Cnr9n}-yK#B(<822N4`-M}6G@fKzKWE%I_!)GPGiq!1-ZzxAxD3f?q@Z-+` zY(`69p-xQhb2{Dril-h%wSbr)cj1Twj1gbIc2CI28w^1g_s?&jPQ5M%)nkKEgz0;3 z^e~H;bu$!2xt8$}S%aXr{2$EddC42Z-@dF`#L)CRc}x-^cDwofx;cc)ww29uWZC5+G8{vg&*)_ zApnram{KpB@~hO*cK2mlW%nBd#%@zRp*DZ27(PQ=R~t8SoX`I$J=U+b-!}qN^4!h^ zmk!DVn97EP6($t8H;wfh=~=l62Hg4WouwCT{;M<%Bo-THFAtcqUuUWq)!*G`@9m3E zJ#r(b#LgI*BJ+Bgi{V#jpyIej|2i)BCv!vQvgVJZ$IQ~=&gLt_Ws>vasVc)B;YJuF zq!Hn3GW&4JVBO8g-Mf+BpQvzQqoA+59GEC^bIixN0NV4j@u?rZBN%Ix!qVmAot*hpgP6zHo~I9UAnMs1`LP;I10Q7-oooS<}*+KC=(#^+7#GVluY@#22xsz z94m-rDr+Af#7B%=qBu?0>!YKMy5C9qVsH6D?>TV)5Kn%E;=w&++vcH&57&~>4`f>T z@Nj6v($#JB!!6$oV-k9)zEDoq?(v$=q4%f4}B5*66Ki+M_6e zclGc-Cm|SN849qj(+9G(0B$vU6EGWkUj2Oj4H2?dJ!B#fAP4_u#jTtAWCUn1VCZx$ zS-_kqTX-{dP3(&C2tD7cW$nJA@45jCu5j;F?svRp%BaUbiJyO=w$}HWfr56r@5Mh> z4AtfhacWg~3aU4hBStytgj9IpG@k0Ce2T9vdce7dR?Fd_piMnoyu*;>&9YWRB z3zEqUea1ZRIJHoxcK~D<4UB4)SN+QWXq#vM{FYow<^zHw&bjm0YJpYDgDk-`?0#dO5IO;^y9!>0CyT>NhT?zOY1RBS90q)!pd`qNAU2vl z?VyU1FvF*Z<48pvGn)mLPe`y1=SaMFzES%3C8P;NZxzXnE{MV)98oCa2t>?eT#3jO zJxRF`)O^mlUW#-)ZmCNRCJ*?o7AGD+!xou6CB6kf-Dm-HhU4Ed+33#qB>|_O|N4k5e z;fxwkR}t-dZCX+pCgC9DB}ynvJOKFDu;?DCcmiG2-BZ%(_&a`3=^2gLQl18$xI-za zeVDA48i}~>{){36s$6`Ur&6{2*OPv(HU2h!wd9@FQC$eq2~Ra0N<^T!?*;XO*EkpLcsKZ(xW|(Re4O`-bb)CrlwLo& z!Crmrx{<_N9InHwcE!cu?~~!LkaRi(s}H(tOb0yEv^^`#7MTe_UAOeH!Y%JRn3aWL?e?rAykzAVJ=^SsnRB|ajwoFK6w1Fls^pFA=yJSc z$^K3S^)2^3*IUA?{k-#=?HEkXa`rI+Z3*(}f^}dJu7QMS%+2e-K|+@3gcVUn+paJ; z*6E0le2n8;M_x(eWEU$DkbNCsll$?Un_`bu4z+>wgBEfK!R^)$y~Vcp`(?aCjbtQ2 zl~(CERn9OWMd{dgmXpT__bx0{+{5(_a5*=9Hb|6YUpp^ee{e06v|le?ooeXr8{eN} ziT9hB`yF&T4%Bi_&PprAiL{E&vQm{cN?79D9v#@+F&?$$;(s9OJYQsoNXj?3Cukpr zvG7zuI$~nulR(4n-yd8LOSAFUi595H@h>}po*L7RHnBsaWar~wTwjstuK)im0JT6; zNZ;Q_YB51yMITVSqK5>l<4wJkwtaoGa(Y0*$t1{MLSfHiCl*l-q3fAM{5!~Q`i@l- zlQr7pN2tic1x6v28~B^91f-u!GE4kI3ZM3v+_d)Dg--ZlA-vX~l+w*)FB z{z)Tqznf4bjv6Zz7F_;-2Q_oAS{9hIDQSa+(suV2=R{GChM6ikBk{9#NOIj~!Sn1Z z%Q7myl;oUJN<^iNT78SJL?olU96~31s@QGtZf?*+b)NX5Mb_B^#M=)`z0fG5ep5Dv z9WjK&lnX~Cn{5sZTrig8i^iM&P+B`axh8u&o01Do`WqOoFxNUE*W}&UWaa4&C1;+RbxXsg2Hbw` zI5Rzl2{M4|rPsEen{(u=b03L(u0NO0m}H)11Fs-bq>mHMo_!@^0#|+g!Q?+NCF%s2 z@-CRiV_^bcD`~+57GCcR9<-DzGDo3W200okZc7%b)XwUya+jWGYNzH&$S!;>gloUB zqD7x8Rb`E-3%VEep9vRZ=MVRs;aP{l>Ra0I{bMdMbO&~Ax!kBV2R-#F{&%`6#2O(X z`#?lQOqI@>q<)KoPM%j7^;R6N^wdYnEAwJMs+x1^M67FBiEbJ4Irg2M89Ps?f-%`c zDblbaSCOoxGzZl+T2S|Pasw+IxE);-@u`Uv0CJ%C@yW$QY+~%aMncRpOx4zCYLIq! z-)G5=fg*Yj{q_0b7YD&Fu%mXVtB81#E?mKot#LM6e~ltEvv|FsJAzCL>lwyRLiPHQ zFcb0lrt!iQH8=<-coX8xG*X876SUtQi8DX0Tq&zKOVHoTc#9A+h`tk|dvt8Ir~C%? z-a%(#ibkmE7q@qJ-^UQ_W|7_kjN)mg0J-h}#{l{9HF%aMPxWZf~7ae zTIjkv3WbymSJYy-UCD_piS{{S1hRc>HQ=kc36||-$cySpI!J9`-ORO z$b^$DrYY%LTsVXvWCRa7cmzxLF_-FeQrupY%!No#BJ^SFH%-cDaYTHQCP7r5U^#a~ zZ)579xac^6L<;pS=BOAT8f=8sme?rSs^LE= zKt4*t^|4tu!78&khoi z$&A)^#EB&t2LU)I9Q{Xn1HTzCEAoZ*0+k+?a;QrS9D=9cjYFP{2r0Us$}dTmlguql zr{}#Qe>@JyMT#cgs13E~?!Q%ppl>|O-K+UOLEcak!_2o>lq?;3lMq&$)QO2a&5JN? z(URG4z_1A-`LIVJ(NIy6If;hvMjCZG!ASgI+@PH4kgRlXRY7p!yNDWyi7-j;760b% zcMXjSwdZdvV*FzvlZX$oApxAk&9)Lnz!IEDPugl7)tocthJk~wE>_kJu`j1AN@s~W zBoh?vgoP6#4~0kJM-Z{z`EYXKNZ(FRl)%>GE!5#EtyeU`nVq|?rnJH*5-Il_BM%C> zI)Oup-dzjj0ad-Fk;3J!qP0BD~tqxqhfH5=`p8` z>0BpBb7|_t3t@bCYsWgky9Sbv;4sT0a^xe37;v;?x7ZAU@?3!pS29VWC$1FYLyhp7 zhry6%DJzlbCfTg6jah8#G7B1i1Zo0s=7Z5iDVCUUAQM5&3qgY|vm56YJ_K~wp?A~5 zU0@MWEms!=7Hz?IzXnVQ%rCXOzvldHln4$_$-e{$WUMX zQf00DjO;i|Gj+ovF4q1QP*{y}gNl4A53V|V%6yDTMn6>s^yZ!DrN>|=cAPR7k$il;rT|x(cLjiSo+=lUal>Y=j{x)YPI{}bDp9_qD*Yl_p^PVAFG%e+G zw5P=U*0(^-x>xAMnO5RwSZyJ4zaXD_R>=uhuFdq${*Q-ur(GI*7wYKE0P~qm)INo# z0^M^nM9ove{bexor=$Q>Sszt40TMeNr};I^=S71u6;f(st|oai#!;{n8G=rDYb_Bu zb~b_X4vlkmGm#siSUEJtdL$*atunsm$~_RAopjlPpKR{(b3S+-{=(e6r%mjkPmJl8 zfLQRB)7;+MqNG_D)5ZVjm5@#)hyV78*5M z8kG)<&vJ#`{m~u;4#l!oASps=G!CC9q?Le!M|#99NQ|6S@Td=!U?uza66kM(G;F|V zx;6c)c`)o}z<5 zPt(ARDi-OT{sXsIoaob8vrLAPV&#(FE@ofh!U8;9gvg=#-Gp1bc(pWn4<$I)P2KH7 zHA;MJ-T=D3)a%9f{%0JO0&wb(ZA5pdr0C$+t?epZ6TJ|>Nz$G&g&-mgE5J^Ws98z zMKWhpsS!jXrJ77D<;tNoM`Lu9^6+6Nhg$1Kxn}kx)D46LqZ9eO^;o*JMc;76c>(Yi zC1sj46nM|#wHI^63({JVK~cAHSa5DdlNXG|vhW~#9{Ez<(m{xP2AUw|poS`c-Hk#J zY_M8%%FMgO%h0a)B8-G211X!A_Q?2TdYBcxu?Xb8GUR*}V+~199R{2D>TfX13fz5x z`F>0s4(cd~+WbO}TzIJ#Y7t111jOv(|Fr)J4v3D?wqhkD67jVQ__>Lg+KEkfT<$dg zhk`Cf`xAqx7@&c+UM=W{4iIl3nl6!))!+}{H?KQ=%wUSVEa&cKc$adPw!2cj;BnGF z=>vougjein@e_PCLAr7|-?@hOGWr%1)emY{({zuuSz#g2;_K&@<(U!G91e2Z) zKef!hYF!EI@f}I4x`o%R7NC-If7pwDincj}r(6>+2HEVB`h0rxN1?$OVAUaBDnlIo zr1FwZB0+LT<$Gs-<5_YPAdaahvrsvJV3;@dg!e05E@PVWV2YZ<>&T1L%ObV#EwB;V zcc`E@7R%6iveXsZaOLrK(C|##ntaq~psh%zmEF8q`STS6Zab(5?24zpl(t%-+=%ZM z>Lt*YlBA3)#7OW>xQ7c6(p8n`zX0#zD33IC`_lq7)JO^G#Xbx)dgIMxZp- zM6{Ma=kWbFlqt3g(|*`C1ZtQTeY(6yv??IuR~=AL6>mh|np;u5{T|cI zt$N|YK7m*cyV)qiUb;JZ8UA^5llni~`~XRx>PAu3hV>Mq#D)QIz&LVg9*=q2QG1?j zrzV@4L86DrSq%LP1Jo$d5A>3`zZ#6NL$o>=oj_i=2GP2U&e1BVx&bOuf|0KCfi+OgSq=^(}S&?fAye8(_^;;-37W zdGq3a*Ltu8J2{l}hL++bLZTvmNQBDL7?t>Bc}T*Q7nB_gMw235z>bh%R}K0ol(d67 zI{44t8uoEl`h|q4Ux&o=mw(Nh6;K~>R6}xo7O0(>E9SNx{gY0Y;7nFS;u-f5-|;y$ ze~*v^QXKjT=NgS`q_L?Dp*Q*Zds6=DFd7iSb8nTjxLldr>Si40z+(&W>*~{pdhv-S zTi(g}l!vI36pXRoE^yp65WRfU#kzd1L$tcO&fPxT|KJOKkP3R`NRg#?KhdnOOMH8> zx185=PLl1HR->`2XMvms;)qK^G;;mUF_4&0Kxk8k6G8lgWuKyl(y=>_<ZnL>dtHuBVp78vOkZ$nS~Z~1E7v#Y%rsAtIX0*dqQPqhrr=k_w( z%JcIXoI#}^aqJ7wvB+12ZLcKLduas-8FkJ=uD8~>fTSJ+%RA2d?_H^Y= z>6afOAS#g;KzM3)$rMR4FNwdzFAdeS&Jg7aSlS~~m(Th6{!UzK(BXf3n0hZ?vk7~b zDv^9%VcrBsBdLm$K(YDz*{3YkXi#;kO7IpH{%L&|?yl+c) z%zlo?nyewflQ`wHzE!a+uOM^lqDgGo9sP7(YlQpf2lTAjYU#1E%AI&8ORt2go+oDdo(COY&BuTK_dS>A zWsFai;YgOZCQo&xIjskmxJJfN!r2E$XCBSx-4`xgIkU}m<+VvzAM3K&LjVnjfPi$< zy3E^jN9Xh6@EaT{EM;eebx@B6t$47q7=?NN;R{PHc#hwkk%8hGwrCHs31wI{xkTws zZrtXDC2SU(4lR-?oaA8gogtbt-fQhE+69oOUn|8<8S!#D1rUBniY6*)WvB3Jz{ z&r4?;_P5s{34mc9h~ZXz=;{~prAE4ynvhBJmnUNV0k<&BkIGnl;28F;ahW4+t#o}0 z(iroy0$)x`j!+3Yny~}I1e-bzyEwM5tLz;yQN3=nE$)@#m zg<}(RG5d>qD3>S_t%!zbf3C|iK0j4Il7vDp1e7tdXkR}<)Le&P-V+#n)D z5{y>s77UhI3dz>U3O%lx&W?Txi7VjhbT9(bdt)wg24}o{)>8JAw##!|%6m7x{|qJf z*S?`@YAaO@1fD*JD+cVm?J<61TA|Ky&-rND(A>UuGtl;1LXe49z6A#I>LJxRXzC3LQ&0z+@194tXKg!@U_)&vuPZ{d zc-r9zId2vDcqtw??$oibegNnju5+RIpwhNGG@TpOSWgPE@JHcCQyCVUQRz32##qK0 zjIsAz|H{5qlOo9w5;5}~^u&$Mkk_J(5{c&krN)P}*pdcqy9eXYWpZj`Rbcg4f#d(H z$Nu!fG*zobjZ(fXwlE?c=2Za})h2w_Knfn?AzzIi$>Vt`)*A&AVz$n}bCoU_a1cqU z6vnG(jYTUYu{l|RXF=y8R8qWxd|DG0DIg-w%?7)U%Mr9jW_~)&rX#;Vr2rNxEl;ig zRiO1qFyzZ8^7{EJAN`MF63cMv1O?BAgEVF?YCVVCQ@s`AR`tN5(UGH@{f4Tno#Uy#?I?ZZ zna;%Gd$7kzs3Mq7MET)Rhws;FTLwe#pe38{L4+ACvLkazkH`X+eE-1_7T_=Y*inyQ z%MbD|^Fw*T^XBcVJN<)9I2v3naslTqEj`(FHlEBGY{Uwyp08Du;4l61;~vMLM}Ta( z0kIawiXRZ_ZpKQ9#w$#kh3K{bHT7gPR;qh zSK?3KhQsya@-5zP*O(0I^ge(Hhw+k&i*A~g2e_s?fonPlzM95p2>f;ornqBXu+YS;cn%mus>vmeg02p*7$ugNN2e*ap4 z2cJV23rZu(A+>jt{#vNM9iHOh$R`!)S>E)s>Lfia756jkN5v(;$w8%is5Qfi-mGjw zKYI39?RgzB*Gc35E}~M{WkBp_oVUT~$f~|wpmT85pOg>)ki~jGuP=Z~HDbKB6!zY2 zu~>WzrLt?BAyMRoPHgyr(TbSg?6F-Xtx_9j=$g?Xoh&tzF2zy?w|;Vf{ZsuR?p-LXyD@;U$|5vQ2+Yemo?!j}-yR;SJ>eQLuIJO~y_ZD7r zSTAFs#YV=|5AKY8eb0HckbIV;AGJZFDwGF&2ITuDYUuMBAtj<(KD-#Ru@t)JjMuZ+ zr>Sa$;~L+#UV)rw_^9>vC~toyfSn~f5TQl;JJqYG$RTAinr!-+*h`KgV#M+0SF@o% ziDI#Mqnhdd zqw~thuRc{V;L>=%iq-0&H#*~Y0pS0Bp7ow<3LR8zxi>NLUhcPE3g(?tD z0z!hBu9&9pS350~tA6STQ1<2>hbE(b1Rz+o-vJ8Bq<-$vlgf_mQ=U&C9rR~{IU^r` zkg)&lYR7wj{2q{8_MrpG-EO1Q%Bh2`Nz7gjR|O!@i>8Zb(adOZEWtO%g$YeZOl z9B@>yq)xi~1`|eW+O{ads>momDp({#By0u%vB_6yFJ(rLi55`en{+Fvqm0xtDZ$X~ z8Aq+{XPWbAvA~M)ZAxcZlE9I_gsdm&J+wgmXW=Pnu6sV#n0Kk{J^TtK3y+w3hTJVJ z``c1~z?oXrVL9y%84cz}+NJlB>TUa*=Zk_s=GU{NaI=^Zw!q4+JN1si9EMC~Lzc#U z&mXG|lw+TiDhyE-{iHnuecuFS0%lT|7YPNYo#B)|F~h8zU9k6wJvLcdLus!Om`UoO z|5lA*-F3Rq3NqE3OF#0CpoW?91P_#KA6`>mI8R+&q9?qBd0`V0CpY@nL_fYWBIKA# zPWjn&)p;{2C!S-kmFAcCQ^CaRV5t?{?l;w_A`)azUbGoy)Z;|6UnKvg!j?MoQKpxU zl8Y@Cc4hEAw9LALmh*F~=5@)4t=$%Jmi2+7p#XH>^WVG!i*p{Q{1zHJi9Xi$=M<+Q zWWB(gmi*U6C|GEI8J_Eh4U&1hlYTa8m21E=b){52Y^WxM9;~ffx8%uIqvO@{9<5%m z4sfD}F{zqI|F=0I`x9Bq$#XvH&N|JUq2G!+lz+jj0Ur~r5l80>^{hyx-H8i`2t(Mt z%|Aq2R=yAu<_ZLX493vB<|(15lgL4a&u6^7P1>QT^T|i;{ivIiX}c z^?asy=?F0PKd}R*kRNzYg@#O+#hJ-R<%H+}Y!wQ|RiaFP`)uvK&O8;ih@=>=T4m#l zzDXw5AfTU^BAN@*{|1xZLXcw0u`Tng1Q;E&Ap>&flfp0vZ&9KSjMClai zae=Ifl7);xcCq^r*y5#daXPT1ybnvK&c|r{O zVX%W7p%wnF{8)bY9VJ)!E>X4qB@`b;kq~BybF*~=>+5FgfqB~9qr0#SEUg>a{{xYH zem~K`b2)Xfl?;t03O98^O3fqV5&3S-=%sbY4J`}T5*{Qc!+$_n$i8v}jH<$=3j~du zOj^tNxsxKsple}CQY5D^ADbEb4LTP5N;^&vdqNm$(Z1DoUR8NlZ1y)Y_NfFG8nwOu zS+xe~HTWN_)JGah0VBBPWi{i3`5oi^97hi{ox?_nKVYmNA1XQvrvg?D zAjnW;%>N&7vuQ^loU6!eCjF)q{p$i|@s(5uI9+#fNvBQ@gozLhuqyi2{{nfHI>Ampg-{D!`{UT9Hl? zIWHq~NVnkH$heHEG_V0K;SE;Cr=?OHdl}OuM2yB+5Wn9_s$VYACUr?_F`g z=;zGaG)L^AlTot)$4T?JzGI2f}$JN&Wj_L?qb|ST<4ttWa_*@cO)NQ z+nRBH;luCypbtOjqu__^i{s;eZ@h)tHqeZ5?YggLjUU?Ksw+^2z^aARM28eZSo7_R z<(j7=^H3`Dj44?H(~=r@n395x(raPfU;k>rR~Q_%08a*=gJdIg6DKE7G8NMr9y2{W zd@Zt1>^kDd*I5?XNj%am^Ts?0)NZPH1FacR-ItMZ+y2EFBzp>*X5bWhmUoe#1aEWM zziB`zv6G9tH+9w~7T>Pod>x9btT9SJ5w&Q=B1b3}zS#qFpVX=k^A4YBhr~*C7V>s_ zvA^SfAxJ09W+h0DeYMRNh9eVw)V(*U4{Iv;grA+?t6lqK!Tm|8Kl9p%VFDJWtHpeu z?geEq0uXX4@3DMizFn$ZbgY)Hy)3^*(nhp;NH6zV9{#WMFMYW~{}gH$d8Ae!^}4_S zwT2_M2$UNn!=c!v9w|O1G?|wkvTbOD4Q9sP35S!r_mbTlmr$*Ju&HIP$CNTdL4qYA zr`|jL9{?b*{F8x7OXz50K_niH6AODvbHT-YT@F*hSqj8iHv)Mi` zBw0?fI0^(gY9d;oG4xCw~ zBwNbknELMVYnH@*Qqntd3(yC-S>e^ITSSy8l!tqu-o{oM360WhdyMd+ji$XQ33(W^ z9^L6xs$@>J0o5p~wQmcB))j%cvO%DNE3Oeow$J1lBd4^KljAt`3VO%sz1dlfKaoy7 zKT?ST*@@wsv%A2cJC%ZAMn@_+opU9z${N;!XWT94bjL*dvr;m^(vcrYivs<9);XbZ z$BjPQCw10!)(~Vy1nE8=yMxdszdR-5M)6qDWdHoT+^YT!TBg-XwDoUufrS}&}wkRXS^zn zk%qs3-AxcUg4Y*!)StorV|^%))2fgt#UmGppVTpq>I3#U@)e$F2bcbB8uN2dT0WR- zzZ|z_zX*nuX%Q_04s|oyBbBkbnxq4aB_eNNYMP^@o3aq}-_L|~hEYXw2p=|pdAP=K z*#p?g*lK!*{RblKmT_+c4`T=xf+I(M@fbF+=V!tGvnH%T38k=8gN6G{%{~-C=tkOx z7SOYU+%!zh=}aqUeB8{es3N$?SmdzJbjb;vY8N5~txotp)~^?dEkxv5LsSBbX^f+z zVemF_@|tV67mcJQ4i$6xaEb^d+!Jk1EIL9K??x%%#~xE9-%?<$Xn_?L1*i zsU0)fI})*1GlH^rlS}~9KHlcUq@uU1C$xqaYropkasNOBp~ z1h?dmjHs#_g+T9SW%%+~+;aQ{y%qIw5mygMKdq92E;GumK!aN@jkfB!WRmS*#S1(7 zZTk1m|FCZ%YNAr@uY=clhHCX(T2A0W+ZLl#Bf#)+PhG0rwEbdQ*SPO-P+e^ykw)4? zu8|4PVSh}+rmud23;+$u$UrC-2_otCo4Qk$Afu`Dsa zF;e;YtYwd6aYXv<05(EJjUueTh8wdiJw(|-z#Ngs%RaTkJ^;}wXxwHL9H2-SrQ^uTY`Mb`%sM(5EmB*G!1C_mQS;of3vIzMVi1Y7wwdiS{9*9uBVzk?t{s%_`emmfEM{~WO7;<)*<{+<^x2guPb>3n+slj5<%T&L@U@i{KTNo!t90Lpgg;*Gw z3A;9VI0%(u=E409;A-MTvQF`>kw*E=E%;Z2|#$(cwZm?2CJ1h9$zP80CHfq&Nns#y?XhwDa`*Lj0NR; zr7;bqdGt(J1KF;ceqL z8r*H~L?`{2V#8JI%@GmIL145i7&9DgB#dm-{Fv0T_iQtic0}+gM&K$X?WkNRhHJ{Y z=4u`=>{q8;lljeoSbs_XkiNkyAcY;>gPd@$`CrzvfEP*|T?|-dySHqd;lJetXT@jm z$<}P3585`Ir_%-AkpS`bDS78eXEZAQ`2?*)EyHMkC&Mi$6PnFtmnjR{_eakJNYGJ9 zk+>ADM%NrhTNHf3vvCcA;Of%qgm$gqm5+Qp#!|6$-BzkIAw;E3t-cf@3q_rjz( zqchZk-^Zpk#x{;b_EyCQ9NS7%EDUO`)nsTUI|<2nrLst5(*Fiwz!F zIuuQUOACgT*DCZnjQ?DUBdM+Sp0{rFvk$za(={`5j3_)Z$~>Y_JK)AYB8*K;S5j-H zpoW(GfenC*&AhUwWThgusnA z)~M=}xB8!33mN^1B5?&PZiM8+p5H7l&MoUU>aThNJ9wDxaU8DY?BvU|H5Bns823sM zQuw~yP+oSohJHd=rv>0jegc88)j47>qGF8UN;C}1kaO76W9d~OE4(<`#i%krQSfJZ z>|c>pSc<9A)S0}|y*W2e6KzGV`A4!m8Um+%8_wNL?d$Hh947^buSF~;JC<|&FTC#7 zpX?qYK zY0X2^u+0zFnQUyZ&^_6;GrSxY);&zJ(%Ez@?QDt$7!-!?dAyX0o~pSn6_UVshUDkJ zmJG^|Ib+Y_v$W$`#G0pMo@FnZ3{ zFJwRDs{2P22Q9WlO=-re?z6CLmRd61yp>(<1$fPWd!dVQXqPQ?!epo1O@rH=K{)){ z@a^06WUrz6ho;nM42-#Jp!|~{+s)*;2G(O-uKt^-z4^4I$`LP!tq9`2mar^`81pw* zDOnz@rnqrib3+R`eyhH5)D?aYr@kK$-Jfo4oJ7D4&oEx|M=?7{BL7_~k`l z5r@|_+SN#Xj!ZQ1n8za!WL0xA1B<(i+Eq-S=Y@`VApV?old9;EvN0sQD{iD3WoLAS z?#~T;|3OhxmvC#@I-YXMd>TKu;(tSWZorLua3X*(X?gB#Tw7QjacSnXb!p=7j{a$c z#znq*&CG!-sAVN*ORI|!&|Dby9nWy6=JlU3r$R@^Desv`6NBtb1VmO+?CQQ(w(NP* z^yFR6-(0$QUp$`kb;Sby)H#Sgwmc4s>kmNt zyPpEVeUoEfh2kqaWN~%m;@6Ur4I+?Zhg~n38tRpdx4xO?k_~rcKKZ@~xW7~ur+-d< z6*tjB6?Qo$Q;8aK-*Km_iNMUIFs7v8rlor}I=^X@7pghZ?Gbx2v-(x>*yf41Ae)!7 z=c+Hw$2atQ5^;#O#G8D?hgs1vdBCbJpNDDzKTs2Pd^)MG&mPk0f-sp`9b1Q!`K|iz z6@)G#)u-`S)Hj_Kr#>Lf?ljfyI4|-KA*IEC(Mgr#@pyI-eh81ZxOW~k+5c2x2EKqd zb;XA5A1k-J_cXDOlZ^x#9>0}XUSV6N0dgly(>U#?Du)bE^Bk94ZfEopR9p<^zBmHV8^{C|hr zCg}9K)~>@XlYl;(E=48iPStK5e`C*0;C-xMhn8pswt-Ahr#naDYT9Q39vO!YB{t9G z*B{vb-i8-_Ri^)v|d7UyReVPZ*i-e8S!KhZSuD-9qm{&oibbnEX_O$eNo(7?UNX1xUj1 z`C#{kAnNEKKZ*iiHnii2uX7#!b$YJdyKlQEJJdb8N`XJRSy^NS6TXq^$@+~i&O>@;|M!B<0K)$%R>`O7wKx&d`|Hi`iWn$(`( zs6O1jEk+$_*7uxubw`AyD;S8U?bv_tPSf}0YCQZ6A%+_{yF}?xydnFq!yh!18l-+3 zF+?_xvbuV*S3-W9;j3dWpfC$tlZ9?A;evDOkaOua-74A1NH7KW=DDVxI#5PLEz6x=EB> zd&p?^a<#r6U}=Rp{xmCv9H6B zZ2shVxZZR7eRVNiO_gbuprq7qLx9Gk@Knd=DYsD}3uCULq5#(&P0HWrJ(zc9vZoNQ zN=q$~H+7^ROH$wPMarL=#__QJ`RdV2@R~IWVC%fw=~x{5?i(v3A1fXQN zqE6{t8?fG2-8}+cSmyKr?>|CF6`Q{$q@NZX!uQthsO!AQk^D(Fb>q;3mXBMPT5|4S z*{~)D;F?M#uez4g&&pTP>jqY{ z#;X+E)2t=0R|k_ol(3Jd1~qd28-JrN#xAUlMZ4KnhOwuo$>RLCDMjDPObEcd3F|-d z)qd?Hr?{9f2(itrY=7p)yBCQ7ge6<^M6O|@htz`1CgSTc(z2;WwZ;2-)eohlUmnQ`0#P|`PAL;QyWkC zDvb3u7C4Rc+>W1XR*VIBx7?^ZcNAfg4sS57$MZ$*{uzoc5K)XOjK-`?%TZ+mk0ou* z&V3twHt5oy#~?O|#~d%hq>`NP@*rLBco(H9R;tf=jl}N{m+L`*Y4X67J2!`%z%ZjW zx+%C}dzlxya)NFb*z^C<^c8+hHeS3crBc!jA}C0A!vqvmR7B~TG}7H{B3&DZq%Z_T zq(Qoy(MXQ&9t{HqW8CNc-FyFlZJ+HqJLi1koa3n)rV~0-w^60W+s)F4|H>L8BhQh6 zerZD&rHi^9Nfc~#oehd+BrIMvp)}qOz4OJTV7k6d>hEC`%W@45@0p}nK=7xgwutB0 zMru(HI;Qiu4=*KJesa{O?UrjXTTr29csYW0DiVoVWKRf`+Zf z2Om`Ua@&xns(m}lNEXY<*H0x#ZsT4gHk_sg3B1%kfL>xr>(Tur4E$iT9fLY#=m_zU zqW_wFJ%skfYMQk90QlBN8!=Xfo0q{SQCEDQz9%@cGI<-_j*MmFa?DEANa-bsW9`rF ze%A0djhI3Y&ZOnfg_IQ=;&8H!ls;jws`kTg&oOI%eH@_owtYYb`6QqbuNurka%z>R zE*U+Hm9HXr4e4+s-F^a(_Wm*Hknzcb$=(XR}hAHVN-k$9d9igP2$K6-dot4|w#Bcv83p z22((jGtoH|fYfLR)u>H++b|oqEpV#OT_| zF5^tY4<}9U;zL7q^!a0Ycq1_Gfhwoh@b6Bw`Kjb2e>QV*6sh2*fQcU~dP&uTk14WZ zrgN;Tq`s-;-30$$bliCLN(hluYOnA5T;-oW2kgOGn9vl+@1> ztU)|H;H1Jc1OXeCt%3v^@J^+2uJwCf_Z{L{(c>(XQ6urIcj?eiWyd+em=TTcYVkYi zUnlEy(J61aWxB@*?AEc)c-j+OrW#|hOVpFAJ7IS%&d{?w6 zU~5Y5gKM_RLQ9iI%imM&z;H6M)6?xgR?#KmF(=Ny{2w)|Seqv+$A+5Lc2X!Vy#K@B z;?xtsi{Hh}E*!C?!YkjvM`MjO8n47eybh1pmOV;<+>??uh7dpW8EiF?w%lj#Rh=Y0 zij5A$4Nv3=J_5?!(<8m44f5313f&=$@S*$sBVf(M+PzExTpzWF!OZJ^3pw?47U_|} zSYe9aEjOvJvAJDmqZ$XX!k7uLL47l40^RAOuazb3e|Du_Cg^FlyT!BgUz$?cKP?f) z1Rmpn2w6k?7a`G1K>rU})$>$UW6^8&-)+#dzk}v-e&TP;n#52PAWzH+DZ)8qI4|x@ zqZ%an!W5Sx`p*KI_7p_S7)M+c;2wm&awP@4Z&#K^;&KJw6c33^)>cNgCTKe1^si#? zXh1|RWSjdA7+>bUSN-|oAJ;PoznO$wyPXlnpVo6V;uF(NT42BP&HyEYhLeM09;UBA z&3C!4J)z~1QxHveEnUWUCU-b)W3oq%#za80=sWEyC#bxVrM!Nm;N9OX^Nv)*$?}?( zck$uIpRB4p$rZZ2h-Iv^(Z#h@;pW3DSTtf`1Gmjo^x+DT<4$T;ZL#1U$XX1w{Za#A zv)QH|^EW;|N9Tz&-Jh|1&rxmBqz*$zC>jub2(Hz=2lLxj{m^1#?9Zm-So0~7W3TFt zOB8)|rZhi08NE`HKuslZHn8R{EO{!8sl?PUL@AVIrI4Ov8(HAe$xE^dLQdRW*wyXUCM3 z@>{;w5!d=JYdKHmfSw!r)ynL%ySE{!)rXgi88O4mxUaOvaLLyY54RDmpz?B9eFDPH zI)Ikdv1-d~(SJ28W+e@*sjV;A`f4T_f9t;e*cnFUO3wP})pWz712*V7t^(z#G`s;I zKor>=@aXjvQT1vQvr$LiQYL;QL(dDebeq|@z>E9Q71W+X*9F#EJf6kOH9Cwu5vDoj zre9ShC&67k6=?7|9KC~)Wne$|;o36R_3U)~_xGY)MTFhsO!|9bRTuJNp&)3v8=3ZxQWf* zB=;2s>fSbiP zf(gBv$BO;ItQYmO|66In-%Z{cgXbl#wq~IuZ@byu=Th7DP@1R*VHK_*XkOB@qgNJz z+b$b+$Q=9p;^G}wA`G7sykS_u^EcNFc=*PYXH|eg@R1G`l|kdrwH))RO?vwX-1S8V z(b!e)=AaS!!_8xbZ50ldltVfo7w`6NVmO5qtgQ>%&XiH42>=06kgUx~xF; z^H^3gBA+B|zJDMoyCg2-H5#B}6p^vWc~W4#&kc0DKa>bK`pHwTqHHp>V1^ZLRImAf zl%or=dfK?uxmO4-(Nv9N;q9j75$!#>^rbzJGwA&5e#Zx^B1GCZU-XvNE&qB{1NflJ zkbxiLCe`D2+tP!1_Ugk&;|I#CzB(e_Ttww&Pp&{YG&Hpv+7iKp;D){gMUQ-#I_y@^x_wn1%PW>Z1Dr{;lnkcLY@G|X zCsWmR{28d=Cp>5IN%|%IBXWK^!6G%)T!W^If(yK-Iwm0X%Mvc7&ndC+OVlL&r&#a< z6@9gSUl@nKJ1f;Eienrb)}SQ{jBQ9-*~oE=RV8RxA%U(9!VTGtL6YV+>Vw`Zct@5jXid8z_qB%yh^QMPa;i&!I7 zO61q?d|Go#aUU|b!lhMq8#zx7x8>r2#v#f28M?Dr7;~+Uw#?}LLB+Y@85a7dc|2LO zr;NhPH%Kh=pQ?x?!Ix?~Lipn+g(MII#cuS1tuNQRR)x}LKe0iccH?nr+|nWdSoq#6 za_gKPeIK8#MJ=z8A(IKtOUXHOR9WL43qMh<1e1z;H&54tXz06 zO%)5)H`cF|g?>+Eq$jJh@-JFJv&_J@Tg)raK4dz#T`%-;Lnt-&LEML>+yDNZA05yg zq*)%++Q-nurOJ5y0+UU3rDo7;U9)D;@^{(QiRtUF=pR2o_E8egT5PC|ll!o=svq#EJK_@sa}L#H0zXW33v!Q8AY(w3$H zS4L6kZSNzPBzex%EuT{eU@1RSDLD7AzM0vUvsn^N2@0Q!t2?=Dv58r^4D}UKE2@NR|c4P9R-imc{Ue89qbbRgj`dCOqV9y6nH8lLpZoZi_=a1CsTwxOq8T19)-kNU z;;OeK;M%_;od(=*1PyI-EXcs*61*-;YH2A_!ogWW; zxBDu8!W!LwEN}bzUfWJeKm*TOwTWwbRE_D`sw%^Rs8+Ww zje~&^B++|Jwlrv6QC*tFIA1=eo=zUil^1^a0)z2ky+;F>aPu$`zqPu0P6GCalf98i zCP1p3zxz(?LwxxkX``37YO{%#8aX?5)6W2Q)Sh_1fz$l}108nh8tzgoLJVDAwvjUx z8>zF@E$jF9rv+R%3zp?aM#w2|F*Cv0NIi=dE;BI&)@C~8rac>^^(Az7jh)N>|1JR0 z`M7v=;GSO6pQm$7V-Tg%UM7|lPbpUUnczhWBiWBNtEQ10aLDmkW*P>#SbCy~`8Ta( zvjo7?3$k5l%Sw6|u9QSeFvpo^5a*NRA#3kgcBRpT(yOjbB2oK4l9gV5Y&tEQ+c-sQ zr@c1whIh`x=&wQ(Jt#_Kbw=T2bf*7C$^ZHSN-H;{jrnqpX_u z#L+PnZT^~fKa+BGt-_=zE_kQUGzk-;G_yAcLuXN%E9T)8NjUxGh_-EdR>mUHk?5OG zmKRrB!A8lgF4$BKEnkKwI*Xm_H8?fQz{QfP(1|aB)b-3@pr^Yv|sj z9xu6X$Y9)bd+Du|Y_3^GD@73Qy9&hEn0MUl`}~QPCgf>vGiCtU9>`owMa`~4d?~pZ804Z)e!6EqD$Pvc;sO{+kD*kvF`J74rk4}kSX(TiBfvtQ-g z)?|iTUm5~c)o2p7rrK0Q1-=4A!|jL^k7tg2p-CV0r%3;Jeh_}PNuDb8)aC?u?6$J!lRn8<%0DBIX9cIWMHHEd3w?8zKnw5{d|+mrQhJy#!7GM^tNQm#fUmU|Mn zwsFZP$>`R?x`d-S8te{}>W{ryp17X&%6eVHV}%Jb`PFLn_{6HSdg{&VorY5HUz-C+Y{NoS zt#zQoiH)K}&kKChz9J-qCJd0yma-s=G8{cE-G0M+O$a$=`!(H+CI7jBs;}%VCG7|{ zcQBEns_Wj8yi~3aGR|wOSUVJ~h z-_aFy44i)tgk%*FXa0DLbO#bYf~iO=tieIPg!`8gzpRGi%mSNL+dXAznN<%Zwm2>76-?+E&P-A4^kw+D?TuOf7}wfn#8eBSdVGr1gtA`fZ< zmIyd%aij3-t@Sa`-k9H(L#DV3(ERxj<1^1&ZjR+!R``W8@s}tt1j_(DT6k^nRG+5o z0o@{4(=JwH?w0!%BZV{9$joYJ&SU??E@2O&M|OKftJU#q91{h*bA1O&cq+sOz0-O0 zm$2gVrPJMt?);wo)~t%*=g={;?hyo;dO0F%rNLXzr^qKy5Or(8$WAj*=oRpDb+A=JTLjxzE#YvBtC)r0JwcA^+pEeN5L7E! zds%33vfsp_g*syZYHEHg(Jb+?VUZov`lYw?(?6c_ehQbfWIF~J(#2KFuC$Np3X8jF zl3-P`XyWS48BxXGQb+UPK(5!b#SOkN`rp>Y9_d*k`!tK!{)#)K1hD=*F)D(fJ4|z6 z|K#YKzR`~uo6;;ZBpT}0|JtH?h+}($oXJPO>GP=C3ecdX&W$|$jJh4n<~g8n(Lf}Q z$*u+-d@|5*1b!Pj18{aPnF211Op`CL?)0Gn4Wk&EHgKBjO$%p^OiG;yWw`143-A|n z(ZuGnx!xqz_kqWunk{#^GBjT=b4uJ_4SOBp{v-1jliaJ`2;U)biASlvvUnLOKDX%V&lZG^aR9&XKr zVaE=f_H)g09eivyAIOb}v*BVxa><;U;eb=Q02v+_4hc>8MbyL}u|G^=TF zQ;^jx)c5hZ*Y9^O{!MFkl_QO?@5uWmc{Qa~C5pLK-Tyd;+)Htt5_eWI-Zox&&B3Nu zGtg&IpU!j-B%^bB${&>qIO?k#)`;kxEIaQ}j(mDE>hL@&i~UKHZ+Sd+DsANat=v<| z8_RY+1+m}jetN8A?o;J<_5`bl^HJ50>>~O(jR&<|Ij!8aBjq7woqpGo=!sHK*&%kK zODjF1D{l!;+CsycvJ0!vW25$366dI&yfk^j(L-8ANcr&Ux`pTi$${5feO`_;1M>EQ zklnQ4g@VVzzW24*tB&z%QQf6B6z@Sw<61a&LE(af0Qk)=rec*w?2IP%ITs2a>Aybo z?+Z{x-zV}#r%GF9$Bk>uOgzKfOQpP-xe~@xq@q=F?X9O(6dND2p%}yY)G*^bKv8E` z^5;1Qnyn!+h25|>MS$@NH6v&zX=3GrL|3ec*NI-{f|r)qNT*f2`h<31bTA!uTxuFK!;h`JVhBOr4D@#l~51 zb4?oevMkGT)Zx{+Gb^9xojvzQj*Sfw9UtJ9OB))|O$T)Wi#5>MbruRdHEk_GTK<WkNk$)%8!AHvYpaENUQsWDoAGQm`_d|w=<^Z1eBqWQ9=!3YbH(yOg5 zKGRhq!s5y=4KDWQhWa&$Nak{N2bW67@{Vz+P40?~NL-^`rtF{^!FfsOO;YHhZz>q( zGgfS7lWQiaj`_Oqrq4~Y%Dw^rVJPdH7MHcV)N3y78ay*Dbf|U4<1Bbw-!J?EoAh$-}dw87By(z~C4fPGQna zZUUjiXLr=jbxc7|{aV*i`R8egUXulAg=&~d=?U59{!8`S=wf&MBBSC@+c&k4Z^eXy zuZ);Jgv9YT-(1!u8sGzQ^^kwz-g}&G4kjp9vQm-pOr>IQ-H=+f*0b&kN2DoA_4wr# z{cv9Hu#TBspcp}*fkfHp#BW9!Or*s+*Fxa&;+(nA(w4O+B2CNqc=eZd#xyMQwdEJj zk|aEaN>x)9Xb5b8t^R|PjCbEccFuPAY&LE&FB-(X-3uEQYycLvy?yJG$Uw! zDN+@X{j-HJY|@rq)0~?tog7#n{B&BR8tL}v$k?(7`j-U=*K=} zbZs42c0ed9)dfFeZ4q$nidY?-x)qZ*^g5&1yy$kqWKP4QaV~H8_fkVk<kC zIr?^)%p*OT_TfElJvLh5A@_~s& zy`9rnK=}8|3u#_Qmo_LcNLz}rljeeli_qyiU(i+KN8D9xDI0DHU3LPUotvb%C@c;1J znRu`;_%a;-Z(SU;DlT+$-moaym?!%`e0oWS<97}iTTg~ylZ9{kHF2Hm;G1k=!@goP z#C2GbSXf4P#Mob-CYeQSRmYY3)2nsAT;;1CK1#B#+{ywSIK8gVlsY=o)h}zGoRq^p z<93?$&yLoFx9@hN(33Ibrvc3on^YHyaQ$2@S-?TlDx^;5w~8$rV8r?@JVgApX6V&qKj)CNyf)DZN&8d)b0kJZQ!xZ{%w)=Roa-n z#dVuEsl4lIY7~puFzDR4BT*7G%SzCCU{WgA%#$p9K%l48~>?Y6SKzT9^FZDy-MjHXZ;gFgeofOu*I+{npTR zIw#&Rkyc{6YwEVx{s0-ZD0yg+yXxa}-15i|pWid{VfbXqad_m~j;7;mPR^9A-JuA#z#9lNe_Fz78uyulRfP^oti!X z4DzS%S~edE)!>+Q{j@(yML8GW9e9)|?}uGM?LqK9=?H$eHsl8oJ&-z`dK9k0E-FxI zSQ!$JKx$1PHtI;wbIjZysAlKA%Jy4i)Gs7r9aQ|x0>WePeWS>~SvB@?i$_8C-68o0 zN+16p>2XP(O4)R*`FG1VP@St3dt@nJa)ISOEKpBT;b>~a$S=71Ue+}49|u5Go`^l2 z?(s~P^893wyrC%~eXP#)X}6xYg8=Z)l%t}v0(?Y=8+Mg>2Y97r0`d>^Vh?O?bIr4U zV=m7xPLUEpx}x}FR{i8SW?iy_{kzA^AoHibE616Tb8izwG#Gjj3_c`EzFEG)YuM%~ z{x9Kdk>O?+PdQ8>$2QQjdFJ~F@dz#n% zZOi((!=LUnlhbl)VTjL3OG;bsGgneQc9A29i~!eC`EaWH(P%`;=+iPLqIb|Ku1tx$&b2DW32E*XhIW7LNp zm235i*cM@AGtHZ~{cgxp5gdY*k%_S{qrz>zVCfF7yMttEJfAfEqBPvdQSl_={)&t^ z+<`W5=J*mu-w&*<5s>j&I#Yu1l$M|WxNUoH7F)?y+H{>xH1s7*pwUD#(8bOkt&fRZ z9OV=$s?HEzgl)^1>vwg&WJ@2>qS^EO1)K2upmo^uDRJ540Hz5XWdcp@rBDyt ztw)z}YUFdQeP~JW)nYABzq}o{-2MYWxmyjj>Y}G*Tc70BHJvsyoA$-sssRo=!1@zF zlSr0QZG}CbDUTB@j|m`ac;JurZd3&PTlnLzPCd4+OuqXcb=VX_-WC+4&vu&+%{%8r zk1gji74Kg3c7U#<1ddunc-a=?w(AQte-vG)voLn~+$Pc0R23Jp@ra0Z>iYPQmbkl? zN9YC@%+ZKL3||)EPEeb9mJ+Wv|I=5zi7z|`0I9UTUl@lQ1|wh$DaI$-gK1x&2c$K~ zfv#xk$@^P9_T2p66cp>vX;H-dD_j$nYzoOKV^xp9hsR-V4-Witq}Ob8*5Yq;W8}p! z>F?=UlS!1Sk$x;*H7>tYj97Kuu!sF})2V-=5CFN-S!K-oRIYgmF z0h?`zcP3XOAn4Zwee5q7mSKD1M)w!LsHqwyb zF{giJioN}kyrWn!R5g1^X>)7Q^Sa1a!_fGmJx*-T$AlMDDjp+x-=VuIdhd5qhqS0C z=GN{`VaWp+%z>LcZ>VjusW1Feb=Oc?^fk!PgBB(UFDYWT-PJt|9Pl;)nP?cFj$3n` zvv*MLD>a;if<|wtze%b#mt^^{HLLXDM1o`GhdoA7UC$RVK4Xi0RX0-C*5yh!Gq_*C z9`sxDzkWNfU&()V)B>@}&p3Sj9h9-hL^(sf5;i~zfUs0;96_V;I|wV!6H< z02fnq^q)&dV{144f4SeE7+@ZI?G;P*D)-uLO#b~zyV4>5CX5->>(1UyEC114J4RQ0;XF9?BW0wW5jU!_DsQ;~7MY>4Z$@vusox5A zCr72Q>s2PoAUVluy*u+P5pd03Tw%+VHyFPFy;7z0Gc~9{H2v55UOXRe?JsR~!_Jjn zUH0E#Nn0Ld!hDa8_ZE+fQ~!b&-CRt-s0#I{SA|?Vr6$!i5UClg7U{z}W3**|#X9_2VNnKn>pANGp3O^vOI;1KIO(1%@HS(tWXK~I`U#l_e z-!{tMO`&(XOT?y>bp|+4nRLdg<`31>2k~sLKB%ZWcKQA(WpovAi}IgU@ZfwCN~hSd z9rg>xWuNZ*S8Z`~hfwk5syLg`r1>pDxRD|UW6^`o@Cl2F+5{TvaM@`M!3+a!S7Llr~*WOJ=09wxPuNdwG8$>~cHF$dP zZ3S+~mWNq9cs$|}hz)ep_W@u?Mjs#a~Vl{S1wVY^Q0PN@@2D<6SYESdh;RpUZ3dmD`0WJ z&Jt&eXi@Ur< zjqJN2Y2NJ@g333;S8&l91Ec?`a*l(o9MM01Lk01FN0S438NiTZBvhdU?RlMLSYq#B za^v%C0eS8>jRIGSQM>)Eg0Z3F-%irS$erd%R90nvn7LgtDfE=mW8=+?;;OoRzVFsc zw5{<9>$)-!AEB<`E4GhRhGoQ11^Allo4<7Tb<#|X+=a-TZE3k+F9?3%j{>Bl;Ro*@ zz6(7Bm^b+_8Qr6y1IMn`JxEt2mG^;d%OjZ^W$L}l3rj_NHPSJ7OCwt}@uVwPtATR7 z=iJ$8fJyF)kAF%s0~H=NR{Pj_J`MhaYeo zwc!OxLwfP@z=3~Mx_2PQ8xDrc7>i8dvb7xedcyLl`+QmknpjT%nDo_;Dai(rPA>uG zPAl&>a1(X-aIFz@S`9lIrCFM5(UW;pzF?)mmSJUit}W$%p0e%)294i7$N+wQz+KTi zuD7|-&kX;%ifM5=+$%j=hZj4YGb*`kYmeR(@_a zlcLo{b))WLpa1hwin%|5`K^f}1~THmDtt+B<6aB$SXkw?p3r!dU_xxQV*gSngQ(-z zx38ox%ogQAfJ>37S8{r-(0E?)UXC&uzqF2dWbaPADwT+o$iATJYt$kDj)s1S$?f0t zw%j0v7O}C5cTX!f$$MVg34#B4eNv}{y3Nr`S0PDHHJPQWm5;r&t{W0r z6}x{vLC5c3S*fF7J7=~9#)GgQ9LRhP@qDV z)aji6;eLZ4$oT>8V{r%7r)tP=dvNTPKtmEro#mT>$`%-KcqneV^7oSmt`*~h^k^F&P)wAiRKq`S>?>&yI|{W;mXEKsbgW*FT##ZHSMnoWHo? zzjM>*RNEPB8-}MLMajO{{XoI*zIETcq7ehke259)08`i-L_Zd{{sWl9dGef_ONmnF z`zpEmLR{(!^IL`IBQ&W|L4<1NSVjs!-B;yU4tw)m4A-2Mqm2%OHOeYU{ss^oP1fNA z{RJQiP86lX{%7Fx8D|k=rq$>)7Vmy}G~2C&-@KGCjS%5nQkn$3Ha~yd%aA&V*}VUo zp$x~`CF~=RAXPhCfVTH@33r$AU|2{0`-cYYH!Sn3D+J| z6|C&o2KG?QPvU)dlnR-=<%du&Zxh{!f1;6H z4o?E*W0;K~Rd>0Hk?#JG9*d)Zj6s>*d9g(U)cGL5=R(qSrkCvis$FjlV+1^V*vW?> zBlaw!=Pjc|VA+`W8b;|kTq)WwG8D|ymG2pQnK<9G)$PZEduttY6lw*>I8v1N2y zG@oH<@V2VM{BWu!DPO)CHyGlLR28XO^Py4dDKXb_Wu#f6`s_l@`5Ai?TRHm6n9paP zJiw7OUFu1H)(BD9yE4#r*TaxE1Fwn}s3Y8)F42E=J9LmLG-g);ubXl8M#n7~doQ(- z)BT9YGEDAV#p9*qqxB$oq9(evku!nE{N3+Y@Br0~J?CBl!;6pUG<9^vPl%T2Cx=`= z5yf?rLyqsbFU0kx%6M;*DR_zhbM6W7!6oME$=?-ZfY@@N)l!Vm@CSe>)eMYuC{#=s z));4|*{}F<&G>OQvTbqV+& zGA|30B4mO&_Aa%%2gIzFR;H+6M(XXTgB&m7*9AX5C_LGd1p_PKG>`V}_AsUvB##D0 zFsfB1>W;Wk^d5{$55CHuj@${RUU0p>kO`}15VLhx`fXwVlYjQ)>@12&ktqZ8clB~) zw_>C{TN#%`)PnSeAg!LpJ|olUVNT~?j$Mh3bmwM~x5!8sA4!?`W8(2ux+JIb!S4!v zy?C4`6zImYDY`40URUX8v4@w{z(CC3kB82;pNwBgFQ#&27CU~} z+3!lX#oCAti}~gi#2v1x3-}bxuKPaB#TkgqMhcKSvRMpI`4f}fHMZ8=GkWp&RsOMqxlqf838RiDQyNIyQzlHCF^J_SV^9Q zT(aEXK2}q$9H+9bq^lWn3DIhc0tE3GUy5( z?V586hg`S(Kv^!tMbaq?FY4D_Vg@i^bd;iI=VgxMd*M~3naRwdy}dx*tr_9R&Msw* zYm2KUa8G^4NzIFx{8i_wURQpnn`fcR2UiQ7lH%PS2WjOaTXY`f!Zd!Rz1h67Y zP{{KNv9Ff>Yg0*<8ZTZ8_tIN?$p7%+OisP*4ZO0)dhpp6^o|97 z%pjobuSxV-<2sjm#IK=RND4?|>0k0Z(rN&M`Iv5i@yOyNZGjRcda%^Lp&LDg?X(5Z)e-e6F%rF=K?t$u-ci`u^)q8>Y2Lcb( zbMqRHgtaVsBfJ=tm%4l`5~M9xN(Ekhb}G*fN^PFB?U~S6A_jGo%irvJ{;00-9cs0Z zATZ=xo zs$|PIceu5vL-9a;hMsiT<#>fAhB2|<)cU8zX@=M$&T%mbe9Ax6f*}ICU=zTYjg#(} zC#!P+q9Yvq;?#+2ZCD7z_GVsB4;(%Qb{m?}$IbyG3_FsRPDLCLRXzs2vQ=Z` zUt@G#eW#aT1|Iw-;ThMilKpe}Pg;okGR5d~(A zD%D&(Vlu1bU5?kIvQDY5_T-68u5!Y;jyK(V0`~KdNc(M{1kgNDLP{_$NndOo?^~8NbJS0OWLvN{~zVI#bKdH!>p@iI;zaJV;CMaVP z?KQk~{{9for`CIRm(xUT?IP+?Y0;zSGGaB zlw42WDnD($&~FQ{>*Gz3_(N)26x_VP9wlNk4~A1JtNR(h$<}ohKJP1ydBv++6;lkU z`eQp$%|N>uD&lAB&{+lx@qXS%hf;Q4=o?hY=Vh>AmF4J*EjItxGwnE*mV^SF}1aUrs#fegcv~&>x4zgwu{ho!m<; zoyPBawk&g59Y+*Od0Lk&ix=!XsLs`vt|@X?0QJPDxa<&x%qZ$T<>to*@*K!FZ~6-@ zUTxms)-#)Ku9+?e7V)#f(8CkZ&2jF{%%-W*!TEJ_{{ePjxUmsmb6wrf6G-jcP~ZN4 zoq0J-K93=6+nIrf!O1shz{3G75kCZI$(;lIL2_oq50REr&705d;G24sNi%jBm)dXC zj4?V%OROj*-9hQ}?>uxinb`MrE&_u#F2#F9Gzu%^YH&sJMLBa1l8HqPW{SuZ!!6iH zikX)Zun`CP@GsSS%qMRpNnPlx-B1fv-Q|~k+3HT0zJ%f8ZwW2WkAlO6HH<`8PD-0Q z(a@D9C6{B_ed!m07uQp5>k2u}86_J%T0^t(TuS!`p2fz_%@5W8d?BLado$J+?r5F$ zHO8?NMyrq1Q=0oA@3c3g@+$I3#BaGLoKBa8dCKEh0o0w@T-}fxbi)feZC}zN@94ir z+qEUhsvoBjD+3r#>Bq#sa710&=Efd^cZw`XEcqw23*Y{&JlqSr=%7w!U!<=^L;~rvt7=CxUV}pgVe71}8 z4)=WA2#=))Fld9{jhRQJGwQFLM)p&ql^z2l{TZ|WIWrU%#Q%XfJ23Ex9=C1ZPEN1p znIg`==YX)schV3gQS`OOJ}T!edXqWa>TQ=#GqtHE^Qf|hl3CGm(RqM4Uue?w>H5jtZO8$(?W(U@<1KDBs(M%sy-%|7^53g@Ex`66pGI z;1NF!^t$DSCAsDN{H8f$6aSwXOle>Ia2m*8$sk-6*dO~Fbe$)hd6u79zlE;1x&{sd zf7Cnkc#-38aN{x9zi8oqP#wlq#L_?kDbDG%X8*eU^A#fU9*HP0xN{v?YFq_rOOSGO z]!MBR0J&p^+7P2KK3VBF+MSiGrY#OR`*f4^cgW2(IMggBeFbI#vZ2eM(mF^e%= z>5P%a4`x)o#p^zmS@HB>q`8aoHz~h9oG9E`Ce#oUa>@T$m>kxmUT&J{t^>34wDz5# zzTq{ji*2EFr>$wmQcf?9ZW(WUyL5`>5M0f!k^AsL=whq5ZZ5cB0pq~tJD{47rV(Bg z$`$5W64l@tC{+CVr>+LxUJ{CZ|ob<&$og6hi z5KANBCEKC=MVwc>Anc;EoZvzEay(PyeN`gy?-DE3r@AqieB2+K#b)8Es3N{IQY3k* z^bX)BbF`e)n%f#cjV-@-0ECMdzi8B%zXd!>W4%)TkdE{Ql36F`@9H|M^#`#VHC$DE#&ok_HgDqj}8i!8_}-G^b`4m>Ge(f=k;u!FiZ;H-;L(U`PH?ViZ8R*OOb{D zjUFsX!lU>%E;_o0?{=+ozD57!)^r>PPG=r|RZUg$Oq3)*xCi}Kr!ABLrVCMPbbz1|#w0o~*fM>|VG-+-bmek| zgp<7x@nMU>o7}~}5+c*QLpp@N$FyEUp_)&iWal5Z?Huv^CEfkNxb{K0kb3bEgHhvoH6676z4$FZ>Y>@b*)+ zZ?d;@a08UhN)*_?-7Xti@p~`S{g;C^vV+)!&Py`i?{#nFln#luxKuu?AI2R$E?Yy= zqt!Hoeav%)aQzGfe!p(-^GexLiZxIK_Ut4OXpsF#*McHi?vovsc>m&jV+vAz&**Uq zKoi*iMjqv6LMGuG>uJ*JiH%$igHa>%@4}-ePv%V2o4zNz2VfB`x3oR1rMm{mjCTlM zH4=oh>isA?Vb+mIs8^=pKQPQum`PW#xGmK!u(xbMqH681aZ-NWNKBHl*(4|A;*epF z!6?|TnJG$?m;Fn8{6rpw*t+n}M`1}C&`X)9cOa5i#GePuy}?HWv%~yU?FPqHXFSH#a(*$`rh%@91(s5Fr~?vU(Exmhf@7E&^eekv&V>I{uUfj<$U zRPUv8GuLc(?%caKvlVKFGlnjenTs$?6&~dZXl?5IpM7?opDK*Ee^%^a^pj7Ps^=e7H1kWZ-qDSzy^hKL-lN5i7IkVO z!Js|db5$wEVG?2|I=8LA3TO1aD^hR@>Vk%#Q;InLk=2SeL@18i3Ee3?d7WvxeVj&$ zuu0`y`Q|-z*z=UD_z~gHi_w)TXFI~wzI?^&q;COEotF6&>9*te>6lOW!);TNLr>(#Ga_&A0cs`4K!uPbiM-)E~ zKB99aC4tmPLZ2(Z)#WUy#a6dlSrE$YAEPS!yau#NR2Jsbkj{;daQcI&-gG9N%LCtorsN zjJ~q?!@niBqQt=tFEVQ-PFkidNNFCODX4|mSM za*bF7XyzJgpwM)iH`Xo}_T(p(Q&ZNP$({~ZZpWu*d>OKez|yqg%5qzYN`ePQU+f!O zvV!1_hqm=%O79NR3!s(gU zy}6OH&kWnWSrPlv0fFKwc-aW$hfq+-_!lBo3V|c1wwW5!>y6^wDpca6*S1>!jXN(X zNbWQgl{0+a`0ZQERRwO(4o$82vwnH+2Hl&s z*^5`URycnA>_KhVuGb5c+=e@iY9CF`zb(ext!BE+y8#A1`aq!8>81P8=~XRtj)a9d zbY)B55wpGP42&4jbp#%GmT(8!-kPnC{k!vj0N+3$zpTAk%Ea&S8CV)o!$?^5n+YGg z&~9`r#rui^={1>jQfmk@PI)T_#b`WreoD#w*iWz;fk$ZgrG^U&1uH-LV6=WYUiA=* z(6Sll`mE)!{IJQj5L;)CE(FK}N7oTPs%q5H1Ys83YuvR*OvaDC_z%%2%p5`1T7I{x z@BCC-)!5@&%B$9VE=l*U1U8tREZ8mR&{5b~t)Vs)tRuDWczHW6X8^zqxG}pWwX~q& zEPSX&P|wS{COeLEw6J2kCtRNSYUG1g2;pTytnuLxFPd~XP-%c z*wPX2h|7HQ2^o=udm_F7F7Zi@*3033B`XTSu#(LK9qm zjFpdXu|b`evyzXkT6oHiY|UYhUgSLLCg#jPMvDsePR&d`DjNAmZ_@75;f>h55(abU z?Yc8TT(!my#xPTN0^MMqj`a4N8}$>?-RszEAdaPoi5ssZuEr(*h{4G-GOvuuu25^; ziuM}7%z19dUoqlM_P9A~3`YDv%F&5e@y-|U9u4?dGV+F#Tot$HXgtDj{jeYj{SdBI z13T)^m*0yCK7;A#{$UejGpX=7Kr=S{oHNCSD{=obfza zDxUnX+NO@X4$=;rdP~2q<6b|x`*+ARS~hpBHSY0sOs{hml1a52^O1x(yS|j-+Tl1` zBT1tMGL)}cHC`zSIb+e8?QFQWU^+gt%_>V+{>C4$X=aY$nIofY=!yrJqd~a%cq~Lu zai00aNAkyjW{ zH81vOxb48TG$UFyT-RGHgL%=O=YWY#rP3*Mv{5g*Q(`*Iq>J5OCE{0(KJqxBef^@B80cqknu`jcqc1Hd^8B-#;&<; z_uyBRkE(F2Efs5frYeu|if1MQ*(sJ!>^i!JMf}FEoO3)l?^3RnMjEP`p$KGb5On^l z$z$S1<4T7TCt(N;JiX$fXby(h-~BU*jrlp zW>m!r5d2#>Y3*Z5Y@5Gi4}rw?+D1F)DDv!D(!VRW;-9GPp|u?0IRCQtcK-D7_i?4a z;cktob8cbMAA<7?9=ggQ{)04b9KTyRv;HL*%g%I0#*uWo4(d>>$LZa7Lq=))n5_fJkyHQNPe&JH5EV;3X(fkpN`Cno~)?kGB zsP5gzGnG89-*fEuE^ebaSafH89K`u~4gUT-qxl7=xY~>Z3s1*Jo9d^MehzgV;W~dX zt)SyAg#e%<{LYup+Zv%K{fF^U@d&&(bH4^h!o|#Sc zjlpaDBcjI5En$id{;YrUN5*`i<@P-Hm1ihd%Z|NHaJT$JW;sq^;R-{W7(6U-^})!} zCcayZJ8|Ru;m>Yzqc`2c1Q9JwvUL2g;*6<)u{lD|ws6%_WPLqb%xbMV)r;CBHR|_| zOs`qj9>#M#e)Qq3)hfDrNUz9#PS~gY%Cf@9pL)z|DUjdnl`mg?f>3I)?&P9o7F_JH zM({*s0F4iRx31gXdP6_fFC~lf!=sA!)K&^J^H;&V_(u5lSe2pYQlIVCnDF1?!9TKa ziDl{0=~_8F0BP&|(MYuZctpX{|0D+{{OIKK^VD$G85@3k-64kU3_b72pV!YMxdOUa zsjV7vSRkK&RD2v49h3XELd4i48oT`U@(ff|I=x5!s$^s$aKImkD z^^?Ag)Xv&dWd`t1M>MS3J+H3ab;yJ5vEH7ql=AZxF@HI(k2fPLE)MwO2v+>>u3I@fMm5}ryA`MYyR35ceA0T5 z-ogAUPieW8T54|RSmDH~?LoN(ZvtMB9#NwoKaKdF#H7RKk~%ua$NyMTl+tj`durk} zqE*ZBSVIPIeX-P8$8FJa1-gSjLQ~+XMv@B=$99j)z2A@#CUN3w{`Js#A^y{FSBsh756M(H_N$sCDgi#$1CU zno}@QnwR499Dxq=0HQ&0rQLH1$Ad~)pAtKE=*gmiC4rAjsAQoW7ZG-(8EAHX`huV1 z8M%ocS(L4{E1DrKB3QKGyFbpYM7R308W1pT7rxHhaPxvz6XkgA_xxS0zagzR1VSuX zIwjjPI`Zc=l-G{AigRq{L`;(9JX*K+{jHki?=rm&=N5*za?$R5wW*ow2dy0^4Q=4{ zhKBPe$CU}3Mf-m3i1#Qj{Dfz-B0bq_kE5=**WFpPv;HhFcE0#W=&qwdTQ0d&TT?Ys{So|xN=;{TFRn*8jfAYMEK+Sx_ z1PY(bXnN@Bp0l^iEt#6 zh=QtPUVrA6!l?C7XDE|kl4@=$ms=B(UFXl|53S#KoWEU%bXmMQ94Lurg6sS}FYP?~ zDq*=HqfB)csHcsbSZj!^pmyOI*N^bU7|l*h^6l#fC@R#ees6c2Vr(6QPj<%(*M0Sa zs)v0-E7=l=@#l<5Fpi@&eQkxOYF1ztHycY09bX7}kG2r>e+Pr$+-AbZqF{`AgUs<0 zFu`Wr;)!pK@l37p>q)j9e{!IK9a&|@53-z+xGi%?o`lb-6Gr;nkaO6(@`kk+ILz;|Lt&o;fLT zIOt9tHQGj877c8J&2vV+qj<;O)m9#I3MZ`x=_P3}+xXFv?K-`!Jlepzx8m{Lw2(uX zCfK7~#JZiwU(PXxySRHE&OhBuxQa=#Cu{AFO$?4fpbOJE59(e`2B{A>PcJ>G+SN*+w~K3$m^1P!4-J&v$o9d){>aE5EO>TaAh9lg3nK~ zdtAY+XJ{zKmVP>6xgqma2cM9sED&}!{*f=btlQKl#xnb3U1QvZvwU|mivR#X07*na zRCdpx7;$AHhLukn7~AoL8h$<35Yr`4k)-9WYr&V*Lww@Cgy# zT{Ug6iR2vgb#K&B{YQMbnX^*HJXD94>*^}Q6GpAX4RGhfxQm~dE|895tgpqLgy=}w zDrOFCXKmC-LBrmq0~8DvKe;R8b)+!FK(A+jaDCp0DyJTfOT#E?xsH!xA!uyi-1FWw zMMVEcf#BTjl^koB+!9oQ;VeG7%B=QCH+nXS$v{PEVY^PSOpHC}MrkMpOYK|RP~-v@ zB_3uT^Vb?iPezZyceZ1`VnjH;c??E?Sk0c*$cr^DR_So$H7rLu>oqW~bPuABwb@qo zzI(_atA}F5v0{N!yKf!S{W~6<{p+VTG=9awbe(DkQ)Gn44GFrMU$u7~<`!J(t*yIr z3mXbUglni2icRY}A7slU4)7R%>CIH6bbbLsrnnq+Gm*m%xPw7d1v56vx%K*UkS1Qg zvZTMVl)thxYT*43a||bKBpNBw&f}v`$OyoNthR#w=vw%rA{B}FDss8(e`U$>P#8e? z((M~M_~t}}V~-tS@EzI^@bpG>)PIOOa%S>gK~&a@+c+Irp)A1NN6c@%QawDFBN3>j zJf$8E;)2Vy;G*B-U}XCS28#yv2rJ)FykigBB#B_2pea%_@M<)X#CDC|KCI_saLyYm zH;zB`kb+w5AJA;C9(U}Mq4+0<{BJMK5p4HvWDS! z>3uIJ7Y8QgK6X&kj`I=Oo?mTa z$647yLqy}pjV&H(j^sXeGrtP}l(uV>4p&8a7Shr|VLdWE*vqeluIL1=WdM$!v85da&<)U z)|QpKKCI5~@!R}5--MA)X%!1CVv6Kh6amFphHJLW?Ra~Nm;EG50Q|}zFI-!Dd#qTV zfgbBz;W+z+;f*2dai2eU*IM1P&Yw8c(sSX3*)e+_FkK6bXPdndM_S;!mVK*-9vZ51 zwznW%=FhmoSG?-0>vhF}^$1+q%sYzjnBl70!l63q{R0H{SZ;$ry@A@luxA@@wwm5Eq zn>`$pFdGS3d-{aTd*VA}p1M5y%(GED)y>^J~=o7 z_pt_7XU$T`Z9cG7dA)ufI(}(cFZS_YbFk09Zpb_pfA(5m>YMR_BR_vqkSA*VQ)@V) z!fY|JeltY@M+dkW`S)1fJctD&Yb74%jLv^y$(`{*zVn7mCK*64AfbVJD`smZEi%gT zQSA+qDpvM>*R^*YBbjB1Ku9IFeVRfN!+-raGdJj(S@w?YsdPh6pcN z$c)O0Ru4u+!Xsxs2}J%Z;aZQ%;8|zv#f>`;`TQ$@&If>DY3~|D^7kA6;I5lH~s18%-cH3p>3I+5Xq0u z%ddA-`G3_*QtYUMetC}ZJFbskG4!ygV8PrSk7)jn-8gQ0o~mPS)$vy(U;8Db6v!=0 zU0Ab|nPUN~I2f!@#MRcOSZd^5w}|qNcCr(;_3X=ioCm3?>qnc$p9y3FF@ESKUZ0B8T4$m}J zJRCpd5T+?@w|O3P{yna~>S1=OO88P*x4nKe(E<%FD~}-Wf0U3saN%6rar`x!_|N#^ z3y%htyw*DQk_RlP-LJkX@abRApFO`ZFm2IIw#08-Kj3OzGxL@MJqSi%YiLRM8~ozd2Yj@rhRl}{TOEP9Tz-$#t!7HzzI0$Y7ez1ENX z9;=u?*GIO>pI8K@wX#^`zyh&S;qc$GJHO+oYQ;oP4h`g0uD>6T^|RMWn|kRV**pVt z2zC81>$mv6A@g+Hka;GC9{)7<*nIx$Xbk2iU{qD{#BT&sm<6YkL63GrW>UrlMt^>X z%scd@zT-G#Nk-_--0QNEe~p}HD7*etFptQG)|xq518{ybFU z2kVHu5=UP1GZY-^e*irxv-Yf{*HOv9$fAoalv`lcyB9O&orQSU+WRyAOro*ViEMdm zyRufxN+5ULyKsAK=N|R(uf0<;EFsgY*e-=k*`DA?D0lKpOPU5df>Pr^VM&9C+w;R9Rkf{nfqomF9>}7kb!Mq^O^De19OQ}{1r8Q5**A^&P ziqa;A?XglZTJ^z88>D{ZyT+;mYb4HYTYu*b88TX_u?P%HWY4(4M)wGX0*sC!h82c5 zR#$)zU!V@of8jcxJhl(Ne$XIq+Hu|-eZo)rVAY1NcHeY{>#H~tad7=~jm|S8R3#jB zsFWwhUlAc!KPRraR!G0dRt20lmAC# z)LYDzFaNsL`7fl7<78yRz3aTF6hA%V{Fxf12Enk=Gdo9~9_102hn(b=?Qz8_k9vi5 zulC@`MBBi1iz-n4vitROD?fks`kDB% z*Lv-ik4TdBFV~PW9_3X{`3V_{n|27QBu_}AH*w+lgv^V}4H-o|bke7lt4Rs8r5T#8FyXSkgee~h)`g=UD=l~lR`<;N$ zft40>;LGjSGH^hCpo@BMmBoUR$FfH8AoX6qsg;Em_E0bcv>4`=z(QB+n#q_HQT;_N z)w%nuF|coQxwxZ@3_n7f{Q)?Dl7hKS=$nu8;LWpjS9 z$eWhQYx9J1$Er;YSmHTz{rVAhB%yv`wRLXyxQvMU33TGbr(WZ0Z>^yyszib>Jo#(1 z=IvTJ_rj-4@r5?F@2%1I;(ObbJF42ZP^f^sg;Sv&>t2jG2{&2D#+ZUV1Q@d{e>QwS z|Cj{?dTwP?erJ71Zd9&q*HyHu?Xe>^W0}|g1a+k&-TwU_8cUsTzNu~gUPGx8DbGxO z&;Y%L>IWM0`6Ga-IIaiq*L-{Ufe6=+@Z#5KP}r^cbN+V5c0KNFp8>H?nb-}ya|?Yu zFL>*0##0&axI$&BjTlyu9tYXu#uO@BFUK*TH}Ti!05n!R_{ikNGp<67U0Sgb-sc}b zRvFD$k;KV;Jq|~1`QhDVD@U~A|Epa;m4ox|*7~v6NnITOb@W7Q9?Roa*ONvbh8%>I zi0`_LS)Y*6cgQ>&H)K@1zw#xHg^uAkhm}cuX`$!rqpvyBcgP@$bv5c)uop2Gr~!WA z#TPCg{DWzoiT2zIe`B79M2;nDYca#qBy!#tQ6D3J9d*7bkKy#|bSg{PZKb zSumUX&MtMn8m@%l&m`mgvtH~UQFuRpcKt_`n`Ir=8qQsh?~qb}t9QqAtsYaYy?)fe zV8uV&LV$(Ug%yL=UGtH#cKwrIxws+o{P>k6@}oNs9bLI!x(=g(e$ReVAwx6tR;ED}!v+Y`DH6Eg8(`(8eLtQ2}& z{T@?an=l;*21_izaegrP38T8kUBAeIpOo0?+_~=w9qi7UP}W&B=t0NGXyY8k24@dQ13N2q7ER)vER%; zx8E%yGP8PbP*QYfn7jQ4vp8l}baj>Vb^4ZG^0w%aNyf1C+=n5&I{Ivv*O&gRc+T4Q zJhsZ;Gf`Y)*Dnk`@v2Z}ZEK2a{Lgv)2%EhtcP=`ca$g_%<~(hGJ}+1_^~Xyowu7bRWF%9Y!hXZCdL$itf+;4?3N=VFwT^#v7X z5znJ%z#5hZ4`o6>6zHi%KUb&M8T9w8zW#Oos|IzbFC894I6^po@2;QbJyoykM?O#Q zKh(s*J#|jb`ofFjOw?dKf8{S0II;c9gM2$a)yOMoi9!|kt=#7Eo&9${sv&jOTB~oV zk9=H2>9amR6jwd?5$x^2pS?pQ{}i+A4T%07E-4&hW@7T{{)zuh>Dni0c0GUQQF!HZ z?_JsMwH@)A=b!Rm?0whEu^-6P&yeAPOkO`ahOv|n4(*xXGRy}uN@(6iVw88)5RDIH z`vP0!y9YAmhg&zJ{J+Ga%6*wU?10q>UDB7joqA!l6&>L}MXwJB59Y5TU#R5)UBs{G zUwJ#m_D^r}a3`*O?I##<@hnkt*Gc%!2anueF_E_ZC82yd-=2J)@^tk{{8M2}Vz=@nQoUW1WcRgCKixMp7 zLD<)y)fa~!>Z=DbTJS>8Ieguqr0*5~Vu~JM`;Z_$5Vf6xYI|d(FGSV@86A6M)A!v2 znf+h=5YorkunEsO^(AKvhv*n(93v+^FX3>A`1iS29Oo*7hkQ>=8P(WFf7d8PYpPY| zoHgs!$m@%T2QjM$Njy9K(vqu5*wf#F8Z= zI!`Ul#c{C7yW_xBoP5U^`}re3UbR!J!u0XFu9v4{?9-{IwsVz(H}hcNr#`X(Lr8t} z#FKunFDqYDwaZn;^h#I{@hK-Mq===%XPuLhE&}~whK_jZVUg{Kk0M$ktgmqBIQO^S zi{kZ7zwQ9f>Uv-svb^l-w z%x&oY)K{SOcQ@X6kG%eU)*0t`))$J0V;YkuiVNrXtfFftfnerIeO^E4iRDrII>+%8 zD&M!(Lz%3+E)MD`hhBX=EuV0t#h2(hjmcBTadMNx@q?>vYEEUr@bHK}bAveVzv4Kh z>y)Eo%7F9!EgTL}^}(e%ImN<<__ymjWPbUJdLYB+`Z}O;!Kd`pDkRp+=v(}&YMe&J z#ym-tt!C$4spv`jReWJHze6U0Sjq2F81PM`mpeU}EbegSoufI+6Ca%D`cE}z6*FGmb^P1%z+C5$P=_{)8^MLr)zZ$o%{K zElaDZ<&m?ct9o6P(Nd%xIU+TP&L!tua3UBHjc`7|2rIllV4zpbeGyTgN19RlD^kb# ztGU-IAKV(fa)iBJTq7oZ;ZASP^Zuoo9JIv~qxyG`-a^o%9ot!%>$?1@a_T4htXb$M z`a7M}URCpVyotA-(PYSVF5Pimcd^GqR!a7M_copLV{;^x*ZG+reb#rD$%7wotub0Y zuwA~G6rl=lZe&PlxEH*LOLlO;fDg@#eJ% zhgcppYrpf0X6l_CG#mVy0jbFTuh}zuBJ_H?`D`-E!F`J19b7r?=wmU_cfQ?w)aV?c z?@r&>pT7P1N$$`3u5orT{Z5o^>@0lzwD)S<=kb+;PhCHW=ePF6NRxO8!E>e7)>Lg_ zsFLPr1^oV;H z8KQe=Odc{kp|@FMZtukR4th(xC|@S%=gAfvlgWFBOD7II>8A%A zMV86=!~^ZS=)J(b%Yi9^lRNqYJd(w{dm!WV`U0l7hX3~xmRoNYZqKeuP+bhf)T-; zqnz;T0~vW06Me>GIr5E) z0pthc0K^eKj~`m%>5(=m1dmKE@R%!#!>ei)>fGs5iV|*KeBg6ljeE~+UfCbqmAW}b z)8+X%dhJ;Y|BlWk+bTx$_=^cfbSI6;7Z=pCLS8SZ_I@^3Fn83LygTO+5ugv_Yq{rv zwol4BZ5`ZmxEO{<#+AH$EwT#1 zb^ZQAHj=d+*B+@Qoo{FQt(9}cyLMRFcWZiN|93a{O87#(2gZ(<1Bm-^lso73QCz*6 zKQZ)I=ld?6^xTD{nw+kFhgH|;pR{b7dwpd*eevUSiaHaAm%N+Pm2_SuXPuCF4o5;+ z_;@I?faN~~L=ue4kNpo19X#fDacabI9XJsW=P+*08m&;bH(0%I7(Mm>)C)aKz4G)x z?5ee0x)MU>CCfBK^y$;AL+vU>%& zGc_VVf2glaf6Rifd$?!gj5y@3ILFOA^FL;WCm!A-xlcH7e{7_d_d)K?+j$d5M-+GR z!HAnfLZ*?td&A6n;+d(Yo-!!u-?O5XPkiC3&QopY@Go#r*UeMEc+48)T{k&kaPM$1 zMQ|cJSeR}%m4Ci|+=JRg>M-9mNPHNde8|7^eaD-4K9KqIfAfRBvrm}bICYL{tXjH3 zSIzUr&0eA(V$jcJ73^Q14P1(Tqpf`Q5W|wyS`U51-Vc9rQg7_=^+4ti@k4$8Iv&W- z3vVIiB7eHpxR&OUIU_jdcYio8WaB2I9=(YJk4SIa57~$TK9o>U0 zUYh$DT=CR3i~8d9@oYA@qOu@F>2M>05ykOk@*H~_qj|#b^6%=SdB@{;!{bbR>IokH z=$k(oyR}gh12!P#e*vDqg>&*p>40`z=hNC$|={ZrQvEKuboU* z7b`7!F893^1{V%dC>RlpSPoa7@{?0soR-q2e*G`>QeSy|iOQLcrkj(Kn-zTb5RLqe zEU)AGIE>?4rksKI^&t&eW8OA z4s&{BF5JDUzq39byq`$h_u_JbgWulHN)%}&Pxpef#-lUm2;8^9$FSispB%@1+&s^A zU*LEBp9(6U5&fBcJ#$};-Geb4B6&Ic7@UfZDF;6D@WGjPECl>+|A;$%;lDcnn$!L1 z+{;gBfHCK@rRoSh&QI}HaSiKrf6Ysc?d(_B-NKcKx3=H4jPbqmfFlgQ%|p0Ir~78@ zzB}_g!cq)s6Hz&b92_u3y_5&z*zkE)_l*;W89*#})vSFSgCpf{f49CvCjQIPUIN%0 z1`G5);AcO7{G)y>bnMQU-R;!jgEk8Z(*v3RYCqIhraxwx^`>9pv4)t0yUu@17w@$y zPYYtJjoGX6aPBbSKh-kp)SSzxs?Q$goL=x!ghQMh<_EqEG6e9-2mO-d^zef4#aMp# z+y6cO-k-Ke|C+jebp3qTA3n|}Cku_ih#y)f&ArfVp9Qj4HeR2InSK;!on2$7eR~G9 zuMOwm0%?zz|FR%pa`K;=0R1*%4^6E`#J69u(f z6E`k!O(qY{%;N&i{D~*rG1vL{$>)yao?JWPHMd99?u_%)TQT4XbuN8fV=IL9L(Nh< zsxh8K>Net|u7(df5c2&mtcBElrP=U^AVjHz@zO|G;+b4ZyMBBREp%_6{IrnwF*S*f z!FSBZ;L^Nanwg{Z?MV3>NkUkg(az1jx;E>IJ!M|`J!xL$>A`&&gAv_lawaCcIJK5X z`}D|l8iU}^c}$*jqw#iVj%vf854>?`okq>BJbKu8pQ_f~;qv4=2BZ_`>H6V;j5v4^ z>8)a|6Vp)VpB9~XOF+*B)OxWzwa00*X4F(P&Rp_YYn~bnvCtSa@+--JDY4Q8Cc^n;@Ur zq&|72)f&Ib)U4T_rI*TSeQFD@-2L`{*27r`Kd-0HKjjsjF4Nhw^EH08upNrB`~E)9 zebPR$KPVFZwRcn}7>nfnLq+esgL!T~kl}#~+oJ>JUYr-g;Ryfshd=&f?AdsF7q3j< zwA3Q|f+|y*It7WUH4Os`5?TvLdIFpcju3<#&+Lr3m9>?Kr7!-9U zhN0ZQ$5$59&TJWeIpTTr6v@Z_xvPG;(!GHCphq8(1XlzjqJg9N#3pCrXo=wZElYp# zuj4yp;%CTc@f)M&`N1Mi9G(CAlOOnW4RC)HDVC?Btvabqy@-*?;?+TYuoNj?S|ZJ` z60z2aoBtPC^3uu2d;L%2zbtL7VVw$HzJ4yq3nProaaGz-)yZ*PnB~0$eUG3ZnfaS zd`973*T+@26^{8>j9CY~^EFCO&I<|SKESVu)s$;Vc*QkSnAcC-8~TBQJ6!t&b#KP>sO^KNu@>i4=3IlH+4k8-G>AFY z0oqOX(e2T6`fUEbkMa@Exlh{;XYLQW0cZX$PM>>yWb^OZYj*FKfAv~U(@(fmyw@Y0 z`*qfJAKv%UD*&(LJmDv?`DV`fno@?(aXnhe}Bw?Ng&!=WpXXWb)Tu|1y54Z%}xl+n1_ga8zTz z{(F9hOkCTsQ!^Q?Y+@ZmiF`?HC4|i%>g%}ceyZ_4$=t9S&diUkKev}M?hA(~4mc6r zC((W2_`N>vaX03;PV9S*^YmcO?jb7A_7EL&91kKqbmWQP z%;AwIlD~T(Bkks`HyiCIx=9)Vs@kOfra#_FQ?7RH1iO2|O;4wE! z4t4zQu&3(>J-%KapNh5CJ>krULx`ovgZqZO;cEQJf`_>8w9ajBR42+_Nfb(p@-|sh zKT_Sp>1M(ho_5Sm1@%v*X2o;1kFWe|hKdzy%~s$A2Q_)-is%)ce?51^E{Eyk9?cT_ zWa58jRkx6L4`k4*hL-lBgU%GA)?}x?IoJBa*azsTh!1)C2bWfyeP4?S8lsw)9*>H( z-W(al0!L#oqT|kU?#w%;F|OC+wW?ULJzS<0d|i(3*4rnE?7`_XS>35*Vwa+0+Pzuf zO5De3yIl>aBLrQuRwjZ*ISYYnYV4U-E1P>@*&w_gkJe~ZI*nFg4YlktdrTME4~3I@0M> zu1XPR&17!+9%Pww)|K%9*LTS1fy~c;@$>Z$ryPmW{7hBjFZ|q(IjKzE--Q1`Kh$@z z@@c%j5C{)_0iqvBISc2TefRZ)e(m)?tpBpKYz+NuNR4E#FJ$-v=$*r%HS$+_aSsk> z@47P0J>f)ZtqbQX^^g*=Rb-F*2@Nj(+d6z9DCS%8!^ufwUySOLFGO|x?#lbNxP5XH zT#U6};ZHB}#P;HJzOUcB`w_?gNqwz1eTwmo~2iZYyaK(&uy6d_D=I@j{V0` zXkBD(3s?2sgE)Kg{`?gEkc|7sEV&=Zj9+`5UkItR^!ls)M;{rPbXwlLVwm*J^6Lf_{I@%GTD@g5~3 zrhJpCnxNjic2Z?KpnnMTeI282LRIeiJz}4=sz)Vv?vVi)0^d7Ob|bd~SgyP!W|IAc zs+q0#mae?WD5+Nv>-NdJal6eCb;&PzPQ4y`6kbOu0R?^J6KKJC{S2jiPp?rk3!j{` z+KGuy|4Z72QOLKreR{Mt=~JD}{^lG_yYxud`Tj3;;puZy%PN4X;p>95o|0lYUJR}E z0>5`_-F37oxe;bbJ|XasJYDg4pErhvdPy8FuimU4ot;qe z%)XzM-|;{uMD^ZVpWo6s4lM8S+cW#DF2BVMGpGL;UqRP4#9E>}qXN|Tz{XukSmA3s zHL*nxWS)NQwb34j!!COec|MS#hbNE(gW3Dz>Q1GK^_G84?_OUSQ*LW?4QsOY-_HLF46+xUztuoiq`!R7F?(1% z&Venkg!FomB4ULP0TMw5E5icS<=FS5n7G`gh0VF0M!B+RE_t@!kZsWx0 zhdlFktBmmGsA|pfp0;+bLjHFZJ3ly5cYSuPvq$Coj$15hmruL(=;SrOM*plINtd4; zRBNhA>udDNF`lY9cRgdaL+6j~fb;y3$DI!~v$`g^kApeRUwBs!Dz8q0n4Gnme0=D; zw_3H=OQJSOs{7ypJqfwhAzPo{Qoqg*_;?UYhq`C)aQQU{_ej!^zTPTh?Sa^oM)_GvCcF}FGUgxax0tuqH^Zawg{dfuO{JYzK_u9_RK7D_R z1wQjS?}g>J^RMfdBLG(sJ#k~|TYnX`@$32#PcQK{*L?VINs%gC@-?~Jet9i8BUhj9 zkHWyWN#wk4ij0Sb^JHH?cKwTU=Yf|Zb>V@LXYP1m&B#%#k`pUCdy!|ndKBDF-LDYm z7qk1Fqtiosr84QEV;djvxn^>(%wHWVY7*(EoP8aPm?j>pr|rk*j3JbjdX$6y?nX~0 zGcmOgHCAoCd~uxol~a4F&gXiFss5dZk0Rq;FZ&iCB(iq?QFF%f!Ku-oEsL{yX}vFzAhCw9vzq4>Yw{xIW+65qgfBX&i5$2*B7AQZ2t|$S6g_)FLxfu{GUJn`8tWz z>AHS`QeXd!PGTqN98Ko>SGaF_Aft-)FT*iphpq=Qe1V;uXOl&HH>H9>7#}_uKe_uJ+Jmy5am@7VUox^zQnd3o(|0-tp z+n~1p75@`#+ue}CCY1YfG!A#9r}T0Dss53dK%wj26SbT__f#n z_j(|+y>#E>6@BqQTAsOh5pof=EkN5#c-}{~g)6%PT(hWQ@bgc`Q0e+jO*t$%Za?rq z=C8(g$mn)1MalV?Sv`7f`5m5F*OD7;9zDxvdTD#{ON`0N?&&gdeWg~0FDT;;RoelN zh!2sTK8bh`@yHncYsqycK--xNdLnr-ecZ}(e|f&gUB5=8EinYLsH5V8u+W*;;T|cvbdd{dqD=}-TBe-;@2#ilZ#Y_ARZ~O-%QH>Hj&-NPRW5z914$FLCf7g5g*`HOpfH zu~N6k1Pa@Gda8|Lz*|<^OMERHzk1&ew)J0(Yyae*BHoUBAK(>h4X*7!@p5UrUEwL? zW4O(HHFBkRu3p%rEZV#42bnuwJ2`74cdgK0HJ&L}{)2$7oAv(Vj5)?vQ7@Itk{OrZ z%D@fCIGVA~{pTLl1DQO9f{kZ$OLNp*pFec|>ANfH(ywy>J<)wM*24J@=d4;BU;Fm3 zrR7ts>BHPT7$5re{!jmKrrzs|GjZi6$9Z5x*y$5V?wyTG{eGE!e5(KL?|!%b%ToQ8 zCF<8kgi9Kg+x~1)J1Nf)zHa&U$K!!aG`vm>eL@7+=@ z6yiF#;ELcxao*wh5aBa-FLFORdXYB;oT|ES=+&BwJ`(hP+Q)Al?04HrDMdmUTzkgeRX(_|C!X0~_V3Nd$Wp*X6Wt2rsz{eC6-` zO&%B#oanVD$FWbwKUP0$;HUguzwn*+iYwmsc_DiFKqfqn;}_S^qvvXH-(1(cLY&3X z1KMxj75>d0HV#mlezkjE=+m9697ysk}K z!WySaMep-OKAlYmDxohtvrc{ZI>RZ9rz6f4Pyek+QSXaW6yyWzr3a% zA0pS^+;8Ahe^x8j6GvPyB6`IdU#}mz!qfPZ1%J4o>UkjZi}h=-&j&Jcqswyc_lUJ` zSE2Y0z4etqB=WSQgSgNn(o`=sI-8ZX|4`pM96NE};ddPL^l-;fE*Hpmh2Z=Th~Lu| zhh-nw5BjE#sO$pIdEV_NC$V)}_=6Gg@Fh+kaddGCuZMSwc_VN?eC~R=kwSlr)8{K5 zrN!a>&FHa-7Xa=~x$Dt+uW<3eiSGSB+t|13t~a*9dF1Z+mmKG>_KFF4AanUIOKA{i zE(O;I4U;Bg&Z*1w%8iz}&YrR=rR*tzR!E}zR$Omy;_3P?{5%h@4`k$4&QyJ*R`LsVtGqP zPqVsvs*HH5&-Dwg>#?wMsM@(a%iQNmH%23Bg$pqNsJ!qpb>aNQeg};AEf;D4!C(>E* zSZ3C}#F;oc=V=U&c;dY!!s`XI9Gy>%+~WWb*ALulLPeO zq}LLR`nwFS446;xl|uEkz+n$_UV|6w>uY?aOm7SqK%MCft@1u|$5Z79ir${Zg}c?PbcFXGXI^7R$eeJ$7BjoGF5lJ_{K0;?%ma9!-x%syF%k_CMAS_5I~~Ad^i~ zfVhP6>wV>Jv7J1S;ag_*Kkcp?63v{lZiHWZ{cHOU8RtIf=@Fc{Yn6M4qs!>zX><_( zG@Xk-)BpSb-=(6+A*Yf-MD?HaX0gu3b>iUG; zYF-ZEL$|juaaDnubcKbNcpWSp=G*gB|8rXR-^)6G!ZQ!ykHBc9A^r}orjXiphCniX zxaFB-TeRG~e!y)~4Kkxc8RdEwd$-8e3SKKO7OJQ^=vb-IFZ}AIhph*Ycez_9LNYs{ zJ3qL=#%N5n%rIMuRYdPjDh&fXfDF+%JE_&1iGck^pBKK?fEo0YD0aS@3S1e7%LHmW z&zgM;1>)PX5Xm(9g(8G19b)UuX+V3`%yM>9F>1(CNd7s7uABJ^^e_gK9&K2;d;d5PkiJA&;q%8vxEb8nn@T>F#$vhXs#;f9# zVL}MsVv$eLkuYgWq~Slvk*i*9ig)Ab5(2CESo&y4s8n6@TE$y+!$^UDJ`&Hwhk?68h%&`x+X7L}`OrxU)^{SLRCsj|CHIC8HV>G{Z z4DR*6EgmFT%I!4st0IKE7}C;jOeMj`YDFdf69$BoOp&f>3eQ3JpD0v}quFQB~X9#1|F`>+1t_<{yB4ikfJPhxGlp0e$hk#(vcka_upCaF{l0 z;w{{@ZT@35=)wBnL-;j>$ma)89}-XklH7y$y&>auTP=*u#g=T&f|+Fc(H z76^22{E3o8Ec&xjp5CgYEQT{Kvgi|U-F`#S5KBr&vbjJnv#bW;qOD)1lAuQY_UBkz z=l?>zMb5{MS3a90KIN+{c3CbKICIl{_&ZS`*ad~JgccqdMvGYIuX*1K3oPUqU9eZ0 zF@xofyDaO}6qonoqjJ$v#^Xx01I$lcaC-&uW6Z*{U@OOpKcH8VJ#3iqb;A*0_`beH50$n4wl5sRjKxcBQM2cVMj1swBm*4|=Jl#e^} z@?AFMc#O3OiOioEB-gXcl-QCdJ3!0gMBgY^ynk+W$jY}%S0eO53#h!FA2_flMX)A( zNT%I%{8oJ#{V37R%D$`~+?@)euXMakPl0BU0htf*=iq=@q zwKdVtpQe9Z1vo84#-Z*fT+r`~yhDfjiqaozUW2cOEbSQZbP(Y;V#Pp%;T&&k%bnE= z9Zr!qxP!knR||J_onHL0fuNb4e05qy%L@x-=XS|RIEl~RgmD|Du;D*lo3q2?K_SP|fG01VV0f(w6bKJ5$^iB9YJ6!aa{jvNn|Gg)vOug9Ux1hp4i4bfb~ zPPwZgSsz4J+PcNjo5hj*2XtvxJ7+ZJ*K%0tUgr=on_BhkF`Yz-8^=vA@jrq>o1y16RGk zoyF9h?)=1UH}Guf+y#}!lF(eDD*ZW@G32RT9g#I~FoG8q2oP=juK4%_#A*x00D7up z1^(K-4BmuCZr@Tlk7Ce%Ai)Z*@)WK=??)QGr@$ zA9JUv9oZgL+2Pl<*ALIh3Pt^Kc5JIbjo&*f1Il+9Qz>|4&R)@!J7e)+XE&+wJ^NNl z)O3@=xIBV^w-(<|ap=qtdEW5H)Fmy79;BsPCoHVqnH_m3DOeQ>hZcjutC}~kdnyA3 z1~|R9eq6n^^}`SF_i9jE1B|iG`W7=TG!NPgKcr?F+96ddl14UFIley=Q74Cb$6)m1 zQhy*%FPHA)r(HAH5$~z%l$g+VfNg!;LM`<8<vTCaKyiIbnH66t=yPe!b=g=1}_j6@Vafcpr~=kV7nxK;JOv z^z@*=W!n9k0G?J%b{DMi;gwDYM<*KRqbmU%?E1%&$;aVd3g3w*+XIJ>dk`PPaJ-R< zhzx7z?+zAYcT3R*X>`Q>ee~%|Z@VGxc-?suLFQHOcRjKNviB=h<|lgYzcB|>9SF3T z9=Hg(-JC0Yb>af_O5{Sb4rb!pf38#HV>i9hA?6d4?IZvqMrcvl2qfZ|UmRK>CDiKC zka=o@rx8KJ`&XVJ+2KczL*frc82;uuhutpK*|k7OBu@L@pT>r?Y-xz8{t=7lAh%aZ%990OspY=Yq|?Ba+?gm1<)46Nl*J-Xm= z%qp)cNLk*hFhv{Zi4U%yTBzO#7Y5ytsdja^y znFl-%fLUX)iNzxAN5`3RI@Jb?qm~X33%B3bJO)h1!xLBuZuc?0`mIW}`H3*FvS@OF z+d^)m!{R8H@HOP|l>uS?GYxR(?vRDWTh+^m(~n@J#Wu8#_yh^~V0+TyfF%GT1L;qc z9U_T@3TDgZd;n33sLBGRuC(8#ugE=rTiQHyWtrKNL5WB5~+kyyp^S`L{4Mo?ZLsx8+IAD`3 zSg4#w=1c4-_9L`zik+Y<7@ja&Mp2&vXT|!RJ1W$$0`}W}x(sY*JmDLDeL66v5k=j5 zMlkh!E0lk5{+ab!>$V}#sFYL7o&I-)JW_ogLra8SUFRdxN8faZ}b333Zmg9@6LWYS(1KKPerX=<{1~0>WFL zWeQVRZ+<(e(`JkaQ3I*{!h=vC^E`hge z_QsX9SBc8>ZNG3DT!D<+`gtpI5u}OV5Kz)e>}JAIGyiU%*VT$Dbp`^1*uxK(c=}bf zY+`m=N<|KnWj?Kuh&~bSY*L4rIFln0+T)HXV(MzqKJy8@kgk%My7dDYWD$6vH@1>G z6(tGKZ+NYa9jjA#6?}qqSe-gzK4auh5MG)da9NflTxA z#>uA&B5j0007|#YPQnmy>GSD=BKsm-EUn8@6@*vuIBf@5ee!{){&eJxTi1Bx#L_1 zXz!^gP7ypOHH%K|ht=8CdGo9Nc&#ql}f&DAn_){hxHN5`x%t_*VkdFrJo&zCxWyf`*eP+8x zZ29EZ&%Bsfy6B`AzJLW#j<}emW1bc{u_t6KCgr3qJ>;IZ@zdkc7}i2?dO&_2pl+0- zzOtd({=!JZlr2S4?4d{R!Z_6c_j0STs-PF*u%)wZuD!lIZQO&C+Ip`ceX{VGK`Xe; z@)IM{?&Z4{#pOvCE0h!H67u^4_7DCvOU;4CEICr%ep;N2s-_11>D|4kaXbbpmc5%< z05-@6arQzchjBjVoIdwl!ug{$g+{K9>fc6^2TO`t@?fV1^EF;`KNi;@k)VXe&{&KnfTW5nDq-E?SViJ725HqoL&=bJ~Ju{;r@nfXpI{uz%IEk>X%-@tJJA^bjF zGGn^$VEjim?se9i3n=B3@^aQYfT_tq-NW>OAo+qX4La$EGb$ECO}x}^8QT;=_f+?V zU$iUqa$);PuY{0RF-H;TI-Gx55>!|2ZFdQm6lldX;A59N{HOlfe3+{T#sCm8)dO1@XqkLDnVsFQ^Mj5|W z_Wgh}E=xoV)Gq4NNp#!}c=1KlqG9`cqtD}fVDefzF=znPZl@VF{bcc}U?=Q$Z%FH^ zPp1&1+B%F(=hZv9-uYkr0=340`s{Z3-bar6Vzcz9r;}jdVA~5|{jb&{Rr@85ck<ujBs^%D=zFLzi+`-)n;kPr02sZ2n{XVvRMqEJ`rZl$c7E~>)@e$8|`@fsi2 z^5rf@Vzxo^U6jb2<3d^+_r6`;CQu(4Jo*Mz2L3rezaiQ9H{$wJ*Nq7EqlnsnSMdnQ z%#v<02as~)7c}d1n<0=v{{BNlx3qT>BE{JRop!dWLc)+tikhfM z(v#9g3}r!&o!x9t({0e{9Nn8wMcgG+6>g>#pT^%>hMEgV5Mr6-$#+Sw2uWvqqwNF; z>q$H7I((`;QJY%evF&EwFgK6SDyqM_6?^_86ScV(V|heQ=Hjh>@jUxeJ9-rj&$5=# zbh9_6EF22VR#PPdCUc}z=ODZ^KhA+E%-)=p-@Udh>&2-}LDwm0poP&kqzEN^3px@!%Q^VHTj=lMfW$_zFk$>BB z#Pk1M%bE)GEj_yOq{D8O_o|DLYr;~dz+9+N!1KV*v2j}#d}l?^=dS9{a^{N17o)xZ zIIiIT<~`le>#Az4K<)o~4(;1{_u_=tiP@uVXBIE&=5{joYRS7Zl3n^>QpJTE=v zTQ_z-VPRhMK3}I#ZKxF8gOs&V#t5BUDmJB>Sam(l7dPBx4r3`sL_k!53uBUh`d)k+ zycaRej|n9Rb3$m{D$xxy&oz|xm)$gDbk0EXeN&@#d)#GCX&>KU3h#&(cPLlb@g#HWAM7=E~2{nDw&>GuRxamP;PpJDjk zK(uM&-sHMq-Kz?8l(%h*{`V-KJYj`EuuO2c?)qS1tt1Q}GywsmKXR$0O6Ec?JE3uu zrLdippj9tWtWPIG-(C&2r+p?r_o%())zSH|d%eX2{A4>0fz`HuA=>n{j8<`I@HxvqYs^S7&j>2;Ab!k@u zDdX9SXO}6wXYHO>)@Xc5Y%5JR=X2W$e^ANGQv0ylwE5>?VNfGYu0y=iMN+Ew_n)eQ zTr?UWak{s*`N? zqNr}CUf?>jt)zd zYjN>k`O9}%b5y%yOSg>etA`1bZn2D%OW$vo*StC*vtS0V?-|!waDQ3v0;t_)(=;Fd zhCsiajSZ~D6-E-)GVdAbbHC9`KCJfGJbh@7e>=y7p|?vFN>%-Eae2#!b`zBHUHeZ4 zzhHP1n;U<>(>jSU1P|SQ=qZTtiF^?6d~sd;OKs1*#e~Q^7x6MLtJ>uH#bvHWx8AlY zuf2_bnc~Bh*Ep8}vaNQdkDu!UJc< z2108;Uf$!`2_8)ZlGTme*-sQv0%1B)fA26OgI~3X`g6H3+tRNLwbMY0gJ-*hY zT`UEc z4f@F}R*O*muAPnCRrAHGMsq<3!F5MtG{9d+<1yQr2U`=%Q%~#S(n(~KPS0g-rr?mT zj1dOTsy6;O&A~$F-M#9qkNQLNyFe)0h3`%t{M*qvZDGy$#oKAJXv}1!y29?%PlR1uqXL%I|+sZz!f57 zQ7)H5ekyRy95{XW-g3&GBP^(G#4Mwa0HhGv#NQr%IH~(R(A~T!gvv+qu_t`EiVuy$ zTz98Cji@@$3y4SN?H=QsUk4UFvF1@$GgJMSODPvz-%Cv4~1UF42lR@HQ*>T@{IU!t#r z*uGB9qK5Kva^v^Dh;aOF`&^kS`zy}HLXhv!@6NiBLa5w+UD>vzX2bV-Pb-)3y!1Vc z)lts2Wu1{~h9~A){_yGRX{`#anNGoKr=xo<3Qi{%2wOxEZA<7S_sC%F2F+cp4~wqm zTUV6M9YMBY@;<9aX!pjnPr3!c>F7em#HJqF)H-K>u3=a0$0pakpku9ck5kHiqXU1A z=#+m{=4&IJXYb3b`XY+ zx^`d1k>iB>*UXDkT15vi2@m_ZJ``(4KvQe^HadL1*Q|mcxSV_#5_loum^{x*Z_8je zf2F#I(688|magNl^>i@mU|AMbdX8)-JG7#9&Evm-tBvpTBiWXT=5}WzStYcOnqOb} zy2#gF=h;LgMK>D+noe-;UwI@F+C<^#gCm^^lS(alpfm4(Fvn^yl{LUWuG+{ntKAO^ zDAjodv@QucjTL@?Oog`)jdVZZWc<2MpT=y?NINvP>zK#cuGZWDK;}x(7BK6zFe8F@ zcmOa83|ie0hn%x*Knz*G-R1uhAx$2&0zNaD#s#`w?ShY`cdTfnCjUbsp5^)OM6H!opiwF#;4P^z_3~8GT+2~4r=Rx9H@@e@ z1|aaeg4Q4g~AR6o&V zGR=eUe2NZ9xnSkuQ&dfvXp|(-kDAA+OFR$`t^S=@`a4V8k<)*Yf1We4*8`>@>5-PR zw8%v_k6Zy`ZB0-A8DD*Q6~^YFABLTjI=>P>)v9@@g0{t1G3#BnjveW^{sfPYu@ktc z1NbWJ;94f_<|{WZzKd8{NV123L&u(z)qfUv%krAi))J-J9Nd7M)I~X8m*bfNhyC0W z6{vYtC18JQwnIZr>lZthqVq~5tD9hW@QRVUDiEjGpjnD1vn)R}E+x^mnH@uZr>yc0 zk*pIn#H7}g;Rp_1C9B+_A#N@b4xTojlI#&q4dMQKH|_`NSpACRfcA@)w#qC9IvYNU zc8!HD`13sxE=?|HygUm?R-^&F-XP!X)*A#%7{`m>C+AtFS(99g9;$N1eL_a?xXX6` zw7VY!ZrE};`e58EB)RFp`t7Ek(9>j{DPPNa3FB}n$hE577QxGil?M$A0Y-%$sT5*7 z|2rz+(cwQYe&tF7>fj>O>79#7C$)7ar}M8QmrMM=h6BYPwu9u^{VuhCU&}Ah)#cW} zMoCN6S4OyyqiNwTT08N5D8jcx7u1EHNd{qN~Ilto>`j0Lpmfc!{U zv?tS~4kN)Z1uib&wn{wTq)9k~-E6%Pc%_qwYovRevLz;{NM^^kJ6&MT&DQ?l<4gt~ zLWGKE<(5WxNgG+?ITlJf`a#iW{vVG-DrYZJ=nug zq+Ahu=Xh0N4?}4_@MrV~Wd;N~mFfOcFzeolLF$hm_xL-zZ#h|l*BZWfSv+j36?lEk zsUOqV>EU{>0Ho;8?Z!h~LSa(?RTqxxL zJD@LZE{^&dj@mOC&$xzM;zcuNG>Ri#C->l$^?y$q)^%mx2Z#Mg)e>RN^*LLZ`|0Ur zjBr|+5uNh>!goPr7Jm|HK_vg>=quBL%O#8Jp(~spk?cvI z)!E9_jowYQyE&fxH#-u)Z(jGC5p`&jbHPMtJc=4qmd!0Eb;fL4R~P(g)=SSSF{om7 z)XN22=#MpZ*$|EO4nE!zm)!ovX(cmKFc@o&C z6k@m~r2&f=Gq(cwKAx{am6e*kko^;Wf?RWN`l-OS@1?H-rB#j9U+%ZoAVK2#QHLw`! zCDv(uNh4BQACRokTGoln^}Se;*{?KTN`u?}FXna|Ey*2zr()7Z+$3{jzu^(=yU;e3 z(pfz>{+Ps8J$L_w~wwppKa(U}vEKV@G2KBH&*Aj}b(tLW{*Lg#! z_pwh=y`)s3{TXJ{Y##SQL!LEd(oLepBk`ZF`i@7p<>e&2 zrox25h*}X(`D|32DPryh=}p4A$~(FdV}6EV9KngO(^X7k!h1=3{Fu zW@AxwvCT|Q_NvCoMGGbA?uc$D6y+x)U`4d;;yI#HXRD}rN(Bae8*@tPdJox?cl+hS zD%9?ZZ#J7#NvW4=J7*5ItHqlzRY0`XC%NZXJ5MU*0f%Q6gliL{? z8{Zu`!T->?JrVV~O(oL)GmKai{^iLefrGTIXm)3{)mLFfe#*y#Yddbwr0QkV5;OND ztY$)4RxO>`IO7>ixPrcWOsp}=t)xtzp%o^K7W`|Qo7R--VJ<;oz$Dme4A3{IZ)Nku z)cjkG$=U^q zA{S>x)XPh_p32CiIeH+y-uvgMXTE3MG&gp}_NF1Be?V!qEN9GQ;bES(#Pel4JE<@o zTmssP{5(z?@BfKd=f*qXI{6i7;P%_m3PQgCreh6cO6Xup4M z1RIIKeF?=I{Q4t;+Z1AQUURGC_K6-RY?wu$QsHyAC9#Y&LcAS|H&J9yWQB{=@6lXd ztda#>Z~o^r^!(kGUrNJ+85w(&Y#rlnkz?FROT4oiR35(~c4&8svKV9486Ufe97+1w zF_ zYr_Cd5&4HdZlUzre+7rB{Y7^=I~-6088vU+9}mTi^I+m8@nbE>jvVF9)*r=cqmOZy zxRKtnm^j@Z{~gDUm3K2gA;2SO{k`=+nZ(ouACh0%c=P6zS*YX}u$g7ZuPOq%9g2O2 zm2Z)L%nmH{Tt5wuX$uI}S%{9ehw$6k|GD18+4rzB?pR+%!d-S+$pXBsqFhi8j6SRc zpCTlTaOP7S@623B18B#OB^qCjp2_<`R58y;tXiPF0EQcpai;?i28U^{aZzD^SC-bG zGQl!X8bDw#J&9Js30}p7PIR4rMV#w6hmk~*Nh@%}LTW9YgiLi^>+%fwrM~shd+iDKQNE8Azv z6;+pU(bKVKn@}8wuQzFw>uZoJ-i;ooRV_G$LAjnZWC`lf^1Y52iW`3|4dAS5^!&L1 z?BWlaVS&fx9Z~81Z~Y_3<|$)I`FwH~=d$k~ME4{!pIy*q7{eOWRerV zOYgi3{|P;zQ5Qc@3=8;=xKjn2lB@C`zDrASY#xIC5|j3&)ch>as*aX4ZZ}(eAU9Q8 zZP{N?&}nH8RQ>LeBJ~H&d6gm@&FuJdv-tT@DI4P9`Irfc_6I(3`{z&(mDuD2u_9vQHqt!0rGc((c(xnRu>nprR$xz+XVnO5F9M<;~t zAmtAUt&)wF<&WP}OxCN$w9Ed~3#R(^AQnc&JuT&P*y|Q+B$+>fqhsFT8y#2;<0sqp zqSN^9XOwO8-D>59L0z|!L8lhI6BSjgZWhVPv=*;31y9GIdNg$63?^?gQOq2{@gBD2 zttW~tbL6XkbLJ_eB}M$WSknC-Js z@=*?+$~}@FsV@Ef>hkHjA^~k`zbmM!@ShIgdXB}G;p4EaR!tTO$T{((cHx$pkf3I% zDCX|?k@9$mLk2B)AdYJ$%W)>U?jJ*1wJl`r?P)je@mj$(*7YA~>$z~fxFCqxRw=}6 z*F<(^z;W-df4}ld>DOyEDVvSYsOfi2Vpk;|Wk1Th(+UeRnQ%GjIDX(u*_M2@Z<*{d zhaO{|$w9?>K_P;SCKW49no%Z=OSfJs?Qc%{^oot_bjwp&H!P%9$ma76hYuq+7teSG zg;8jY=6F#`{PB-l38QU!D`~cWXlJ#Gd|X_7FHic$9;PzO55J1{0iUeVNeLY-S785H zfJ~}I`s?x*ZbkAPKa?3jMG-kJjWKxy!|%xRl_P(^rol}-LsyLWeRu4o>j3>cwfv&( zTY{N+Np+GI=CZ}x3!|w&k7|{@1hK898Y+v3!t3n;x8IF>2Q~+))lm?e7OF46?|05g zWskoDFkzqBwrF8WpOXJ~FC!lv7g&`+Co|v@SXJ*+1OMIduq=#9`hNfM&lTps^$%?t zX!wl+V@^|d_2j0X)tB7{;mhVQiWk7KUn9q%}Rca&7|$yks=~6B0GcD zXrTKCP|obW@vm1iC=FR$v0YOTK(hS4#r=0_iOQ|LzAk)8D+Z`vL2_P>wZ?G(-TE%V%Ae1K}IR zp^6j$7qOD^a?F#LCKKf9f8Vw^Eiu+fK~q%jJ%+VIJp^MrNA{)HW8=e2oUf~MuFY9{w!QD9~S=nYV*eKn`*>J>VUBRl;Bza*h z3GKXGmn2-=Y2ZQ@r0k4lE(Qk-9Bm6YK^YE}bWlg5kpkmKu$ zE@mt$F_!nD-%S!;^lRVXj)LwK^CJhFMfH;FqfZA7BZ^MW4o?LJ)#X6m5;c=2$VtBU zwHh1KEFPnn+CUDkVd6(Ydb%(B)}6AfT^k&zkJkUT(MZaFwXr2vBs|+&NJm~A0uE76 zrve_75YzSRy#KQGXDQomxeGyZq!wk|@@r@+j{8xe^x)hk{m)+XI?87qtP8~%DKvyO zBE7Cn}(_&GV!D0bCKp@J?8%2wKh+s z(;C^?JbRVUKyEJSAzD!8R#QlO`HOJTg#W`;$+lhUvkXdA)XKH?^KhpaLs-U=m~rm< zR``_eS<`cVOli#Wr?(jBg12h4Hc%<-4J34lRgcFg?EbpF4AS*Gx>7gF>ll1~OLY!h zdA`4|_4H;BlgJKfFLBYC$2U>%4Q6k75~5&bzKChD_Hity92*?jow7`owJ)+}Zqnxe z!&J+5)&5-qbhy3p&FM`sp3d`{ZEHgZg=K2gF~n$Mz1$qfQZe*w$wL)vcl{!~U!<$6 z5_aNemFlbFN$utrvmzID?}q=%8b_bGNPm8W*3KWz7vWyFMIC;yI9w-?7_sN=$D04h zcaCE#j`zxDFL*a-)_y3hzp%I`D!$nrr1mRP%Iy5))Hy8GHk&eh&am_MLGIebGt_!jA5m-Xy<#Ql)^SU7&`T{La>#OKKLqG*$IYPXFaetee%7 z*H0{U-X|M@=8SU;;<@VGsTr&GXAL?!lST&mSVzuUmD8wsm)ozr_(bj?+;G`#$;^;? zlE0@kl@-~Myr>tgS9g_u{uTgVBrXIF(@gP~!pbS|xG-@~9ridpF(`LaJ#^-64t{l) zL(5INB_+0-!o8^+IZY6KAl!1cC7f`|pqYg3?EbYNN`oQ4z)cwWJjl{eN{ z{NOMo!vC2NI=2nXqNfM76LCGb4`t}L`}^7X3J&LrbDLHYppowkCH;Y~qWdeg1i&xC#48}6Q2_0=$qj>X*8Iwi*wXQfWkVhSM zR+z5MvQa9P&Ig^kc0>sIAU;(4+{gq1BVf`8!3Iet;{8<;M^ELui*MVmO1fgQ;$x?7 z*$RUk`&9gspZ~3*yJ5Cuzd)>LeM9x6cYhDvJ2kk%gU6M6{J7`uwD@+sZmO5>=G4Fa zP5jlAQqM(v@-4%`hObho=rjt@dg}wd^rG%j&UtoK&Vlt;0SCF3IS_G$vyxHQ5i4^@ zA6~%8(%!A=%HK*Jh36_oKG^2E{BTu8w;-}&>Sg}@^fD?JlY5eV77p#5W?TUcIg3X-#hD2 zf|;5-_7}GFRR^^k!LKE*yI@?23RmQF*2k(oL-^V4#{C*QqQUk*aQR$T5MgmSF)KiHu3V^d%- z>yiGOaa^1sy%c4{GQV&?D-rq%(57~s8if$3_{or+dau2{A{lfDbyXv^I$CMneVcJyuiW#3HJ_(Xed}sLaABk_ix_{ z0cfwiK;_F~{KXe5RDJy{8*j^_vmUzE+ljWQoS)TuEH_U(8U#hc4f7B)(jLlgbLW8Wn4$xwh2wMhpXD%Taj~uj3$9k8+#{#t6 zoXP5B{`MzT!fLM(4tAKN#@4W9qad+t8mwld&O41=C|%6I(#_RiW$&ax%rQ8D$v==8?73YC|BbBN>p zXD>`9dXphs&`uI??nOB8C>6(oB2?7BbC|-K$&*~ATbS7s=2CEC_icK=OLiF-(3h8^*CB>B|rt`TTTO zC@ExpW4#0pU-=#?DYBDv9rk%O;zk!zbl8L+MYiuCWR8Gsq+;S*@17&%s(jPULpp1N z+QvIJ?Ot?^y}7Megh48veu@y&71`~Q)Dxa%^ggWBLivezCfeLS4+QNR90y+Wh(>13 zj`n1_mRhCtc-UFSXz^;Mc}In>Og)`Hl4Y1Ts(GF9H3sM4iAlW3#wm8(Sm$+ z9X%8utH`j;m6ixw>}eMD=Qbk2Z5_W?dUBPub^;0h)GMqjl@FO{+`BG@9T zrEN~S69FxZSE>mL8uFGUFqauI${#y3!e{_kJx;y5-!Sz}uBmxs>q2pt3EWNWI+_xe zq8u}v4=7cYc>Aob7S;4_SYg-|JExm`ECGTyF?MJ)1#gY5_JAzFm;=ThJ}-~vYE-*+ zswFtN(aZZ!x^I`^O|*@y`Sp_O&h0VkgZhVx)bsR0W;YXR-2BmeXbg^_t1EZ5IF&D< zJUUs*+x6DyDs=^)Tq#E8!Vj1u2)+hFE*oo~_LQh|55Uv~MorD`*13bx_c5wlx?nmxhac&Q?JcXljFue-5d<>DAqffFg_ywZDqSpThn_=P^%qfgOTD5MCLi}-|0_Mg5C zL6AS(m0%0mef(X3P4YfgmL1O$QyG&$EWa0~sf<-N=vuRUV4Ec?v8KhSV8 z1!4@$b80yxUZjmH;^^0VdFs@G3iGCr_>0}LqX?*9&QX8uB0xrxz{6LWt>OPO+2k;5 zt!91oL;$R@tDe{F;u=0JxwCD0-N34pKFAZrv4;6g{h{_k5;WGhm@V@MSBktb|5b4@ zzzKj5tG~U)xU^9U40H40)EgN*rshuXh?k8_?GRbcMY&JTlEqK3g zP#96LWAs(uCd-Y*brB0os)=I`W`+Coi<-#7tTbFt_%P2UH7W&qs7Yb!Q%3KOh5g~# zja4S60(aG&#R%cfvlJs}ZHj8dxQ}f(2gQp^)_oNBb!#VA7UB--2v|+%xECoE1l6!P zTf=nfFOPA!a8G1Uo6YTsssqb8sTCr16mOT}7F6I5n7mYfvNS$HH8^r(-9m1y2WzF~ zdYf4xrr!)B7erHK_pN;U$Vb}E2zpJY;G%Z2?~8_&{`8t%2rla3fkU6`QFPJjM9662 ziT$s|`bhY$*9?6n9(4q%Ga>vjKF|55`@13zvUlamSOL!T->4ljV>I`+3`iSQngn`j zt4da1BPWFBrr5(c`L)bhd|`jD@a}u61x4aEHd}-Zbrc3q!D8O=XEDiqx+WdY6$r}^d`jJ z!W!0AU5g&AC_Lf(Vlr=3O4!oiaBl9B8pbaLvvPz7N4a%1KbcDIYQvKlg@TWUeKgf< z2^2y?!f&*4Py->qWL$6-%sTt1lf%8JI@Wt%*~kiUy@+f^S1p6{55gt-n7jQSx5|96 z;Is<+)f$Eim8i$_Ao0s~>?X>{>FsDWQTr;*hq4~o^>Au7f)zKI3`^2;M1n#@Q zgp1g6-vX~_(@7yW)KVwfl-ho3kMWR@R&#SzaelUd5z z5DBL{JS6=pRiHZH)PTF|M!k}Zj!Xehg{R6qV&XSlolzS!M4o(YC`_HqKKolJ z@5z8VS^eBxU&z9;=Q&X&iNg7c)3x{Cey3F7h7T?V<(T*)vOBFv+_C`P#I@-#H^rR! z+{Q)KG^xGZu{fH&>_^{}n)wdqVSV&?I@7w}17p>Zr$UPqKI)p5u{xWC13RfKmyvpiwuDH%s%X z?FGf(%_!Gg$I)a@S;B{ngeXbUD$ui~!@%%)(azkl&m?HT3HwaGgISnSt*bMlv!JTC zj_2FnsGl~iUWnL@jBWH&_^&gFg(F!>AJ};{Q}SDUeQ_j=HC{ zL!wNx*PkwgFw=SdHHCl=n=t^yexhKf%;Lj2a_|XdrFC193Yv$D)TQ6uF9s>w13yUt zj-hqaM~pI+y7SV#K57RYI=m1|4=^!tFL~B(%IkQk8dsv6z5Ohkw|lvFckce-TO{E; zs$ARjVse=mkT5+ztgJZ`L*b1k$m zA)@W4F}>FqIn8nwwBU-5eRPWz2XxUm6JnpSk-BVB&@=u$AIS6kp*!mGC%SU;K2u`j zaqlLAy(~mO?;1HEoy&Sl)O%`6t)xK2xALct(I4J^^7AF5ydN`6gA9X6(2#Dp^xe9b z0QJMP&XH1;W9@kmoO33qp(eHbuDO*vs(t8>>~vSSQ{vaD6Y8_8KBs+`L=D(%o)-5$ zRp_LK)qUSf z*fyJF(W%~j-qy171ZG;)JvZLv@w6lLZ=mS_c#p66qFXQy_ z6<*f)3Yr~Kyvh&qWj*hXQ-3<4Colft36fQ2eSPYn*&bTqIbGP{d`Eo4mrEtB0GCU! zWEJoHZXVgh2l@3U`+(d;IJG6I&N&u@QceS*hBCRI)!T|g6y9p-TRUB}c&uT)aZ4lO zrv4N0yYU%s&rJ;ojOK=f+Qr1ddm)|R~4_JN@0IVgJ#BIXteX-r7_>V z)Fp(3DHHK|X;A&1%^hpW63bp+)wAB4heaKQVDHID){xwx?uYx=$E{cCZX2)cS|eSa z(He+PO0M`lfk@>Yvu^V39mT+`8cWiGil+9h;5RhywVw5P^R8LVdWDqWeLPhEb~5|5 z@xKn1I@zSe4!X*$ClEb1pWZI4x>#!Edf(nuvb!R6^f?Fvb+@WZTmlk(|yE_zzVxbhb7Wd%p z8r)rqyL-@=o^$VgKk_%pH+%NXnptaR(-m>(^h%5pMKzpl0N82@t*q-=#uY(=t!$7P zUcb#aDx{~c5)48o!*;W7hSPz6Y=0*Rnn87uhlEMlH@i+XV|D8m`|iQzRT>Hu5pM(~ zPeY#OlX6WIs2)U@Sh@xC+o;$_2hQ->{bsTYjwgEWQR|+md0l?54e(wS+-b^KfZCZj z`3JuXkThp)fdKMfaDUd5%yDSF|N4!}t^q*}lNU92Z1_7v z)UE|Wq+wp&g<`9pI#9k`BakNp9e4<-kyMpH(Xl^v%}I)RWX3O7X{-pGzM~z{(L?7w zkUuq0c92Harl3;(HZTrZu;^w-j)PLpH10Q_8tx;LtW4K$_7Lc-IoTxv=vX;x&9M=_ zo8gu!0wn*M{FI_C8GozVS2Si#K(QDV|Ci}6N&Le(%jZd)o%yNHLN~PvsO8lIskZM! zg4Ew1d1lhbi*k^sIR2re1uejgf6>v5o9r_Nd%TgK51Z^U$GzDL^{YhlC5JKQt!FJn zusT}|qoCUMyM9VC;;}&@8RJ)Xq^P!7cwzHW@C=;~8WODhLn-q+u5~i^wTj+u+6FpL z%$o)mPo(-ngI^s#6nC^@MhkmIFT==!eZukdYk#gUEqxeHj;W?Vua5uYgrAMf7W7+9 zURP4Lp2*Zj(^HC7v}TiVb!T|^YPlPcLbn2VDyGvtM-bwa)`c4b<@Q5~~v^|=pj?&06jQY@XtJJTbv|*2Fd{qbie9NclQ$ zu5hf#SpoKXPIcl?(`%FDia2CkQul1)2NufpgL)ouYTz7B-KKh7M#-(9x#sVIoZh|W znF@3N&bM;wbW%U#NO~THVkjVCZNL=TdqW;XWlVf2!m^Ja@oO9=c-!uW-!PJUX;*$? z^w~kT8N=8^WGxxxS&+v=d#=!>kn}ifJr*pN(M}V;eqY?s8?=gy*Zf>NI=KEig+8hZ z^n>frk*lXSyj{$tW`61i5;?2jA~>hFMEk8)U8%uwnWg$ozne>I7^Z>N7j8jHD!&d7 z`?@zrFm?A5jHB{41iO6EBYVtb3l(uGp|&FSM+H=XiM9iU6oHaaC)(r{TPgeZuXo>@ z*WXU_J{bEoIs`yWZ)lqar~&bVEDG%W5kx*_*~RH^P5W+8BOC(oq;0t_g;dp*gdmlo zZ%S)VbWT5*24j9kW6W#@ViIHP&L2jTPz%V!SC+vlMZ>N-{x~MXqT!ne_gkr)M;Epw zaKa6=-r*4%QF`wLQDE9u^shdJX_Ya$eMY1`+1xA@T^(;MV?5wB&Kv!!k5-2b#O}IL z{rBK5QdZuszybn_*2Tx$BlKT4qheP^Bq85N=RA#(IW}vj0Jkv;@T-{f_!tUmz;UVD z=ZbTwc%Hd(A{aeH?UQC3VZ5tCJ+sQoxR~HH`kRA3lFaUrd#EbKPScTDN89`|j_YXt z4(ECCK5h8?8VA=`b}*>!$Wa)*$KzQ9@v2)rhx*W|mC_e&RQq(W0kwGXMhkvW zO$ySpDQg?y1}AXto49{nAk)Lo*WPy?IcsBCXt4s%dd~^;b_YXJgy6t|?2^ayAYH8TDI>56`9U3Ki6uKTtCkclsaH_NWb_DymiKX{mo}2=#{A zJE#%wBbs|-0*LnlM#RMeH$l_JH0R%$*MlcaO3l*+K%p(Y%zv;@_`RP=N3San3#E^< z=}44ZP@fh22aci@gT06+s@h)wedF!XF5*dLLE4OIuiNPmtOOfmHu5!4F}v31-)UU4 zKY<=4g9wo{pbCVbsK_i;y0JUYNRf8Ile=o0mL66jFJF*`pO2GtLN*_^LseK1Va9xR zY+pxLe06TkGaBP;R;!=k%ix!1=BP*F%7NG9r8OHAc2bkLa-j_)_Vj zLdN)9CEveEO_?0Dh9l`lrH?k72;7^pwk)vSSGu=02~!TP+!$m7#Dp(`TGKxL9yxlz zO1IrZF+$eocp?ww=FBhW@HAPe_=(+SfpqMzZ2824rIZ>R6n0rA%;#Z&#;DCgD;-=P zj)HwQA4F1T!m1u%%q_#YB>Yk#CwF*I!nD}%H{ka0a~Q)qQp(Nh^$OyTb__?{T8jb!@r2%91H&SJa4aMNL;uEbmRGMw$17 zG?C1zy1tE!$TNeLaE5g29gm`j$DEpdV_ezwcrpx9-dvvJ@8_L0XW_T0 ziV-DVbi1Jn5V<%kKn=X+s9XD9gfjTC^y*&qyDi4a2{nF1Elr$DE?sFsZg~;RegCk& z4|VA7bN7MGpEycbcZ_<+^AVe<#-F*iMYgRm!@yW|4>N!tiK)HsbC>nyC8c%PZh1tl zh!{4}&zSmd)fno>p$B}K#qjDW#dY|5sc@{p^c8q1LSQY|yv^9%a?Wu zw`ut>^OnVZ`S|AeT2cQ9GDb}aF%NAXLZ=~?e1ggb$Y!g zGOszZwqIM~_AZ&UUz_{L*m-e{1}!`E#@Hq!L_<`i*6R&gDz6DXv(t60C1xM8<*w#v z2-ft)7s2|k=(G7piupDT#ejQ2OqUlY+t%iMewhT$QiY;-hJ1nK_;eUB1wp=cHL2D1 z`A>!R(95!<8Km`g7O|G+A}1xV%zB)kuBzXoI&c{I8KJj0n<6QCm90Q1LZl2W^G}9PC5!f@ zx8Mbjv(C)^UmOC-;bIgQe{!{3!xzO@lUYL7al&bR-~R0Eg#I?tgE5er%ipjrsI&N3 zAq$;LF+f0|fw%eZ%+nd&x(=-hA1LR$i?HnU@eu;TPgN6)5_CIY8F6k{aJ{-pX!hL6l zK49%yE4~(x*FI>fRcpoKzrXy2Ou+FVxZzS@+ARf1%a#Cy@wJr4eTc5rl2{8X^lsBx z!5HtoOI&A*ee=%f)v7(BE9P4npu2B7I#gOQI?K$?$|)~#z_R#@V*j;1M-@)aT}RhZ z+60ZLyjQ~nZMi|Fr@H61wZ7Ww&8Yh@EQLOE$rFyj2~_<23b%KkG7BGDR;2U?ATJcJ zAbQ%C7uL^gzjWr*Dd1u%H6?D+N9S1g5F7#R36 zrjH5qB*@ChYnFMu(tSNlZ+Qqh-}CAu*w(j&w#qIR`<%4os2gXxUZM{(SA>*(|G9Bo ze)c-{GjhreDc`^}+IfC34r-ZAC<&DN=H<6Jx&Ewy9_f&%4VQ#+A?*;{y}&ms8Vc-PIghgbeco3dN>X}~5ZGV(%SQRMGq7Eg^|Cx? z5_VkA{vftpw5>Jf(G+|A<7?X?WVp~|%URSV64EpNdpD@RFetp!BhN+rs?&}~kfe{& z3t6`Pfd}m)LTa{aaX2hkI1Q?xGb;xP199NSM*JLNVxZ=CGRkr&5?TYwssGAp|+wHWv)lqd`v?F+QrE=kf-N7neK`ziTJ>xK7>jwt78FecOQIAQr@P zq^dgsqI(tb(lWx#0qMvu>j)vz*YtDTN!ecd`{D=d;%ieCJwa^F%m zfHZG{gH$c*XhN16Ij@%`AVw_6dFW63`uoBZF+uGW$sP2Y>bXsCDA->OT2^AQCTU^N z+xmW0>}`psio1tlZO4R$JjthAX3*%<%>MRxca|w_{QS$C$(xBHLT2uWG~{87Z7Z>2 z)Uc){eLps6X+?Imm3{!L8*(2q31T$&TjZ6*r~#~!t#^hSU;J_HzC5(Us}f#{nu++z zizt&%!!vJ1J&3y@v#vbsag;oK=s-G?^|kZ0RL7o!<&AYCi&%Nig_k3p{KY%CADx&a z$t|MzWA*FR^SJbEVC&ajtq<(64EY+VR}FdHfnEm2eDaT8>iu;@ulJRk@5 z4-1;}z%JMhNOch2B{B3^eUDN->(9tZIWpdn_}7CumL1lSr7Z#qBq@vV8PasAK?M0~ zUiq#eB4C%f2rz7I@-9t`J?p&A-&(G7bU1SCtl{IYBynL;9AO<4Gd;i->e72Xx3PeJn^+(a~6Vhv>+y3pg$hAzj%8dSN12wYB|b~!C*0& zS=|f#l)*0BBD^3Z?fTE3n||VO@X8b6TVb!m1zV5$lKEk9GUzIS{x4}{?7?tx`4aVa zjm>4so;MA3%=R_V%rw*UiN3}shLR?xRlF%H#&Ud@gGWqFNRnThWU z7+2}W@uS$Vx_-}U!k~P+V1_76>V~@RQf4{8SX-Hal!{qh9WIjqgkTS7rdgC7U zB>&OA_wv02m(}rS3~NB82*Z749rT&l_EAE}I&HdfY~=J&i23Lk%Q!rCoRR4U zU>sR1#`>RJ4lX?(|6MM*xb#!5!Kt#swDOFW|Mftyf#m^O>vOtFAu4qt$>N-3-3Yx7 zDmsdX^-Ku=c9`^g91z_@`d0EtFCggSrf@Yrlv8)k$(CD*Dxc=M`P|D2?U+Ibx&`UR z){v`=G%IuTxFJ2>$3t37iY5g5<_Xb(9`ilCiI%u68#Kuiyv`D4juU1BEr`~J$h>(} z-P}+lHBUtHdADgz6>ceZD8R{q0L5dNy~tW+UR;j?h_cC#X5-wtt@Pwmcke?#7zvCC z&v01;X2mM3wOkjIJ9KTx9;2PYpPse71se@%4iKB5DzO4N7Tn)UfJ%(omj(r8R*G*y z*L{3T37pE|4=ChkVYkd-cJn53ys{pQDAnMF*l9Z<*!DqX$N&KmPyFmCz>2Gx4O9|8ZuR!8CRCb!%AYg0DR|sfmHt<) zSfGAsoQD__HjBPE)T+xKVs7D~tFE}nPA7tQn?*N@@at8xvuiGV!@1p9D;APr`VY;V zrD+B1&WCBrE`GCKT)kVITS$m{`b^>^KEy5XY5KC?@fQgV`DQv-Pr)Oa|IVw}Qlq2v zq<09??nZ&kcia?9^7*VDQAi!=yX{63x5+LPcll52%-8_iX*5Q!fK-*Q6BGU9d<|hy z{=co70BOIY&2!g+R?D0uWQbYw#RQ6|Htj!X9{0V!pR_wzDhG3>c*t-_FxIQ;VpV6i zl8hjy)O~d}*AL*SMI1s0ruOhtf>9F(mU=XhU;oT%6>T5{xy;8|@+K)$C>jNFDn(%x z1lY!?O30wTf7oq=U#e~-z_Ud+Da!ee#(sjFOpO9`AjxST^U<0$*p06AX9Q9<_W~Ov zjFPJu-jRkeHJcAp&W{NmcxFX%y87;#!BaB``TgMDp*-coc-X<1$?Y7wzz=n!oV*di z{E!$<;@V8Ij4%b%584$b*p?HDI#d}u(#@qK%U3h+P>-{HrFAxw>0xSaoe19S(9$hb zmzy+`fqVXoioQy0<}#VlF=tK&$k}q1##((lBEIRbEHOKnzzK{eW(X**gvj%<1)#>D z+N`st5o0olvhAk`+vG>Whd9q&ei8-K9U2As`;2=@_o4W#qdo4_ovovKZ~3#sSgBI)SpHN> zD+deo+a1VDpy*okD(bM6@N4X2gs+ zjr2vmkeI@30RiOie}DO@n5A`i9wf%hl=oA5?6IRMbpSm z17AobbD#WHg?C2I-}15wYH!tSe3ZrTH@qAC4!k?_-0~dJ-7@!8+*c&S3AD2N7d;Iz zv;z78P8Oa^533p{@SmG`R4{}k*qr`7RDm(JUku>=_9heAzaEEWnb zq4P<92n`Oz6iy)bxU0f_%)t2+Zo5!n z6f{P^WBdLAz7!|4=Y!6XW!k7;U`d-RIN+(oe2W-HhVJb~P2VZmqmj6wk=W#XRim)A zQNyYkrN{xVIHUemoVI1(orb4HKx`*0P@GKP11fZCNtq&Ec>OTDBm%;&{OGhQC@bf@ z`Z3A0ksX7>&i3+B*vfV&YlMoD!P^)c(kJ;`hD_p-=84=^@_fXDoKi_4s_j91TW4kQ zH7m&;zQ|a!`gk4(H8*r3{d(-XRhD}X#*mCq$~cx!`&p^^Yj<~1IFR{gG|hm^<4ojw z-%i|Z9k-mnyX0;b0%zX~mP*_mts$4ih}8<%-Sjh&jLwCWZDr;!GqKZ_Bm+N{bCt8> zdG4r{E!YM`rSr;Bjr>^sKfKT%cYKP$#Qe{B(R`Fm8%T)Piad8z;j|1ogl(BO2b^lG z-X{u(?S`a3T%y}hGynD;T)r^EVP{;5_RI30H`-1?1*p(QYE<1^ZD6uo;J^YB=USsW zo3TUQ$)>j`?y!4Q`JOUAA>;C2;9Q9~Z|h$nApwck;$r1Zf_#T-hjBc#9jqh?%{!XH zhG+G`3sZB7^8)X|fFV zGMVzfNu}Cm;1y`5OY_+ofhq74fi8-NpFELgN6~c&kyD@sbtpVWd*Wbc98W1^lJ+9V zHmg&FI`&gX;ot;LX zj~+F4tEWQU^NS>L>Tdy|XC+vAD<%+=u&0l|2oI9lg-di_82H%C(;a1C{bIgHkJg`A zHsGnb0=c{cu4W9mnCE*mSulDy9&&Goja04FEghKpMASJxzggM!~;{U+R{u{m3V zO-V=eL*~B%?Vc~@@o{X-`vm^;mKi%IEr>dc=zDWX6t3ekV`BxHe){&Xl&CfK+eC!T zZ{;EoKkX6A!x@c>pNQ{wu1!Ud?3W3g_21awFvHgj1?U%2QRq!E=)nd3&o|?p0_i6r6h00kSJc3 zr?#T%1>0ApLJvV~=7e5*iv>e^N~X^qTz|5de-ZQ1jBoPkfjN_&`?-ifHSaEYAsvf4 z0Yk-9Ighz?+l>Qw#s*|_3*bP7=8q`IDkMvxnykJ~tG~mIKXO=+n=tO&T)Jj+27C&( zY>=DkvLtg@NbbV&Ld_#@gO_<{H3Oh3ot*0V2ZR~s%g`I+MBHtp!Q|IsBa3gUTX%f! zaAki5VT;r(ALsKiovxgIGWp>t{I9gpnTJ2)OaNh{e7WaaEU0K1Ww%Qf8S@}@N#!)} zD`XeJd1UK;|KNqq^L-ass~)VfsK*o(p^RfpQeUETNYIpWih86 zkBFO9NU!KU?+(Bi$>IybZf;&{rUgsz9!pif7*&x8z8JDD*4OCs&z&6poqq_u+j~Ee3*86mBe|U- zE+Gk%5S2R$uN9LV%x24S?8Ny*#bN5sZi{*?nhO84M(V#Wp{lMW0Nu<_wN-&oDG5M# z$P()U5Jpz{eA_=%&H=4L8$=TDA?C62Vk$i4{KncBpRS?T_)*w_RyYDK_vj@1`Mmt& ztlfV!nhi|?8>!+oEKbIlE8NuAs+gb_&xsV)`lJTb-3MpYb726VH`)8M@9;ds)Ehs; zDw%ZOqOuxYy;11xv>9k50)5Cg<|2Y~X0g1B#4ya2Tp|z!M@PV}IH#F7JME)u>A(?m z6$ALuH1b^TKD@{+wRPC8amjAGz5V`Mv*3=So9l2H0C@VJz&NCbQ2uklbxv=H=(tVU z1{9Fa{E>+CYHj92yG71O^75(Y`?JY&_3!og4&%nBWlmMLi~VG;;}>6iK4NCx3E`xw z+HH!{WE1cT*W=m7_A2$QMC}Pl+%B1>P0>EvRkMf+Tk4nYj9bku{e_>5kE?<~QX?Ov zBe4ZhmmW^e<)vl~pn3H^d}DKKD=K}PV#JAR=?4nyn3sFevuO5e`=Dz+kYmS=_b7Ca zm?7o`shcIrl5)k!iVD@0>m2_~0a(%53%wljTl>-s#q;UP$-xpz4xQvo>iZ#Qpw-mk zYic=$<(-b?_Bt8OV6J^bZCa3{`~nrp8$1)!y+>-HOwOCNLH(w#<-^6Em__gKu?&X!4QhHY&UfUJ4yXs^HiA#}%ZlRCL znywUL%25IUce_^72n3Q}@x-MuP`@-xA5e=BYa2<%7Q_tUnz1byDr3ZI+pf)wC6|+Y z4x`oTb-Ox;H?7pc3nmaOHj7&@C+b4z-gybn;Q3O*wIez?fnU%!#JE>mBf)YX?QBh_ z34r(ic*+tBr&pxbcrB@S$+)7$6+CJHMN27XiTpCSI!VJ&v8*E>9~9bX?C2$bDao7u zN);^7usY4xy^V}_k@h+u;_DznSkHkhdB0(b69`M^W+4_#PTpin?0}yt#l)fkJ!&2k zRg$6TRgP3fX$#Q2!ZLoFhr2E$$QejU#zBn(IXFE0jJmXacFE3M`Q=}4XHhete}SFt z>CjwMC-X(O*>mf;iA8_uE`ic{{aU%sP%cO_Dc<6VMrZuT%V3o(J`5~NK4&{V_0GS)PoYV)&%{@*76+@JJ}V}Q8R3w5 zD_O1T+|A>uY4UOC^JYRyKipv!n9nSPAYRQ+8$T~D0}`_y%wUX+;9qZ`23jk}*|Il? z@sp7VhQ!(dAx4GKuYjRoix635W)d%LW7>9%oHGAyMzF*8em{UngJH@d4z9^VWiRJ7 zv;6(SA-`Ej$OGSp81pK;>a4M}FO+}>b)9A^2Bqufht$RBZz|Z~dnGV>7AUa}{V-~G zcxT&V9=%?O8ks>Ctf zvo1KUPNJmM<0+sEC7=Y+8ii!4;X0O%CACjgpj#AW(lX~8rSCPBs^Iut#eM1d;{85) zib0mMo9DPFPWZNS*Ry3Iw%l0=1iEWH)E3lwn)Z#yPBV?shn!r*Xng&a8pm5x;N;Fd zAxZ2}c`$3$^R{w8IP)KTXLUi=zl~5%ENHnthN~q!#6A0UR=*f5v2saGCkDak)@KtM zR-NSUQQwtpXU#oN`GU`dB=Krug5U$0(eG)Q-Sw%Xs_bnI<>D5CvO^8e48?DT{hZs? zZuJ$Yh_{gZ_$>=O4ELKpsG8ucSac}wBqWi&7qkrzin!@a%!;iYIx>I_x3l_}XFvb9 zeVnQpkP{$Z|0v3X zS$dsEkUvn`-?0V1X{^KlbJL6^WY#Y?%s$-E+}g4je9~4NuAy)FK(pYWW&kHQaXeHb z8EZ4MK-Nh67ru4){(b29jQvL6m2!$kuKCL!N#7xP+50kPJaK&=8xmw}JM3{0RrhTZ*%|LxEsgL)cpXxO3#+sg82IC?o+CMc;RGn6;{WWn#&PVUlkHcbUEY1dq2y^SjQ##zbqyyxAmpBv9KPp=ReEOW_)COJDvXt=Z(x~ zB+cx9^R zjG1>?Oa%3mlt{HI;0Vje+==jNTM9$+h?fm~&|(@nt_9bmA}h2Z;&>Law6Ih84Yuql zi`?JxpK6w;60woWxz2!k>QWIzzFqzcz+i@K_i;PhuS*;E`uk4|;3O0y6JR+2bkZI9 zKB#!3e2Q}=@8US6qpZavIXfRMrdtZon%bC#d1LN1a<4d>T%S7spjGtbLry_GGI?A5 z^N({8SFP7ib~`On>KWf?2wNK2=dZn@iLCu-MAI292DVAo_hWi5rx-ZwLW?WAySD#m z;@Lj*$E(unq3cZ8QPn24(620?mj|A`R3h%G@?ET-5_EyAn9Ek{DT%CR;Ypx+IySuf#ti+c zzdw%AA}*Z4G*AoN2Bcql?OAN2Et(t>0M(FF2*XELuBrTWhDS}Y?+$+Yn||sMTuA!) z{dF>W3YBS&tq%?fBBt%%=1oBt1;XU?A-1%aPp7VkW^lsyesQ!FZm~&pOmra8b4BZ9 z(l$0d!$p@zCLM4qrzUV-=ln8(p5SLKlbF}xg7PL@&-3YLOCd9etyIjk?Da9@Wul6R z{Aa@@Duda9sLi5{RA{LsVu6Dvq2=mELx;}c3?JTUWnh%Q=|G#hVa(yIAa9>@EIOKn zwpL{ufH^;z)RgDS^|wJ#NB{Qmg<{!?7R19+XnxK9zj%E}FW{2Br9$@_lBNEybMNQ7 zlfpXgVS93v3aKQUQ^ut-G*d zaftIpd4a~g`Le0(I-9KAq0`e5{E-jAmb^8sBS4}j%22YW0WrQ;h8^h z0H{fRgG&dv*gWF#bR=ZjhHpd|wIz0roR7SF;4gP^dJ}i6O3AR)NGg;*9>k?q_Wcc|e1>{8@NIyUAo{5gd zJ~IY=!MEP|sGQasG*q?SkSqpM^UO$p6RR!KgD?gcS7_EJXb?}E(|eOmuBCo}2Cr@a zzr>EtoBfU|mV(m8!sD{+E5A!$sCc^jmGJ z6O_|6UH(NZRxgFXPhpwF0-7K3>|bp#t4XaRr{KX=0;=E?uT5jd8yaN+fI-b2ra@%wo6Ud+q6}16eszdch$r!9q=OEB0BiSLACca+Xd(jUBnw`vB}kZPUz-p z-?rR`SwVkl!gqtC4$AiU0`U&ag$>H@WLel}_(oX}@GU z&+^?MVynH2XS*t00IgAI;+85_@UOjjGdlRpU>gZFd zSIz{qcrF1Q*wqI@rB-6`+tX>7ojCQ_#cE$&v}1#A&7{Cmn`2ku&s_#k$D! zNw32RM2S6&;+aMDjxd??-En^HnK^gVeM#~ub-aeze_I{mo@QDS`K^7jQzW}c4kiL- zTR>$GwA{eqd*)*mZHYcim7)A|Hs+eIujE#ugzcj&)`yZE4T$eRM0FgmNh#3f7&Fhx z)xl^F!`hITxWb7VB^0;XPZ89w)Kp!jV)V9Jp>3N!u;aSR?AgbJyf!mud;Ngg3dF&n z?!c&QeV+{W(GZqefSB0;p^2!`1jG&%sCf}_qM~-qEBlwlKc}FVh}CJ`+d!kr0z+Ak zEyc9|!`c~m6PLz*{zE}AyZ-t)4xw%oNwlg!pLpQ!5d)@-_NtzP(SP1SP;Eh=pUEZJb>iE5j>KebZD)s2AP3cGcqup$e?L@W#%L;GBFcJ~p^8EDI?tod=4je-_ zpB=x-IU73l#YqAsgWaBd4~<-i;BOV{LwXngzi2Z&eK1`0A6J8Wcx@nNBi(M$a*}9l ze{~{`Z9t_jvKn|n)Ba#+WGpuk`BN*&*BGnScT^3%24NWTXT#kViFT6h9-~^-X+I8Z!AnBPM0RvEH91{k0MPg@EH$&TQ-8Ig~&*HfT@N@B{qIH_%zwp%%Px)KJRd*cOSUp>Ys0OHj93 zl9Q^4Kj5-6O~*0*g%3HljHW{gxcdpFb+5P{+1RUpC(2b>-V!oHgMxh(#q~vy09028 zVS)2Rw&c&8E=40)fr9OTyx$mN-XnQpo9Ia$!XN6D?>M|ooMZH;os;#b@w}Ew)c9cb zN@#r`v<+x(aUwIfS<8rz+Ft@f3w3y1yItOqrwZ1xbt@qNVOOv22pgd_+8iVLMj+QX=FdCV;U9!qThgi^hEA=D`(!?F3M%97>0VmXR%9 zMZE2!{t$asrmju|&&lY{5)AAU|Cpwc((g`_8w9{$O3#2fVXeTrtc7beKb9Fc-9)pJ zKR&`;_DMMOewjKMHJ3_Ake&KVEN|m6iuY?62?CvRJN9fFY^f-~Pbm2kgeGI>=II5U zfn*PJ{D<6`i#<8J?$3jh|KDp4lbAt$w|6ucTgwF`abySYyd(|QVaPt%E)^|>-Xc6~h99z4j9~hg=ppK3)G{;fF z$#dv{B1KoErg2c1TPfvc4dLQ^|9ok}xJXf$wEQwB%PBmX`7S|NzJ)mVBFt1frg{%c zr1(10Dq(|%DPh^YDu_zUa8#iI#hf#zfS<6sy10NvZQ!s6n@GWyg1G+SUd$ozI#gYf zoR%Xhr)+Ze{UMY4=OyQr`BUyWI_F45ZFC^>B}sF}djpq=5W+C6H#V_nB#Gaq0qB9cbMgqGFZhGDXeR(if0-hu$P40Kg!))aozw18q61+Rz?f>iYDHg{%pvYV zGR?jk0RLi~Ia=E`7rhH@q(=@!XWE-HpW=~pw&4s;o-Z(#gGr2~3U6Q6$)k=ZOX^3D z%0AX~ZA9`OPLC10SnYbl6La#v#9T(8!bVC2^3sn~sAgegNn^UJ(3kon%heZsO|Idb zk5pH1dr(671J^EF4##MelxQZ zX@~18)YNVJ;O&9!vhVusYRDw|n`!Z6ee-LD0^d@&GsOBnN3rJ5$-dvf4fLzKX&XV- zd*PVIWbyxe+V1utunCT*-=Qia&%Y8~3`NlSV+E8k+5J*rw>NVT&{oUyob!a9a%~zb z!sjszpUSy8i_ixMbI;akA(N2ob^PR8k<2Fm4eJP<5woQ)AXosT88j!&5x>L+%?=JD zr_f{YMZ(+6T^evR4^>g&?K9l`qLkBt2Unq{B76Pvid3a(hc`IbOZk!$uLHze!ug_~ zy^Uj$j2;GTg@|J;mLU!A;T_A55;6_xkj<$T>H?0%!sT7%?-LU!z>CWJ&H3>|^b}Zr zTWR0_9A_ikoiVv+_*jOJO#mjf3n-bTW5icA^Y;Y43^ice5_`H0K8^t z&xE)|gHeryNL+)G@742NTf0qh$pJNK_o4>3;D}yVDf(|*6*$hfc_+V=oX$5*0=n^? zC1gst?cuK0`VRxfV*mxva*Nn-aV|pFHkTzNcEDb0e-lA#yhsgtUr_;cC{2S|j&wDd zXi26G4tv_zpRJA=bC{q}Vi{n23VaX+2suSp3Qc{TW^(=(keP#~HwueP+6wy9n@Q;3 zIKmg5?r+Kj8U7c;hYSK7CGyQk5z#59NHER6J#{nOd~CG*%EvHe$z$)df?- zXK==zdCbU+1u5?x9T~GFmX?NTnfM2G0~HJk{xM0~Itbgp{X&KpGTS|TxspsF{#P|5 zL;w*kG`#yLba|B$(=j_sLIF7q%(CiKzj{VO8p@p-wqp_g=>r6fnm^xOuQi!oby&Y< z7y?p=03D!~agm(&m9h}^u~VNAOlE<9nz=G~S?TT*qTs)BtNYVD;aW>cpZIW~U8Kac zF|#OA&uhf*e407Yr>H?0@D&l;0w!XrA=PMJzbFr%_ zcvW9`Tsom6^;)vO*W>Qy)#-}P&Va9nMJ1GYU_5qvZHa)5K;ilLe$NP^j^n|E#M!s% zn7FhX_h0(G(<+kplx9Nr zD{Z$I=&qJm72P;qF9~_qheP_;=8%c07>JvF=T}4zxWa@ZmuhpJ?DL&Ru<2Y_uk)b@ z=&E`yt!ka$#9Bd%J0>Zo1~o?vdVN^Fo<-viSJ0!lUBc-AIBY&5&{PL&4P(jW1Eq!? z8_m41zJj&H9C1j_*G6=PICW~XgDsIOvK3xFGkS7rd&cYezfN0qXec@j8B8V)6TOF3 z{BfP!nQ^O)2r|&hrrH$mmttp$yCH7Q9pf9qkL2~~hu-8vpW`;e)NiUX;92&cBJp3A zh5Utk-NYs7T(Yo4n(G=@ObbK0*wxH%s#^B(-Wn+|#(7L7O@u-5U{E$zaMN3?leJIO zIHqKwd-`0?tDOAs?Bmka@w}RV?nI64Qzcow7ZRejEhQG5dc{e32K)!4ccH%*+hqoM zWZ-#k1!i`R$4d?nbWNcm^*87#;!w1WMxmTWlD?hjdP{L0=@PC7Knuev%dr_r?+#v_ zeLPU*F+3z@5dQp*a?mZ~?&Feg5tTD?Gdqz6G?)Ur`uZipMC$h+3VC?jk$^2-wZk#C zrm)eZ@OO=4$Z$R(2qejJQ1F&Vx9uA(>#td-rJ*t8lm7o5r+>Mxh#3UMm{_C; zV2qI4dl$!L!DkL<@pzqZQU+0z?g+Ae4*#Z7v&UbM{NYidhG}PAc0$OGy?@7M^IK5` zt0VPwTvMBL3C6-;d6a^>!ZsE|76m%=dYeD1Og8c5FBDwmKR^GO?^BWOM_v zEjI2y2Z|6kRI}e3ioA1I=b<_IT%g%UgzQe7LqO=vEn-FUgAniq{)EES-t;8VC|3!F zj7qqo~Tvr`_lRT3_68LJ^_O0DUs+@0W@D>_ece8ih>?F~*+HNiQxvK6wAQR^x z2C?j|h%89zYJ$~@ke;>ueLG_=>*?o zIj{IUrZfer`P@0#EvbSh2!LLG%Q|}g0+wSEJk$L4?+z=y;Fb2K+NwN+DjsPe*Dphg zr3l$>vyrO$6Ej~B{3tfm#nG(cS`zU?`BsJ+(x%-ZBSf+oUfUlVs}>qi8x#0P_|8zd zOHd^oL-ZE%S`wJhfspv}s?oy2oF-P!LAig%FE23421SP-BK(7zU%+CEp9v~lNC;YX z5$6TG73A-e{Mr*^un(|bI})-fh()cM9~6Dc^f$F^sk-!l*X)&ugJW<#(OXmLXCC-z^FY` z&S=)Os*(CJ37RZPcl@(0swT-7v=e9W7-(`!<; z`akEqb6t5M`Q^@ceAb;Y$VQo2IX~f>5=OSi=jic9;(46HetX^w@@NT2Qe+xH)ghwBbTP_bZDOh7@Jz{#D7U={1d#Iz`dHN0$tqCtu zz&Yq=aH+?I2Y|DP$MVr4$PQnIZ!ErPpX(}d%Do&6!=m}tauB~54#p8(wGS4u@{eji zlkZ2>7FOPqgoToTphR8{ExISmJ8c}YT!AcW_@5lIJI}^aDapF&!4SD3)JumytSB?yge692x zt%Bd^-o_dnhNeBFAdy(%A0J&$pOEL8pFoLT-o!}JUobGv8tAX^50wd3+BFa+f09&I zA-&f|0$88c2nKTyGI;KVI7{GL64#m!{!G^0mHr_>1KJ|ZYz36FxgyA_vb(2B=kN+v zxwlut;~97CGeGIb*?#_P`rE--dW1>|o%a_rTS*|{NkrhTWy6`gLk?y6mk-jM-}#}4 zaVZ7C!paKDRbJ(fGc>h1rv9Uub}vcy>$(#|=YxOhskZ;n;w1hbvd{6tlc%k{88nxl z4h@8x00tUuF-?Cwn3%RSX1d5-q`00(o0_czNtRX;Ylx^46g@*&kM#kUpEk3<+gK7W zpcGmD^L+DuJzJ3Pp?V-={KJQ#49;GHIJ=#PDNv1!@mx~TiL8foxR)TT3E7{<{FjfG!fySxSiB};khw* z1h>~|NK)9Kp48v;J)@HhdO|{Y`2Rs26g>D6;QT0&bJoo;GN?*jKG_^(ZdlC^Dz6Qe z@>}T)>ChTm>3GpGrA{L6Oxr;1==Av5rU4R9<##Bluj^-9<30M9CqoB9#bRvKzko-J ze+J^Cb+-7K8LI%jW`fdn?EBXv;SU^@Yu$c3r;Z*<4VW7u_@Dnj?3C%bE@YH!ORW_{ zX&aFx>b#nkn@ZL6>XI&FCy({Ie~zrF&JC_wEH?BoMgGofqL8|8KMKiZYUsY&cj1+g z{E#JxobpDmU`-cYecSH$g>9pYXIk>s9x&rd3LrHdrKD3l9@eRt`uACP@vzPZVR)7) zL4v-dK0LN!$H3bs=lm(`RanN5fK);UvG%S(0)@b?Lu08vD~pleM=>*i8+*bIFoH^m z)mr+C8R}_nPoxno=mhD_=%BzggqD%$qgvo5lXQA`Ucy8NdRf509`^os>y0fbV6ukr z#qExF$m2B}9=Nn4^m>VxuMI`WJM@cH`wvRq#0K=0)k#0VF!Qi;lA&Nbf3ATn_3dijo&2Q9UmyFyf=l1SrlZ7XIkwN(4dXM$ z;I=z9aIrPYU5D#jVSo6e|E>FVFS1VYs+bJ?3M!0Gi19LIyo^Cf8Af0`)tgy}oj;Bg ztrB+_pVTh*5$$$Z^#6Cwxuqt+^%xtAQlWTy_A>sEsqQY_-yzy1%lvv;(3TwNRr#^+ zW{`QDKNLUX+N-^O;r%oe)j926{-kFD zGqvr<(=KZK@SyW;c>&nPVFu>A#B}ja&@OWjK>m7(0h8>CYc#QmbVlwkEPu}w?^L7m z-f9RjX+NDDM@&}KDK*VFR|N6`hCgIh-<7|_ZDJ-|Z-1J2_cuR~jdcDr zlQ<-4F(hJ?V(3GeS0L6L!N4Wx{|Af2@nftxzKWu#ag$HEZ;9!oEVCGT$r9hfb-G%u zoOIb9t7WHNhiO`++glEaz8WLbsK+~Iq^jlILm7J`pEp>N$7}u)3k$KZ5|)>`Z@Ks% z^wgR;y((qJsi>)MN;(`Xdy9CqE%txiOzC`Zo+Edcjf#tOl!35)?~kI_DlsNLC!?2! zdoQDuH+slFqCVfdl;n>l&vx4I6@Fu}^W<6AW9&n^5OwZ47Op6!Cz7R$f3DJ3@TyIu4!-1OqgSmYvfji!_N8hY-rdK`J zC2H=L-e&K4#vJ`V$DML|T&11gf-yR*SNlh+T`UH;lMeajS0*s$6%9NtaoqEv9G+gN zGJUUW2H#DTwi);j4pTm@zBK4a9Qlcw6wVqz0;6Y71wXHAVYG)U6aC1ASsG~~S=?zg zrcH;UMl<>5bj_bYKbjj}oi6gL0U3+8#A{VZ_&n7jndS@*7_IR%t!Thj6j5&&lKBebMT>nnxc_Uc;HeVGJ_FUeqz zzVOoFaPp;Zg>pizBS3^r4NIs8B?0dpFV)AnG0-0>HWAPEz$k=+009}Ssu7ktztEVcs?nQLn-a_QA1dqd*sdQIyP!)=5*{L9>l_}z_{Q{5$#G}(c1H+l_E3iP&@ck zkcr-?e^K8&bGU12k>o7ch?!uGA7o{^c7yK)Hh2;XFED6b@Gl$x!sI+iDel#_8*sMS z6#n8aqBja)D$bAGAsr!1}S-Q)vs<3JPF{+XJYu%_`B#_ST73IJ7NMiZ7cEmMSVLV z_(|RLw*dLuQDI~P*iCt6k569!g_|!i&l@>8t*HsQEUB&uHnoo|F~5A|ep4Uksjz0! z5;@?vZTQ2ncg7<2>vF_fEgo=eT&#-;WNB6Zy<8zol}5VnInE6;=~m-eg%PH;7Aore z7N%Z;?cy-z&|_lX`f7qR;fbKU%U?wY8XD6GKA)=8kO?UtrnnowADOT^beJ-9J=0EH zI*whL1uciFEsG|sqc^m1qAD8;0ResG!w5)e3{8bOyNJO%l*9?+=xo zch29t=6fj$?BEykr?(6XI}O$0iZTqT9KKI-B5?sM`E1mNe(Bp$DbsCrChI2R?@Q{a zGf~T!hbkKp#jmQ@*%N1r^qMD00F}fg8n`8t!Eord`bJq?grWZNF+Q)LZ0z4V@c9R7 z9MS?VUsPm3DAQ>J{_@6xbOMedJ$0AkOXQx&?67}V5jUsy+p1&m+dJ!?+0EQ{nO^(;YBKJvB(D4G^F3!%MxCT$&w<0fYy6X`4U4_- zq9~o5pp6b8tF|=gZD$<@#SQ%>6$a=K1U_kq|F}9q?O*yEIrn?d;|)F;%R)9&VUg)Y zm{aozovb+JQ^^4ZV0!q+G6g)gXdDN?lbS{YZGvFJWVc2|+uv+mWN>mR^behw1)H4i z4+6}kF`98Ig?#P+T}(ut>@5(I0WjW4|m2gaL57>({P*Ua$> zT-MlS&T*tV*!GsO=_C+{P?q)PvuidOcGq19Dz7PvzS+=&ynP1b9PDAiM;wW7ch;`F zT-MURnsykD6MXDwy6pO9K*dy^cs?Qi^;x0)ypKEKiV4J$dHvrW^lvw(0j18b#AN-6 zmE{(B3rfL>98?@kjTW97c<26OMib7WXBT7jXwCC=oJbA79CV2F%twu8P9cv0Z&5A(DVt zI?yYDu&6)E(m&`m$d%*MoNxHmy28ka#%z2HSxBj4UVh@g5)VfVFU_4S!6F6Mie9mZ+$K^wcsDg8NV&0Lf3=Mbrtb-NS^=0n`ksW8@6s;4}5 z+J3zx6u8Zr$Dg>bQp^BM*pV0xN@_zfRjBN z)V!;~1tJzd^sUa`wgxR`L;0b&MHB)B1gp+q0`&P|;K=Q?z8x^yrFz20MEa}@@44z{IYPDZM(Vl3D_Q^UVn$^SVawig|#d1>Q z%S(9`(-ObrTw3x5hZ~_rp2;}t_Rz8h*fs&xL(hk*Dl+=e9#i*0h4dfMl+r3ujlA5IGkTyj=e<-$zP8$ zV0s%`-(BXb?C81pHZXDzw_QWe&fNyKfqL(5rZf>!cDUa=AN5P-Hd0YSlp)KeNOG5E zH6kO;=_K4P40-Ar5LbO}y|>yt-hN$k)XTA;@6$d+5h9#^dv)TA2t;4a98C@fSC|~C z8mDE>?=@pO>&;NrndfV}rmoV(|HkB_Q`n%X>*{aKw%ZbnQeSY@j?Zvr@-=YP2Sads z1;so|3O?qSKY&f5VK=7;4rsu>M~`zkq`0*rkb?Q(g7QpMAPL9h2jK!c{Mlo)7X$*VC;SZw#7Pf0l{siZ`B z1Ug%7i7|4GQ;G32+UN`;T2G%z$tDMF~qW5)p`uc8%Mh;y@ zzsi<8nbT&?z&i!+B@0rLIZn7_H%;2HSX4VF@T7OVhs=~LOrYd{&~`t5I`#Cx`g&(} z$#x@>hAMDa84!(#=K_cmTXoM(oQvEbv`Zb-wY8?}yPLng>iQX^;JhgQ{f%O%dj`1> zGAy>8<>;aX)`&e(fgL>f5PJ#p1kdL+_3XNCNNi4Oc+Civg$>Zjhj{p(<)h!fyGBE$ zPiadJJQphdocri66w11PHav{J4dU?yDW#-i}i^NPh>%Nm-Nl7>PU3S!6UdB8N+Rqn8U z6oP?sZ$`7DC-f7W0~!3-VxS|-LXa1IPqJpbgS(%y>Uc80a!{ToIFl5>-yDNC5TE?~ zr|HDQ6Uk7CHnx2K*JlTEc@8JTsd{)s#?Su5N&Ttc zS5RO299_0%?#UphbbXu3d`GC(+H8$2XT&?ceEa#I3pwOm!|K^)DQTB9iFUTqgEm7p z`C)zd$Jc9kk$GLs|9aciuvrBY7(`%cl=rK+%n?Aa{&f}RunBR>4e5Gvdcc1iM(If@ zHSpEMQ6@7%e7EPdf-=BxB|-0KU`2(2`d-}3cGgVieBL>TnUQJZV<1!LY%)Ebo^P|P zh#CxY4@oqVb<;CYcWlZg8m|y5Kn5dpQ%`=ZX zR-Ry+%@b(0E{oT3IC4{7zGlw4%NZL@bz^eoi(8KjdC_HTe!6e=P-=_d`kI#M%5WEo zJ4nOKBm`Jro}u2In)@~P0UN4I%zaLvf7i%c7Cu&>&r~s%U~lM=3MIPCWHefKAKbT& zD4_t>XSwRDVt3V8%~+bza?i=pX05j}NLVZO#wN1Fv_ST&Ic@N7}EKEIyxI*NWm#tVn1)jQX> zST)V&G@L+vLoU#BNE7gY83 z5BEDE6(HU|6l_d%-v^%;VOS}^uK4@VU@NaB z-OJm;Vr}0P8VpE&izNk+W>kOIfiT#z>s78@18c(-=*Q}xnUM0m;-pR>>*P`Q0%=7{ zh1n(f7MUB8nARNp!V-)gntlFNR^WA89)fQb_xLt-KYJWVJj~7~z>AU_D zJrz`RtkHp<=fg==!*fWkTs)e2O&i;7Q$D{QOSFEA@)8s2PNJr6c1pX>emxb+vJRF6 z=rJv}v7q0ycvI_k9n!ZC!8XhvojCM#D(x#*$MXYb<&bU`yC#sgF`~YBb$~TLgcPt2 z4Nk>;3c=Mo@yVeLQloC~ym&pVJ>P7XRQbV$9afuV4e;x&L}ChGmJKg4;=49kh=!#9t~AEy=ui{i1#K^`0?}7{P#F%&6)K{O#c3*Rnz)H zgP<@_I^^;7H5&KUAMQ+1$1g9$ki@ORhUOe|hU)pVZN{VJUZp!q5eW@o+@eP|dRt;L7?-r59UpQN zIp4=#5$H5?U3HUI4D=eHL)@VZ)-Z(4ryx|oXVqs+ty@w)FUCmGrsA?kLzqelV0}+f zyj2jy6q8?Kvv66Ugd}%mF*nV#0O&#z=iBgoIm#FNnt$=l8P~P+I1(D{)%17LmxuKN zN6L8=cd1Sby61NlbNf?vAX?P_dYimLfEA2RBkIXMZ`-ZGbf~nIn|Md=A~J!v226m& zo_Xjv8ppAJ5q5N1k)>Vh!hLh3o3W?8&A(z|qz~u$l}e`<1qrHt4`G=7U@;v16DGN_ zL<;;HmCs58&rrIVHe@ATH(=WMPdDPxp?|T!vo(?oqP?1gSMB$29zGp7cpoLyIM+=& zJ6J&*@5cPMo$mfMS6SN5L_m@h&F@~zs$=3E6No*(xG$c4q2~WFhpK=`A+5PG1&~yt zRVlTJti7a=0)E4UU3$Af(2Dr+?*ww^USFq^zbUc2@~Q;!V$aBpfZteCKcaIsahk+3`nRt}6`ybhVh9CVh_(2~aqsk$+;y)>RM2smt`dM53IYO^G zc98zG4zFrut)52WnMJ1*`tltW$<=8;1>0Z0Vyxhy?lQ*|I$kSb>T7=(3^qOF)4v2d zF*$;1+6$6`Su-!H`QRHlPr2LnOJ18B&)l?2klxzgBJ5c87u>o7RYQ^lUl4v(^N1Lj z_YF{$I*#>3B)SG#HJ!INXqe@fREWvpa#SuBZ;F@(M|Y~8tl-TP}49J~Q+k{o)_1z8EVAv|o+2R>~I54_v| z<~;i{=DPJ)X$!XbD8tiiKKjLg9Uelfkj!>FIyO3tB5n&x9E*nqo>1po&{_M@1E#3uN894Q8SjsEcnMitt za_h62t28Ko(7L*wl`mN2!a`Gy$1!y(R*DhoHSZ>`&ENde6jeK!@iWZ=-O z6Qn-H-lc*%-{bjH2k?ry7@{Wz|IIhA56TB0WsKCOR(zPpic5Ty7@2oX&`(YR9;A4{ zf_-bQb9bgHn>;5h_l3-FkIoTL=EhE#9w#$lh6?ud27wa}ar;s+U4%a)y1{?;4D%Rg z*LLw0cWG%EyAgR^i1%q;Mz9~Y~T+ZvqI9gEO6?@DL-8FA{@}JWW z#^aD((xPc}_b_&U*?Rv*^|JFkWVsTrCD_>3h61lAtsP*0Z7VWmXE9LVw(joQ=)Z&M znb|V;6WbMeZoVje^(!G|egAyz8x&16=aoOta}|l1sR;Tob50y1VT=pC$#!i#tO>zO zHnPO3W9L48_|E1|=9RY9%NXoY}1<8j^G_?VR(4bLsTBs3P8 z8rcX!Dp{j4UY3ck*WaP9n;w@Q3R=f*o-ui6|HSQ_&ftCfl+)aPHtFIGZ>^`sS6n^C zZeQ>eQL6OwdDAA5uV{t8)Nm4?axewSyH}sq&-x2xRzNHh|Hr4}aU?k%efYueBF?^% zr~0V}AxN>%_un_@ORY4%zFPfDJFYfKop*xW8SVUjaK@ioAFN-*^4K))fCoR zYldonG=243CALjGeu*^VI>CNWkS-d$hWLFQreW&-uSJ z1FiA-ZUSG7oIi{l-RZ`R&HR%--s5Q5F&s+q5?81{6OxI(%i~icV?me_aptz=Y7RBOTJLk5I2M- zqO&5{X=dI1P&X5QvVGCV;+TRyIo_&Q`mk3ge~Q{|4!+!kO2{+@?XQkMuG9&Ol)D6C zHifUP^|c`V`VTZ_Vr^(ZUaucX91f<(Fj5l67PTEDAUU8uQ1u!7FQsa4w*4S0^j$xm z`}16ipi8A>Juf&K$_E+&kC!SaTzrB|TU>_{G_Nzz#&-BPFJz7#$x{-C|NVuw-qBCa zIbSt{jLz+OOy0;2p@I)S^_zdwrt)Yw)x0XB1^rFRMZsVPE0Yzd;FEya$`y*Yd5BO{cntbXS{s0@fB8y*OQd@}pXn2EZzSwVd__V!eVCkp?$yCBuU6Hm~yNlRB0KHhc%3YN0q4aslbgYqW0kMC;!o?_P7xWw!G zoc-5^@Z0P^Puwmc!!_btTSvc=Dwq53S-{t?aPd^fWT@ApkN2m_z9ha*=FYPtd8Der zvYJsGahESU2;=Uis`GWC&N-IYeo~#M4=48QxXzH&B4MrW{@AYK_{km3>s;VUwfqHs z$*i*spTIqvcxc6w$U;B;U+|-uiq~-=%8`Vpgd!Yu^wOU5Qcb6qGmf_@oSRrfik9{# zlOOAPnJ&k9J-zFBl#Gi*F5fMm->2B#sl zJ>ZF~E&+R4uMpvI3b3}Y0Uq$d(K%t6j3%Wa0f^ zL>H5sNU>5%S*=Q9=*eHW-I^*R83YnW@q@J_?R#waYP*?Ps9;lB6oMp~lW5)-F7Zq~Mp1nENNuvy4whvQK z=)H4F{DEgz$B^MKhwHsJ3S9VHrC%fi^)oK^=v^!Si4-~DV<$2S5@u>0_r05i zQJJ@%Ms24pvpagM+rErXWc&o2g3Lkc&tq7<_!R;A0KId71dx4a% zf;ESZ_A=%(Ty1^MJ=U{ie1~(aeVPtrRN)&$v&;Y8O%J)M`S0PhRLKF$56e-GyCo|` zXOK=owVTsa&U8(o55{Qo)&ZGiO0!tf2EGw4_+ zRnPjU0VT0;XcBz}s<;0J;q8nGcB3xHPjEiD-bZG64ltx{b$M3)?50Z0XFirXJ1o$$ zqVVrDM^}INzSe9w{89#bCnzKXvSh70gDG?;bbrM5K92N(WNZg7ppka*W1p~LwyXMs z9~P=_#U`25f@@oIute)?(14%i8qD?IXkohMqi5?z6Py|wBN}4)xKXN1BIc%GCDkXz zsX^=3Q6^$}SB0Nu4&7(`U)k(~4?+i|u9?=T){8EB5ZnV7lB>9ppu6@&zAP^ z%f1Z!+tE?%rCxiiqJ*u|1jouRC9&E7-)XKoBp=Azq;e>`0}*31n%d+~9=E?2jriH~ zgqMxyZ3{i@hDKYt<=v#{#}crC`e9A1pz%)@w-FHd)5;|&U`va_%&h#iV&S&>SdG4B zUG7K5PmWE34OIH@U)7iSmbJC}*<4o-!Xz8tGq|glc7LUKh#;bP!eiaV0uXN;S$Ry& z*Qp$up0j7FR7ep;z|ODT6QV<$F^{@!lMT5&uHLrHM22$z?2{fp-;$JKp1tg&rISqXOi zRA+^SYv-PL<;^5tI3`~diZGPe5#}#z`uw%M`18Q(nVl*Zs4;-Jp>)0|^FQC5dapxf{*=JM-|h9il4w?SxP_QUG7X!qad?)np}uK1Yip^z3q#*X+e6k( z4sU=aVg&rHcPV2(51gh2{q&JT@{GFpO{)#>GY|r(Z9daSGj7kAyuyy_q2d7T-<+l8 zQ^=c$&%jwz%r3@~`TLr$EvWhJrH5sc2F zJ#8Ht4V8Ew`UMx$VZc83F#~xK(^Qr71fSd&L-S?YbSu-n;o)zfM7Ne%Af6|MKi0`oBO#2J!JM4LO);f`7rN~9Ex7^$#(frfM>Hhgy1UxxO z=$eX_=30XjbP?fI%qs9ypKP1XH1DV=RB3gXdk$niS9ns`UWtc}GBKKc3v6#1v71xj zHcOmi`QObM594>LIyam3`>#DNHfQ_miCq4pnu{4IO9NLsCVD+sCxxu9H%Xc&0j zQc$p!S}^Jw%mXvnG-$CY32LD{(~`18BUk^z<~U;nn?9MG@31vTXI`x}IK8$cO_@4_ zWX$2z`#%pLhub)fD@M@D?necING{j`w_2#^R*&!MB5)m!6uxMe*_C5I-pKwgyXD?& zp|g}EJgYXf-{ukIDt7*) zR(F3^_bLq`z4Fo436>}VREV%EhD2*G2qVJ3d$ZzZxCoyF{Z(KDY%oR$fC*nCT=7Y! zZh$os1`ERxRQj|Y*VP;2Zb-Gfdk|t8Z~R;Rh7?LVMHy?xPjss%kV|}!Oq}p~(x)kU zWzG$XPMY}Xy$o|rTKO_3z<6Wf-{g*N`~Jh>Oa&U3)^mN9grPHfGlaI3G2CsW1{H1= zCi%W|ZATf&IVQ;yH2$(tyq};O2X%kZpCH9!9j_C|hPVe6t>oKVz1m=Reui9poEAS0 z{=@a$KNg?*R)v?HS%_JVdNBEKl4?K{URwU@E>zlqA9j4d90dJp&fwD^;v$yW3&KlP zU~4={=FYEt7J?}+RicJdB~ce-XgtrE`X{LJ&TrQb*^qwjpXDgA5qDcQO?VL1BLI#U zB`U&Z_z=z!N*X&(v*P(ul(%st|HYBQp+|1)rbhbC-S5+GR80I^ zf+<;yb@uNgV$&H)L?{gMVy~Js;BDEc0*$ zRpE0lR*5eMr`FRMWwSr-4(dS~RD#(wfs)#0AvkYC-x}SMw`vKqYc0n6RpmP#ev~a` z#SRhhpF8XXkt_0-xuM~=S^OXDts|9aeLHI|Yt4L0xj z%WG74LcruT;5w@Yj&Ky>on@`fEv;Gt;EjX~)2r6GTITC^{74e{E!&0_*`JJ*l&ojf zWjT_!^-4AotiD9us*@MIj$i9jA)$M#C;w{of#J0MU=jwbWR4GHOn`cy+;%Glv`i;a zwm!QJ^!hm3`eVb*QxyogMRarnQ=LY? ztjt;D7)mfI4ABDQ@%IFM*osybU|-=H2=Ql`uKV%myUR*Xn%S7Kxc;;5M`Dc!=iRY2 zQaZZp5z_3`MddxJ*${j8fV_imDl92&LRXO|+v95WCiVsRSMAm^i`j5%#0-8*>sg0) zpMR#?zPA+%74qa2{n-Sh=`131rkS=>lP9vvI%ZrRVv8H1<&O; zbdF#BAfjFE=lt~bwcLITiaN4xPTtGFLuSXNn4VzBJ@ntC)qC)Umve}k z;=ir%WgBs?EN{j8H9Bbl_oWxe*npl9WYmBQlcJ4CQBk9di|De?HYUEwtR+_WPv;Z* z9xZpQAPM4C4~horyEUiXjn#7{3B<=mw6(i0AT~Yz(gZGVPT_Cawi2}!Y3oV*{Ryw> zpWLkbl((^H8gB8m1)SH<5j!vEvYzRWvRI8DZdKH4_-WWn88YHe^*Gh@jyS{wGWM_NuV`3~Kl!1=;aa@JKQzq2(OC{xBkAUwxA z1P)Y#|2UpLIL{k*NkS=a2`XJ#nz~i_Kc8vL!(K@+60@|#b&oS^h^L*6YT~GH-#J0p znhKQ)KarX@W7YD1LKU&OAyT;7EQnSeWZABpudir>=*~M*_Z_R`%;(Jbxz{8n?g!#I ztHbc>-$i0Js~B=y~orLz0IEK zs@F1aw{C=1O!+tWYWwz{K!uSr8jsc~a#{t7huF`pAmumJ zokhwaR*+Yp0DR(z%)U3hkKQ*djq?2+!TDe4NXL0@65;SEXrB@ujkEZD`h54Y>!ZJH zyb#M^xy%=<*wNP?j-bagbo-h0J9KvnDy;`V!=kTcPzH+1`tXyz0$b6L-Kt{i)!$UR zt7eSbn^mafVxic}w#pF2&s86en&DN=boeXx{tFMa_ifGd)*nPu{K%7Rml|?j(HaCN z5i^|JO=u5UZl?l-VDA|Qw-wlgy7Z8)t$&I8CVWEG>38(#cD%Og+&Qf6^5~;!@L_f9 z@n3pFs|Z4yrpd=ChU@9Y#X{n#!I(?monr3!)C~Q?O_RCxpT$I#qJo7InP{ry%<}LSf$Sii%9u^EN`wH0Gh?!qdM%fePXsf(4BbN2RQ<)YRCO5S%T*+Zu?{9;U7v3FUekV?_0*S?A5 zy2_wY^)Qk+w1nl7pHa_3J99&^Yn{|Rj;BVeJr|CJw{vbdY{ z9Yu#Id(GhEVD*YxzbGG_gREDvU;4y4|tCnMr_7$sZz!5PGUcpuvs0{ND$rTPu3PQQM)g#K6;Al3V5 zz2G7XnJBXV=hgR%VAqv@BtJ(>aGZJ$KwDeDnbaSt1i^k?B04ox0PieCadA40FW zDaFg%mhR)1C}IXiuqgY=TV~pbkSf-nm4q^aTp%~?YHA}N?PX%B`xd2HkUOkjb~UuN ze#f~IhzR%>wdyB7&;5<+Fj;(6`#QHnjBewpfS)t%A&ha5&*8V>{}e0zRExk10JAHGPUm>UI`k84XraeoQ; zQ93*Ax2(k_+v0m#7CLTrk4TAZM;-3O{0Lw>QR#4}{*R$}2)kC!>D%4Awj&lv!7oT` zh!F3Z)71S_)ZdH#k+V%t4}D_h2QDi7Jdo}V?caB8F}2NdF_OBnlLKv(uqS(8FbdhQsX64 zd%yD00e=O8HUY?$knn_4%xm*$iBR*eovNwtW5Ih;r1jH{hs8ycYc)q>E&Kgj^N*7T zy};k0JZ)Q5mPgS*1;=aFGIr8IasDM`GCHi#Q z+M|t_!*bl26>asUd57m)GC2MQ z-~~jsBC|laXxiLiZoh3vnlQV0G{pA5!wh1Ex$5M;FCJK%im#Tk{nP-+A)&131ZM$x zeTiF#NQkkyQ53Y2fU(KcB-?1``APr}C+Mi=&JCTjh*F|)MA4QaW7EXd!$*L}x1@!w zi8U50-xU?_=bB%Krp2?~UMTSmn~$u=v^z6htpz|8ht;(V4cNoMe&6rJR=YJ8mJW0l z&;mLXlxxbL6egZ z{>HiEDA)mx2=hZtIF0;;=<^1b0AEa3!Yc-T$IY}bq|fZ@wJGymq)kJNa0)+5fWe;J z@_R$?#0X<`coum7W^Dh85tB3w6cZbFME6EEb!BBYbk!e?7o-X_HtIu-YGU>@k8SDY zBRmJCOR#!LA63i)+e%vo07n%B=LH*qnuj~V0n={#gBH3vc*#qWHsy09`~Qdg;A83&h(#B3_OC9K7~5ky_a;x zkVbUozi4lKH#UNPe_=nbkv*q;2$^5$J}DNrQy>X`~lVo?qlh9RhAQ!2a07>J4u`#idhHs{4>@EVMte z&vPquOuX0>O#hOytXuOUUvR=coTfEZwLH@e5%Hfw>nbaY$LxeAamGTSeg;JA^_r$d zm)Z0Gd>1V#89X1}%jvgNRK59rgkBq3Rv{ zgQmt{nca^9270k8>KXqs9ngfDYd@BB(76O;o6%Mxd)w{UY=4LZD6_V--`vTB=h|WO zIK|A>zI>Y1R=nTKAn&JO>I0u16>u?lAF2Jg^%I691NZ$+A$f;iCpp3)|u5lc*-akd9K<)3|>yLuT9Y9RViOEmM<*QeTGkoPQKj<+8h0Sy&Ekx%1{XJBJZVC24bZVY7_wYJVslc)~d zij%;MseD1qG*=U0hPp!d9XEcn(Qx5W3geH7-BNGB!8el*C5Rm{pOj1}UVhMc2?f~h z7|iG;DbwvWwrw7WXkE8}U<&m|8yPqi#+DgDwcAnug=Nb1zw2uvrUkW`z~X1 z=8wIPJ`!%)o>>Aw=g++SOo~6t1?kD{y<)t*LJD!frhui@2Z~2;M*NMF4AN_)*}HtZ zt#uqUj06+-n{2J;h)2z7+47lErk2yLN4E7Rm9npX)=rfQF9@T~1EeK>>{3lS>%2pR z{B2TUo{p#k^!qXiK4Uzyb`tq^WjK|e_SD&#);eA}sF%eeYB>AFGj)tq z(!AAtche`os;4nBT)ZAH;nFm-;JP8g9u68Tz2f+z)v_aCTn=k*%|G0$2BZ1x%tNFs zj*9ZH#Y?+I6@o9`;RnQmGU5!YokJI~v&6OZR*L7Mm!CYU%-7MPkXD>^KdkH` znKQD9z+#8Rq-=4lYUJyGtDia8#eA^zQfHsrqt!W&shc67(mFyVf#S`LT*viGfex}8 ztZcM{(eO(-0_XHl3FUU~A|{W5Ip1v^fYy0}LTAs+pZWD97MOv?nY>GkxIsqHgS$f1 zfF##~DW?jtPo%dkd+oD0O%3uuxLX9-LITE{wl6pmr(B3U!)NF1`r34j4^?(|^~0u> zDMmgMtEv$*z^}1xx=miFY$ujcWvSh>Th-fb7l{Aq)4)}Cg3fPRtg(rc3G)4VUrZ{I z@zwKNB@EC=0UN(<5-&Is5Fr*ZS9*)-TtI zzR_8~!Dj}a?yyVuJD!iyes4d?xe(mPsScRQ)G#uN7Sz2Ah=X;f+V#`(_da$J!K0B~ z`s!w5MQpAI4HR!)=Ch<1#(3a}XlNN&b<#Z_pR>wTJPSx)8TbhCofGO4cQAncU8}c> zEfMO?8yOAUl9YO&>)V&}Cqp%xvI)nnBMWDJHbV2Q9V3G)-;A`zre(fAADO_7If_jb7-$Wns{GGZH zJaGGiadr*n=@&mArtPX|mn#@tqhRNHJhE8br-6>0KeX^HJUm6Y8x1s8(>S8Q!W$lU zT+Yp_Gxj&1oV@h1)8e(P{No+f&O%wQJL+O2L`HqaY^%2kVxr+nK3Mytc=7d9R$J6&!uE1vRi-D)U`18Y5O<;7fq=CZ4XGh5l*Vg>KeqR7znt;= zU#C$GcyyE1wc*BgDfbai^=!WO+~KFx-GRi?u7SiEJcw(_nA$mDwcM{d9xztxx}bQ@%d}%gfBM!|okHUF_}1;| z@cYJZ?3tQ2)95@`gKN01hIWbxwQSSZU;QcQoYovBVaxT;*c)!*TXpJ6;ic-1e-pUi z7dbSJN^=&Be6IeNxjP@*4EO)E94F{s)E8eS=}p{dVt|&n0yQPO&~f737;L@xk20SV z@$H@pnX*gcGX(EsYU)=1d2LD*Jgr} z;{z0$W^Eob6otG+Xg7JTmXtKrX#P1;mC?wx@C~?S_=)PFKn7&3{qDZTy+|6=u~Q%y z%l-PkHP4Sgo`Z_}l9?8o)is3aNJH18X&&>>Z1*CN270E+7JZHy=z7fj-dcxV$3_#@EW9CO^p%7`U6qjOwJQp$gI(+6{e+YU4ie<4tPzLSS}`3krT8U_@2dYXXBJ*-JapFy3tJK8*!;3GOd$r4Ql99c?0yY z)MJ=aUPkPg;IGUL;dTb=oc31wq!PhWd94P~^CZeW;ng}aSzv!&F&EgMudg7+jlK`R zH=IP>kQH2qq2#KsTBD;Wu9?DcosG$vf1ojsR>5^2jqm?dJ(^b;lSJ~``{%|d%`f%! zRjhSlF~E}w=(9XJ1%Sab&}ZXI2lv;LcOHk7$~$PS^1Hno*M_csLj@gwco-|J%+N4W zV%$9rWZEJwGOt%yEA|*b@g)HYc5RBgdvQv?I2-Iojn2xCN}c;1{DE=uHX5PD@Reb# z+W2VGC?XLx1KCr`=@6x=pCo2QqlGi7wi z)6EPAZ`5ZQg^(`TI*IpEWb2z(qQMFTj#F;4%x$vxRii+I*J}d9TN8@g{)6}I%_Cuo+@gV0z!l|vEOCL+5Q&z}& z0)}YZ^+zAwJw|a;pB3)*OG|!zU(iitGZhbNApe*4++1z|x9P2I`ko-{7S+x#H`2CG z)})J1*O(rRn!0MH{Nn2ygM?^ZuhX8*@)fdAu6|?=etwcpV(1juC|+?BSSyIYP2wwiM+|Ks!;xWu{w zl*{|BzkXIiD7?Y|N#U!F;G3f@4qFFJmDq?zniM=gTX9Si^(=kh!Y^-1vcOwFN+%JR zva{&;sZ4r@5_HL+YwK@`vu3chP&+=OI13NyEZC=;Bed3gG>dvZcNXuptQR8lnQy3A zI$05bu_#)sxGgtTUr>Z`e>)#!?T9BnYL^~qIpS~HAD2e8N$k1QQuy@?#N+J=P>iW} z8J@qN0tk2X?j(j(6wRKWc!Cjr9Zbs>dPp-7c3x;Dc#xf!>o4}g$#;EFHt7Fs>GZ89 zV{V3vRQ5GzpR8vS9Iz8V)k-qsYm#AdGvJOetve1Dsb`!!ImP>LUv{&5T^Q3A;zQ&* zWm3|i^wxY)PYBEVg?)KO=#S8Ej~QQcq=e?Onb&>T|Goa(Xw+iH`ca5p;^TWUc$%Nt z7YtC>YV`Ys0UgECKqWkw!edl+trEgs?Y{!8FXg`zvfeCEK{L%nuqZY`dIc{L0AK;s zp>`jPmKtl#oc8VDQ-K3+paTy0J;~mCv)nxjuzy^3k7j;7pL@iR-Q%Q4OMo;NccGc|N8)yu_gx6#k{y_}}F#koit&%BKg*lBoO8KFe@G|j|P(ZE#J zaWUb(Kk?&EX8-n3R-xk#uqLl43sIv0dLTiI4}ql$Ge#EHO_#qP!?lGEzTav(j<;RU zOLZ379zJJAHUFcz?O1%8v=>m7$AFVDmeb81wt39A)}8X69FDYzRyLiqSf-UkHF`^O z^F1J$+L1Q{7DO*)F`zT_DZ{ zte)|BHA{B0EixGDb6cuV$aq)IclmDjbbXK3=*VZU*s-&2Ru|$K@m?z$``a^{V6%IE zequa~z)*o;Z(e=b;~Aw%*-rP*dbO(S;6zNz5rzNNSps|$q@x?h)n)vFe)i!>V>dsl zuU!5f?q>mT4P)m1Qg0(_T(FBz6O;?gA4CzTu{OO_YG<P zpT$0#4l%BrRP9WsI|#ZBwv^)u;56mc>(~MbHhe5C}!H-Ksdwb`Mm3 z?7R@@-|nf|aQPRB$|X;twGzaSapcGL@ii%z-56ON^AO0e+0Xu7k0Ad_hw~-J)1r z3v3u8L?$`pG_Pn|hRE%Q@geSjtuU}Ub)eX3ve6N=H|2UNX-Jx@{JMmpaZE%lsr{UN zRx&)&@Lu>(fkPki`LiC_-;Ky!!B|mcNIwnF=AW#*zb%nkW0{p4HT0_KK(6Vg{r+|8 z$`?0>f2pY|oj z5Aev8%OMON#we>tdy?jI|Bwr>eTSXT9F<~Z)NpqC_eF^atP=HaZ&R59>9n zV`9`(e!Q-cw&hBx#5|1Na#1N1-r#+&ql)a1M``qdUVfkOq~y;hEbZ=_;!*&58P@dJ zF!Z`Vtj-lq`A}-8#J6F>ccUFqMeykTeTYaGdY*CBbN}b*abq^sMvLgp6;^6TUqL;{ z9@y?-*xn49zN(yWlEulPs*f>Ot6_?XyP0zKWQB1Arwuy=zR{$1$?3(Jq>`V5p-yK z_ULkl*z7gt@cDLTeb%9xyMf3^mF*bj?$J=JdJpM%9wmjW-Vwtb{!Tf}u}gFR8sob@ zy5AUMtbk-HAhN1%*iklZ+KNUq&7dEJTDi9*bak)oZ@>O|+4PL)f}$!Sh8|P9zM8Ls znM>X z@h+{V%cXiSrRVzVFJbtuH0`sIRQuzV27u5gBQ`Rf+Udi|zj8c*=~gk{IG|GRmY^m3 z#cYho@T!O1!7(-chqNj97osZvv6A?UlxDv-Y)Ri?(|R|qGp8)$NF`0Q`HK!wZc@Wt zn)RzIaq%??_300ik=n+ksIS^WM+!7s#R$=j{v%TUpt6LwVylws0^Ce3-DM(|k z@k!RiPIG+9qj#rElgv-NkO}`;?UIuPMnJI+#b(=4R_~19CpfyaZ7bvuW?qL~`Qzt4 zC7~$96X@)FRtWu)I9)bwoNjxYC4~f@eyxMw= zD!o?8(^);g%sxlCRb-#N;<{F#Rn5*t3*DM&Gd`NQKWZ6Hz{By>xJ+vv9Y6_wRwB^C z8i>~epFb6vJ6{g;--Yc4V@G+lKj+6(=PyqL1fa`hMDslY=#q$60Jm1flKa?G_05tK z2rannvRLK0@!792=VdGPJvpP@9vS=Q@b{S3IWfDw=F8$jq|zk`G|vM{r^U|uJ|P$3 zOkP#*GIt&v4{`6gw$!EDMSo`aw;xC7POV3;)h}Nimjqyaweudy1e8a?ga2Wgvj+e{ zk}ULxxPkG|^WVzG>&t7C70L~C)5%RSo@aHX@j_P9>7j`GBm7LO%RriU4eAff%7Pe^ z&f-$rS>QMHtdehrViTD{zEwM)wi-A)*!rvbb0%#%5BwD$j~o;P$E{~&wyS(B(7Mn> z(QVgga}Q{m?QOoEjOcSE)Q!B(6?MNPXl~SttXjMGP{}XPZX0%uY>s{MzxtL_LuciKgXiG!$1JuWNV3(xdNKe zTWqIPzPZ5P#}Zk*$|uKuJUPEhsCKNtC3k&H>Z4j!dtmy)RsXl0Tqu2v{_B-~q-aDl z?B)H3TWYk_yg-g{2;~Y^V}ccj?WZ&HEMgtVRh zFiSp!w^XsA;W7XNeiOrripGr`nfw*2DeWNnU9pOZIL(SgugL;!SO5Vw*AqKEX}FYn zCwQHr+l_oFQF0{5g7fVhJY&Db=c1FMve$UUZs`5sUL%n1YungVcLbL3s`+)7KE@}M zOA%RPmscwx=e<-R8gbGucWcqsoxhy)CGt0}?4z}12EUyyKr_Oa6@{6Ejs@$rF|NLg z@qO<*gcXuMBldU|ih-BQ@M5*gaSWF9W#XT_uF~~)T`BEB!TXuTvsPi4x$NKdO?59b z>Goi(Z_#f%EjpCj?n;JAJ!ZaUl^PKw+i<&CZOC|da$V^+xpcIxe@>RDXcHT0Tkp8a z%ehQ4)S5A9O%=>8RVVv!TV0l?E0$EhlB@hu*I`@|Hp}Hh;Zf0|1#E}@qNs~ChZtbq zr1M3nQq3@%|5Ev!rKS(TBlT6egiong2KwLFIv<~))+}WVki(zVApKB4%#`8&h`sBu zHGf5P3?tB!;hyOp@~j!5kZ{H~TY7WZZ<*O|bZ2$Uae=$Sq%)0Tywi1H*fQ|>ZwuV2 z=!n(7&$Obt z=ySZb*ZxR(hYv0@JJRF=5hADWKFa5;Z0Ebl6CWOGO-Q@r0As;q9x znh2Cdk*J}2nq8)wgjBn`FLtl{n`ZPJaNus7xKXt9k1E%Ry3+Wf zBMYQV08%2##P#{f9ys204`2JyaGtydk*XA=;pzo)rL2>ln1iz%!e-0klP(>f<26kO zdCg?vmiTzrddAh2FpU8Q94lUM^-LVgRWtcZMTfBP)d$CNrSHUzG_%707M4I3C`8X> z+2^QD1N@4=TfRuj76c%J>CMF@(Ns8Px{P_iD6_Sjk5*l(0>BnpU{hXU&r?Ep_SuOF zI7dZu6l1_G*k3plL%2~bGI;&4kHB%X(61>(Xf}bNy;K6Z?>aPe$mK?MI(|#7GP7ya za3XPIbkV8wv0`n2pSC(QRL~!7GNLichhSCD+#~i`%y(-Tm(-7!&ZAz}HGR2eHAVwY zhEv7L-g|YCpZdmN3`-IgAG|t_CKo+iZU!aKVwd;+KvKQuMeK92H;NQ_z=J#S3W(pU zUaS%El;3&=IZ{lBDjGh3`7R{9CVDe36Voogp)6`&gc4tqITZaWiCVS*6_)B>&3qbh zjuN(bzD5|UX~$q4P0HW>O1>%v5as^}P5W zzkQB8;+baKJT|5it~gL`9<->Ta}v+LINYNbA&R)Un0-5{;iICE0&c~8!fT0_g;*6Ft?~v`y%omd6rR!=m{xb;NE9%G7{K8?I(KYI9{E! z_PZtkn+^~1mPj3((Jv-Yg!YG(I~(M|X&-$Io&?m_-K9~Er!4#p)CRgZl*RZE?3#%q zBH)(z+eUquBN4)@h!4ppgGh0P;oPdpn~4gQSm(bZy5A>*aO{#nP|&fyDU0<#e8k(! ztkS??uQ(7fLbSMap9DagRn+ur@fpM@n<8{7hmtb2&hoI@-@PDQuPW_R^r{l4d4ENa zmV-vU#X+wV0pVPJUPXbc`{;hrD!B;FCta$Zf-x9?{jc}dYep+~iB6BKn(liDVMlPw zSgw}uJOgfb1@v0&CX)++7O7m#dbM^w@&STA4^^nb$mF}Ix0KBaUPE&Z&1x_sT_t=5 zP35o`WTmQP#Fr`M3~<#MeMeci1pRB_@7Lk3vINP_+%2ed%7`WJ6XdHV&rjNAg0~CA z)!l+l^Gic)HiRWUHYJSSo4SgxOkr4fGK~A5vlA{~E9iTgh@{RtZ@#hnzQ%00DQ)L- zs8sKS;)+8n1YY+y^E6>XN^%~MaRRXx#W77URg;%C^`lpHkMrDS(M+=IVQdFOBJbYF zNv}Ebxn&vA_FbNAQn@-yxiBsCUmB>zdw6D0hKlX7A+H;$41deONAZ`6Q z>~jp_RtWm?U}RC%165S~xXq|94C2uun_A@79#8Og+Q*ckfxh``EzM^X=du3naw9zj z>La1u`>S?Jty;?UgF1R<;<3#&ZYDe$UR0WQ3-9w)-QFfSY>jl0r=B4yH`MBG*dTyLjPHHDa7@!@y`FzYxi$AMo4$BlnXfk%IC2`JD2Q@HV9{pYH z9!lEK*^@wDcQz!ag(iB?(?EY;u;SGpkj_{Iz7vceixUKQjfkano%2K8vX$hMN$i|8 zOaAY@1Hd$UyrzR+&jaSy?H?(hu7ify*1t`7HQnZqj(q7)No862#xyE?Rq}9uCcV*Q znhX6$y_Mk1{p|MujoXAZo&_&I{`ghDtX6%dC(k)ufx7aDwHh#y2cshI};uNg#L(DInQ^L@ZwMtsl}_0?YB5as15o;S*FYplrp6L-2kr~`!5}Fn#tgZ zZ|*2D*AD7N z^sq-N9if-&O0|ozz^JG!>zsTs2TZmB3^=`s<4<_wYCAftc3Em#zT>yOF?{K!zBy_r zx-Yojq4~!)5AhQxp;yIz9N^?rez`6ICVTl7mS=*WBsuyP(@En@ZCLTlG2*IU`7WL} zxJ)X|35ejf4y*pe_u-RQRM)1Jg5e#CvzMO3r^s9?k!`nX$@@R&qz6TKc4EVilUjh~ z!_sT76c8}0uV8&Uzy`?MM4M^2Pc9q=A9mt< z(iPwOJKKlAT{QgIMdrGE$A0+D5c+4^SclYDm}kXX5p=Unntxu~gFRe+)0OV1wNkoK zT0J}72Fvuz$Y=81lWKwc)l)%hvVq1;{GPhv$gx0GRF-+or z#=K%|pe}C+%X!u8B9sx@D(`03HahOAMV|u^D^gR!td9`LMb;zxAyn@4c}arC7~Iu# z%e`}c+)7(Y=8@La{P;Z>q1^mE%(m*EZB)%@OrLXR66vgw{8Q~;*4%@|qT20%p(_Au z$G#gpL38(*Qf$^%GGdw4Z@g;VF@x936T0O*k5?l|YPQ#oo!c0|UC110S?PU2{V^?X zD6OoDRnKGIa2m>7_v}oRGdOK!F+uN&&)Aav^9MXF8`UL30(ncM2UK>xP z2D!ektZ=C!e@8sR!w)$)oz~Sp`nPm8|9CyFp;#CTD#C}5-bVnaoyn(%hq$e(+&bKC zoiN1b?%3uJlsYPM@YC0Oris;bp)Rg=i`6`iM>Gp_iL zg}D6Y0C!VkmLl)Hxo|ziewy1~Zu|3nbjcbuV?6*_6yrE=-myjzeW+GiMpT?E!9?=4 zs|+HXSVT$?oSSCys#BP?M~ej>-h2mqqd0cEm1?<~fT8mJGzw+Gi=1!9?qa(hM_1Jo zQ7wbPxgrSycAjcVB5QKDUk|pcui3qz$H-3#-@n~wq-MtYHwZ3DRZf@h_Xp{hQ8*2Y zq%L_)#+@~&+r^Cb`D7cv>;R6Vl`{`PzN5E}dU}9E@|A=v*d*Z*S3iVxU;`$sjS$62 z{;%z26J2{j$eX6}dt1KLk9;lb7AYx1T(R#e;e)vkzw=B=ulRypqiB%-udx)N!#7@H zB-2TqD%gpocn~&;$0`5Tt@Zw;9)^2)sb%JwP-N-~yoypl-H&CtwV1t_=9B%f-(UFE ze<_2pj4o`aAey!PK0~l%a88=Ohm1wy>aTvYg);daxBE;6KE(CKw3d(HDEC8;Fz-H7 zRX~{%aX}QVj{?t?MSMTP`PlH>9e$M--AVQLbm@FT zndg#}0FP}|aW+4UsqXg+$4^lah0Rc`-l#dvWdQ1lcMP|;Jx>+~jEA2_I9OiNOf?q9 zV}OjNwr@TYV{|@ob=A=*C#o?wNIp_s>cvFeU<;$=#{Q@rqf0J6Qz=x=OI%&vZXKb;>8F~S({yiln?}bvY%4#pVBhq^SWuA_WSekJLgS&*F ztEDAlO69LfZrQ>3diCn3tP0<^7>4k_a^DOXN*V~jG>X=yhQ^cL zcm2j7`{~Ax()cQB&ANaV_3lupr zy8YE{A{zos2h;-Yl+)oT@1Vj0MXy1pbHjRvunDKTfI^;2St*gRygz2mY6UE1qrVxC zeLhltQ8(V4bxy>e zMEwch{i$P7amyY~>vMM!iL{!k8`tCcRj>^;lN~qC7843#x*%xD@t&6*Ceo|wum2@m=DIj*r5-a06-X;~OqFKF~ z)VsFQgfY$say)jSI>-#Ts(d$;SLsqUGJrmH1|`46|F@Bc>b#>aA?s`(0olRNrtFsdEmYhCxb>lNb?kWDfOojQt_BNWEIhXV7`{otOQNd8fov*!5V`}oe`=d#H#Oi;_3(7DL; zCqak!ff>@^>gICp_k-8cYhH9te_yd%;UcI0(pQtQSy*$yYS(~3qj`Z=AVC{3#=<;r z#MPvTX@#A7_ao`tl*)O6+5x7pg*p^S(b0we#hxJmp+NpruXD9QwHo-N0$XceA;(MV0Gs~HTkx(pVY%^GVkjdz;XBMZC6;Wfs>*9 zCDqnv)zK# zr@_nGz3D~8V2lTtzIhqB0L8ZX5R#eBZDRg~8%~|*LkgeaJDyduZ?aXWM6GWHh?3|O z?~)m%2&dZEo+=MHb?MJi<#&T`ssjrgd)?@y@^!NL@@rQi6o_Hpz|Vhhb*ET;Iap7m zu}!RktAvvNJ(iJs7-sZ08v+`L?i(78i zPnl0T24$Q=u-T__n&E_?fy@{ir)2 zEs!Taq6I8;GO)e|-^9%{H5lWFN&5CeJ&kwxb2|)u=!F#O`*oFKllgx{3vQu06 zI?NyMIGhZYW&fPFb9QaHx5A1~t7iEI_HC7gP%S@Rq5%o$bwF*vSke#U$G+;#=FU9&9gjk^x>U*66ij`QyQnOdLdh&&bLCbW#maZ# zw=Gw9#5U?H%U+((YOT_Atwuq0OBEPkOX@Kt83Ym{4-0qm4QH}CH4!5%)P(1|fy=8V zuP=T`q9`p=^6pJok^2EyG}T^7j^b!-e;sw(VAkFGjsq}{!Cn#LGE=1gJn{RZULj>U zd|PRxu>H1~1br4anaKG~@y_)!R*!>ZG*Y(u+fGr??E~@u-&vgu$_O^8cf#xWEcokR zZDLI2hPeh~9mQ#yvxQ(;12^bi4Y*9G;;vQ}76u$;cG=!JxL>``v=#v+DkAU6I_i;i z$dCwmga%iG_fWDv)a?g`VXZX9XRnk$2aU{vhrfW+;IZvj%q3QBAxPBIHZeN%5$p7K z`Rke+a0V8*_~4p!@TElP+v zg{fe`8NLKK!-27vH$Wc8xwSnQv)knNPbusNu&Y&|L9Gf8_Xj#M@p7e;iUONLzCA+zEu!j zMma@WC{1M@gE?w8FhEzxNF%JC6CK!;`zSZ*?JCTrZ$re^$kARJ@h)zXzRZMY0-E2T z9zW{2h-0+$+{tdb{AaeWz$u$SoI0sIl+aL;A7C&>)g(>o`6tb zrP(+RYy3vJ@ge5&%DMyQqRSqiD5GS&O-a_#MN2L;L)5b|)WSi%xTfM3zDHnxyvHVo zurE8t8TP)`QiVtye+%vy5Dq7}kDnAPK6#fTJ<`$r`|%V3m}1I-d?FMET!Hkl;K-=c z>GhV+l#~_JX;;6LQl+xZ4NTNXe~GI}(E`ehfY1_Ip!TpwdrO=QKvf>^ZnlL<%@1Cg z*UTYIOk>`hPQIRd`iX?kmiVrzTms(UiuDw+1Li%ctPskgyd+qUi|hg#)Ogj7ivl0D zkRr)Fhz)AcA%4MWtRrvKQCTANw@TuDB@49O5iU8q`eqxsVZ2d{_0gZ7<~GM`jKtD2 zJ&7N%->Mi^SZ1vcA+6adAUw~iymy42OkNgoc`n;n`F)|90%RIp?<6#Lx2rIm7kC2g z5$tXcnE%G^P_1yc2bzQi`YC&g6biha2FOhyZJzEc}bzRG-m)d#IQDH9OxjrBAU?s&U;1EP+!>ri~{7 z;7Q*2&iNpZ6+EIQr|9NT8d$3vuqk7l})IB<4^ z&Q%8x89&wM$n=5Mi1x@s)n##21Mbqy;<)$z9?>kM9d6jSMe}ohW@0TgZm1aIoxGqSK40#uK3^ytp!-Wlln=}YD9AP>KbZ&it61M7 zzA?!Qadab><|s-i7 z@n&2r+N=)~>10G5oawHMXNBU;sPz=d*hB{3LLIOLu4L^!6by zAUt=ckKZ)iwuStL$c!D(M_uuZ!*_&U1()woz_WTZR!nqz%xHAuc}f67Vvm2Uw98VHJmxJ*2@C z94We$UImT$KRKuro%-ZLGj6w|MgF>kd^2|O<0bj^V;)$2=ny*jpy%sh zYQ|s7ipyT-5}>DUMu_?BE-d@#e1IdWO@kJ?7GAdkFcAL=+QU%}oWUb>97)o;z0lR3 zT_i^|!39c~c!5u1bhc3PdKg(QmIM8{<&{RVR`AO_yg#DF$|-{;D4ubuNCgCuefmG? z`-7J_KlScc%<9`^gKk^j7D}hU*-YJ-&Cp>;SgrCov3<{G?*4Gem!sLLIXoMq(&|~ zrvC@!9fl%L?soNm2ZYT39uT4#(YI^riU=Lk+6%4eZQ-Q&O#-Rk-LJ9DyI0LhdbA{| z@`T60PD^h)hSbKe(ZuFF?&z*ri|AD`0x?HQ-egPgd`<)-Mb^uvE5Ej~HdPnf!B5R{ z^~1kRH}zbq<}4F#D%Y+sLhHVM(S>SX^Zuchyd|9$sz8KCp@a64l9%1rF7!h}&q$UN z1NkoucvZ7_!7v7B`itI_u)vP|E-A=OLqiInb)mJ!3rt}DXZdaHX4xml0cMv}RpB4+ zwTumvTHMLumf7=8_0ucfhbVlzzJr@p9+`1bA|#9)qmbO?iCSM0mtNM*GpD0?AET!% zl5Y>()r0?1xp!v50Upcsxk-HM3pJFD>Logw5OI$5YcyAL7&;Zyf~hRpRKcIdSjC}z z(fk2rg^%oc6}@q%VvKOXH<55Jwz+%&@lgpC3JQw&nE4P8PW7)iKe^F4$FS>*bw|HAf)@b!n9Z+uOEwe1Ku=pk(0(KG_ZX!@Uf z59!%@HBP`-)D?yDTU4A)e2#V(vY1d?l*2@5wD$YCEFT*4U(aj=y_xsLXPvJd=C=0; z*9>?Jd{DA_!#ai-Vpml{6<6yvp3+e)9_RLX37zvZ{N8W_u2BQCbV5h{NVfU*|-nP^bfAIlwq~rRst>DU?KkG&zSDb_-)v{%+J*luxZh550 zaWdK>lW92fd0Gnl24LZjjrqWndUiyFA*DYV`3m49QEk8(?`JcYl@68QyV}MhITa6o zNfTgHruj}=_JwAaaPoy1CfTNjGdad<1vebf?2iEI1{ottOgswN&Or+H6kLD)d(B_y zEM|@E|6zuOEI>FzD+nNTb;Z$g`$e8f(v4&mE?odoIKbRq#cD=Lkm(aUjL2Hq70O$F z-WdNy)DeP|pH9Jl8ali=g-lJp-XNZRMEsk<#uranHzV3FoOO?vC9C)oI0Ch_3bX;^ zo%_9-I5E=PO151wdep(1^#!;rfYQPg?QN#MGUBGNgZ*d0Tw))3n3jnS$s}F2acP#k z0$;sJ2A$JhUw}N<$585qxe<3+WX^X61~1l5!;W#Yv&Q|e$Y-L3vozO=v9=7s*X_ey131CXis2>unGuf$EPUrKss4v=mPDIJAQGm+Cq5Q6wqBd* zJW_+OsEat5RVF6MtiR9-Vj zXp+%~=Gw;Bh>OXRA5^orN|Mt@O9cH17}C=JihIFG!dLrGN9u*IaGp1326hOtWWSB8 zkiF=I_8H41vpl!*W%mCxJR!~sh@04(wds!{_gQX3A!3YZaa3g~C<_kohFGa_+RHHh zjUS2B{djpj1iVI$WfnwcvQDIDLmik_l*T!41aI?d$MwX6u0Sz^cm0k=PRuu1TI z7@(2Qiu<4)*Z=x9z%pF*gOX8Nt@q_XIvUH1STv01yYyJRZb)R7qCnOO259p_RzX~X zgLTS2qfUc9vXclS?sGrI<0J#ylibb}P7a+VUt#0b`7kMKhSiH*#V19|Bnf1Ka5U5bc>lj|fV|6hGvF+u-v;%((~$y(MSVfK zIZ&jHuy-kV)2!Q&y8+~F4Ys4y#|uTxx3cZ5UpIX!Z#rSwZNK>8IQ)!80CmoN97YmX z(iAWF=VUbQY)1E!-7j2=?m8O8Ymx;n9(9#;E`M7qBS%xjkcHFD(Z2b1YF>Gchn$ZR z(f>l&x^hz}x4nT5ztgC^i6MCV;ZufSzV-{k;rtrL?2AM0$k zV8lSabuAzNbez$e=5$D|kD3X+Q=3_-^Ko0H8m#Cv)_cz_5mNhh6S`Jf~Yuw zv$-TuUPqU2XSl2spfBwKzsM zn;SH4fd4$oM8oCqit|%X<#F?>IyUz~0tbX|Ue!O`o(5(>6^JfM1IU5A-Zdllye>38 z7YnM#-M^K$TWBR(hzPtgDN{mljwlZoTV~}Bi2^AyjlR5nMDakZJ7rG1Ro^>(`X>!t zt6s)Mq{!tW^k>5dGfF7QHRxO+?B<48PWZuf207^7-23hwZi-K zQ7%&|l^W`aM~W%oe_>61YhGa;p!Mh>`9Z1#cV5P{K{D+HaGk8TdadxyQQfR@*v9cl zw-E;O4}LX-g>DiidT$@AxH6~%Lr`3JP(rY8F+s0os*_4#CkgXq=L0qi##BlmC?s=at;g&)im{24_UGZ>8%&1%?PK{`0336w3x&| z-;lqAZf+4wXW8dZ8cxoHz&5vhv*}kX62SpRQQtHk=rqsPy?AqMmjt({->`m+ryf!H z@5vGUhjIcA5==M|aqWG0$nLoj89p_EqF_NhShpp_1=%fcwYlyS2R^fw6LC{|w=x%E z{_~8-6=k0|I_rRpAAOP({4-jiU*k_?;_)V-wNMJwAc)$4=zf5LaF}x&a3FF46~S&I zwJ@x|&v#T{qw0pxK1`7eg{Tu>*sN@=`O`Qao%t&h0%3312GzQYawNXvS zU@{s!McgkE6(THsB^P_+R2NEGRzOr7&p;hmito$JH9yoOlt7ZY(>cZyE zy^=lt;3VL;@XxzzVfw=OXa|MU02!wXzh$uwB}5*m_zQ5==lyrp1MEki`sM9+xFm5D z-^+%W2^u;hcJ4rT+aR6mMEO=FKI(32@8SaUU}>DTU2U_y&Nus0RLu7#Ek7sFMtJCI z{cczZo;%B~aAY+@xiw`xU?gn+{d7j>7GOy@fF()506mVKyR$^nv;=hnlTPfI`!J$N zKGnUGUd?41wbAbHlmJKG<*`gBvI~OO1cifxUyI{Nx^Xpx^y}9qo|o(au3gcn zCo$91-;+6(5AK>P^jkN3)|FBL81Jpks3#_8yD zojY3lI_Nh>K0DZ-^=z5FJKT;LoC~+4Ql|NA^+_Bp)M_cCD?njK4J&W6xaJ0p#_v}N z5>vH}!a}5L88D_v=FrT582AYi`6xV>fB9|qQlUWcit!lx1D#L#mDRc|n5g4gHbxN- z^1EImYen+7u*XBEBHG%z;Ipm}%Ap z%c5rH7#8m*%?F^9BcqXEf-Zdk`u)KVlUsBnSD}r3u;AT?y>*AsSf%y1^-A^in;MSN;FMZ=e(p2=^Mn?fAp$H`5!tHSztpM1K@Sc{WbCCduO4uhya) z=OP=LD{QT6%Rb(ljdzhL%CQ>pA}okw!D=#a_+sD&QqDALpbPxg1Ms9;|KT5+ijJ5d zu?IUYu=>#}>{!6|mk|vX@oriBDkUcW*Ctm%0O$o4c(u8HW%#*pqwr?4fjN}45jrM+ z`l%S*#Dik#am4{=p9POR;v?@`1&|B&roHq*Is+t+Qdxl6PZJq0GlYs+bUTZ$3RIX@ z8;sBoUsEKnv6(htSPRi`<$SHNBzubPhjplq(o|AMbSIjeEx9?MjuU46Kg9Ru0RmoE z)|I2A^LdKP(=j(dG=<|b_XDwD?>zVLB`~pIdC{s#@esAX#;Hi~lkP#yJ;x;GdqlQ0 z5CSYpe{XNgucnl8fg&x-i&7Lz8&I;vze~Lsr`L6_54=6jGoL+yB{TdbdK@61$K2Y;6`Od-Qao2O_4SvHIG8OhOkflSsQ zruuMLa86tI=$EOoKWb8f?0Ku($%0_@!2YUQjL+=G@0q~4%*qHlejahulL1endhNGZ zBpPkO3S#X0*oQEt=Kn|3TZTp1eQ%%w5)vXJ-K}&tgESH%Al)#uv?4uH(y0hYjI?xj z49(CW9nu{VLl4ZH=l%W9xz2U*i7!0twfBnqUh7^$i|oxIE#BE$Fusq;IZ&v1-w-{EPsE zpEI5dLXFEdcU!-i!htYyw_B(=K^+uRs;U z!h1~wP!ipkuOl&;*tZ3OvFhjL^!r{MJTp&8^oaecfSu!3)f0wy>5%a~KX6UDtTV&zK*8HD(4S6E7Qg@Ti$dU z@g`OTf^9Iszt6lX#{hVN;r}8CK<)TMG+E6-@2nartz#*x@olVPF}_t}0RIsu$+wB` zUw%wReY;3zpUiJ&+t96~Qvx~G&w-0SWcZ2m{d(%#{UR_?ORdy3?`$Ll4;bQkgdLc@ zv~e9`8Os%vOB|>*S^bt&m}zsBO4^-6=`*3yNo47wizEMmhNH7Qd#A$Z^Feog-h`~! zy}hx&oK0`x;BUGHK*I4_=nJVDB20r)6%R9(;1}>yWz|=UV?M_skVL{+7Ycx&(pQpY ziEqxH3Mgwo{sZ|%Sk_ECQC0J&?U557o;=!mjpy>4tEyG!TY2zf>5L;vfoZB!TL@ljYc=opu!6-EDB^`>>mSC<8vfEj|Q-?02fyAF?vT z2>Vk9FzR&6R7K8XGTk4ZMiD_*= zUaWcE;lXjob1MhY`t{W7CmALzvD3C6v^gBt{v*CJQ~!h5s)6?W)&4L-YrDwjcSpiI zF8`kvz~r@7D}B#Ehp(_@fcp2Okm*k}ESz?6l-K_dd(y`QS)6$VH`qM#f8{0rfVT6u zIKVY>RIm7tWG=jDo@|Q1pYv6=l7I4W(WD)dn=^;x$y~J|BR!{j&SB+fcWVUAn9PS7g>{Z1yPP#JJMAUR#E#&OV%fBPY{A?(%g zmxwqA6{M){?(K2D(P~ZDLa{mk0NCTW%Uj9gXc?8keIVv~p zr8E?!81VHt8aKE zzw(W-qfUgWN4#@f_LYjJ#AfPQ+=twLV>?Ow;E{i0ws_C=?24-s z=m3~IF!>>V^8beK4KF|oSztQvH9aiaqiX;@A56ZT4;U@MN3Gdt0+iOqjG|)jCk8Fcb4!#uU#=^ z@}n@r(aw>Fsi*SUJrl*^mEfi^1A3bxdNM5bZ5br}UA1H*X?CmL_w%EIu;}%;mu?5n z?F4TPi8(*q`IDNM4+`vAHoqH-N-I{+0@NF||5n)lVi|e#_9dX&7FcmN;>8VUyDnLO zQ;#K9(tTURD7=ovhAtp=aXlE80q3rcJoz{!D`cIKlg_nPwowEUz7-Vhoe-nxg5l3g zSsDta_6t&_FIvtN(`$D%UD!5q$I)?_ScPju_VI7D?i~*DC)_*(QwoKg+)(JtJfK8D@$@2-Rex>5qIW#?kt*fll7PP z-(9j8h}_@q9J3d=1N8PJ5G-bII9D}QZ&*@@CgCfe3u4-Fo;8y}Yp#1znm*&=X!Q5c zKe7QoJNn#qzp@?S;Nqs$$QU(f`o(yOvi!5rT;Qz4KfJn^zeF?687{o$4@gp&K!oIwC9jewxss(Ky6qAx=v z)2MAT{lRx&l%PF=7GW=VoE_on5^SX*^|}w=M0+*UcNlD^;AYfvhLdjoZMhwJAz%?% z{W6Vs?Np{W!1*s*qMtyseq2xm7x?!xpo=S54viR89zUZfo0LP%J4ad)gio6kliCIu znHLabx6VYpR&_`JFmIv&jE>nvISB-QIxVK91%nOP)b-2>Iw2 zzrTF5!w;6yVuUHnrml;3#2=mp!>(slv@D^E z3W*sFgo7}`6_ywAA-B(`#NJ3gp6j2`Om{K4%~v($k1R~T%GRZRr4Nq<-YU)kkNV4- z%N_^PwkKRqE3oAz_%sRGS-syd0dd#*xwAgtE-i7Ml?^?i^3DjZGVRg6*b^hvgFdbRW+}(L_`9@$v2>=_4AHr?gW-83X$?M!~<5Z@;L%& zpT&3XTLcUxvx6;bz@qB(&7n1!t^^HSN8h(y9BnE9>0`Hpr5{wK0zkNj{vS46NGnEJ zX%jFRZFTVlZa?QDDg3SUuj4y8La*=7vDgdvi7ssXf_L+iHUpfVX_%&4DDFNK;XIo- zhaca39MO)V!ATT&&O2Mz%_l&z7hh-e&wg6a;YPDjn@2Gy_l;w@G!7hbW4c=NqNuI{ z$a0zfZ~m-mUs4t^DhVCdHGP@uMabGqlCo7yPDe8@^3wKay7$moj5pKgtb0}T_MTFC z>|(Q`%VQO$aqBslE8DzYx1rl2{zYrtG_F)JnTq|D`b-E>iuTM^;$$GH|MfpQ;bk4) z$7|$)FHC;Qo{^9(6qwgdfaUEH*}&8Ic8dN+DBBinQg$^|`w!w?2}v@Huy4k5cRvbd z3+sLWg3G-2kC*@J4`jbB01WmwB5Y^fw z?vMt(WsTn_p|47!AOiI(c1Z3+}`)S3a31v zds+Vz%vv+=nd_`GxM}~_dDj{)*BOT|xEeeY(%zAA4Rhi^#Y<9#43#IY+pgw3Lk>#sma z=@VqoDkX#^>mhL5$iMQ^0uB6j_sA>zA$!+1)*zvIX?mw`*L3DsWVK36^=6%6V&|}1 zY%FWEtE1*g;(gS+IQ@L=p;r;3)wW-#Echj_ry&fH%I}fHjjXq^Ws~@sOcyXj{^-DG zPgdfcT`xI1>R+|K@tt%a%%9N1kk~)%!dhi5J+z@NS1>cFw%YfZLz;hg-w5>w6#HAO zSFscL7P1NNHCxO*xGAc&uG)xk-u5ueC5Aulr+mLoCep05`1tkLsfM#8w!q>_%E?xWwb&rT9^%d;>Z`AU9^Mi7xO8Nj^D9`>&-YYsh%XYw2QaA!14>x$P1tLGD-LP zR~jEDCHrCd=xe+5zm@wydSUl9uXRd@M=2|$u9~IVfLj#Y_jnji)**y{a*%X6JlosF zA(zpSBn!w(5nqgzbzwyYH|%QvXM(Ru65A4(T5%*3_ah;(9h=VCTuyuy z*l~Rzr%0-X&2ldlnTbos=9*1LT`ypFWH0^{+J{23Ci+% zqfZ>-et>G;Q-C{1I_mtHaR^uTx?JUyxikbxGGTS{VZ9(;_{%suM>oa%=C{FGhSnoN zbT%pe|AP1+Q=Y&!H)%C{ByF&i|KoUa@b^!$1LZ;6((anEK6<^o76-_R@fCWXQ;dYZ zkSMGVq+Ewo83Bn~VMl&ust%M1B^_xSoo&*l#)%~1G(_W``{Ses7`VK2d-x4u+j(4M$bEGMl(W1&l}wdBqK&2VCF(K?0dm zy37%6Y!+!Mt4`Wxw#s3}}3y|bprXFET zgO{h8-3?ra=uf8}n>6j7>Ft^X=I1|ndm%M`@?U91#ArteMrbT? zXw?u8AL7mOaIs_U;#5Vb@b-&lTyf&!pRVxygnkdP@wjHfP7&*%1BmaZHxgU z3kD3BC|Qhs$hvNuCb&@DFEMi=Tj1(XG3yzjGdauXX^1y5O+{kDnN%l|o z@V@q7F=!=CM8ZP+BpN)Si4sOWp62=|(vUXb{v3RSLH?%wRAtCUuY@kXbFw57Z0I}% znlryV?Lmdb23<)8ut0Jd*R#<4TpU*a{MsPkSL`0g1~YY2+q49N%#Gw9*fOl-3itHb zqDsDE(i}%O*SL=DhK!`-eZ&N8%cAnyYZx$)`Wxq!-(CYws7??9Pl^wI6Y>VbyCXfH zs>JxyqM%eSnE=Fzex^Ixy7xl*ZO%k(yU!jFU9iwhD_4cbV5^FlG=(qBkUxKHy?F>UByTN!)s6Ld9E-2PJm0v27m6r%$7*T(9f5%7)i!U>O&+0Rub7${kve~5<0wr!pzG(xj-J{uA1?=Ay)M!*HHvz*ev5Uoqdpudn> zncao)m7s?>w>CkOG2cQA@TbSRpXz!2FH0|@m*%`sB?4QkCnhnveJ?I6{IYH~bS{R9 zTLNx%-uQ5SwVZb_Xzn)fFAvoO4G*R?Gv?XN+q}dvx~>v%ysrm?NP(N!L8~{FG1Q9lskg=wl7qa>|~r| zzWzK$%~ALTKD;K>r=3o!2pRYrtz0d~+$CA|p^9RrzJ#lTrGhG>sjBT@{pM-9zlA$= zU`LKzBy6pqAz;elC-;MWWlzs?06&rCIWN-ecLpz9(U0oOqBiEdL5(L7%1)dDtk2?o z0GI-y@{1SVHX39bC78MSlEY{>Y!IyJs_4>PN)dtymfHz#VC*4fdVRNn8EmZM<(g){ z-{dW)12X--R!Kp#x;(_0RcpY2rjR%i7>CVLOq0Q`Gb>;>rL(0cDo@OM>RV%Ik} z)l989BII=r!=?5T`qB+Gdeu_tyMC2~o`k5vb+i9>N+3+x2RFzk=$8o6aO=u(J zQdjoV=7)jX^%|+ErC7<0N2|v_owq+kOQLTieW2CJbP(Y>DTc4VxXS}Y0$4!OPL8My z3h>p+0CiJ5iGfjUR|lca+(PTW`0TFY<^0>VgnLye6tW?O?z&2X9B7#-`7DnA?@dw# zLmQAKT6f8Om$$d}7U(+=`s|_dC*h}f+0jPd)xwK(T-J#694cYRE#Xf8T9Sq3ltqhs z@Fai9mA{2Ek~Nd|+)?a@`3%m=+mEbA0UX7Rayk%I;awQ&_QVv=3|*oj*(Jw@V`c2!!k#kFi7JEqK(C5Aia8ALC`QN#&G(t zUDQp0QwgvU_pSE+sDz==(eyWYAYvew4#v&x&E2(llPs6;Nw4M)Ld+@NAB1RlQ7K+h zgw}q?PB&;fPg#24^Iy24klo2^gq)_rEKg-DZkxVtKUuPZ#tF=6bY0P1c$|oJ@4O8ObThxR`=ak50vR{BMD@Q zQpNbv*j-B3zB%H~3+HDpl2%zO7Qy@^w{tzD^24==;wP6dWWPICB9{*if5%P{2;Tp@ zL@F51Whb{;J^r!yUwi_W0s7noO^|6C7Ix-tZhN40 zU0GZIe_q+O4}TP{ZT|m&g}MfMQ1`_kUdX|@c2^`NaxX{ARFtUBD5UT@gyutw@EAc3 zoqb|`#T_>0=w6(DTAgrtRND2(M*nNpw@$d1tQR{rW+A!{&PQe&!56=Y^2OMKPRd|U z`I^>^2inTweEJ_naS6s897CS%6`Jo;J#1;AFRq?I&tEMeRu0=6(8%?<_j9#D@tr_2 zdWv^HIDRv7tO-LM1_tezYfXjB9oq?$hYd(XIw~Ynb=kHS^{wtI3c(8{2#+QcpIIQ1 z59m4cd!+`FPG*+Z)5VGZoHW*QsF2M1-AK=|JyvHNm_0h}z2fv=Jl((E0&F^K{k{Lu zz|Nn6fyJ@q{wK-`s zhr={xpem9+{REW^s|oxU5%?RKmGPfjYVuN9i&!}oTB3mLejd0cJ$(@H2Gy>O=raB! zP9wdewQqT7OKj4sT>l*utbk>`ZT5*swy8Y81%<=%IbthUT@Xh#5Gy{6j0$8o0|^E@ zwAJX*(*7x(_U;Q8X>`<0+%VW+H5N&m(p_BK@CTy?-0r^~BS6PfV9@aY5}YoU(@^w1 z)D3c5a7~Qx%G`Hx&VBnt#Y@O+V}C+tc5er zAXQ{Yr>V4wlVDy(_9ZvtWP~EY!Hn#E)kDp1a_~h`feQkgYUYvf=l-a$tRn8A@7Pqn zkEJE>XtD~nYl7(m7DAU!+8;)jhRx5C$vt_JNu{U_6=WOaai42_+A&Ss&-2g+ytZ0g zXB8tNEwK3HAm2^q=lBE5Z+9}TOdD-c72uE13C5=%oeKNJmy*j*CwFlkgd!UUg|lCd z(0ZfdqTE&8RlC5YeES~<&wNJtdEt+l8%vBjxPBC0UsV^RE)M)a9YN0JmSi7*VlT)3kT%%s@xLaL8^r3}s+%kd0jp}ix4rTxs01E& z->Zi(S()EqGgrPnoOg<`b4$|mZ_!@`0f#AUsWVmdN+p8s{WE@a?Bd&%!e-IkH)gJ$ zSbEZWE}9WaxR|qzI1}h>@cXSaSgTL0>1+CfcjZUcrX&OhDHg+88VR-Jv+# zgL|jRC{K@1^h-eM0W#g)p$#rt#)~BoM30>})&_~6<*AQQdErt_fMBtPC~E5ajaHZp zQ_r~hHSC9|j`dRkjlr^){v$~h7X#0csYMIY;j+Gj(*J0FL}P`=zCKB$~*DeyPW-W7xEn8D<>C0xxpTm-2S5SdHVNuD`80W zj`{55TOq971)!7yLaqH{AIY7CalhFsH^@3gqdf>gm7Tehl!myUzVhuuCGR@Zj|2*w zZE-(S{T%zW<<+i=hgvwUj^)C<-;AiOotK_v6}#3Qw#`P(4uW(wHEsNgSYXk`gQC*nr<<*8;t&WRH-zwr>kr2Yv=sssAmR{Rc17ygEPy=!i@Np~Rni`8F?=ENK zWw)ps_jnRznKpBb-Kl=heniqxeE(PH;%9>p^_Snt0gC-Kc}ag~ar@16K5AhkwzjXI z#*+8uEuG-eS0LX$eNPkI!iZ=XsAIG#&?s}dLcsP5O2dnP9eBnk%^K@X`qVKZ2m>QV zu!yhrdnN3;4-PMzrlsxjwdQ-nZyog~!(Ur$kV!LDE5y9Oq0x~%f`cGp;mnd9nmY-g z^WS{%fQG$(7ZA2{)_JjuQWwS9X!)re}a*6?6{^+`bxiyZelqn*)~6b{t6W+gS=b&em(kw z;3sb@uLnh*ZHzg-p`vnPW(PSn&DHcdc?j=D)sqXKj$TC*X@GKiM*{}H=Yg4P3L#mW z?e`f+!+|2QuoI*!`i=pzDj_TM?!WBeDRuC`hagnqI6CAwHR#d+IVFH-9%=qpn?Uu~ zb!cMF#hWmG`uA5a2a)#Eu=-QKC>}~(oR&TpBK~>Ss3=pb&d`1HimJGOZpSsbc3BoS z=Jh0}NQ*Wec^C*=2yp_67v6~|{ka276IO&{r&9n)#f;vi!_q8Q^3?`v8KLtihA{^i zbJ(y2;dqC;-Te}q&BjXboIOc=lJN8_dN=Vxn?*7FM~s^I#=@T{?I*-SSr!K-xc`Th zQ3ds)Rz-DImhGVv2lntFC% zCO+M*8e4c%KVUQee{-U2o?%Md@s(iGeN0ygIqU4(sxJl&dD zR|8XOWLny7gRoIvKMmPM*A(-Qtz_9m5#|nz1ef}lIOO%sA4_RU1{*hw66iZGQ=JgG zs|5747Etot_pvE&e+M7}BbHO8t6{ZOH&|Kkec6$Cs>A@>(#WRhQTN+J5}o?zE0+EQ z;W@&%g5zYLpeEfMdy%r^3X0#tuwI#Mzw@`CHpWyL=Su96HWLtx3Bc#y#kJ*7nY{1J zi~rpmsOc}SOd!a)#wx@ekP307%Q+(vk!#A#V`Nbsxbhmefd&Q*=MWk=4{P?+`$jBF$P<5)Er%CUWr;FVl zP~O`9tjH!Km3OPM(ckobtSw-DK~?*zT7V-$Mv#eN|9z*j zGfXpJj~zCWg*rXP}oik%mMiohME(|0;)8x$18&@rO zXv!aPgF2z5u(=PM4nMO4Em&1HNmYdn-3O^y+YZY_5QC)B2#RT@(rbrc0;6(weY4q7 ztRifFeIf43ES?v&eq;VxGX7xp`a&S`&Vb~vLXbdhkHx}!rhX?F!)CD1R-w3r#EIK6 z`y{7a;W^De2FugTor{+c$4GZW(UuACfPXGnLCFFXzr;25Mwhs6okJ4pZqbY7Q@tM zy8)J@{T|EqqSFDWaUj<&S9F4RDQqe9T?%052IUNw{)C@baZR}<)`PGyK?x2q@oV>? zEf#tB$zArzLPy=-J!p*J=6q&3a9Bnxoz}4%kljWI@=D3#COjK|_a*?8S9y#7&C4c3 zRqRgQ{<0s5+hG(9^;h#eqRiOpRWGY(#P^VvrIHq?BPyoZ|6yoH zEqd+(k6|`U$qIKwxCPY58Fn9N!6!(jhU7;wOErC!a&VJH1|SY@HyvBUHcUIF-6v53dTn_mXc zPwG8-?getmB)QK5aAjL%@&sw6S&$v|_VYIR&5i+vr-v?Vc5*EL08qa5T`4w|N3_LM z16ttM=xUbU95k(sXolHIP8s8(9gkY0UU>3)b<$gh$x%a*lcxyWEHg^50>*N(ISgO}`geFH z4+=8O*hh-$Z^w6`qDFU0V+sy_aT^eq361^ci!eXeHR`sQ_{Wy`dXAVco|g4<9Yy(F z)Eg(>{ynaBk^8FMDQR8Z9l)BA+NT4)RL=(XZCRX$yz+2hJyX4CT(t1(I0)zam!W zAHvab3rB>|t_#u8F8_68vIoNMncc~bh}88rm_epK&yEeHQ=4AaEtfv&&aLx_T(3ld z;#b>v+0Abp+}wEk-NAfmTjn$IX$JGtgX;GI5hM0>3EZE;Feo!J+j|afeLa(Mys8-y zP2}LPS~^ZIG@VQF&~S+>akOD#F@sU^-#-k4}M)p-6 z4FQdP;N;1iEP5OY>v*By^Os>;8*Hb*d2Fie;U#4u*N_Se?jFZ zBfFBjH`<<(Q`?*N0U3Dx*w{N&nNO_ERh6wpKMR%R4VecP7p+Q?-%|-VmA*{jI?K8A zdDJF{6ZM{FCCzXfQt*`$&$aEBN$5apY-iS}=a^XsH!KCE*cBx<@fFhZ>p9mQUngU4g1hW_?5_@eyZS;1 zSDB{TX?SjfRdz@PbI$lKQxYUYzTr$U3=qc>JIvd>CgVoQe}L6o)SLkia6@7qBX=Y8 zPc6;oxASy689!R*Vur^GAMJ){#4j010U(bs=A2QY6>r)u#$8m_RqQa$MwMkHf#5lr zR)@8Ku_3$HhwiVSosJvz!tYntx9{L2l8^1c)R1&P7x<6(Y{6Z|d<;>sfxlJ1kq&*HyJ5oU#9E}X+Jvv^0xc+*VYQrq%_YnSS9mMqaU#wk7&3afhN^1BD;Uy7TQ%P)uq7f zso;1}*l@H&qFg&cqA-P1eyG@lq+ewt-{&ChbSU~rC7}0e4%Ka+MIb~t z`yW+~8~cUa8ijJEUqNlttgIc4j59oFGdQ}8EfcoQP1mLwv7XicfXlpX@UT+ImSC7^ z`a~b>H7-6@bOR_9c<3K~^Tkyv=}paFD~5WlmTiO>9%5SC+n~28Pr_+m2c6LL4EY*QtdzA&k$oTPnnGW;jsLe2@8YQJB<%3S6r|&nx%2nBD(d#4R+48 zguaV`{SVfglF5E3vv=XUH-R|&4;nx|38p1LdyexHRA4T%Rku9vQ|&_Y*VQKvZ+Xpk zaH~7R%De?~RMaA1tbX}NKiz4kQrb6ms_UUJ+Pe8?pWW~Y@o3Zrpav%x=inoH>RHSKW3_t06{eQbBICU5Q#x$#D* z94GO8Qj*D-Z2TpSYB(;Lj%fQdHIER%Z0{3o1YW{Cdy7-01w_vz(KiSQe@hKKPDKu%09HaF2JC1i+mdm%@q$)x4V`m!XCb@@H=1Qs*68>#0b5q=?DDOyY*LM1_=X|H z0K7ijkKTrW3i586%2Yt@bFUI^CnFZcwHrw20y`;_v|Fh0RIO(wN4e6wEi$!M%kUo5 zVx|x_h@D<#f09e2Rc8$-iY)Lh6$Gp`cMvuh$lL?e{RjeJEfL5B=K8>2w2@I^tH|HV zLd$^ff5Zd-z1&B(D4oAf6uPubFSRGu zolXBeE1aCXtN_bEL1K(3=dy%}r9){hB-WhyC3d{89FFRi-OGnYfzuB4g{a&=8KWdC zaQz|0cj_AAIc_g_F>TsU)p`f$8!m>76{p540ntuW>M`1q~K<~Z^>VD z@Zz|fM5?z&>CJ5{q%{e;ODK2fn?M&u^oH;#d0ze}J4$N0gLD##AA2eR89HGPr@S$p zvk+&{{v3Jhw$(E2)X^`u)7ux*aGo2l*-|?6)EPPKJF@oY8d@Qf9x85&dkXRl{nV30l)o%Yh^S-<&E zH$}a#zi~M)g|v+O7ss@2sTEh55Fe57KZujoIiPdGX81Lq+vHqgioyxj`#UV|yCk+p z+22JHgFmnqmWdOC@qzfUw1V0>?n;HtuVyxB5v)!;NSU^`Qp^ZmKODHWumgBqK>Qlm{HR~nL-Cwa5^o42-fz(M>2oS-hEFz7}7FG2|Zk{JNfAKfP$MA6npv!)5k0e zS@{}&|LgR-^huSay&z^pFRxLn?xB#kEmmzZy#lflTbs8+s0yI|d>ifEj8*lmm@mF^ z`Hcpi_Dp*@UU1KU0eYGS!NqBL{C-hA78Q}{DkcyiCJtm83D=R6-+pl3E&L)(LqWu< zw?pls$?QcoIl}WgLCvi&V($BM+x}mbp8a94%Ra7V#&2!Nn-mrgl7hwj^kXxdub`mr zdM>l9h^r))uMJ+b`;lx>bgx?R(;WI5tgBeUJOBN`8FF#S^OzcF!+cy*lW?i9f=P-) z8>`w?#Z+)6tucE>Fmze|v?JBGr~@}=AQiIxY&^QOxyfB>)1De#AZ&CLkbtNXY*J4k zDUX#qg7-93R3*4I9vVV9cunWJDRvT(guF{pm1hRe5z-}-pTC(1L2@di*+xtkCxNvB zcwV`C5T{w~*~R6en{jbw=4zJh2qGhwzfQ`RN|bR)-nue6XKV?a_B8jZRYGt5_~+&O zZ}VagZm`LT4dkeBS!}`3q=R^Z75bQTs*&UT&F{9-Cme|k<*sgpy0C|L=5a& z5L0TOC6IG7*>18)QL=WV^7Drj5aGHgC)^t2c2C}hCTxwExVX+>u)RmaB_hvmTz;=u z)bm|ZwdJezeO1=xsX)tipe%+U^ejGX*Fgdy6?4_|j|3oGu%pU~H%|IMEUb@h7;tc0cOl4^q;1@%&v(V<3NnE)-o2 znHW}Ag{dR%5~q^HT;UxNq#F(aSMqkr0bqtn4Uk34|w_+c2D5gg71-9xpnm#1h{1G5?FfAo9 z>r15ZV7p9yDw$s_`S#LSpHf^ZbquCXPnG={THVkt<^y_1*DQUJMTs_hn zCwcv?dp+UtbKk90&M{hQbmtn0=MGi*A$c|H1DOSFCsH5=jHVWt`y>eUVnmg32zoEV zS){Kx!~Y^4E3`qPBz@I&c>B^_e3;e~rMvY5WrEhy1}Z(rE|a**PDczE4HmdKK*^Fn z7t77@y=S5}ys|dP*M4Dl>C7!0=ey z{vZTr4#M0?GQt9x(W5=9uKbY7f8G;+(uVGF%>zH`$LH?oRT(mOnLbm%mt9)>-Oz$g zgB;#s;dZ1j&|7s5juEtOncC4WtUn@Pzto=6zs~%uIp=n7myWWDT@!P@2XLC6hD{eGyXp+UR;M-1CocbrnU zwCnLlrg0W7K-0rr>uT-aYpQC-ylXmC_hf#PFWDzq(Uel2#J719i5kMq&U45;TcC*M zgSWAE8!6&+eWs85*%f{#%~ju|lk-T7nEmDuK34t6q6ULPnQvPk z(ZlOYM(iCB?^Ct}tVZtp1Gn4X~+=OXKDUI>C8cf`9GG zqWEWy2Evl8^M{sOq)pLZF_b+=t||(#2LCo~b!EW8JS0mJ!sh;%={M@8gIdI_QxrT+ zb*fV)Jdk5evbHMq+^L|V3Q|?zfczP7GSk#z9n6=XS0coO{W9(SGvVaJ!9vuPiBhYR znSnww8mhm9RQ_F-=@FusmTssR$wh7C*j4xke)_2TtSN7hc)#-r&#b>;4u6hA>&NN` zu6izU_oAVNXnG`=PcY$kI-R1L)w&HVLCf1Tg5_3UgiJEK=TKVS*~RcNXTm=dlY-XWm>=bRJlWC zyM(f~9Z`keRNHc@D(zUq{o1g_7SJbSbSV*G5=D%wLo2qv{;_S)I`&Sh&6hhe&g?l| z&D|FTF`FtrUyTJm%};_MM^6*E;d@W0QWjr`8VF>U*#bi5sze9-&t+U zobGVMLeyJtlS5fMk^xxF!olLexuke4YR)0M{+G)8enm*2DPyHn%vc-BnJ0To9H%=; zO3)L~xT>P-!&QML-W0d@=ZiX16(F6_{w%_l)#|x18NSAovI|yyA!W2fy4WfLE9zCT#kg`Q+A8@q`%#S*-Fk)qU23 zd*c-o6cY%^o0O;Vb3+DZ}!JmVPg;?B*%-m{zM+w#mr54Rg=7ckFKk zdM;tKsIb(Y%0Y5aDRM${z~OJ5VoOw_+1bt{ysd!f0VTC)M}iKl;p z;k`}TAP`X(fs0agtPEvZ;Xc#+F(jQ!$`$CG?w+tr_*Ted`xkv$HTK|90=@8YbcRd! z`_@IvMAJ_LlUmD1vR>}egBJ5tTC}%Hj?{#s!8hNii?SPEsH8bSVO@sTGlbU!!YE4b3H3X(uV;1VYO6bBRP7sGu0(Jbp^QS z-MLJ(oJ!V&ia-0(uiL?z1HYNtt(v&ZnI>!)5PIkvx^rdRgK=42(ejm=o4IFW(*E`@ zQ^@ltC+O^2bMBp-v7u#QOo7^vzBJFm>OvejBwZcL$%a>q(WxO71}Y`yrh; zw!%a5jia-Eyy8%0R`z>kuEkQ@ZEYFf5+!NhsMsG3!CK(i?jIkXntxDE!2VVGyx6#b z8Mi`Sk@mVnwF*oWkD8n5=gfyue+j2M)7TOtSX-IXG2yDM{AMkSWT9f- z!|h3Jd>x=&z-4lmXNI&I&8%s-o_D@N>Y=KD6i7B8sqh?=ys}P(NJ{#K&2?wc(9+-NNu^1oju$FlTUJU3GbwmR(|5M|2egw|+G`8l zQ7ez;Ef?7-k1At6OJkYgk1JPz`1~w0U${>9_cRve`u0cyBov_hHE?>U)#>*fEuU?3 z^wWq9?FiyG74wejW0d>PDeBwIaj5+-l_J7gt(9HtbKeu9l1mZx;o%(lzM{_u8pOaU z2I4-6f6T&}7%IrRhoQ*e7Dxzza&_snLr$@YQIz)OG?#44u;&=_##|X3?^TDBLF;0P znRq)GH-|_XxwJyIR<_z1)cu<$Bi@C-q7lci&-zwNbp7B1 zRa426ScLd2(5MdF^#tyPadUrKct-nm2Rlaao9SU}Dx#MJg47!A7p3ZK%CL!JaxkG3 zMaY}(VCn9PqjTa4TGR3JT6P)%1L-Kq3KEsb>fHfQXx|fK@|dXfw#3jA5$X%#y?A>P zhNGk~(kE}1h0yp|=7L8@MW^Ox@1Y$34*>N*3ct1P30Qm&o!Mt?j7EPX!5~3`p*`EB z2O|lWa4W3v*C}v_Sf$=IU+xhT88&CcDk z1?slOa5?^d+%GN23`zNwp^cNBzaGb=;**2xtTE>xyXH^c49G49m&I9CURL#@2=MCc zaeJFT#1iw}w#^GN_(MkNGRRd4m zJF=%f{Hl*MvprC&Z6a0c8e`Q|+EmRB-VZF*ELut=e*9fo{E z*^YBVfuj~2ZSHG4*uatn@sK@h6fKcx!cuYr*NAD7FL6SS(Z^OG@UJ#>BiB_e^TM%_ zW%i>lFl5byJn$VcBTw~H9qi0ey}(qR>iD1sCy5O?W1?+HoaE!+*`p)9A~^g)&S>lc zV?#EP3aeG3h>0GIX-c0b|%-xnwya@mmOgLS&AVUX`upr6nO4$h5 z(UnhJJRSs348K}TQ&_Gi-Mk zIK^r&*@97uh%KIoso3E!jAB0e$fMiu>pNuH6Edc{Q6UYSdTdkleWyHBEOsW>R82AS zgf-!p?ARo!M;@(ZLZPGdiZ{nrAmC^Y7e)17`R(n2(zebm3ZgwT=IJBo@sp?Rf7j-Q z*vP^~ZH!;l2Ugp;MEuUu&Yg;y_r}ayv8rRA_4K0J zd)zYqetZ}>ad^&*x7QYbk8kyP{I1JRp20VICFi*B@Tq8eNlrQE`;lL* zk&Uv7FE72s3pcVc4soy{!T29jS|IgX_8Lznu&S%T5v|cKEcoC8HTr?UI&j)pz@r+6 zhUOLJ=JSg=s_yni0>_*`F;+3Ve+dE)eq-jTjheA~vW-2Gbd#&C<~h%9i}pTlj^zSw zyeig%cK&;FiZ9)z@lgLM;)~aqNg@JSt?qAtK8|hYz$K5P_`!C(o#TQ$iZ8qF9aWCv z!-<@+=q0I-T=Iw#_WhUA&;5l5znZ^v)n4ne%~h7UxOD%fFJD&ip&5=3F|>## z)N;-?hVoG%C*tW>mh>Gm{)CM2`y5iwAM=MRWb-6cYu>e5kWunlkP(Iz*gBxj(FiUr z$dI)6VA?eINqeE`WR{spzsMG7jOWP64$yFpmg0D!>i%@V2}YLM0yfaGGf+-Dw5zE! z30LKxAlGWnZyNfHZQhCjKN3>eB8S*&^kxj$Lusyzzq1&@%yMJSDOh9|${kmU?+6DZ zDr-qv`J|mO*pb+g$yCPA!1F?3#a*XPycxTSJ7eZH$Amch0)jvy+Z~^VQvNdDFqrsP z=TD`nb)}#pKb4TYnG7A{2qv`q)MmSIH5Lmp4?OUIQw+zY0o~sx8LQd?m7*xnn5Q_s5JwD&;K<^JoMBM_)@Ek8SqmA8#nE*?5TeT3e9$gC}u8#v-=TI%0w@ zxxoS}HidB;Fm4RZu{{{XReh;Pj5c;swxIBZ_wg2D}y`usrMD=&4i<|6I=l3`x zOYqxPmG<(f#b8{wL!I$Q2DxP{+2S>$ltqMy8e6ejG0|#HFnee8f(uXI&Uj3n6U8LS z-Tei&m)Er!NQXvYA;Q{z*%~zDpcZeWYky0y!zyuRDTmsXIZN=P-ZE;2xKTBdmY9yu z{&6ICy51W~;_N$M78x6W`E$k{j|YyEN9&NRQ5}0Yp?6J=XuOcHRR_nGJ`4lifyscX zWo})k)`!_B_IUiNtiD`|f}G7pg%fR~iFpjdGmF=K!B zn9N~y{=k(z9mi(2*EX^6o!ekXy=nE|UzQt5H4^SI)KtHf{mjF75EluKHYbkjFbBDO{CHCCc6$TwX ztJs0L4l=zfpFI2VW7)}8?RZ@k0Xuk?ZNJ|Cl2>f^yz8^Jx*+2*I9>?IJkUwZVE-e> zkeKAcOS-uq(p5Vc$zF&jDFWpqCl!}8LdBQ*@+(WBO&8jiq{56n$rjO~FXGMl3|n~b zS+L6^tjQ~%w2yk(vVplIFRzGcD2){>y9+Sl%rFBxpk0j^uN`U zhngFLploMtXFhP_h5(A$U)i-k`Z#2|XE$AVtmtJykA&ZEPo!)H7F86aW4sbA{BZyS z8ap!WV$8yLu>X&7_>f?0?;KK*HL`?bFDrJoM#S9GC*kC$7drYO8zt-@LQ5zRaJA{=53jeVMZPlaiZ9yPR% z#xw1K@NEOT&FU_-vDbWlXa7mHj!kyWISkUM0T<2c$Sbf`<$UWz5O-+C$AlDQA3q$y z%y#s6%FZi1qmPNp__9kMHuH-A?j{Zy!j^q^y_t|<2j{N7!CsMI1xS1Ps0Ic*68(tI z)>U@jFCw!SPfYe`n&j=m1BYtFWq)D?Lu7EYd(SY;4n0bGPzx6#hW6H6{7=Lbs1&W zzUqe`j$c`7U+O#b+razdv&FYC;v%=@f=q@rFnP5UbHOH1->_JaDJ~A0ldt?&)jQX4 zv=^^3Ar);OPU)@_KjPSyfCs!ahU&nhp8niu%=lYrD~2CRR$@W}r< z#p9LphxjpR-)x)|ZiG%|wUX49z;$Q8;T(=OX;~Viy+k(vf58Q8g z>QQXldXN^WiP@h#&?DiO9gK7<`${jJ7v*QKSU6Ko@ft|Mnta55_S!$PwD^^!RUTDW zXnf4acC-Q?H+brc+HfF+)IJLteI9Z32GL=@!5^8Hb!{F?q3~VVR`!a+7b$$TSAXwe zGy3sqM)}2cu^l^3lzo-|bqhK8I9i+-&pak&pL1JrlUtlXc3hjIv2-XLlBvylwaW>N z_sWcM`UHwa8Tjk@15Xd!D)8N|edbL*tEu6MS~Pn{p#0Jvp5y@!b|m^4H@CsX{%WC7 znv6Fj;Z?ceLE?`qSlJn0+0cVS!bAI#JmR?I1OJgP_G1p~p`4w->czqP*W0}^P|aHM zK&IU^QkMOnNAW<}8*oU`_k;N>FCEi${JYassT|crUhq23b2HxN)cbE>?poFyp{=pt zN8(p8(jD2WK787?^}u%a@mHgTn2OGh^^4 zLxXfo*ulvw3OL%})W(LC4m%PX56Nv@SgInZ*;iBg z5~pSn)>Y&qj>>mbU-7N5`xs4Qf@2Ac-(xot`igDlMo%6{;beQYEyz6mo@Z?L-gmE$ z+tYOau0fRDgfP6Mlf_@AlvHl|clfi{0(lXO6-aerNClx5WPb1MZ{GOX>job%@^^(~ zBL*jxN{<)K4hI)@CENh+G<&73!cx1BQ*FM(aP$GK*30#9v+d;Ho0r`$TE!7OOgq*O zSG4|{AWkJ(jgD|Def7IkpW{?6KcpI<9_%yU%z=I63T@SaTS+)2@j+g~%hqv_49b=F zURPZ&xmR3|yQjNlP37N{%vPq1Jrd3Afqu?iIL;s02IdBaOA>!%9Kg=t)l0_@Iowo= z{r(rfVte;@d)9WpePl^F`X8BkHj!BLCC(}GlccV=^hNbrI;P(|HH`^)5Ao*HZ;1RI zbs5De;3JR5f=vC{>mIuXnCdeoX|onCfvj45k-}k*J(putG9UVlFLFw6*Mxs(D2j>^ z@6288xZMOp-^8O=k6{gNy!J6dqjA6{NxgG4K3TY-(=0FE$T3C|KZEZ|@HT{z$RqrN z0bgWj;ZM5YiS~eL$1?scvF;cdX5VHWxa2?`F4~>UDq$NDGY)I|nMO+G+6Qm;%J}%f zgG4X6_ulK$FKfY(-m>T=4O*nI;koy(n8c|TWwxVV$;wXsD*sD%I99Rtx&^waw5zzQ zsVL4GjxBLwm()x7k)!Z_nA*m_Gocyq2WKvwhkEeF*1va2tp1-YGLC%a5}v88v7}}G z=m>M@7^|%NA82sIKuV9T^Og!R=|6r2GE~9q`Xiq@e+Y;_Qaa*GqDNM{>T|&teUER7 zu;sRLQpX<|JKyr?3VKe2%rP~@O3loV1$})AQK5JV2S!paY@QNEYfzO>Esiqs<_J5!v2)WIkMCq9h{-f8}8kyjkF)ge9^r!5OF#}k> zBLxmIV?$OOk+IRPlH-xltNzqS|Io&FujYc(h+&jc5_86(M_#f=;BX@XH z+A(HE?7IJ6m2PiF+AC~jxK&)`SjDKi8l%>877sMGkBdhsQ^|zfT|^7U69e68GU5wRu^P;u{=_2WaHJ7 zFZQdd@?910)5_3v@rEcGM+fBvce^e`WPHivs<;<9%{cS)ebb1sOA-rtiTlV_dB9wX zk+!Jm5Wf^3Jlqv~$p(HUZ{k<6JvPd&SVh_EaI2OTMa8Y`)&ExO#6VsZ=g@8hd2_SR zX7(dqOe%pNj)3L>4#v*Y3_cOMa_|OzTT&tJfAPz0LFU=pOCNl|kI$no+pY7Q(&2TK zz|b$v0ZkEIv90}=ikEl>&NgrGW$L;ku8UH`LbUl_cR{A5;jxG*Nx|L|ZU&8h+CYc& zJ~b5>-*2YFR`(v5$^x$Hz*QSMr0~_QwrqMy&&#Sqss1voIHZTCV!}6ih#i0W2|v zyFlj_RP~n?J;-XWHtN-mPvwz)z)X%btMBFd=#j_iI8LCXB|b-(Y`he!<|urdLIcyB z=k3s9wt7X0TP1*?aQ$VEc%B4SO#5W;^NB z??waT=c?T&fsxcp_KpeKj%m$qr!5Ijt%V)7u*@+66Gt!{k;8u6u!qm_795e0M;@F% z=@%ZeA9nGmj@?HQqbYOtjfmi)m0MhpvVcWaTY5$fj9M#PwXZPME`K%l6 zQJu=$0_nws6gP_7rZ4sN1(}z>B!W6RupY3r-C6aZ?L4Sl?U@9Ic#`lmSGinSSQzo8 z{)WD;fA^K2-WXjOtpG|Nq@@Ysh>LzYZfJz3RB(x7=D`-|y&e4}Idae%BT0(}38!qc z*Ng#sNx}i#HB&5I4tR z{&WJ{*neMCae}xcz%BQI*V^KEJly5Giu5W-)vH}?l}q+& z6Bd7@>@%j~R<^Lipq4lSi57X4KX&|sE%T%ePVL~1<6U*xwg2uS0#P27M5`dE2_mALAB=b216m}Fb^ zA^Q<_j#K<3!KBVKO{w!|Zx1B#(ob$9y2itT4F9nuE;OpweyL3XGv_eJ?_?b~_NS{G zv3M&7*G1w=1|~H4BdhO-$o&IH*k@GLuIu!DwvvPND@?WHyT^?^VE^F@p7}fH<;JKt zR1GYwW=A!7^~6jeJ3R6ZOdRuS#~ZiUU&6{h$IEPGoyXX`eicROlSL)`^ zcog`eY7EV{oGD{p5UmU^Ai8ry#oUig0yfLS}olfxjg24{2Nb?;N=q+*kaS z13PkXUNvwwTRn!Z?Z4u@pex~)`&M=^%HdKT;A*&vulB01c-1RT?G+Y`BzB~Hu&Z86 z#h|Yo=vk(Pk1CMQvp$T6WWb#XW z6*Dm`00>P?{E^ibtm#i1IdjZ<+LD#O>Q{AmIo{Gg_6zpeml)WO{N)Fxa^R0${@`Te zxWHbe>fAsHho5BBxge_*{m=!Oc?~={Aj3cM&{q2V!w4u-mNIjB1|Bx?V31glqNDeT z3G0Z7{sktsnO`OaM{1%rbB*@mWmWR=bH{<{4Kg)x%!Phs%}DrwlErnrlW5R-{-k%q z7R8RI*r-;FbmTDGg~NuNF|?~)ej2YDhy4Ox5<9-gu!WU3iq&5@e1%JNTjS+f zleNTJ&HnqMjh#2{*XGYvY75aqpYG9nYSH6I~rqT=1~}YYxQ9=y|;2m$vY%{<6&+NA_6{@9Zx<*lJT9 zJrWMdij8*mKzQ~ZzB67vD%dsNjxj}G!=i1+SREeDGdSX-a%wfqbics+Lb%3k!>`-4 z?>&%h^wDQ8$W^OBOu)+@rR)4VGK3HP={Dxka=y+Ou;>S-rp&lGW);uTto>B=*^_bH ztA4JBfAqX_SwEB~c|rl!v8b-Rup#9Q4yJBv3OLw8!xpfH5SKB@}FuVUEjEbyH0Um$b(?yFC+s@ z`kf+GG7otcxNKh9iOEEOS`up2XCCQpwTKt)K5nLHT{r6(ZETHE;gNQQ!#>zhnh){y z24=vtFa_$G6-EYdSJ{P~G58{}BjrN@L%ZJym1C}Ab{rVZpep^J&plv@Lme)cP1*fR z{GQtlD*b}&J}nXfFYxp}w0xAtdQTpc$n{dFM!h8H{XdAx&tBhe^OlO@)L5OnR~)9p z(FwYP;2{sj6rDoZ5)l9_rm{iQ$&o2&=+JPeBb7m`hG^}%*j1S&>ZKR z2m#3(BH*-VJNhHstOq~yfUEk@U`b#}?$cndY)h=7z zQ`nGUH;@%{l2Nyjw@N$(&{&$oyicU2gnwkDtvK}Ma4LSBcs1G)XSF|%EJT?l_|Vg8 zQD?CCuVMnX%42i|CoTR_udU{zYoOk-*iPJE2P;$H(F7BGXh%$oP;v;okFSilU*r4% z3a8{4H7#IgJ9;5i$BwM_WJ}!Y_!X|6KPwLSxn9XfIDeyPM%nA}FS|O;lQB3`w5dhPzJF325X!=Y@w>YL!*vG?(- z7S)LXL$#xuCH>{iJskcP)f(dP{tjf>$Xdo{A*+tt3a4B`Z-H81;(MNMWVBu5n*q}~ z!am!it+4~u!_nMHc&cx$%K^+>lU~@sALpNZ=F!L`yxNWFFI9uNq^^DCa8n=++J4%) z6G_!_^6J1Q`Zv)N!uf15uAW~Q>21~{kn`xV2u)E{PG1v>qRWSqw zQ#n*ekHjuXT{i4UY>6_E3I+q}SfMG2FEa3fV18#mT)KaQQ#?e__1pE@grP*t*dEh1 zaL^v5Sv?VxcK-Byz0zBQ3qO0!f(-8g>X-Y;QTT|f9Rm>dmII=|T{VyA z3b(?*v$BZ;JA5l##kie|k({RE7wn2h@UZLrQ!Dm=#o`~<+(um5@;9W#zMXyMvBVQp z%7y?#HI-SDultt~9DBD|0-<6n4OQu4aCTz>X^Z`Y1rAjp83!cr{MqYVkO@a2%-64IT&_&j3Ouyoqm8V% z6$bmtCXORJdST?N-iC~}GFH#)8%kjhi;chMV8qVj`@GcmOR)}TFw63t;olahyzxX|kV&8FkvU+eKYI&^Sw>XXS~dKm z|3@5n)J$r9jxe|Jm&|+{s+~{$*EC{B6pAs@*s6SCne7mfgFL*TD94Wu0@V0bzS#z* zO7LK5gF#Ol2@X3FJ(BU0-7ziyqhHyY`MKc0A8*KykALBa9nv&7i(K*BKGS(bsStFM zi3n}rtXE!%b8!TP$LuHX?yEZfmdl78Tk;z z=Ky6!Rej>MQJaom`BEyMoL~rdZSzZ!@sR|FE_h6fLDeT7w|!^9?SDOf@Im52dz6-w z0J?wETvm-SvG!DPusa63L8%TAdd4 zNGI}%@ek(8FSo05SWquj!PCaRA%?MEdfiM(u5%y6BAfRXP*Vz9`+uWB6h|qQ8nx%r z48-TSI_f_>lJo@$uOv8hD-5<tdmU4Fs>> z=QN!-`vuj;o<76tcI}IPq{?Av<$$*Yd)QVXZ&65v{n7rj*L6V#Cjs9*Ud;AwsAq8M#?jSzw`LGN3#5NSDhEn%AaoGAJHwg6GuyaD}Q2( z1Aq9iNjgj8^NaBSj>e#Z#C_G!!d^(8kWot|W|^qNfvh(J{$@#+hcWIx>vGS|E$`4G))^cH^>(ad14PCP?c~i-dj1Fr4|0{CaFjhx*|za%$FKU$JO&~abo-Sau|BP%6B{W{ zkdn&c==_1R`pSPS;V+)~K6Li4Jmes*TCbRD3w>^xzvRvQQ+dW5uh*Jm;}zqiRv`So zuARS*F@-S08*)$P<{6O#Qv9&!{?YZeQUB-p7o2?Xm()u(>_}4$IJK)y?YXS%!~zRu z*7s3?&q zPfmI;&%wUb_u(Jj-o?Ju_rA}t?~t*7kNJM({IUKt+-o0|+)_!msM-s<_|VT|BL z{_YeTj!&RErB00&xP-_C-JsCjt^g=+iU&tJWZ zwwad{l*s&sdbeK$wvul+quq8wYnZTr#FqRg(}GN-5o6%{%sYJf97^)2wN#e-bH=L8 zm{uDcRafh?PupvK0YX@gxjI%yy#0b~e&Pz8F<78=?;{)Xz(bDsd;9}`^1wgXVv7wP zX@wiH*~(h@Ya1NKU=_DdAibd!MrusRj2}E;u!~=1#6>Hv@5b)r=HX81nYG}Uw{Xt- z*>A=%{v5yZ$A|r=eTg5sISy@bwBe#H2_|IOwsV>NpPAHJWlAozaMNDJ$F`C)e&)jm z+&q55d;XGsBZbF4*RZtyy9l;b|C1$akwYA-(#pHXH{)twMYPC;J|JO+C%ERZ4>UgT z;TzJl7yvdvJ%6OajKr_792v638IwaeNg8DMq%2s8Ugcb!5^VX4D~tnnE)w`oBp4&sb&2F)Lr%F0S? zr2N6v`xo_J#{!E$ejJrx_)F>~j^s&T#Yqb$5m~!rOH|hO+)bnNr!9#1<1uYvAI~`s zJnD(+-d(1(_EN3o;hP zjv-+*9FpXOzo~{|?vnkt(m#U6J3a4rVgiI_m@n-+VK3R?_#>5c;C2$b4Gvt@=O-hDOFYE7oj6bW zz9FegiJR>i_NBgjWQiQ+e)P%RMpCpaK&vh+FB$WqYEL$_MH#=Cwl_rc8zaT6sn&M# z8p4A`W-Qo9r56h_vih7YWveSAZiuA|2Gs%^HMZ4??vm_*9K;2t1_?yNRfBo-XqZi> z{Cn|>vZ3O?d(^&noLi#R|H@>|;EE_d{O0VfP*wM+Tj&}e|M2O(s~HX){d(85nQQ4& z_S#WLsdQp@4S!+%Ca^i}SWN+++pH0o*s2F^kEVwhaSBjwGu!z4*x3d*Vsr(`_IM~l zY8=C-dyW8F9)KuH#p>q5$h)^E18>NgH5mrt&Vv>?;@c5m%DVqo=dIr>L1L0T+3zw{ zZt_Gjuc(hvqSBkqxn*Plx&?=DC5V|jry+dYl}!9MO;mt)l2Y+l9iU3|$~?WUZS9|H zJ3eC4Wc=cO+;}jE!yTKndhj<1&XlV5{>!H!!Mp0x#<_w}KCAPma~aI*wh0f2oByuk zVIQ!5&NRIrzwM~7eOKeaEO|g%a$p2HgS*oj$2GIzSm4+Vd^?Y=kw`|>WKm{bC`%ud zw`VbLOdJ1A2oXMG4*1fq7T*HWC$SR0#=$O0|I(@XR|gT1>2&_gFz0UjFNE?ltqg9= z{JfdWL`aRujl?|~1?Mku>sTAwiy~r=nlZS4L?o`j;KDPv@aFg>GJDXvbcqLB8xzJr zk!l`F#3M9iMr*j{-(wHX1QDyszW>s4{CK?vkjKm?kQLO)nAfmsXT030#wYJ~+yq~2 zk75SLw1uZ-gfHD7Pe~oq(#OlT7CYX+n^nz5ExS%umrbYfbvw!irrBUFWSY;PzJDrE z^_G7|55Te?U_Gd9EPt%94)`#QT{5ymoa+AZg-3s-_@Rwd4YaqNnFEzVt>jYJ}W&=Z-!bwdmV&^)PEfu z0&7s+5XrE@BQFowYGZu3ja!8miMyUMI5vd2s`$JoS+!71cu@k(Gmr!usQus=J05WN7-19dBA1 z{6Lsf^8?^l2!ixjfJQ@aVTQy=U4`C&f#o_xKf?hYNxj%##od8%^$MRp5UuH=IWQ{S z-vbb(I!yKE(2-UN6^z}o3_r2;CQn{53UMulJ7CUi!HD2-TYRGH7+^m`UUAX|PrGhM z@~)g8&m|r_lH}#}YG|V{jDN9{bAyQ~s})zyA5tskvG3K;*?J_m(%N&?@9weg8*rIB zCs>OWe6g)#F{EM*M_ACIm*M8tUTukWTyHpXwzzOQJn;zx+slDGKvB7B!EFu2TQj8A?Ums(i# z_Xw4G{t!c^?3ZD{_H5*sk;PMs@YNqT&#-YN9)%H-fEm#yu9@3`>!RflV-fuq1vGF} z>BoyHdY=HBSB`1gYkcv{=a0s(UcZS|{Z#K^8h}{*m0#I+Ou=8p*XNUZ4AJdY=xc=xt6Y2Ew{cpiKr=vo_@CLRZT6NCR)uzl*HZm16kfWO2&jJKhBb3p~ zVLuFyuRm-iu(|eQ7^CT^O6sKuoIl-YO6j==sr%p5y^xQhJ8vWBBX1pOm1c64xK%2z z+sznzlO%c^hr6$8{(9al~&Z?~M`sFL_qr+VaS6&(wuu-c7w&O>^CWUu)Q$D2FZM*q!GOiGTf=NRnfqe46;RnOfGZtmlcy~a_!7i(1N zo_7Nfr{k^S{>`Z$zm1gRS9n1xw{(K}O}H7{*8GT5{e`a_?klC0=4SRbmINy2f@4e% zHaxZMS-YwqTW)dHZd3EX==n_NPo4k5p7EW-l=Jx$$6c_JnT|{ElFlEi_<1HmdgFC6 zG3j8nVF(rH4fqQ$5E{IA~}vb*}=jo9d&F-^h55=0gU7kPH|<|u^^0iTy_t$Hw--Bm?IU= zF%747zcG=MO5sowwdzqKJ}ojqDDauDU+7^T9csB6=ckQSQ#4)&ug~+s$cLEXO)PMog19vkUoz zzV*qJ%Bhz~us6Mu#4#?W)G^0(f$K(G*Rp>#p3{!3JzwZ$uDR4Ym=!1E*6Ga=yz8vt zTvDpi?BpL;OH&Kyn0Ebir_S2Xuh6t(#2S8_1Z|@aM^ULrQg7j1&!!*026E4Kd|^k9 z{p?<0o_uOd)#*x&ue#?o97u-c_$F|X@F+SlINrLAHE4I17@L@k7G`8~pTC zc`3%uUr_{ZrQ3<$^-=LWEvL@kaQJ7Qq3O87!~KVIHNDR(t_ZERUL!Yas2bNDde!b3 zg>ATc{TDSn*rikPc#_^qoSGiD>9Mz_i#abTXI$r)40V5nDYoRN{Ui-;@L=}@%nAlR z;o=n@_#@#7|J>$ChK~Jzgm+Zi|DC1f+}ou>{G#$|I zzbS>)^(gGtGbi&+-^s$LG_4%Gjh=fY^5R;HT~V}e++WHF(&sfNYa3#Sll!OtzrC~F zk|ei{s3pDs8(VAR25>=UR`u-4_BsBcVRt4;01*H|FzaJxc7K2V>3uR!&pI!iNz4Jn!h140+LGpy|9X5-McU4=BG1t8^4Mn&^}qLiM;m%37eGyk(NJ@^{Fdii?wYgcWXusC6v1u3E3#k4Qg-1?s z=4oXbqpYUBXD1%k*Zcz*g6z{6x4!MfpXW*w#AXjEdyM5q6CYx44Hm)HKG+(1u(3bt znC*H7_pbMsxzw}Po?@dHo}-SD=NULCH@obAw!JAj~!Q94pa~K z4v04EVx8fdIqT%7DTYV;!mebvp_4K?ZEDiGjuQ&nuCq6AY(|H^huiy{(207_K*xXZ zjI`0eAIi*;wZ~f9##j&551{&$6#KjBOPP9m_3oQ?Z$9nT-x#_+^MGC3W|#Kd{p_#v zk3RbhZ>_t}S8=?6PBDA5sgKAKfKjy7@?4nXg9{Yq8KaAVrr}LBCmkIb>uY7?1@R7B2{OahN zqhn-jyE&Bk!FS0~bO4&9-~B6{JoLQ&(a>{R^qoQ9qW5;kgRIeNPye=4qrJU$8=r=w zSUU&w=;~s)XP?+h(Ca|oXP@V%Eih~pRG)pTzIkT-g>9R@AU)vG}D`}{nrP#sek{supNWe3vbqd+Ov4` zob&U_FQ8`~63uW|*BnjFU9a@4TYWs15IOknL44VV7FWF#s(;(AK>gkTbiWs*ci$kz zPd``COwE)q6$ck*KXzS%Fzp5SYVE5z5o1TQ@7d8iU%h-IaH^gC#=n9dXrC#;_qptO*G{y(@_}u)dY|v3@%*ge zYan0ZE}ixI=8GLQ`up=w&JFo6ccgS{rW4iYi~^uludR9gR>wCdcCo#H>MzlGort4~ zLzgpO$1lC=jfXE}5U!xch;-WL+xE#}Z4|G_=$XYJpEA(WSz zqd8|yozjvpn1YENw_G3^IPfdpX6m|zO-uTN=Ov(hwLQB1>(|gvlYav&ISBSYU&>UB z*pUDJur`S5V)x>?>j?k(uXF>cb)~C&_XE7Tj zdKWeT^I$B9;qYSq)*+Keyw5*$>>n2#*7eMmS3D-PHxGX@9ek@RhR()69hE~$;P2Jv zP4U#nJ1^^poZx4Fn%Ef4Z@f%_+M%N}@kvyRdkzjIe!cr&X0UL4%kNWg8k3JgnQXGX#4omL7 zSy%5EfZiN;e7DZ^nzlP%9oY~5EB5eApGn>15xW8nRxyIC37kI{$K(eW&#Jf_fzR~d z*Hr;irxEn|xu1XM8GiW8<8sV^u*to6@_xr<9>tl*d%5mka#Y~9IV;2Vwug(?Jz8|- zzk=wiWO=O_9yqbFE;sK(7@wuLz2nufcJDaE*=uL7T-=+#^ICgE{MA@Bw(JiQFLbu( zYa(azUzW>J*t5R}rG0t>tO*s8{VQ4W6oavjzT=RIezYsP<3y+4_3CZ`V?ek5gClnE zGA!ab7(#i$(452BDJ=RKwGlO> zdG34mdGBoBIX*V%KJ+8_+kZ~x*}!b#93(e{YRaa&(K5;J&aaa$oEzWJW4ONLSZu{m z{!>K8fi8f$)i3TFKP6s*gfu#riWa+FIpb>YM8>97W9kAM?~L9t<{d)|stK;OVosr^_SB8n@4=zc;| zX`Y{~Bf&u4Tx~J;S^wh6L+q=Y81Ka1AIwo>dS9^K@vcs;5~#;EjTr8I05m~l$V_|y zt-g8CYl}(pT77>ZGclIUesMGF4DI+!7oOe|IM|oFwP{tS^@Z1oA6@FUjZ%A{y;^_b zMBX1{I$reeFJwy8Eu2oeZ{8DY^UulK_}V`R{bvlgnt94POQwzI?>s|u(i&8E?cBHV z`dr*vUju;E#(fQlpjyQ{DrS#y1an)Q!l|3j`fWDsH`W<1v<6v!8QG&tX-xjKh2?`M z!33Oj#I|q|mvQ<-FpKKAZZYHU(zn&;JUDWgtMvVDGK~7i=6yb3GETb33vAb4qsOSu3MWHdh>f;f6yV14-2n^gyaQJ>=-0gymXw1tRtSEnZ_o= z$4&a6%cfb%r2K>IuNE4z{!q85?-wrlS64=7eT`|>g=Ns8^&NeaQ9-N{WoPA z+?;3s%_Y76Yw!Nw15e(Faq^1bs?^BzH97l^Y?4k7giD^^pMNST z$vBbI|kH`RvdY^QT_e;1IGHX z1~`{j=x1&BxxL6gUT-Flo1@3x+aSL2e{o^qwfctrM|~mFf7}y5tmX?5A0+fwd(Mtx z6FCcV15CuRd|fq{Qj$y z*o(Cnw+DIR&kH&LJk+lR>F!$)z7ULMy+Q0iee?FnTb$`J>EE7&7As!qr^1*e$>J6z(h?y>-X%vc=-~;@S?ll zu*GKLhaq}n7y{RJ&e1%#+>gRPTFpOV$6(Bi+df`Pw|_;wH|N2T81PfSWD%;pTxfLY z`})umLp9gaGXz8MUdI@~llIDo;`dtb{RWaaYOp*dBbPqEJU{nyiI<8$rVw`>iLcgP z0&)O%_$&Ul>AaM3fwASaF%w3l%iQG(AoVw!jX6;Hy7 zP2VDp%@MK=?0o$*Z&<|ltmXFm;OXR5Kx3gjMkSITJ~=|nIuTy?o{vOhg|V&MPx955 zM{Fhz=$S@@cV=&GXaG|61GRW2Se1=(`>K2-1c6639J=r-&Ngf2!Xhu{L)ZXfWeOf4+W; zf4_hGwm;!ATvkxytRq3@mPpTN;xUG3y>%p>!_9jKpRC|YcXIB&#PfN69(?mN4;J9v zO4y=T{3VSK^|d(ggqcZPN$LaL+5w1Dy8}JU0?_ zb0)UBJ!eW2W#ISo6Yc#B*ZM0JUH3l}{vRh5vJ!RnH8uD+BO~ibrBX>MB=ZtkXRqT< z6w0_tA?rxlBiZAeJuSGHKswp zkyia>FABEbKAsu0F{S?)F?q2?IehTxu;MRekI5f9I4$vZcUA9oH*G+jG= zXc7CQcntEmOwq48XGcG-3-WnJoLl0OnCdC?Vz;w`dn||Yi}V-Q^yL9JcU$T*MMiw8 z7RXNMinagF(i}fJmtkwUKehSVQdJj!vMT6Uw+ca#-)83vCVluR%BH8|eK2y6d^qf_ zS{znID;eizW<+_l%~kQ)i`rh9o4L2S?LhH%{`c~F^kN^c)7T^dyRl&&!=9FHb9ex; zZ{P2z)=CB~Qq7ML0S3{B)aJA##s()Zavx`sOUlqr6H4anc-L9`WdYd=LG7p{+->)x z9Mm#;>;Y@xp#eJ^uzcLgY-XNb*Ra@603Nwr`lH`Q-A9!nU@GAi;cn#|V{g6u$bnQH zgX_0U=6IEVBHz%4e~2x9V=_qt(z-|vZwn~l@G8VL*&1&d`ufyB^|-3m#}O$r(9Ck< zVqE!~{pvA2k5E?XH&hn;Uc0w(LWzNcr^nFTsJZ*kv{^3g64U4Kq&E5%s#OM<{mhJI z1wChu)GEG}nD5y-ypQrdFW(_{H4!56oJMZ^`7Ur}cgE`F+btObB9qn3{;gpql}*{= zFk|d9>dW5gQ~aLHH^r2G#(e!$nkuKtFpdvQ+{)X5!55Fu2tO;^H>IcRNxdQa#hVo6 zC<<4skE{;=(7!99R8pdu4yJ&yt)Kojstg*rDr#ANElT}0j6t);P@K_dqdErBYREEr zsxMF1*uWbNj!4R0UDZlnO;O9=p1?mZeYG(j0%iK#bKI(Nyl~?UVjnk$xe2~8u_U8F z=pQ%70=2Hf25rM|W|l2XR`xjPzs|}2y3mR>`T|M@|KDnXUk*IU(p~x-mq>7>4eae8B>}qQJ-A;W2Gt;{K7A-3|M4$EFVlZcR{Us-D6eQqX)lcB6 zLmrqQ)Gy=z#gbit(`)Ejmc(Gd+@HLYuMNN}pjG}r&YbaN{mvc7Rk|VAdw3<{FzkUV zCl_!DF<8^~WagEIL6{dM(7bIe!D?;lvQ(44cCWD=mA)RvVAEIUFiR2CB8hB+_s%P1 zeL9!MO6K-IidB7mzVLF+jmaM)Q)#Fm+zCGv%3vG zX94QTvJO$6=!t!Mh<>QG?M;VL3*o3kxvvq+hZZ#fylsdd$F-*P@L#uo{2|4`{v3r5 z6Di1(Z3@|9JvM*QDblj-rO27C4u}>tguaS`2Fxk9MImL8_hHNse9m+bd^^S%s~ONi zsvf4)wp}j`vyeCQ4^sGD>pX&uHuVwAsPLx}joyQQSqAm_!aN*Y+1)?UL;HEo>>7to zuV*n3rK`GHmFZV0e@E|6CzWBvn87HZ*E(0YbSBTZxpl0{nsFG*->w4>a}6>} zd4L-SsE^rOnJ@fSr16sZT^HTEXUu|08f`OX9-h^vwFJjUTkOoH(5Y`bj4%?uGNSqj z!&Ip)aSN0^M9L78)Ncx_T49ZJYBTK2w+D%Bu}5T0DpuA5Wz6m+Mzd1cD^4N1svUpc z*I1=8K%yvIQPU$8doV5X72WhW7t0{k;MD;!ZniW<_#176>Cf$Z9nLr+3{EpjteWfU zhIrV`tO5`(K;%Bar_6@uJPdm1j_lB~ldvHZE3?s_0o<1=xL?N{epD0TEo8=N+oyXS z_YJHm;es~s7>>^W89gH>2ESA#HYj(jVT+tevOKj74z$EvtmSos%JoMh zKo5&>rs%8w%tbH7(t)5??x%JM-k4kfW(g$Ue2 z0{CRX^*1G}N-CU|(knP8RQKBzWI-!e2e(pP1JE*?x_pZ}IvpCVWI zwQWsJ9Zm1kAD&hj!h*KDiw;|xFtcgyQhMH1+_F(hr%#A(5S0t(QYg{`0?XvhO@_m& zX%dtkjW+~OY|;y9e__(ll&2E(36M^D0m{w}7JMYw9;y=R7j(>=wRtV{T;KtnGoPKY zx=J!dX#3y61sb)$xFb$Lg*MGsEqw;q_l$K${Dr1}T9X-+iEwlsR^B^8uc#7!E1{V! z|HZy~i%!cGD82(h3MbauX|q``j!@};Q#cgN-dVH*=99*rR!b2C8~u7W+(&-UvT9eg zB%kve>>T3OJzEu}Scm3!BeC8uxCRlKe~)3Mr?^{zP@Avfl4IY}p5KXxtDGJaW*fPS zUKebZ9#@aH`51nOI6{tX3~EcBQs)XV>ld#){ZBO@bS`3mRtMt4tjIEn?pcV5(1 zJ8^yIc$nsFQ;b!bTE_?(k~nxuKgu1{I8mJ+xK@+9W8H|%f3q$iMW(2-q7ftdR6l$snj!jO+4*?C)95ip#~gD-ZLzm3%W^ zI+@YvCHF(%#pcMl&X?1mke?0oSRyS%{p~+pJ5Rw!+R3qO*%N~0QfAmc{!kVbu>EeK zMszFe<8CK=PD_*oX4IGY;%#+u^~a_dzaY5&$V-WMIJ_K1WJ&k5K@MrTx^0p*hLH(# zf$?0|?9%x;>2coFX8OMpBpPbzti`A=&> z8B;Hc)wxC=_vjJ%p`W|@a!--d3nQXA6VK~l7C9cqJ}a z0nhq=D{701^cr9N$F56K0ES#pQP^@%JIXI20a^k2=71KgWO;^1hKExMx<`&T#Fb0B z0^OB)j*B>I5{Bj&*H&C5d+(KP1&uv#=!dRdzUpPXqD zFk%Pu=`*W} zhmpmf&h6;{)6}6`z4?@uY`{&)^Z%4$0;lhb z*|R}D+^7zut>?6Sa9wo!=Iyi_4Pa9^-f6Vcz+WVjSD80~n-pWFT}k@p~=KU-gwAe1}{-T+B4&q(3Gbx zVyZUY276BX2%u(fcc3pWq~vF zt;W}T$X^GiLP@J8zb^z>7gs07GVs@V%q1HJ?ZxIdWQ5A!IGUz{FP9Owd)_+L2PN>e zFg_izTB?}kXSkIEuQy)=AJ!!EwOH*&8nA+QC5D%5Jt`>&qJ|1!|Ik^=UF(ek1mZNY zV>dg)SD9ZGBKYW5t@7Ap;!%svRd3EVcwCC@l!V9EmUY20j=a7RfHU#*gB{9`=>) z=wm?+nwgoSIq76PXHBqaHc!g0M2p-M=8fDjJ!n!2r zYu9i@4Q9bbcEpFn+H!c{iUCHmg`U{1lM0oS!ZV^v)AzEapI~oaW(?glujQ178Nxwmi?3pGZ(1X7}8qaC@=4Pp|ul%LhITmv9+Nh)15n zw+o*Mbg%4TaWhDBx~HF6ZT7DDjYxGhto!F>}ta`%4o#Iw?!ZLE8wHHREHaN*7v}%wj?-w zuG|?RzT4Sb)rY=3iAe)Cg$tY>zA>>&(jPbZN{8g$1dqc%o@~_tHJ*`EU^xVt2Yh{+H2mB9mO%y6 zs|k{LYkma2bqvMGi`gh}_yeXQwm^qs#(=KJGoY)nu_zl+X=Jd`gB_Ex{j)Ow~c^&I^sUWo7+?+pBgA?zdp-!r488&6@6 z?sjm7Td6}}Dj0qj>_mRXQ~$ckcO!36%WM3Jf48hc(FX6PS?95xQW{3*Ld~0>!jYm&2x<^~7%#l%TH2?8(?@l7xyozy1zt!-u`WS|iyn_Au0n zgL*=-P*358C&Fw_0oL5LP-?>pM&IP)`z_{CC}amV^;n7y;n|DKi-5r$lga!|7M&b< zf!c^sbolC($rd=0JBL!fKooy){m>!PBiE4ufdkKUx+iL9A7eK(J^mdx@MeswV6o{B zm6Fpb=FP{vxg9UF+B>?M5<8`gO-*IyDsYdSgX-*nr>AJ#pB`S6)LiU^?uG7)61Ml} zcdxJh5JZQCQOXGg1f=zV|GvvNF_>d@Y549yF!yFNuaw;&^}jIHr%o9c)$;1`ciCJH zD_PiUZeVjG)G$oC>vG6sj^$PS>JVA107k3vo?=YfPZa*V;)wXMm7syAGs)c?OOk}6 z%i}f5?R^l3W0dst@tzIkC+v=-Liq3s%kliFN?+A{g8)rn0AsgnW1CHNi#KXHZdN@&N{oO0}HL=nL7-i+`Qxf!@T>J#p^fRUx5+93H|<#>IGKNibqMb zLMdzhFAti7mB(sA2nYaAEnI=geTE4C47nE9Rui!3ZpcJT$2CE;ykP;I2hR!JHwYX5 zm`MZ^dK7){DWBE`1Up9d7!RpP-&Xhlh!V;iipWRPH+=?mv}ZtC=Q zufK|{dSg1Ib~TANV(rVYHo~lR`EIjm@Q&Wc-%#8dK@{#Rj3K!CWdX(2;dmiX!cs69t6>-Q@$(W22%NAeSe=OQ=ky(`0&WhLvX9_0LV(@Z)HWO!^V-^E?%?#N(zrGzy4AT}8n^y~KPf{Ov-%h(f>agPfR1 zKUgf47ME89qcJ!{S>TTyL$E%-fyRre$`I-qS~?2~=IXb6Xgeb+yf)2unP1K%Lwv9q zXxd@n$uyEe3puBsO`kiK(PMF0?`B3}+wl4)d$hst&W9l3(89foZ=QYmn-qIPWri&E zyay=w-c-5&L!388-nN1VH;*H%s+r^A4&q}XS5Y@3Ot{20Pv>>(48F~GkgJf3JG1d< zm!!Y5jEL7wZztqpMFc*qNq^o8GakRn&M<{u5DPPms+B3In;QIK!#(t?eSE73>+ThwlF{?zc#r0Q{ znuA#B{v(Hry5G-y=nk%s16n4U>qdca6`kyH}r-@9o{neyZ)-)}8 zg&kCVEc~1$FtT@YNrVZMC5nF4>_BxH0^D4xi-!g z#93*N@o+h5La8{P6`k>;o2(?3?soEX%!Pga-)%TpUr4x%YF)}Vx3i@0iTjk_U6bdq zhCgHUIET7lvZza?CWb6LpJ??qDva92e%ulsO(o|ED8_nShWqaD!s%}dGWTVh;3*Ut z_Qm>&X7fY3VF4E$1#3NuLr%xEtU%F{`%rmk22x{&f1WA^e=wRpf$n$RQ7Vs-&% zukPM0t|36gFqpWsp5(w0KsFt5_Q<0YI>5u(xG-0TZ7s`oLdiDkR5m%dnV*z|(2VSO zJ#K#>{H|WHnN~R)KoSUH@P+1y)mP67)_f#-a%| zfD%rWH{4scyV6A@&<1SR=S9c22#dp7}MDRbylist~y_yp63aYVo2ICluLA&BVj zc`=pom_Ew2rUKp)nK4UnlkAY6W#q*nj~iJr9_*1Q55=2o{kF!85EFA5%flCn46P~Q zr0M9ay8+~EutHKFI)ksUemfs$+Z}?SO3S^sY9RP#8cqv}em=$SjGpyM?z<4+#y-A`CtvJ5KyLTwKz@||r?0=x#v=8# zqF^fT&eM|TNX!2F2f)n+ej~Ds;>&_V18|LxO#hHG1exUb4yatFA8Zv%HeoNRr*Bjk zg#`I7;HcdE6Ds$i7w1 z>WScLabnBg&8f*V@$wsWgat z0tFFci;e^(c#_#h3xhs;&nw^8@xv{N`D%&Kefn!5S%uqgclaNP9H|5zw6Dv~*xw#O z)L2;aY_&?E*xCvaowpQ(UjcT@fk30~9hOx~pG*0-tyq};E+>Cx1iJ-emAf6a(%~(^ z=?agZ+pHm(TZ_-op+ipPq7Oixf2%1cxj#tyj^@+#VN*04@L7^(=WXXns<3}@>>YRC zjg5s0qil@B!i9vyPDir^fXYP9jzN*}<-f@|&cpo?9PHHZg(?mK5uA$INZf`ZDpKXK z4+nQ*%%AC~=S&?RF$gX3wjb!5g|H{SBpkgZdVcR)UJ-<)#5UK58iu%OL(Q_OD5EwaEj z5HGv-YpLDnof~x3Ug=^wXb8KfGe0n*bIo;@bSD<1K^!3M{DptS#LlAbz38IOW|q`h zd5ayLMT@QZx!l(*#znyS3Tw$E#iVlY3#)Vl3V6S*Zk#wM92+bP>B8-=!RfDm9mvb- zT@Lh^PYCtVkqB;(w*a5l`1U5D2C=vf3lIBw)tPUd!QG=z(vAXC&Sud1Oqin@y6n;V z0wA6BDl5^g8Q(PCZ_0t@C3w`Fl=T=gKS*8goxwh!_hPFw6;2RCEtK=172w|a-NuI9 zT;MXp(wjTReWK^PzYxX?n_}W%D-tjs2`)cYp!K5I`7c4E4*6Br$BA<&E1leW=m?)y zCQygc7 Date: Mon, 31 May 2021 13:35:36 +0800 Subject: [PATCH 212/343] doc: Remove hash fields in lock wait table (#24922) --- docs/design/2021-04-26-lock-view.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/design/2021-04-26-lock-view.md b/docs/design/2021-04-26-lock-view.md index 78faf9f6b61de..814294c8e929f 100644 --- a/docs/design/2021-04-26-lock-view.md +++ b/docs/design/2021-04-26-lock-view.md @@ -64,7 +64,6 @@ Several tables will be provided in `information_schema`. Some tables has both lo | Field | Type | Comment | |------------|------------|---------| -| `HASH` | `bigint` | The hash of the lock in TiKV's LockManager | | `KEY` | `varchar` | The key that's being waiting on | | `TRX_ID` | `unsigned bigint` | The current transaction that's waiting for the lock | | `SQL_DIGEST` | `text` | The digest of the SQL that's trying to acquire the lock | From ad9241f92cd625edd1a19707945e8758258cb33e Mon Sep 17 00:00:00 2001 From: Song Gao Date: Mon, 31 May 2021 14:03:36 +0800 Subject: [PATCH 213/343] executor: hide tx_read_ts from show variables (#24926) --- executor/show_test.go | 3 +++ sessionctx/variable/sysvar.go | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/executor/show_test.go b/executor/show_test.go index b00379b72418e..38f513642df99 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -1279,6 +1279,9 @@ func (s *testSuite5) TestShowVar(c *C) { c.Check(res.Rows(), HasLen, 0) res = tk.MustQuery("show global variables like '" + variable.TiDBPartitionPruneMode + "'") c.Check(res.Rows(), HasLen, 0) + // Test Hidden tx_read_ts + res = tk.MustQuery("show variables like '%tx_read_ts'") + c.Check(res.Rows(), HasLen, 0) } func (s *testSuite5) TestIssue19507(c *C) { diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 33d09ac853e58..64287ba756f07 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -825,7 +825,7 @@ var defaultSysVars = []*SysVar{ }, GetSession: func(s *SessionVars) (string, error) { return s.TxnScope.GetVarValue(), nil }}, - {Scope: ScopeSession, Name: TiDBTxnReadTS, Value: "", SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBTxnReadTS, Value: "", Hidden: true, SetSession: func(s *SessionVars, val string) error { return setTxnReadTS(s, val) }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "ENFORCE"}, SetSession: func(s *SessionVars, val string) error { From cd10f944b2e583625861b3bdba2f4fe145c578b8 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 31 May 2021 00:27:36 -0600 Subject: [PATCH 214/343] variable: Add IsNoop to sysvar struct (#24963) --- sessionctx/variable/sysvar.go | 6 ++++-- sessionctx/variable/sysvar_test.go | 15 +++++++++++++++ 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 64287ba756f07..23e17b0cec704 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -129,6 +129,8 @@ type SysVar struct { // This is only important to set for sysvars that include session scope, // since global scoped sysvars are not-applicable. skipInit bool + // IsNoop defines if the sysvar is a noop included for MySQL compatibility + IsNoop bool } // GetGlobalFromHook calls the GetSession func if it exists. @@ -518,7 +520,7 @@ func (sv *SysVar) GetNativeValType(val string) (types.Datum, byte, uint) { // SkipInit returns true if when a new session is created we should "skip" copying // an initial value to it (and call the SetSession func if it exists) func (sv *SysVar) SkipInit() bool { - if sv.skipInit { + if sv.skipInit || sv.IsNoop { return true } // These a special "Global-only" sysvars that for backward compatibility @@ -599,7 +601,7 @@ func init() { RegisterSysVar(v) } for _, v := range noopSysVars { - v.skipInit = true // by definition a noop can skipInit + v.IsNoop = true RegisterSysVar(v) } } diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index eb5090bbb5a42..05f8fc83d97de 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -448,6 +448,21 @@ func (*testSysVarSuite) TestSkipInit(c *C) { c.Assert(sv.SkipInit(), IsTrue) } +// IsNoop is used by the documentation to auto-generate docs for real sysvars. +func (*testSysVarSuite) TestIsNoop(c *C) { + sv := GetSysVar(TiDBMultiStatementMode) + c.Assert(sv.IsNoop, IsFalse) + + sv = GetSysVar(InnodbLockWaitTimeout) + c.Assert(sv.IsNoop, IsFalse) + + sv = GetSysVar(InnodbFastShutdown) + c.Assert(sv.IsNoop, IsTrue) + + sv = GetSysVar(ReadOnly) + c.Assert(sv.IsNoop, IsTrue) +} + func (*testSysVarSuite) TestInstanceScopedVars(c *C) { // This tests instance scoped variables through GetSessionOrGlobalSystemVar(). // Eventually these should be changed to use getters so that the switch From 456e72c56edcc3721024248da1b681479989cd67 Mon Sep 17 00:00:00 2001 From: kaixu120811 <78292293+kaixu120811@users.noreply.github.com> Date: Mon, 31 May 2021 15:07:37 +0800 Subject: [PATCH 215/343] telemetry: add pk type of clusterindex feature usage information (#24825) --- telemetry/data_feature_usage.go | 104 ++++++++++++++++++++++++++------ 1 file changed, 84 insertions(+), 20 deletions(-) diff --git a/telemetry/data_feature_usage.go b/telemetry/data_feature_usage.go index 8e9b2e6cfb69b..b1bb9f7e996e3 100644 --- a/telemetry/data_feature_usage.go +++ b/telemetry/data_feature_usage.go @@ -15,32 +15,66 @@ package telemetry import ( "context" + "errors" + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" ) type featureUsage struct { - Txn *TxnUsage `json:"txn"` - ClusterIndex map[string]bool `json:"clusterIndex"` - TemporaryTable bool `json:"temporaryTable"` + // transaction usage information + Txn *TxnUsage `json:"txn"` + // cluster index usage information + // key is the first 6 characters of sha2(TABLE_NAME, 256) + ClusterIndex *ClusterIndexUsage `json:"clusterIndex"` + TemporaryTable bool `json:"temporaryTable"` } func getFeatureUsage(ctx sessionctx.Context) (*featureUsage, error) { - // init - usageInfo := featureUsage{ - ClusterIndex: make(map[string]bool), + + clusterIdxUsage, err := GetClusterIndexUsageInfo(ctx) + if err != nil { + logutil.BgLogger().Info(err.Error()) + return nil, err } - // cluster index + // transaction related feature + txnUsage := GetTxnUsageInfo(ctx) + + // Avoid the circle dependency. + temporaryTable := ctx.(TemporaryTableFeatureChecker).TemporaryTableExists() + + return &featureUsage{txnUsage, clusterIdxUsage, temporaryTable}, nil +} + +// ClusterIndexUsage records the usage info of all the tables, no more than 10k tables +type ClusterIndexUsage map[string]TableClusteredInfo + +// TableClusteredInfo records the usage info of clusterindex of each table +// CLUSTERED, NON_CLUSTERED, NA +type TableClusteredInfo struct { + IsClustered bool `json:"isClustered"` // True means CLUSTERED, False means NON_CLUSTERED + ClusterPKType string `json:"clusterPKType"` // INT means clustered PK type is int + // NON_INT means clustered PK type is not int + // NA means this field is no meaningful information +} + +// GetClusterIndexUsageInfo gets the ClusterIndex usage information. It's exported for future test. +func GetClusterIndexUsageInfo(ctx sessionctx.Context) (cu *ClusterIndexUsage, err error) { + usage := make(ClusterIndexUsage) exec := ctx.(sqlexec.RestrictedSQLExecutor) + + // query INFORMATION_SCHEMA.tables to get the latest table information about ClusterIndex stmt, err := exec.ParseWithParams(context.TODO(), ` - SELECT left(sha2(TABLE_NAME, 256), 6) name, TIDB_PK_TYPE + SELECT left(sha2(TABLE_NAME, 256), 6) table_name_hash, TIDB_PK_TYPE, TABLE_SCHEMA, TABLE_NAME FROM information_schema.tables WHERE table_schema not in ('INFORMATION_SCHEMA', 'METRICS_SCHEMA', 'PERFORMANCE_SCHEMA', 'mysql') - ORDER BY name + ORDER BY table_name_hash limit 10000`) if err != nil { return nil, err @@ -49,24 +83,54 @@ func getFeatureUsage(ctx sessionctx.Context) (*featureUsage, error) { if err != nil { return nil, err } + + defer func() { + if r := recover(); r != nil { + switch x := r.(type) { + case string: + err = errors.New(x) + case error: + err = x + default: + err = errors.New("unknown failure") + } + } + }() + err = ctx.RefreshTxnCtx(context.TODO()) + if err != nil { + return nil, err + } + infoSchema := ctx.GetSessionVars().TxnCtx.InfoSchema.(infoschema.InfoSchema) + + // check ClusterIndex information for each table + // row: 0 = table_name_hash, 1 = TIDB_PK_TYPE, 2 = TABLE_SCHEMA (db), 3 = TABLE_NAME + for _, row := range rows { - if row.Len() < 2 { + if row.Len() < 4 { continue } - isClustered := false + tblClusteredInfo := TableClusteredInfo{false, "NA"} if row.GetString(1) == "CLUSTERED" { - isClustered = true + tblClusteredInfo.IsClustered = true + table, err := infoSchema.TableByName(model.NewCIStr(row.GetString(2)), model.NewCIStr(row.GetString(3))) + if err != nil { + continue + } + tableInfo := table.Meta() + if tableInfo.PKIsHandle { + tblClusteredInfo.ClusterPKType = "INT" + } else if tableInfo.IsCommonHandle { + tblClusteredInfo.ClusterPKType = "NON_INT" + } else { + // if both CLUSTERED IS TURE and CLUSTERPKTYPE IS NA met, this else is hit + // it means the status of INFORMATION_SCHEMA.tables if not consistent with session.Context + // WE SHOULD treat this issue SERIOUSLY + } } - usageInfo.ClusterIndex[row.GetString(0)] = isClustered + usage[row.GetString(0)] = tblClusteredInfo } - // transaction related feature - usageInfo.Txn = GetTxnUsageInfo(ctx) - - // Avoid the circle dependency. - usageInfo.TemporaryTable = ctx.(TemporaryTableFeatureChecker).TemporaryTableExists() - - return &usageInfo, nil + return &usage, nil } // TemporaryTableFeatureChecker is defined to avoid package circle dependency. From 702c86b8f956a2c748d4f2626222d7933325eebd Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 31 May 2021 15:17:36 +0800 Subject: [PATCH 216/343] *: add global/instance variable to config top sql (#24934) --- config/config.go | 21 ---------- distsql/request_builder.go | 3 +- executor/executor.go | 4 +- executor/executor_test.go | 6 +-- executor/set_test.go | 69 ++++++++++++++++++++++++++++++++ executor/update.go | 4 +- sessionctx/variable/sysvar.go | 55 +++++++++++++++++++++++++ sessionctx/variable/tidb_vars.go | 39 ++++++++++++++++++ 8 files changed, 170 insertions(+), 31 deletions(-) diff --git a/config/config.go b/config/config.go index cfadfe3fd94e7..14467afaa0b4f 100644 --- a/config/config.go +++ b/config/config.go @@ -137,7 +137,6 @@ type Config struct { DelayCleanTableLock uint64 `toml:"delay-clean-table-lock" json:"delay-clean-table-lock"` SplitRegionMaxNum uint64 `toml:"split-region-max-num" json:"split-region-max-num"` StmtSummary StmtSummary `toml:"stmt-summary" json:"stmt-summary"` - TopSQL TopSQL `toml:"top-sql" json:"top-sql"` // RepairMode indicates that the TiDB is in the repair mode for table meta. RepairMode bool `toml:"repair-mode" json:"repair-mode"` RepairTableList []string `toml:"repair-table-list" json:"repair-table-list"` @@ -527,16 +526,6 @@ type StmtSummary struct { HistorySize int `toml:"history-size" json:"history-size"` } -// TopSQL is the config for top sql. -type TopSQL struct { - // Enable statement summary or not. - Enable bool `toml:"enable" json:"enable"` - // The refresh interval of statement summary. - RefreshInterval int `toml:"refresh-interval" json:"refresh-interval"` - // The maximum number of statements kept in memory. - MaxStmtCount uint `toml:"max-stmt-count" json:"max-stmt-count"` -} - // IsolationRead is the config for isolation read. type IsolationRead struct { // Engines filters tidb-server access paths by engine type. @@ -666,11 +655,6 @@ var defaultConf = Config{ RefreshInterval: 1800, HistorySize: 24, }, - TopSQL: TopSQL{ - Enable: true, - RefreshInterval: 1, - MaxStmtCount: 5000, - }, IsolationRead: IsolationRead{ Engines: []string{"tikv", "tiflash", "tidb"}, }, @@ -958,11 +942,6 @@ func TableLockEnabled() bool { return GetGlobalConfig().EnableTableLock } -// TopSQLEnabled uses to check whether enabled the top SQL feature. -func TopSQLEnabled() bool { - return GetGlobalConfig().TopSQL.Enable -} - // TableLockDelayClean uses to get the time of delay clean table lock. var TableLockDelayClean = func() uint64 { return GetGlobalConfig().DelayCleanTableLock diff --git a/distsql/request_builder.go b/distsql/request_builder.go index d34ecc9ab0c47..0fd44b044ae3b 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" @@ -278,7 +277,7 @@ func (builder *RequestBuilder) SetFromInfoSchema(pis interface{}) *RequestBuilde // SetResourceGroupTag sets the request resource group tag. func (builder *RequestBuilder) SetResourceGroupTag(sc *stmtctx.StatementContext) *RequestBuilder { - if config.TopSQLEnabled() { + if variable.TopSQLEnabled() { builder.Request.ResourceGroupTag = sc.GetResourceGroupTag() } return builder diff --git a/executor/executor.go b/executor/executor.go index 3474d7e33cd8f..f9cffcdfdba54 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -977,7 +977,7 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *tikvstore.LockCtx { var planDigest *parser.Digest _, sqlDigest := seVars.StmtCtx.SQLDigest() - if config.TopSQLEnabled() { + if variable.TopSQLEnabled() { _, planDigest = seVars.StmtCtx.GetPlanDigest() } return &tikvstore.LockCtx{ @@ -1822,7 +1822,7 @@ func FillVirtualColumnValue(virtualRetTypes []*types.FieldType, virtualColumnInd } func setResourceGroupTagForTxn(sc *stmtctx.StatementContext, snapshot kv.Snapshot) { - if snapshot != nil && config.TopSQLEnabled() { + if snapshot != nil && variable.TopSQLEnabled() { snapshot.SetOption(kv.ResourceGroupTag, sc.GetResourceGroupTag()) } } diff --git a/executor/executor_test.go b/executor/executor_test.go index d04c202e1e221..c41b6f2a7773e 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8364,10 +8364,8 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { tbInfo := testGetTableByName(c, tk.Se, "test", "t") // Enable Top SQL - cfg := config.GetGlobalConfig() - newCfg := *cfg - newCfg.TopSQL.Enable = true - config.StoreGlobalConfig(&newCfg) + variable.TopSQLVariable.Enable.Store(true) + variable.TopSQLVariable.AgentAddress.Store("mock-agent") c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook", `return(true)`), IsNil) defer failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/unistoreRPCClientSendHook") diff --git a/executor/set_test.go b/executor/set_test.go index eda1696497cea..4134cb128d990 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -1376,3 +1376,72 @@ func (s *testSuite5) TestSetClusterConfigJSONData(c *C) { } } } + +func (s *testSerialSuite) TestSetTopSQLVariables(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@tidb_enable_top_sql='On';") + tk.MustQuery("select @@tidb_enable_top_sql;").Check(testkit.Rows("1")) + c.Assert(variable.TopSQLVariable.Enable.Load(), IsTrue) + tk.MustExec("set @@tidb_enable_top_sql='off';") + tk.MustQuery("select @@tidb_enable_top_sql;").Check(testkit.Rows("0")) + c.Assert(variable.TopSQLVariable.Enable.Load(), IsFalse) + tk.MustExec("set @@global.tidb_enable_top_sql='On';") + tk.MustQuery("select @@global.tidb_enable_top_sql;").Check(testkit.Rows("1")) + c.Assert(variable.TopSQLVariable.Enable.Load(), IsTrue) + tk.MustExec("set @@global.tidb_enable_top_sql='off';") + tk.MustQuery("select @@global.tidb_enable_top_sql;").Check(testkit.Rows("0")) + c.Assert(variable.TopSQLVariable.Enable.Load(), IsFalse) + + tk.MustExec("set @@tidb_top_sql_agent_address='127.0.0.1:4001';") + tk.MustQuery("select @@tidb_top_sql_agent_address;").Check(testkit.Rows("127.0.0.1:4001")) + c.Assert(variable.TopSQLVariable.AgentAddress.Load(), Equals, "127.0.0.1:4001") + tk.MustExec("set @@tidb_top_sql_agent_address='';") + tk.MustQuery("select @@tidb_top_sql_agent_address;").Check(testkit.Rows("")) + c.Assert(variable.TopSQLVariable.AgentAddress.Load(), Equals, "") + tk.MustExec("set @@global.tidb_top_sql_agent_address='127.0.0.1:4001';") + tk.MustQuery("select @@global.tidb_top_sql_agent_address;").Check(testkit.Rows("127.0.0.1:4001")) + c.Assert(variable.TopSQLVariable.AgentAddress.Load(), Equals, "127.0.0.1:4001") + tk.MustExec("set @@global.tidb_top_sql_agent_address='';") + tk.MustQuery("select @@global.tidb_top_sql_agent_address;").Check(testkit.Rows("")) + c.Assert(variable.TopSQLVariable.AgentAddress.Load(), Equals, "") + + tk.MustExec("set @@tidb_top_sql_precision_seconds=60;") + tk.MustQuery("select @@tidb_top_sql_precision_seconds;").Check(testkit.Rows("60")) + c.Assert(variable.TopSQLVariable.PrecisionSeconds.Load(), Equals, int64(60)) + _, err := tk.Exec("set @@tidb_top_sql_precision_seconds='abc';") + c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_precision_seconds'") + _, err = tk.Exec("set @@tidb_top_sql_precision_seconds='-1';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_precision_seconds' can't be set to the value of '-1'") + tk.MustQuery("select @@tidb_top_sql_precision_seconds;").Check(testkit.Rows("60")) + c.Assert(variable.TopSQLVariable.PrecisionSeconds.Load(), Equals, int64(60)) + tk.MustExec("set @@global.tidb_top_sql_precision_seconds=2;") + tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("2")) + c.Assert(variable.TopSQLVariable.PrecisionSeconds.Load(), Equals, int64(2)) + _, err = tk.Exec("set @@global.tidb_top_sql_precision_seconds='abc';") + c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_precision_seconds'") + _, err = tk.Exec("set @@global.tidb_top_sql_precision_seconds='-1';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_precision_seconds' can't be set to the value of '-1'") + tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("2")) + c.Assert(variable.TopSQLVariable.PrecisionSeconds.Load(), Equals, int64(2)) + + tk.MustExec("set @@tidb_top_sql_max_statement_count=5000;") + tk.MustQuery("select @@tidb_top_sql_max_statement_count;").Check(testkit.Rows("5000")) + c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(5000)) + _, err = tk.Exec("set @@tidb_top_sql_max_statement_count='abc';") + c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_max_statement_count'") + _, err = tk.Exec("set @@tidb_top_sql_max_statement_count='-1';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_max_statement_count' can't be set to the value of '-1'") + tk.MustQuery("select @@tidb_top_sql_max_statement_count;").Check(testkit.Rows("5000")) + c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(5000)) + tk.MustExec("set @@global.tidb_top_sql_max_statement_count=2;") + tk.MustQuery("select @@global.tidb_top_sql_max_statement_count;").Check(testkit.Rows("2")) + c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(2)) + _, err = tk.Exec("set @@global.tidb_top_sql_max_statement_count='abc';") + c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_max_statement_count'") + _, err = tk.Exec("set @@global.tidb_top_sql_max_statement_count='-1';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_max_statement_count' can't be set to the value of '-1'") + _, err = tk.Exec("set @@global.tidb_top_sql_max_statement_count='5001';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_max_statement_count' can't be set to the value of '5001'") + tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("2")) + c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(2)) +} diff --git a/executor/update.go b/executor/update.go index 194f8e415a17a..cbc61f6121b88 100644 --- a/executor/update.go +++ b/executor/update.go @@ -20,10 +20,10 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" plannercore "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -264,7 +264,7 @@ func (e *UpdateExec) updateRows(ctx context.Context) (int, error) { txn.GetSnapshot().SetOption(kv.CollectRuntimeStats, e.stats.SnapshotRuntimeStats) } } - if config.TopSQLEnabled() { + if variable.TopSQLEnabled() { txn, err := e.ctx.Txn(true) if err == nil { txn.SetOption(kv.ResourceGroupTag, e.ctx.GetSessionVars().StmtCtx.GetResourceGroupTag()) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 23e17b0cec704..ff53d355e4443 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1627,6 +1627,61 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBGCLifetime, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64}, {Scope: ScopeGlobal, Name: TiDBGCConcurrency, Value: "-1", Type: TypeInt, MinValue: 1, MaxValue: 128, AllowAutoValue: true}, {Scope: ScopeGlobal, Name: TiDBGCScanLockMode, Value: "PHYSICAL", Type: TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}}, + + // variable for top SQL feature. + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { + return BoolToOnOff(TopSQLVariable.Enable.Load()), nil + }, SetSession: func(vars *SessionVars, s string) error { + TopSQLVariable.Enable.Store(TiDBOptOn(s)) + return nil + }, SetGlobal: func(vars *SessionVars, s string) error { + TopSQLVariable.Enable.Store(TiDBOptOn(s)) + return nil + }}, + // TODO(crazycs520): Add validation + {Scope: ScopeGlobal | ScopeSession, Name: TiDBTopSQLAgentAddress, Value: DefTiDBTopSQLAgentAddress, Type: TypeStr, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { + return TopSQLVariable.AgentAddress.Load(), nil + }, SetSession: func(vars *SessionVars, s string) error { + TopSQLVariable.AgentAddress.Store(s) + return nil + }, SetGlobal: func(vars *SessionVars, s string) error { + TopSQLVariable.AgentAddress.Store(s) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBTopSQLPrecisionSeconds, Value: strconv.Itoa(DefTiDBTopSQLPrecisionSeconds), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { + return strconv.FormatInt(TopSQLVariable.PrecisionSeconds.Load(), 10), nil + }, SetSession: func(vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TopSQLVariable.PrecisionSeconds.Store(val) + return nil + }, SetGlobal: func(vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TopSQLVariable.PrecisionSeconds.Store(val) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBTopSQLMaxStatementCount, Value: strconv.Itoa(DefTiDBTopSQLMaxStatementCount), Type: TypeInt, MinValue: 0, MaxValue: 5000, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { + return strconv.FormatInt(TopSQLVariable.MaxStatementCount.Load(), 10), nil + }, SetSession: func(vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TopSQLVariable.MaxStatementCount.Store(val) + return nil + }, SetGlobal: func(vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TopSQLVariable.MaxStatementCount.Store(val) + return nil + }}, } // FeedbackProbability points to the FeedbackProbability in statistics package. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 5d7897ff9273a..63f03e4c14695 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -531,6 +531,18 @@ const ( // TiDBEnableDynamicPrivileges enables MySQL 8.0 compatible dynamic privileges (experimental). TiDBEnableDynamicPrivileges = "tidb_enable_dynamic_privileges" + + // TiDBEnableTopSQL indicates whether the top SQL is enabled. + TiDBEnableTopSQL = "tidb_enable_top_sql" + + // TiDBTopSQLAgentAddress indicates the top SQL agent address. + TiDBTopSQLAgentAddress = "tidb_top_sql_agent_address" + + // TiDBTopSQLPrecisionSeconds indicates the top SQL precision seconds. + TiDBTopSQLPrecisionSeconds = "tidb_top_sql_precision_seconds" + + // TiDBTopSQLMaxStatementCount indicates the max number of statements been collected. + TiDBTopSQLMaxStatementCount = "tidb_top_sql_max_statement_count" ) // TiDB vars that have only global scope @@ -673,6 +685,10 @@ const ( DefTiDBTrackAggregateMemoryUsage = true DefTiDBEnableExchangePartition = false DefCTEMaxRecursionDepth = 1000 + DefTiDBTopSQLEnable = false + DefTiDBTopSQLAgentAddress = "" + DefTiDBTopSQLPrecisionSeconds = 1 + DefTiDBTopSQLMaxStatementCount = 200 ) // Process global variables. @@ -697,4 +713,27 @@ var ( CapturePlanBaseline = serverGlobalVariable{globalVal: Off} DefExecutorConcurrency = 5 MemoryUsageAlarmRatio = atomic.NewFloat64(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio) + TopSQLVariable = TopSQL{ + Enable: atomic.NewBool(DefTiDBTopSQLEnable), + AgentAddress: atomic.NewString(DefTiDBTopSQLAgentAddress), + PrecisionSeconds: atomic.NewInt64(DefTiDBTopSQLPrecisionSeconds), + MaxStatementCount: atomic.NewInt64(DefTiDBTopSQLMaxStatementCount), + } ) + +// TopSQL is the variable for control top sql feature. +type TopSQL struct { + // Enable statement summary or not. + Enable *atomic.Bool + // AgentAddress indicate the collect agent address. + AgentAddress *atomic.String + // The refresh interval of statement summary. + PrecisionSeconds *atomic.Int64 + // The maximum number of statements kept in memory. + MaxStatementCount *atomic.Int64 +} + +// TopSQLEnabled uses to check whether enabled the top SQL feature. +func TopSQLEnabled() bool { + return TopSQLVariable.Enable.Load() && TopSQLVariable.AgentAddress.Load() != "" +} From 11019d6f346e65d326da51ea1334b782678956ef Mon Sep 17 00:00:00 2001 From: Howie Date: Mon, 31 May 2021 15:47:36 +0800 Subject: [PATCH 217/343] ddl: add admin_checksum_table compatibility for temporary table (#24867) --- ddl/serial_test.go | 7 +++++++ executor/admin_test.go | 14 +++++++++++++- planner/core/planbuilder.go | 3 +++ planner/core/preprocess.go | 8 ++++++-- 4 files changed, 29 insertions(+), 3 deletions(-) diff --git a/ddl/serial_test.go b/ddl/serial_test.go index f78ff3d16577b..8d9217e1df8e3 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -534,6 +534,13 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) { _, err = tk.Exec("create table temporary_table_t1 like temporary_table") c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error()) tk.MustExec("drop table if exists temporary_table;") + + tk.MustExec("drop table if exists temporary_table_like;") + tk.MustExec("create table temporary_table (a int, b int,index(a))") + tk.MustExec("drop table if exists temporary_table_like_t1;") + _, err = tk.Exec("create global temporary table temporary_table_like_t1 like temporary_table on commit delete rows;") + c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error()) + tk.MustExec("drop table if exists temporary_table_like;") } // TestCancelAddIndex1 tests canceling ddl job when the add index worker is not started. diff --git a/executor/admin_test.go b/executor/admin_test.go index 35e3d08345d63..2e1d24bd32bc6 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -24,6 +24,7 @@ import ( mysql "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" @@ -82,7 +83,8 @@ func (s *testSuite5) TestAdminCheckIndexInTemporaryMode(c *C) { tk.MustExec("drop table if exists temporary_admin_test;") tk.MustExec("create global temporary table temporary_admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c1), unique key(c2)) ON COMMIT DELETE ROWS;") tk.MustExec("insert temporary_admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);") - tk.MustGetErrCode("admin check table temporary_admin_test;", mysql.ErrAdminCheckTable) + _, err := tk.Exec("admin check table temporary_admin_test;") + c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin check table").Error()) tk.MustExec("drop table if exists temporary_admin_test;") tk.MustExec("drop table if exists non_temporary_admin_test;") @@ -90,6 +92,16 @@ func (s *testSuite5) TestAdminCheckIndexInTemporaryMode(c *C) { tk.MustExec("insert non_temporary_admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);") tk.MustExec("admin check table non_temporary_admin_test;") tk.MustExec("drop table if exists non_temporary_admin_test;") + + tk.MustExec("drop table if exists temporary_admin_checksum_table_with_index_test;") + tk.MustExec("drop table if exists temporary_admin_checksum_table_without_index_test;") + tk.MustExec("create global temporary table temporary_admin_checksum_table_with_index_test (id int, count int, PRIMARY KEY(id), KEY(count)) ON COMMIT DELETE ROWS;") + tk.MustExec("create global temporary table temporary_admin_checksum_table_without_index_test (id int, count int, PRIMARY KEY(id)) ON COMMIT DELETE ROWS;") + _, err = tk.Exec("admin checksum table temporary_admin_checksum_table_with_index_test;") + c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table").Error()) + _, err = tk.Exec("admin checksum table temporary_admin_checksum_table_without_index_test;") + c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table").Error()) + tk.MustExec("drop table if exists temporary_admin_checksum_table_with_index_test,temporary_admin_checksum_table_without_index_test;") } func (s *testSuite5) TestAdminRecoverIndex(c *C) { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 6640af5785b81..2189f645e32b1 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -3471,6 +3471,9 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err b.visitInfo = appendVisitInfo(b.visitInfo, mysql.IndexPriv, v.Table.Schema.L, v.Table.Name.L, "", authErr) case *ast.CreateTableStmt: + if v.TemporaryKeyword != ast.TemporaryNone && v.ReferTable != nil { + return nil, ErrOptOnTemporaryTable.GenWithStackByArgs("create table like") + } if b.ctx.GetSessionVars().User != nil { authErr = ErrTableaccessDenied.GenWithStackByArgs("CREATE", b.ctx.GetSessionVars().User.AuthUsername, b.ctx.GetSessionVars().User.AuthHostname, v.Table.Name.L) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index c23d4c2df170e..29ad5c848dbeb 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -608,8 +608,12 @@ func (p *preprocessor) checkAdminCheckTableGrammar(stmt *ast.AdminStmt) { return } tempTableType := tableInfo.Meta().TempTableType - if stmt.Tp == ast.AdminCheckTable && tempTableType != model.TempTableNone { - p.err = infoschema.ErrAdminCheckTable + if (stmt.Tp == ast.AdminCheckTable || stmt.Tp == ast.AdminChecksumTable) && tempTableType != model.TempTableNone { + if stmt.Tp == ast.AdminChecksumTable { + p.err = ErrOptOnTemporaryTable.GenWithStackByArgs("admin checksum table") + } else { + p.err = ErrOptOnTemporaryTable.GenWithStackByArgs("admin check table") + } return } } From 238cab806e940a331ea141d11a531fb31ce7b904 Mon Sep 17 00:00:00 2001 From: xhe Date: Mon, 31 May 2021 18:17:38 +0800 Subject: [PATCH 218/343] *: ddl compatibility test completion (#24983) --- executor/stale_txn_test.go | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 7714f42e122ee..b90ece5b77224 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -360,16 +360,28 @@ func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") + defer tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key);") - // test as of schemaVer1 := tk.Se.GetInfoSchema().SchemaMetaVersion() time1 := time.Now() - tk.MustExec("drop table if exists t") - c.Assert(schemaVer1, Less, tk.Se.GetInfoSchema().SchemaMetaVersion()) + tk.MustExec("alter table t add c int") + + // confirm schema changed + schemaVer2 := tk.Se.GetInfoSchema().SchemaMetaVersion() + c.Assert(schemaVer1, Less, schemaVer2) + + // get the specific old schema tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) + + // schema changed back to the newest tk.MustExec("commit") + c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + + // select does not affect the infoschema + tk.MustExec(fmt.Sprintf(`SELECT * from t AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) + c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) } func (s *testStaleTxnSerialSuite) TestSetTransactionReadOnlyAsOf(c *C) { From 7af4feaad1ba4c1aedfdb8f438296e44097e52ac Mon Sep 17 00:00:00 2001 From: Shirly Date: Mon, 31 May 2021 18:37:36 +0800 Subject: [PATCH 219/343] store/tikv: rename func OnSendFailForRegion to OnSendFailForTiFlash (#24992) --- store/copr/region_cache.go | 17 +--- store/tikv/region_cache.go | 172 ++++++++++++++++++++----------------- 2 files changed, 96 insertions(+), 93 deletions(-) diff --git a/store/copr/region_cache.go b/store/copr/region_cache.go index 9d6bd4dc973c6..23c49725b40e6 100644 --- a/store/copr/region_cache.go +++ b/store/copr/region_cache.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" - "go.uber.org/zap" ) // RegionCache wraps tikv.RegionCache. @@ -119,20 +118,6 @@ func (c *RegionCache) OnSendFailForBatchRegions(bo *Backoffer, store *tikv.Store if ri.Meta == nil { continue } - r := c.GetCachedRegionWithRLock(ri.Region) - if r == nil { - return - } - peersNum := len(r.GetMeta().Peers) - if len(ri.Meta.Peers) != peersNum { - logutil.Logger(bo.GetCtx()).Info("retry and refresh current region after send request fail and up/down stores length changed", - zap.Stringer("region", &ri.Region), - zap.Bool("needReload", scheduleReload), - zap.Reflect("oldPeers", ri.Meta.Peers), - zap.Reflect("newPeers", r.GetMeta().Peers), - zap.Error(err)) - continue - } - c.OnSendFailForRegion(bo.TiKVBackoffer(), store, ri.Region, r, scheduleReload, err) + c.OnSendFailForTiFlash(bo.TiKVBackoffer(), store, ri.Region, ri.Meta, scheduleReload, err) } } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index d3eead0444940..2fb006138dff5 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -733,32 +733,41 @@ func (c *RegionCache) findRegionByKey(bo *Backoffer, key []byte, isEndKey bool) return r, nil } -// OnSendFailForRegion handles send request fail logic on a region. -func (c *RegionCache) OnSendFailForRegion(bo *Backoffer, store *Store, rid RegionVerID, r *Region, scheduleReload bool, err error) { +// OnSendFailForTiFlash handles send request fail logic for tiflash. +func (c *RegionCache) OnSendFailForTiFlash(bo *Backoffer, store *Store, region RegionVerID, prev *metapb.Region, scheduleReload bool, err error) { + + r := c.GetCachedRegionWithRLock(region) + if r == nil { + return + } rs := r.getStore() + peersNum := len(r.GetMeta().Peers) + if len(prev.Peers) != peersNum { + logutil.Logger(bo.GetCtx()).Info("retry and refresh current region after send request fail and up/down stores length changed", + zap.Stringer("region", ®ion), + zap.Bool("needReload", scheduleReload), + zap.Reflect("oldPeers", prev.Peers), + zap.Reflect("newPeers", r.GetMeta().Peers), + zap.Error(err)) + return + } accessMode := TiFlashOnly accessIdx := rs.getAccessIndex(accessMode, store) if accessIdx == -1 { - logutil.Logger(bo.GetCtx()).Warn("can not get access index for region " + rid.String()) + logutil.Logger(bo.GetCtx()).Warn("can not get access index for region " + region.String()) return } if err != nil { storeIdx, s := rs.accessStore(accessMode, accessIdx) - epoch := rs.storeEpochs[storeIdx] - if atomic.CompareAndSwapUint32(&s.epoch, epoch, epoch+1) { - logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) - metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() - } - // schedule a store addr resolve. - s.markNeedCheck(c.notifyCheckCh) + c.markRegionNeedBeRefill(s, storeIdx, rs) } // try next peer rs.switchNextFlashPeer(r, accessIdx) logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", - zap.Stringer("region", &rid), + zap.Stringer("region", ®ion), zap.Bool("needReload", scheduleReload), zap.Error(err)) @@ -768,90 +777,99 @@ func (c *RegionCache) OnSendFailForRegion(bo *Backoffer, store *Store, rid Regio } } +func (c *RegionCache) markRegionNeedBeRefill(s *Store, storeIdx int, rs *RegionStore) int { + incEpochStoreIdx := -1 + // invalidate regions in store. + epoch := rs.storeEpochs[storeIdx] + if atomic.CompareAndSwapUint32(&s.epoch, epoch, epoch+1) { + logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) + incEpochStoreIdx = storeIdx + metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() + } + // schedule a store addr resolve. + s.markNeedCheck(c.notifyCheckCh) + return incEpochStoreIdx +} + // OnSendFail handles send request fail logic. func (c *RegionCache) OnSendFail(bo *Backoffer, ctx *RPCContext, scheduleReload bool, err error) { metrics.RegionCacheCounterWithSendFail.Inc() r := c.GetCachedRegionWithRLock(ctx.Region) - if r != nil { - peersNum := len(r.meta.Peers) - if len(ctx.Meta.Peers) != peersNum { - logutil.Logger(bo.GetCtx()).Info("retry and refresh current ctx after send request fail and up/down stores length changed", - zap.Stringer("current", ctx), - zap.Bool("needReload", scheduleReload), - zap.Reflect("oldPeers", ctx.Meta.Peers), - zap.Reflect("newPeers", r.meta.Peers), - zap.Error(err)) - return - } - - rs := r.getStore() - startForwarding := false - incEpochStoreIdx := -1 + if r == nil { + return + } + peersNum := len(r.meta.Peers) + if len(ctx.Meta.Peers) != peersNum { + logutil.Logger(bo.GetCtx()).Info("retry and refresh current ctx after send request fail and up/down stores length changed", + zap.Stringer("current", ctx), + zap.Bool("needReload", scheduleReload), + zap.Reflect("oldPeers", ctx.Meta.Peers), + zap.Reflect("newPeers", r.meta.Peers), + zap.Error(err)) + return + } - if err != nil { - storeIdx, s := rs.accessStore(ctx.AccessMode, ctx.AccessIdx) - leaderReq := ctx.Store.storeType == tikvrpc.TiKV && rs.workTiKVIdx == ctx.AccessIdx - - // Mark the store as failure if it's not a redirection request because we - // can't know the status of the proxy store by it. - if ctx.ProxyStore == nil { - // send fail but store is reachable, keep retry current peer for replica leader request. - // but we still need switch peer for follower-read or learner-read(i.e. tiflash) - if leaderReq { - if s.requestLiveness(bo, c) == reachable { - return - } else if c.enableForwarding { - s.startHealthCheckLoopIfNeeded(c) - startForwarding = true - } - } + rs := r.getStore() + startForwarding := false + incEpochStoreIdx := -1 - // invalidate regions in store. - epoch := rs.storeEpochs[storeIdx] - if atomic.CompareAndSwapUint32(&s.epoch, epoch, epoch+1) { - logutil.BgLogger().Info("mark store's regions need be refill", zap.String("store", s.addr)) - incEpochStoreIdx = storeIdx - metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() + if err != nil { + storeIdx, s := rs.accessStore(ctx.AccessMode, ctx.AccessIdx) + leaderReq := ctx.Store.storeType == tikvrpc.TiKV && rs.workTiKVIdx == ctx.AccessIdx + + // Mark the store as failure if it's not a redirection request because we + // can't know the status of the proxy store by it. + if ctx.ProxyStore == nil { + // send fail but store is reachable, keep retry current peer for replica leader request. + // but we still need switch peer for follower-read or learner-read(i.e. tiflash) + if leaderReq { + if s.requestLiveness(bo, c) == reachable { + return + } else if c.enableForwarding { + s.startHealthCheckLoopIfNeeded(c) + startForwarding = true } - // schedule a store addr resolve. - s.markNeedCheck(c.notifyCheckCh) } + + // invalidate regions in store. + incEpochStoreIdx = c.markRegionNeedBeRefill(s, storeIdx, rs) } + } - // try next peer to found new leader. - if ctx.AccessMode == TiKVOnly { - if startForwarding || ctx.ProxyStore != nil { - var currentProxyIdx AccessIndex = -1 - if ctx.ProxyStore != nil { - currentProxyIdx = ctx.ProxyAccessIdx - } - // In case the epoch of the store is increased, try to avoid reloading the current region by also - // increasing the epoch stored in `rs`. - rs.switchNextProxyStore(r, currentProxyIdx, incEpochStoreIdx) - logutil.Logger(bo.GetCtx()).Info("switch region proxy peer to next due to send request fail", - zap.Stringer("current", ctx), - zap.Bool("needReload", scheduleReload), - zap.Error(err)) - } else { - rs.switchNextTiKVPeer(r, ctx.AccessIdx) - logutil.Logger(bo.GetCtx()).Info("switch region peer to next due to send request fail", - zap.Stringer("current", ctx), - zap.Bool("needReload", scheduleReload), - zap.Error(err)) + // try next peer to found new leader. + if ctx.AccessMode == TiKVOnly { + if startForwarding || ctx.ProxyStore != nil { + var currentProxyIdx AccessIndex = -1 + if ctx.ProxyStore != nil { + currentProxyIdx = ctx.ProxyAccessIdx } + // In case the epoch of the store is increased, try to avoid reloading the current region by also + // increasing the epoch stored in `rs`. + rs.switchNextProxyStore(r, currentProxyIdx, incEpochStoreIdx) + logutil.Logger(bo.GetCtx()).Info("switch region proxy peer to next due to send request fail", + zap.Stringer("current", ctx), + zap.Bool("needReload", scheduleReload), + zap.Error(err)) } else { - rs.switchNextFlashPeer(r, ctx.AccessIdx) - logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", + rs.switchNextTiKVPeer(r, ctx.AccessIdx) + logutil.Logger(bo.GetCtx()).Info("switch region peer to next due to send request fail", zap.Stringer("current", ctx), zap.Bool("needReload", scheduleReload), zap.Error(err)) } + } else { + rs.switchNextFlashPeer(r, ctx.AccessIdx) + logutil.Logger(bo.GetCtx()).Info("switch region tiflash peer to next due to send request fail", + zap.Stringer("current", ctx), + zap.Bool("needReload", scheduleReload), + zap.Error(err)) + } - // force reload region when retry all known peers in region. - if scheduleReload { - r.scheduleReload() - } + // force reload region when retry all known peers in region. + if scheduleReload { + r.scheduleReload() } + } // LocateRegionByID searches for the region with ID. From a80047c6049ac6611f7af39771683b0d1cf5167e Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Mon, 31 May 2021 19:03:36 +0800 Subject: [PATCH 220/343] CTE: support explain CTE plan (#24986) --- cmd/explaintest/r/explain_cte.result | 129 +++++++++++++++++++++++++++ cmd/explaintest/t/explain_cte.test | 31 +++++++ planner/core/common_plans.go | 28 ++++++ planner/core/encode.go | 28 ++++++ planner/core/find_best_task.go | 2 +- planner/core/logical_plan_builder.go | 3 +- planner/core/logical_plans.go | 3 +- planner/core/physical_plans.go | 52 +++++++++++ util/plancodec/id.go | 21 ++++- 9 files changed, 292 insertions(+), 5 deletions(-) create mode 100644 cmd/explaintest/r/explain_cte.result create mode 100644 cmd/explaintest/t/explain_cte.test diff --git a/cmd/explaintest/r/explain_cte.result b/cmd/explaintest/r/explain_cte.result new file mode 100644 index 0000000000000..4092fe73fbd07 --- /dev/null +++ b/cmd/explaintest/r/explain_cte.result @@ -0,0 +1,129 @@ +use test; +drop table if exists t1, t2; +create table t1 (c1 int primary key, c2 int, index c2 (c2)); +create table t2 (c1 int unique, c2 int); +insert into t1 values(1, 0), (2, 1); +insert into t2 values(1, 0), (2, 1); +explain with cte(a) as (select 1) select * from cte; +id estRows task access object operator info +CTEFullScan_8 1.00 root CTE:cte data:CTE_0 +CTE_0 1.00 root None Recursive CTE +└─Projection_6(Seed Part) 1.00 root 1->Column#1 + └─TableDual_7 1.00 root rows:1 +explain with cte(a) as (select c1 from t1) select * from cte; +id estRows task access object operator info +CTEFullScan_11 1.00 root CTE:cte data:CTE_0 +CTE_0 1.00 root None Recursive CTE +└─TableReader_8(Seed Part) 10000.00 root data:TableFullScan_7 + └─TableFullScan_7 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain with cte(a,b,c,d) as (select * from t1, t2) select * from cte; +id estRows task access object operator info +CTEFullScan_18 1.00 root CTE:cte data:CTE_0 +CTE_0 1.00 root None Recursive CTE +└─HashJoin_10(Seed Part) 100000000.00 root CARTESIAN inner join + ├─TableReader_17(Build) 10000.00 root data:TableFullScan_16 + │ └─TableFullScan_16 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─TableReader_13(Probe) 10000.00 root data:TableFullScan_12 + └─TableFullScan_12 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain with recursive cte(a) as (select 1 union select a+1 from cte where a < 10) select * from cte; +id estRows task access object operator info +CTEFullScan_17 1.00 root CTE:cte data:CTE_0 +CTE_0 1.00 root Recursive CTE +├─Projection_12(Seed Part) 1.00 root 1->Column#2 +│ └─TableDual_13 1.00 root rows:1 +└─Projection_14(Recursive Part) 0.80 root cast(plus(Column#3, 1), bigint(1) BINARY)->Column#5 + └─Selection_15 0.80 root lt(Column#3, 10) + └─CTETable_16 1.00 root Scan on CTE_0 +explain with recursive cte(a) as (select c2 from t1 union select a+1 from cte where a < 10) select * from cte; +id estRows task access object operator info +CTEFullScan_20 1.00 root CTE:cte data:CTE_0 +CTE_0 1.00 root Recursive CTE +├─TableReader_14(Seed Part) 10000.00 root data:TableFullScan_13 +│ └─TableFullScan_13 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Projection_17(Recursive Part) 0.80 root cast(plus(test.t1.c2, 1), int(11))->test.t1.c2 + └─Selection_18 0.80 root lt(test.t1.c2, 10) + └─CTETable_19 1.00 root Scan on CTE_0 +explain with cte(a) as (with recursive cte1(a) as (select 1 union select a + 1 from cte1 where a < 10) select * from cte1) select * from cte; +id estRows task access object operator info +CTEFullScan_21 1.00 root CTE:cte data:CTE_0 +CTE_0 1.00 root None Recursive CTE +└─CTEFullScan_20(Seed Part) 1.00 root CTE:cte1 data:CTE_1 +CTE_1 1.00 root Recursive CTE +├─Projection_15(Seed Part) 1.00 root 1->Column#2 +│ └─TableDual_16 1.00 root rows:1 +└─Projection_17(Recursive Part) 0.80 root cast(plus(Column#3, 1), bigint(1) BINARY)->Column#5 + └─Selection_18 0.80 root lt(Column#3, 10) + └─CTETable_19 1.00 root Scan on CTE_1 +explain with recursive cte(a) as (select 1 union select a+1 from cte where a < 10) select * from cte t1, cte t2; +id estRows task access object operator info +HashJoin_15 1.00 root CARTESIAN inner join +├─CTEFullScan_23(Build) 1.00 root CTE:t2 data:CTE_0 +└─CTEFullScan_22(Probe) 1.00 root CTE:t1 data:CTE_0 +CTE_0 1.00 root Recursive CTE +├─Projection_17(Seed Part) 1.00 root 1->Column#2 +│ └─TableDual_18 1.00 root rows:1 +└─Projection_19(Recursive Part) 0.80 root cast(plus(Column#3, 1), bigint(1) BINARY)->Column#5 + └─Selection_20 0.80 root lt(Column#3, 10) + └─CTETable_21 1.00 root Scan on CTE_0 +explain with cte(a) as (with recursive cte1(a) as (select 1 union select a + 1 from cte1 where a < 10) select * from cte1) select * from cte t1, cte t2; +id estRows task access object operator info +HashJoin_17 1.00 root CARTESIAN inner join +├─CTEFullScan_27(Build) 1.00 root CTE:t2 data:CTE_0 +└─CTEFullScan_26(Probe) 1.00 root CTE:t1 data:CTE_0 +CTE_0 1.00 root None Recursive CTE +└─CTEFullScan_25(Seed Part) 1.00 root CTE:cte1 data:CTE_1 +CTE_1 1.00 root Recursive CTE +├─Projection_20(Seed Part) 1.00 root 1->Column#2 +│ └─TableDual_21 1.00 root rows:1 +└─Projection_22(Recursive Part) 0.80 root cast(plus(Column#3, 1), bigint(1) BINARY)->Column#5 + └─Selection_23 0.80 root lt(Column#3, 10) + └─CTETable_24 1.00 root Scan on CTE_1 +explain with recursive cte1(a) as (select 1 union select a+1 from cte1 where a < 10), cte2(a) as (select c2 from t1 union select a+1 from cte2 where a < 10) select * from cte1, cte2; +id estRows task access object operator info +HashJoin_23 1.00 root CARTESIAN inner join +├─CTEFullScan_39(Build) 1.00 root CTE:cte2 data:CTE_1 +└─CTEFullScan_30(Probe) 1.00 root CTE:cte1 data:CTE_0 +CTE_1 1.00 root Recursive CTE +├─TableReader_33(Seed Part) 10000.00 root data:TableFullScan_32 +│ └─TableFullScan_32 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Projection_36(Recursive Part) 0.80 root cast(plus(test.t1.c2, 1), int(11))->test.t1.c2 + └─Selection_37 0.80 root lt(test.t1.c2, 10) + └─CTETable_38 1.00 root Scan on CTE_1 +CTE_0 1.00 root Recursive CTE +├─Projection_25(Seed Part) 1.00 root 1->Column#2 +│ └─TableDual_26 1.00 root rows:1 +└─Projection_27(Recursive Part) 0.80 root cast(plus(Column#3, 1), bigint(1) BINARY)->Column#5 + └─Selection_28 0.80 root lt(Column#3, 10) + └─CTETable_29 1.00 root Scan on CTE_0 +explain with q(a,b) as (select * from t1) select /*+ merge(q) no_merge(q1) */ * from q, q q1 where q.a=1 and q1.a=2; +id estRows task access object operator info +HashJoin_12 0.64 root CARTESIAN inner join +├─Selection_21(Build) 0.80 root eq(test.t1.c1, 2) +│ └─CTEFullScan_22 1.00 root CTE:q1 data:CTE_0 +└─Selection_14(Probe) 0.80 root eq(test.t1.c1, 1) + └─CTEFullScan_20 1.00 root CTE:q data:CTE_0 +CTE_0 1.00 root None Recursive CTE +└─TableReader_17(Seed Part) 10000.00 root data:TableFullScan_16 + └─TableFullScan_16 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +explain with recursive cte(a,b) as (select 1, concat('a', 1) union select a+1, concat(b, 1) from cte where a < 5) select * from cte; +id estRows task access object operator info +CTEFullScan_17 1.00 root CTE:cte data:CTE_0 +CTE_0 1.00 root Recursive CTE +├─Projection_12(Seed Part) 1.00 root 1->Column#3, a1->Column#4 +│ └─TableDual_13 1.00 root rows:1 +└─Projection_14(Recursive Part) 0.80 root cast(plus(Column#5, 1), bigint(1) BINARY)->Column#9, cast(concat(Column#6, 1), var_string(21))->Column#10 + └─Selection_15 0.80 root lt(Column#5, 5) + └─CTETable_16 1.00 root Scan on CTE_0 +explain select * from t1 dt where exists(with recursive qn as (select c1*0+1 as b union all select b+1 from qn where b=0) select * from qn where b=1); +id estRows task access object operator info +Apply_19 10000.00 root CARTESIAN semi join +├─TableReader_21(Build) 10000.00 root data:TableFullScan_20 +│ └─TableFullScan_20 10000.00 cop[tikv] table:dt keep order:false, stats:pseudo +└─Selection_24(Probe) 0.80 root eq(Column#8, 1) + └─CTEFullScan_30 1.00 root CTE:qn data:CTE_0 +CTE_0 1.00 root Recursive CTE +├─Projection_25(Seed Part) 1.00 root plus(mul(test.t1.c1, 0), 1)->Column#4 +│ └─TableDual_26 1.00 root rows:1 +└─Projection_27(Recursive Part) 0.80 root plus(Column#5, 1)->Column#7 + └─Selection_28 0.80 root eq(Column#5, 0) + └─CTETable_29 1.00 root Scan on CTE_0 diff --git a/cmd/explaintest/t/explain_cte.test b/cmd/explaintest/t/explain_cte.test new file mode 100644 index 0000000000000..50032776f85dd --- /dev/null +++ b/cmd/explaintest/t/explain_cte.test @@ -0,0 +1,31 @@ +use test; +drop table if exists t1, t2; +create table t1 (c1 int primary key, c2 int, index c2 (c2)); +create table t2 (c1 int unique, c2 int); +insert into t1 values(1, 0), (2, 1); +insert into t2 values(1, 0), (2, 1); + +# simple cte +explain with cte(a) as (select 1) select * from cte; +explain with cte(a) as (select c1 from t1) select * from cte; +explain with cte(a,b,c,d) as (select * from t1, t2) select * from cte; + +# recursive cte +explain with recursive cte(a) as (select 1 union select a+1 from cte where a < 10) select * from cte; +explain with recursive cte(a) as (select c2 from t1 union select a+1 from cte where a < 10) select * from cte; + +# nested cte +explain with cte(a) as (with recursive cte1(a) as (select 1 union select a + 1 from cte1 where a < 10) select * from cte1) select * from cte; + +# cte with join +explain with recursive cte(a) as (select 1 union select a+1 from cte where a < 10) select * from cte t1, cte t2; +explain with cte(a) as (with recursive cte1(a) as (select 1 union select a + 1 from cte1 where a < 10) select * from cte1) select * from cte t1, cte t2; + +# multiple cte +explain with recursive cte1(a) as (select 1 union select a+1 from cte1 where a < 10), cte2(a) as (select c2 from t1 union select a+1 from cte2 where a < 10) select * from cte1, cte2; + +# other +explain with q(a,b) as (select * from t1) select /*+ merge(q) no_merge(q1) */ * from q, q q1 where q.a=1 and q1.a=2; +# explain with cte(a,b) as (select * from t1) select (select 1 from cte limit 1) from cte; +explain with recursive cte(a,b) as (select 1, concat('a', 1) union select a+1, concat(b, 1) from cte where a < 5) select * from cte; +explain select * from t1 dt where exists(with recursive qn as (select c1*0+1 as b union all select b+1 from qn where b=0) select * from qn where b=1); diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index a20e2bf12a359..d13a2d3313288 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -955,6 +955,8 @@ type Explain struct { Rows [][]string ExplainRows [][]string explainedPlans map[int]bool + + ctes []*PhysicalCTE } // GetExplainRowsForPlan get explain rows for plan. @@ -1016,6 +1018,10 @@ func (e *Explain) RenderResult() error { if err != nil { return err } + err = e.explainPlanInRowFormatCTE() + if err != nil { + return err + } } case ast.ExplainFormatDOT: if physicalPlan, ok := e.TargetPlan.(PhysicalPlan); ok { @@ -1031,6 +1037,26 @@ func (e *Explain) RenderResult() error { return nil } +func (e *Explain) explainPlanInRowFormatCTE() (err error) { + explainedCTEPlan := make(map[int]struct{}) + for i := 0; i < len(e.ctes); i++ { + x := (*CTEDefinition)(e.ctes[i]) + // skip if the CTE has been explained, the same CTE has same IDForStorage + if _, ok := explainedCTEPlan[x.CTE.IDForStorage]; ok { + continue + } + e.prepareOperatorInfo(x, "root", "", "", true) + childIndent := texttree.Indent4Child("", true) + err = e.explainPlanInRowFormat(x.SeedPlan, "root", "(Seed Part)", childIndent, x.RecurPlan == nil) + if x.RecurPlan != nil { + err = e.explainPlanInRowFormat(x.RecurPlan, "root", "(Recursive Part)", childIndent, true) + } + explainedCTEPlan[x.CTE.IDForStorage] = struct{}{} + } + + return +} + // explainPlanInRowFormat generates explain information for root-tasks. func (e *Explain) explainPlanInRowFormat(p Plan, taskType, driverSide, indent string, isLastChild bool) (err error) { e.prepareOperatorInfo(p, taskType, driverSide, indent, isLastChild) @@ -1134,6 +1160,8 @@ func (e *Explain) explainPlanInRowFormat(p Plan, taskType, driverSide, indent st if x.Plan != nil { err = e.explainPlanInRowFormat(x.Plan, "root", "", indent, true) } + case *PhysicalCTE: + e.ctes = append(e.ctes, x) } return } diff --git a/planner/core/encode.go b/planner/core/encode.go index 8dc6ddeca9473..2418c502aa964 100644 --- a/planner/core/encode.go +++ b/planner/core/encode.go @@ -34,6 +34,8 @@ var encoderPool = sync.Pool{ type planEncoder struct { buf bytes.Buffer encodedPlans map[int]bool + + ctes []*PhysicalCTE } // EncodePlan is used to encodePlan the plan to the plan tree with compressing. @@ -59,9 +61,33 @@ func (pn *planEncoder) encodePlanTree(p Plan) string { pn.encodedPlans = make(map[int]bool) pn.buf.Reset() pn.encodePlan(p, true, kv.TiKV, 0) + pn.encodeCTEPlan() return plancodec.Compress(pn.buf.Bytes()) } +func (pn *planEncoder) encodeCTEPlan() { + explainedCTEPlan := make(map[int]struct{}) + for i := 0; i < len(pn.ctes); i++ { + x := (*CTEDefinition)(pn.ctes[i]) + // skip if the CTE has been explained, the same CTE has same IDForStorage + if _, ok := explainedCTEPlan[x.CTE.IDForStorage]; ok { + continue + } + taskTypeInfo := plancodec.EncodeTaskType(true, kv.TiKV) + actRows, analyzeInfo, memoryInfo, diskInfo := getRuntimeInfo(x.SCtx(), x, nil) + rowCount := 0.0 + if statsInfo := x.statsInfo(); statsInfo != nil { + rowCount = x.statsInfo().RowCount + } + plancodec.EncodePlanNode(0, x.CTE.IDForStorage, plancodec.TypeCTEDefinition, rowCount, taskTypeInfo, x.ExplainInfo(), actRows, analyzeInfo, memoryInfo, diskInfo, &pn.buf) + pn.encodePlan(x.SeedPlan, true, kv.TiKV, 1) + if x.RecurPlan != nil { + pn.encodePlan(x.RecurPlan, true, kv.TiKV, 1) + } + explainedCTEPlan[x.CTE.IDForStorage] = struct{}{} + } +} + func (pn *planEncoder) encodePlan(p Plan, isRoot bool, store kv.StoreType, depth int) { taskTypeInfo := plancodec.EncodeTaskType(isRoot, store) actRows, analyzeInfo, memoryInfo, diskInfo := getRuntimeInfo(p.SCtx(), p, nil) @@ -102,6 +128,8 @@ func (pn *planEncoder) encodePlan(p Plan, isRoot bool, store kv.StoreType, depth if copPlan.tablePlan != nil { pn.encodePlan(copPlan.tablePlan, false, store, depth) } + case *PhysicalCTE: + pn.ctes = append(pn.ctes, copPlan) } } diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 2eb7c2b5a1a95..5207390aa8276 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1962,7 +1962,7 @@ func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, planCounter * } } - pcte := PhysicalCTE{SeedPlan: sp, RecurPlan: rp, CTE: p.cte}.Init(p.ctx, p.stats) + pcte := PhysicalCTE{SeedPlan: sp, RecurPlan: rp, CTE: p.cte, cteAsName: p.cteAsName}.Init(p.ctx, p.stats) pcte.SetSchema(p.schema) t = &rootTask{pcte, sp.statsInfo().RowCount} p.cte.cteTask = t diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index ea529dae66423..e6b2c6b6ef20b 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3668,11 +3668,12 @@ func (b *PlanBuilder) tryBuildCTE(ctx context.Context, tn *ast.TableName, asName b.handleHelper.pushMap(nil) var p LogicalPlan - lp := LogicalCTE{cte: &CTEClass{IsDistinct: cte.isDistinct, seedPartLogicalPlan: cte.seedLP, recursivePartLogicalPlan: cte.recurLP, IDForStorage: cte.storageID, optFlag: cte.optFlag}}.Init(b.ctx, b.getSelectOffset()) + lp := LogicalCTE{cteAsName: tn.Name, cte: &CTEClass{IsDistinct: cte.isDistinct, seedPartLogicalPlan: cte.seedLP, recursivePartLogicalPlan: cte.recurLP, IDForStorage: cte.storageID, optFlag: cte.optFlag}}.Init(b.ctx, b.getSelectOffset()) lp.SetSchema(getResultCTESchema(cte.seedLP.Schema(), b.ctx.GetSessionVars())) p = lp p.SetOutputNames(cte.seedLP.OutputNames()) if len(asName.String()) > 0 { + lp.cteAsName = *asName var on types.NameSlice for _, name := range p.OutputNames() { cpOn := *name diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index eac52f90da9c1..16489edc02cba 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -1190,7 +1190,8 @@ type CTEClass struct { type LogicalCTE struct { logicalSchemaProducer - cte *CTEClass + cte *CTEClass + cteAsName model.CIStr } // LogicalCTETable is for CTE table diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 353fb2897b1f9..5602399a63b74 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -14,6 +14,8 @@ package core import ( + "fmt" + "strconv" "unsafe" "github.com/pingcap/errors" @@ -31,6 +33,7 @@ import ( "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" ) @@ -1367,6 +1370,7 @@ type PhysicalCTE struct { SeedPlan PhysicalPlan RecurPlan PhysicalPlan CTE *CTEClass + cteAsName model.CIStr } // PhysicalCTETable is for CTE table. @@ -1384,3 +1388,51 @@ func (p *PhysicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn { } return corCols } + +// AccessObject implements physicalScan interface. +func (p *PhysicalCTE) AccessObject(normalized bool) string { + return fmt.Sprintf("CTE:%s", p.cteAsName.L) +} + +// OperatorInfo implements dataAccesser interface. +func (p *PhysicalCTE) OperatorInfo(normalized bool) string { + return fmt.Sprintf("data:%s", (*CTEDefinition)(p).ExplainID()) +} + +// ExplainInfo implements Plan interface. +func (p *PhysicalCTE) ExplainInfo() string { + return p.AccessObject(false) + ", " + p.OperatorInfo(false) +} + +// ExplainID overrides the ExplainID. +func (p *PhysicalCTE) ExplainID() fmt.Stringer { + return stringutil.MemoizeStr(func() string { + if p.ctx != nil && p.ctx.GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { + return p.TP() + } + return p.TP() + "_" + strconv.Itoa(p.id) + }) +} + +// ExplainInfo overrides the ExplainInfo +func (p *PhysicalCTETable) ExplainInfo() string { + return "Scan on CTE_" + strconv.Itoa(p.IDForStorage) +} + +// CTEDefinition is CTE definition for explain. +type CTEDefinition PhysicalCTE + +// ExplainInfo overrides the ExplainInfo +func (p *CTEDefinition) ExplainInfo() string { + if p.RecurPlan != nil { + return "Recursive CTE" + } + return "None Recursive CTE" +} + +// ExplainID overrides the ExplainID. +func (p *CTEDefinition) ExplainID() fmt.Stringer { + return stringutil.MemoizeStr(func() string { + return "CTE_" + strconv.Itoa(p.CTE.IDForStorage) + }) +} diff --git a/util/plancodec/id.go b/util/plancodec/id.go index 626b8fc3b1dce..b0e8e16e2de62 100644 --- a/util/plancodec/id.go +++ b/util/plancodec/id.go @@ -122,8 +122,10 @@ const ( TypeIndexRangeScan = "IndexRangeScan" // TypeCTETable is the type of TypeCTETable. TypeCTETable = "CTETable" - // TypeCTE is the type of TypeCTE. - TypeCTE = "CTE" + // TypeCTE is the type of CTEFullScan. + TypeCTE = "CTEFullScan" + // TypeCTEDefinition is the type of CTE definition + TypeCTEDefinition = "CTE" ) // plan id. @@ -178,6 +180,9 @@ const ( typeIndexRangeScan int = 47 typeExchangeReceiver int = 48 typeExchangeSender int = 49 + typeCTE int = 50 + typeCTEDefinition int = 51 + typeCTETable int = 52 ) // TypeStringToPhysicalID converts the plan type string to plan id. @@ -281,6 +286,12 @@ func TypeStringToPhysicalID(tp string) int { return typeExchangeReceiver case TypeExchangeSender: return typeExchangeSender + case TypeCTE: + return typeCTE + case TypeCTEDefinition: + return typeCTEDefinition + case TypeCTETable: + return typeCTETable } // Should never reach here. return 0 @@ -385,6 +396,12 @@ func PhysicalIDToTypeString(id int) string { return TypeExchangeReceiver case typeExchangeSender: return TypeExchangeSender + case typeCTE: + return TypeCTE + case typeCTEDefinition: + return TypeCTEDefinition + case typeCTETable: + return TypeCTETable } // Should never reach here. From bbd5b32c2f539f4a3908f25af3e6e8965b6b58f7 Mon Sep 17 00:00:00 2001 From: Yifan Xu Date: Tue, 1 Jun 2021 10:43:37 +0800 Subject: [PATCH 221/343] *: clean up the logging component (#23534) --- config/config.go | 2 +- config/config.toml.example | 2 +- go.mod | 5 +- go.sum | 4 +- server/http_handler.go | 29 ++-- server/http_handler_test.go | 11 +- tidb-server/main.go | 10 -- util/logutil/log.go | 262 ++++-------------------------- util/logutil/log_test.go | 167 ++----------------- util/logutil/slow_query_logger.go | 76 +++++++++ 10 files changed, 141 insertions(+), 427 deletions(-) create mode 100644 util/logutil/slow_query_logger.go diff --git a/config/config.go b/config/config.go index 14467afaa0b4f..8390ba050b98f 100644 --- a/config/config.go +++ b/config/config.go @@ -297,7 +297,7 @@ func (b *nullableBool) UnmarshalJSON(data []byte) error { type Log struct { // Log level. Level string `toml:"level" json:"level"` - // Log format. one of json, text, or console. + // Log format, one of json or text. Format string `toml:"format" json:"format"` // Disable automatic timestamps in output. Deprecated: use EnableTimestamp instead. DisableTimestamp nullableBool `toml:"disable-timestamp" json:"disable-timestamp"` diff --git a/config/config.toml.example b/config/config.toml.example index 6db0b5c517277..1cc2674d01727 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -136,7 +136,7 @@ enable-enum-length-limit = true # Log level: debug, info, warn, error, fatal. level = "info" -# Log format, one of json, text, console. +# Log format, one of json or text. format = "text" # Enable automatic timestamps in log output, if not set, it will be defaulted to true. diff --git a/go.mod b/go.mod index d29f89bfade55..ac4ef88fd215d 100644 --- a/go.mod +++ b/go.mod @@ -54,7 +54,7 @@ require ( github.com/prometheus/common v0.9.1 github.com/rivo/uniseg v0.2.0 // indirect github.com/shirou/gopsutil v3.21.2+incompatible - github.com/sirupsen/logrus v1.6.0 + github.com/sirupsen/logrus v1.6.0 // indirect github.com/soheilhy/cmux v0.1.4 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 github.com/tikv/pd v1.1.0-beta.0.20210323121136-78679e5e209d @@ -77,9 +77,8 @@ require ( golang.org/x/tools v0.1.0 google.golang.org/grpc v1.27.1 gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect - gopkg.in/natefinch/lumberjack.v2 v2.0.0 gopkg.in/yaml.v2 v2.4.0 // indirect - honnef.co/go/tools v0.1.4 // indirect + honnef.co/go/tools v0.2.0 // indirect modernc.org/mathutil v1.2.2 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 diff --git a/go.sum b/go.sum index 991423501bec3..6336c1749649d 100644 --- a/go.sum +++ b/go.sum @@ -928,8 +928,8 @@ honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= -honnef.co/go/tools v0.1.4 h1:SadWOkti5uVN1FAMgxn165+Mw00fuQKyk4Gyn/inxNQ= -honnef.co/go/tools v0.1.4/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= +honnef.co/go/tools v0.2.0 h1:ws8AfbgTX3oIczLPNPCu5166oBg9ST2vNs0rcht+mDE= +honnef.co/go/tools v0.2.0/go.mod h1:lPVVZ2BS5TfnjLyizF7o7hv7j9/L+8cZY2hLyjP9cGY= k8s.io/klog v1.0.0/go.mod h1:4Bi6QPql/J/LkTDqv7R/cd3hPo4k2DG6Ptcz060Ez5I= modernc.org/mathutil v1.2.2 h1:+yFk8hBprV+4c0U9GjFtL+dV3N8hOJ8JCituQcMShFY= modernc.org/mathutil v1.2.2/go.mod h1:mZW8CKdRPY1v87qxC/wUdX5O1qDzXMP5TH3wjfpga6E= diff --git a/server/http_handler.go b/server/http_handler.go index 4aaab275430c4..0642924359210 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/log" "github.com/pingcap/parser/model" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" @@ -61,7 +62,6 @@ import ( "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/pdapi" - log "github.com/sirupsen/logrus" "go.uber.org/zap" ) @@ -625,13 +625,6 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { return } - l, err1 := log.ParseLevel(levelStr) - if err1 != nil { - writeError(w, err1) - return - } - log.SetLevel(l) - config.GetGlobalConfig().Log.Level = levelStr } if generalLog := req.Form.Get("tidb_general_log"); generalLog != "" { @@ -1090,7 +1083,7 @@ func (h ddlResignOwnerHandler) ServeHTTP(w http.ResponseWriter, req *http.Reques err := h.resignDDLOwner() if err != nil { - log.Error(err) + log.Error("failed to resign DDL owner", zap.Error(err)) writeError(w, err) return } @@ -1134,7 +1127,7 @@ func (h tableHandler) addScatterSchedule(startKey, endKey []byte, name string) e return err } if err := resp.Body.Close(); err != nil { - log.Error(err) + log.Error("failed to close response body", zap.Error(err)) } return nil } @@ -1154,7 +1147,7 @@ func (h tableHandler) deleteScatterSchedule(name string) error { return err } if err := resp.Body.Close(); err != nil { - log.Error(err) + log.Error("failed to close response body", zap.Error(err)) } return nil } @@ -1664,14 +1657,14 @@ func (h serverInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { do, err := session.GetDomain(h.Store) if err != nil { writeError(w, errors.New("create session error")) - log.Error(err) + log.Error("failed to get session domain", zap.Error(err)) return } info := serverInfo{} info.ServerInfo, err = infosync.GetServerInfo() if err != nil { writeError(w, err) - log.Error(err) + log.Error("failed to get server info", zap.Error(err)) return } info.IsOwner = do.DDL().OwnerManager().IsOwner() @@ -1694,14 +1687,14 @@ func (h allServerInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request do, err := session.GetDomain(h.Store) if err != nil { writeError(w, errors.New("create session error")) - log.Error(err) + log.Error("failed to get session domain", zap.Error(err)) return } ctx := context.Background() allServersInfo, err := infosync.GetAllServerInfo(ctx) if err != nil { writeError(w, errors.New("ddl server information not found")) - log.Error(err) + log.Error("failed to get all server info", zap.Error(err)) return } ctx, cancel := context.WithTimeout(ctx, 3*time.Second) @@ -1709,7 +1702,7 @@ func (h allServerInfoHandler) ServeHTTP(w http.ResponseWriter, req *http.Request cancel() if err != nil { writeError(w, errors.New("ddl server information not found")) - log.Error(err) + log.Error("failed to get owner id", zap.Error(err)) return } allVersionsMap := map[infosync.ServerVersionInfo]struct{}{} @@ -1905,13 +1898,13 @@ func (h ddlHookHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { dom, err := session.GetDomain(h.store) if err != nil { - log.Error(err) + log.Error("failed to get session domain", zap.Error(err)) writeError(w, err) } newCallbackFunc, err := ddl.GetCustomizedHook(req.FormValue("ddl_hook")) if err != nil { - log.Error(err) + log.Error("failed to get customized hook", zap.Error(err)) writeError(w, err) } callback := newCallbackFunc(dom) diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 208714ac082e1..5d2b654262a54 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -36,7 +36,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" - zaplog "github.com/pingcap/log" + "github.com/pingcap/log" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" @@ -57,7 +57,6 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/versioninfo" - log "github.com/sirupsen/logrus" "go.uber.org/zap" ) @@ -1155,8 +1154,7 @@ func (ts *HTTPHandlerTestSuite) TestPostSettings(c *C) { resp, err := ts.formStatus("/settings", form) c.Assert(err, IsNil) c.Assert(resp.StatusCode, Equals, http.StatusOK) - c.Assert(log.GetLevel(), Equals, log.ErrorLevel) - c.Assert(zaplog.GetLevel(), Equals, zap.ErrorLevel) + c.Assert(log.GetLevel(), Equals, zap.ErrorLevel) c.Assert(config.GetGlobalConfig().Log.Level, Equals, "error") c.Assert(variable.ProcessGeneralLog.Load(), IsTrue) val, err := variable.GetGlobalSystemVar(se.GetSessionVars(), variable.TiDBEnableAsyncCommit) @@ -1175,8 +1173,7 @@ func (ts *HTTPHandlerTestSuite) TestPostSettings(c *C) { c.Assert(err, IsNil) c.Assert(resp.StatusCode, Equals, http.StatusOK) c.Assert(variable.ProcessGeneralLog.Load(), IsFalse) - c.Assert(log.GetLevel(), Equals, log.FatalLevel) - c.Assert(zaplog.GetLevel(), Equals, zap.FatalLevel) + c.Assert(log.GetLevel(), Equals, zap.FatalLevel) c.Assert(config.GetGlobalConfig().Log.Level, Equals, "fatal") val, err = variable.GetGlobalSystemVar(se.GetSessionVars(), variable.TiDBEnableAsyncCommit) c.Assert(err, IsNil) @@ -1244,7 +1241,7 @@ func (ts *HTTPHandlerTestSuite) TestPprof(c *C) { } time.Sleep(time.Millisecond * 10) } - zaplog.Fatal("failed to get profile for %d retries in every 10 ms", zap.Int("retryTime", retryTime)) + log.Fatal("failed to get profile for %d retries in every 10 ms", zap.Int("retryTime", retryTime)) } func (ts *HTTPHandlerTestSuite) TestServerInfo(c *C) { diff --git a/tidb-server/main.go b/tidb-server/main.go index ed1da6abfb7fb..a4e81f51862af 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -17,7 +17,6 @@ import ( "context" "flag" "fmt" - "io" "os" "runtime" "strconv" @@ -69,7 +68,6 @@ import ( pd "github.com/tikv/pd/client" "go.uber.org/automaxprocs/maxprocs" "go.uber.org/zap" - "google.golang.org/grpc/grpclog" ) // Flag Names @@ -588,14 +586,6 @@ func setupLog() { err := logutil.InitZapLogger(cfg.Log.ToLogConfig()) terror.MustNil(err) - err = logutil.InitLogger(cfg.Log.ToLogConfig()) - terror.MustNil(err) - - if len(os.Getenv("GRPC_DEBUG")) > 0 { - grpclog.SetLoggerV2(grpclog.NewLoggerV2WithVerbosity(os.Stderr, os.Stderr, os.Stderr, 999)) - } else { - grpclog.SetLoggerV2(grpclog.NewLoggerV2(io.Discard, io.Discard, os.Stderr)) - } // trigger internal http(s) client init. util.InternalHTTPClient() } diff --git a/util/logutil/log.go b/util/logutil/log.go index 48088318b2d6b..b62484658495f 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -14,35 +14,26 @@ package logutil import ( - "bytes" "context" "fmt" "os" - "path/filepath" - "runtime" "runtime/trace" - "sort" - "strings" "time" + gzap "github.com/grpc-ecosystem/go-grpc-middleware/logging/zap" "github.com/opentracing/opentracing-go" tlog "github.com/opentracing/opentracing-go/log" "github.com/pingcap/errors" - zaplog "github.com/pingcap/log" - tikvlog "github.com/pingcap/tidb/store/tikv/logutil" - log "github.com/sirupsen/logrus" + "github.com/pingcap/log" "go.uber.org/zap" "go.uber.org/zap/zapcore" - "gopkg.in/natefinch/lumberjack.v2" ) const ( - defaultLogTimeFormat = "2006/01/02 15:04:05.000" // DefaultLogMaxSize is the default size of log files. DefaultLogMaxSize = 300 // MB // DefaultLogFormat is the default format of the log. DefaultLogFormat = "text" - defaultLogLevel = log.InfoLevel // DefaultSlowThreshold is the default slow log threshold in millisecond. DefaultSlowThreshold = 300 // DefaultQueryLogMaxLen is the default max length of the query in the log. @@ -51,6 +42,8 @@ const ( DefaultRecordPlanInSlowLog = 1 // DefaultTiDBEnableSlowLog enables TiDB to log slow queries. DefaultTiDBEnableSlowLog = true + // GRPCLogDebugVerbosity enables max verbosity when debugging grpc code. + GRPCLogDebugVerbosity = 99 ) // EmptyFileLogConfig is an empty FileLogConfig. @@ -58,12 +51,12 @@ var EmptyFileLogConfig = FileLogConfig{} // FileLogConfig serializes file log related config in toml/json. type FileLogConfig struct { - zaplog.FileLogConfig + log.FileLogConfig } // NewFileLogConfig creates a FileLogConfig. func NewFileLogConfig(maxSize uint) FileLogConfig { - return FileLogConfig{FileLogConfig: zaplog.FileLogConfig{ + return FileLogConfig{FileLogConfig: log.FileLogConfig{ MaxSize: int(maxSize), }, } @@ -71,16 +64,16 @@ func NewFileLogConfig(maxSize uint) FileLogConfig { // LogConfig serializes log related config in toml/json. type LogConfig struct { - zaplog.Config + log.Config // SlowQueryFile filename, default to File log config on empty. SlowQueryFile string } // NewLogConfig creates a LogConfig. -func NewLogConfig(level, format, slowQueryFile string, fileCfg FileLogConfig, disableTimestamp bool, opts ...func(*zaplog.Config)) *LogConfig { +func NewLogConfig(level, format, slowQueryFile string, fileCfg FileLogConfig, disableTimestamp bool, opts ...func(*log.Config)) *LogConfig { c := &LogConfig{ - Config: zaplog.Config{ + Config: log.Config{ Level: level, Format: format, DisableTimestamp: disableTimestamp, @@ -94,102 +87,6 @@ func NewLogConfig(level, format, slowQueryFile string, fileCfg FileLogConfig, di return c } -// isSKippedPackageName tests wether path name is on log library calling stack. -func isSkippedPackageName(name string) bool { - return strings.Contains(name, "github.com/sirupsen/logrus") || - strings.Contains(name, "github.com/coreos/pkg/capnslog") -} - -// modifyHook injects file name and line pos into log entry. -type contextHook struct{} - -// Fire implements logrus.Hook interface -// https://github.com/sirupsen/logrus/issues/63 -func (hook *contextHook) Fire(entry *log.Entry) error { - pc := make([]uintptr, 4) - cnt := runtime.Callers(8, pc) - - for i := 0; i < cnt; i++ { - fu := runtime.FuncForPC(pc[i] - 1) - name := fu.Name() - if !isSkippedPackageName(name) { - file, line := fu.FileLine(pc[i] - 1) - entry.Data["file"] = filepath.Base(file) - entry.Data["line"] = line - break - } - } - return nil -} - -// Levels implements logrus.Hook interface. -func (hook *contextHook) Levels() []log.Level { - return log.AllLevels -} - -func stringToLogLevel(level string) log.Level { - switch strings.ToLower(level) { - case "fatal": - return log.FatalLevel - case "error": - return log.ErrorLevel - case "warn", "warning": - return log.WarnLevel - case "debug": - return log.DebugLevel - case "info": - return log.InfoLevel - } - return defaultLogLevel -} - -// textFormatter is for compatibility with ngaut/log -type textFormatter struct { - DisableTimestamp bool - EnableEntryOrder bool -} - -// Format implements logrus.Formatter -func (f *textFormatter) Format(entry *log.Entry) ([]byte, error) { - var b *bytes.Buffer - if entry.Buffer != nil { - b = entry.Buffer - } else { - b = &bytes.Buffer{} - } - - if !f.DisableTimestamp { - fmt.Fprintf(b, "%s ", entry.Time.Format(defaultLogTimeFormat)) - } - if file, ok := entry.Data["file"]; ok { - fmt.Fprintf(b, "%s:%v:", file, entry.Data["line"]) - } - fmt.Fprintf(b, " [%s] %s", entry.Level.String(), entry.Message) - - if f.EnableEntryOrder { - keys := make([]string, 0, len(entry.Data)) - for k := range entry.Data { - if k != "file" && k != "line" { - keys = append(keys, k) - } - } - sort.Strings(keys) - for _, k := range keys { - fmt.Fprintf(b, " %v=%v", k, entry.Data[k]) - } - } else { - for k, v := range entry.Data { - if k != "file" && k != "line" { - fmt.Fprintf(b, " %v=%v", k, v) - } - } - } - - b.WriteByte('\n') - - return b.Bytes(), nil -} - const ( // SlowLogTimeFormat is the time format for slow log. SlowLogTimeFormat = time.RFC3339Nano @@ -197,125 +94,34 @@ const ( OldSlowLogTimeFormat = "2006-01-02-15:04:05.999999999 -0700" ) -type slowLogFormatter struct{} - -func (f *slowLogFormatter) Format(entry *log.Entry) ([]byte, error) { - var b *bytes.Buffer - if entry.Buffer != nil { - b = entry.Buffer - } else { - b = &bytes.Buffer{} - } +// SlowQueryLogger is used to log slow query, InitZapLogger will modify it according to config file. +var SlowQueryLogger = log.L() - fmt.Fprintf(b, "# Time: %s\n", entry.Time.Format(SlowLogTimeFormat)) - fmt.Fprintf(b, "%s\n", entry.Message) - return b.Bytes(), nil -} - -func stringToLogFormatter(format string, disableTimestamp bool) log.Formatter { - switch strings.ToLower(format) { - case "text": - return &textFormatter{ - DisableTimestamp: disableTimestamp, - } - default: - return &textFormatter{} - } -} - -// initFileLog initializes file based logging options. -func initFileLog(cfg *zaplog.FileLogConfig, logger *log.Logger) error { - if st, err := os.Stat(cfg.Filename); err == nil { - if st.IsDir() { - return errors.New("can't use directory as log file name") - } - } - if cfg.MaxSize == 0 { - cfg.MaxSize = DefaultLogMaxSize - } - - // use lumberjack to logrotate - output := &lumberjack.Logger{ - Filename: cfg.Filename, - MaxSize: cfg.MaxSize, - MaxBackups: cfg.MaxBackups, - MaxAge: cfg.MaxDays, - LocalTime: true, - } - - if logger == nil { - log.SetOutput(output) - } else { - logger.Out = output - } - return nil -} - -// SlowQueryLogger is used to log slow query, InitLogger will modify it according to config file. -var SlowQueryLogger = log.StandardLogger() - -// SlowQueryZapLogger is used to log slow query, InitZapLogger will modify it according to config file. -var SlowQueryZapLogger = zaplog.L() - -// InitLogger initializes PD's logger. +// InitLogger delegates to InitZapLogger. Keeping it here for historical reason. func InitLogger(cfg *LogConfig) error { - log.SetLevel(stringToLogLevel(cfg.Level)) - log.AddHook(&contextHook{}) - - if cfg.Format == "" { - cfg.Format = DefaultLogFormat - } - formatter := stringToLogFormatter(cfg.Format, cfg.DisableTimestamp) - log.SetFormatter(formatter) - - if len(cfg.File.Filename) != 0 { - if err := initFileLog(&cfg.File, nil); err != nil { - return errors.Trace(err) - } - } - - if len(cfg.SlowQueryFile) != 0 { - SlowQueryLogger = log.New() - tmp := cfg.File - tmp.Filename = cfg.SlowQueryFile - if err := initFileLog(&tmp, SlowQueryLogger); err != nil { - return errors.Trace(err) - } - SlowQueryLogger.Formatter = &slowLogFormatter{} - } - - // Setup log key for tikv client. - tikvlog.CtxLogKey = ctxLogKey - - return nil + return InitZapLogger(cfg) } // InitZapLogger initializes a zap logger with cfg. func InitZapLogger(cfg *LogConfig) error { - gl, props, err := zaplog.InitLogger(&cfg.Config, zap.AddStacktrace(zapcore.FatalLevel)) + gl, props, err := log.InitLogger(&cfg.Config, zap.AddStacktrace(zapcore.FatalLevel)) if err != nil { return errors.Trace(err) } - zaplog.ReplaceGlobals(gl, props) - - if len(cfg.SlowQueryFile) != 0 { - sqfCfg := zaplog.FileLogConfig{ - MaxSize: cfg.File.MaxSize, - Filename: cfg.SlowQueryFile, - } - sqCfg := &zaplog.Config{ - Level: cfg.Level, - Format: cfg.Format, - DisableTimestamp: cfg.DisableTimestamp, - File: sqfCfg, - } - sqLogger, _, err := zaplog.InitLogger(sqCfg) - if err != nil { - return errors.Trace(err) - } - SlowQueryZapLogger = sqLogger + log.ReplaceGlobals(gl, props) + + // init dedicated logger for slow query log + SlowQueryLogger, err = newSlowQueryLogger(cfg) + if err != nil { + return errors.Trace(err) + } + + // init logger for grpc debugging + if len(os.Getenv("GRPC_DEBUG")) > 0 { + // more information for verbosity: https://github.com/google/glog#verbose-logging + gzap.ReplaceGrpcLoggerV2WithVerbosity(gl, GRPCLogDebugVerbosity) } else { - SlowQueryZapLogger = gl + gzap.ReplaceGrpcLoggerV2(gl) } return nil @@ -327,7 +133,7 @@ func SetLevel(level string) error { if err := l.UnmarshalText([]byte(level)); err != nil { return errors.Trace(err) } - zaplog.SetLevel(l.Level()) + log.SetLevel(l.Level()) return nil } @@ -341,12 +147,12 @@ func Logger(ctx context.Context) *zap.Logger { if ctxlogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { return ctxlogger } - return zaplog.L() + return log.L() } // BgLogger is alias of `logutil.BgLogger()` func BgLogger() *zap.Logger { - return zaplog.L() + return log.L() } // WithConnID attaches connId to context. @@ -355,7 +161,7 @@ func WithConnID(ctx context.Context, connID uint64) context.Context { if ctxLogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { - logger = zaplog.L() + logger = log.L() } return context.WithValue(ctx, ctxLogKey, logger.With(zap.Uint64("conn", connID))) } @@ -366,7 +172,7 @@ func WithTraceLogger(ctx context.Context, connID uint64) context.Context { if ctxLogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { - logger = zaplog.L() + logger = log.L() } return context.WithValue(ctx, ctxLogKey, wrapTraceLogger(ctx, connID, logger)) } @@ -374,7 +180,7 @@ func WithTraceLogger(ctx context.Context, connID uint64) context.Context { func wrapTraceLogger(ctx context.Context, connID uint64, logger *zap.Logger) *zap.Logger { return logger.WithOptions(zap.WrapCore(func(core zapcore.Core) zapcore.Core { tl := &traceLog{ctx: ctx} - traceCore := zaplog.NewTextCore(zaplog.NewTextEncoder(&zaplog.Config{}), tl, tl). + traceCore := log.NewTextCore(log.NewTextEncoder(&log.Config{}), tl, tl). With([]zapcore.Field{zap.Uint64("conn", connID)}) return zapcore.NewTee(traceCore, core) })) @@ -403,7 +209,7 @@ func WithKeyValue(ctx context.Context, key, value string) context.Context { if ctxLogger, ok := ctx.Value(ctxLogKey).(*zap.Logger); ok { logger = ctxLogger } else { - logger = zaplog.L() + logger = log.L() } return context.WithValue(ctx, ctxLogKey, logger.With(zap.String(key, value))) } diff --git a/util/logutil/log_test.go b/util/logutil/log_test.go index 4961aa28e98f1..ecc9fb9d03679 100644 --- a/util/logutil/log_test.go +++ b/util/logutil/log_test.go @@ -17,7 +17,6 @@ import ( "bufio" "bytes" "context" - "fmt" "io" "os" "runtime" @@ -25,19 +24,15 @@ import ( "testing" . "github.com/pingcap/check" - zaplog "github.com/pingcap/log" - log "github.com/sirupsen/logrus" + "github.com/pingcap/log" "go.uber.org/zap" ) const ( - logPattern = `\d\d\d\d/\d\d/\d\d \d\d:\d\d:\d\d\.\d\d\d ([\w_%!$@.,+~-]+|\\.)+:\d+: \[(fatal|error|warning|info|debug)\] .*?\n` // zapLogPatern is used to match the zap log format, such as the following log: - // [2019/02/13 15:56:05.385 +08:00] [INFO] [log_test.go:167] ["info message"] ["str key"=val] ["int key"=123] - zapLogPattern = `\[\d\d\d\d/\d\d/\d\d \d\d:\d\d:\d\d.\d\d\d\ (\+|-)\d\d:\d\d\] \[(FATAL|ERROR|WARN|INFO|DEBUG)\] \[([\w_%!$@.,+~-]+|\\.)+:\d+\] \[.*\] (\[.*=.*\]).*\n` - // [2019/02/13 15:56:05.385 +08:00] [INFO] [log_test.go:167] ["info message"] ["str key"=val] ["int key"=123] + // [2019/02/13 15:56:05.385 +08:00] [INFO] [log_test.go:167] ["info message"] [conn=conn1] ["str key"=val] ["int key"=123] zapLogWithConnIDPattern = `\[\d\d\d\d/\d\d/\d\d \d\d:\d\d:\d\d.\d\d\d\ (\+|-)\d\d:\d\d\] \[(FATAL|ERROR|WARN|INFO|DEBUG)\] \[([\w_%!$@.,+~-]+|\\.)+:\d+\] \[.*\] \[conn=.*\] (\[.*=.*\]).*\n` - // [2019/02/13 15:56:05.385 +08:00] [INFO] [log_test.go:167] ["info message"] ["str key"=val] ["int key"=123] + // [2019/02/13 15:56:05.385 +08:00] [INFO] [log_test.go:167] ["info message"] [ctxKey=ctxKey1] ["str key"=val] ["int key"=123] zapLogWithKeyValPattern = `\[\d\d\d\d/\d\d/\d\d \d\d:\d\d:\d\d.\d\d\d\ (\+|-)\d\d:\d\d\] \[(FATAL|ERROR|WARN|INFO|DEBUG)\] \[([\w_%!$@.,+~-]+|\\.)+:\d+\] \[.*\] \[ctxKey=.*\] (\[.*=.*\]).*\n` ) @@ -53,156 +48,14 @@ type testLogSuite struct { buf *bytes.Buffer } -func (s *testLogSuite) SetUpSuite(c *C) { +func (s *testLogSuite) SetUpSuite(_ *C) { s.buf = &bytes.Buffer{} } -func (s *testLogSuite) SetUpTest(c *C) { +func (s *testLogSuite) SetUpTest(_ *C) { s.buf = &bytes.Buffer{} } -func (s *testLogSuite) TestStringToLogLevel(c *C) { - c.Assert(stringToLogLevel("fatal"), Equals, log.FatalLevel) - c.Assert(stringToLogLevel("ERROR"), Equals, log.ErrorLevel) - c.Assert(stringToLogLevel("warn"), Equals, log.WarnLevel) - c.Assert(stringToLogLevel("warning"), Equals, log.WarnLevel) - c.Assert(stringToLogLevel("debug"), Equals, log.DebugLevel) - c.Assert(stringToLogLevel("info"), Equals, log.InfoLevel) - c.Assert(stringToLogLevel("whatever"), Equals, log.InfoLevel) -} - -// TestLogging assure log format and log redirection works. -func (s *testLogSuite) TestLogging(c *C) { - conf := NewLogConfig("warn", DefaultLogFormat, "", NewFileLogConfig(0), false) - conf.File.Filename = "log_file" - c.Assert(InitLogger(conf), IsNil) - - log.SetOutput(s.buf) - - log.Infof("[this message should not be sent to buf]") - c.Assert(s.buf.Len(), Equals, 0) - - log.Warningf("[this message should be sent to buf]") - entry, err := s.buf.ReadString('\n') - c.Assert(err, IsNil) - c.Assert(entry, Matches, logPattern) - - log.Warnf("this message comes from logrus") - entry, err = s.buf.ReadString('\n') - c.Assert(err, IsNil) - c.Assert(entry, Matches, logPattern) - c.Assert(strings.Contains(entry, "log_test.go"), IsTrue) -} - -func (s *testLogSuite) TestSlowQueryLogger(c *C) { - fileName := "slow_query" - os.Remove(fileName) - conf := NewLogConfig("info", DefaultLogFormat, fileName, NewFileLogConfig(DefaultLogMaxSize), false) - c.Assert(conf.File.MaxSize, Equals, DefaultLogMaxSize) - err := InitLogger(conf) - c.Assert(err, IsNil) - defer os.Remove(fileName) - - SlowQueryLogger.Debug("debug message") - SlowQueryLogger.Info("info message") - SlowQueryLogger.Warn("warn message") - SlowQueryLogger.Error("error message") - c.Assert(s.buf.Len(), Equals, 0) - - f, err := os.Open(fileName) - c.Assert(err, IsNil) - defer f.Close() - - r := bufio.NewReader(f) - for { - var str string - str, err = r.ReadString('\n') - if err != nil { - break - } - if strings.HasPrefix(str, "# ") { - c.Assert(str, Matches, `# Time: .*?\n`) - } else { - c.Assert(str, Matches, `.*? message\n`) - } - } - c.Assert(err, Equals, io.EOF) -} - -func (s *testLogSuite) TestLoggerKeepOrder(c *C) { - conf := NewLogConfig("warn", DefaultLogFormat, "", EmptyFileLogConfig, true) - c.Assert(InitLogger(conf), IsNil) - logger := log.StandardLogger() - ft, ok := logger.Formatter.(*textFormatter) - c.Assert(ok, IsTrue) - ft.EnableEntryOrder = true - logger.Out = s.buf - logEntry := log.NewEntry(logger) - logEntry.Data = log.Fields{ - "connectionId": 1, - "costTime": "1", - "database": "test", - "sql": "select 1", - "txnStartTS": 1, - } - - _, _, line, _ := runtime.Caller(0) - logEntry.WithField("type", "slow-query").WithField("succ", true).Warnf("slow-query") - expectMsg := fmt.Sprintf("log_test.go:%v: [warning] slow-query connectionId=1 costTime=1 database=test sql=select 1 succ=true txnStartTS=1 type=slow-query\n", line+1) - c.Assert(s.buf.String(), Equals, expectMsg) - - s.buf.Reset() - logEntry.Data = log.Fields{ - "a": "a", - "d": "d", - "e": "e", - "b": "b", - "f": "f", - "c": "c", - } - - _, _, line, _ = runtime.Caller(0) - logEntry.Warnf("slow-query") - expectMsg = fmt.Sprintf("log_test.go:%v: [warning] slow-query a=a b=b c=c d=d e=e f=f\n", line+1) - c.Assert(s.buf.String(), Equals, expectMsg) -} - -func (s *testLogSuite) TestSlowQueryZapLogger(c *C) { - if runtime.GOOS == "windows" { - // Skip this test on windows for two reasons: - // 1. The pattern match fails somehow. It seems windows treat \n as slash and character n. - // 2. Remove file doesn't work as long as the log instance hold the file. - c.Skip("skip on windows") - } - - fileName := "slow_query" - conf := NewLogConfig("info", DefaultLogFormat, fileName, EmptyFileLogConfig, false) - err := InitZapLogger(conf) - c.Assert(err, IsNil) - defer os.Remove(fileName) - - SlowQueryZapLogger.Debug("debug message", zap.String("str key", "val")) - SlowQueryZapLogger.Info("info message", zap.String("str key", "val")) - SlowQueryZapLogger.Warn("warn", zap.Int("int key", 123)) - SlowQueryZapLogger.Error("error message", zap.Bool("bool key", true)) - - f, err := os.Open(fileName) - c.Assert(err, IsNil) - defer f.Close() - - r := bufio.NewReader(f) - for { - var str string - str, err = r.ReadString('\n') - if err != nil { - break - } - c.Assert(str, Matches, zapLogPattern) - } - c.Assert(err, Equals, io.EOF) - -} - func (s *testLogSuite) TestZapLoggerWithKeys(c *C) { if runtime.GOOS == "windows" { // Skip this test on windows for two reason: @@ -211,7 +64,7 @@ func (s *testLogSuite) TestZapLoggerWithKeys(c *C) { c.Skip("skip on windows") } - fileCfg := FileLogConfig{zaplog.FileLogConfig{Filename: "zap_log", MaxSize: 4096}} + fileCfg := FileLogConfig{log.FileLogConfig{Filename: "zap_log", MaxSize: 4096}} conf := NewLogConfig("info", DefaultLogFormat, "", fileCfg, false) err := InitZapLogger(conf) c.Assert(err, IsNil) @@ -258,14 +111,14 @@ func (s *testLogSuite) TestSetLevel(c *C) { err := InitZapLogger(conf) c.Assert(err, IsNil) - c.Assert(zaplog.GetLevel(), Equals, zap.InfoLevel) + c.Assert(log.GetLevel(), Equals, zap.InfoLevel) err = SetLevel("warn") c.Assert(err, IsNil) - c.Assert(zaplog.GetLevel(), Equals, zap.WarnLevel) + c.Assert(log.GetLevel(), Equals, zap.WarnLevel) err = SetLevel("Error") c.Assert(err, IsNil) - c.Assert(zaplog.GetLevel(), Equals, zap.ErrorLevel) + c.Assert(log.GetLevel(), Equals, zap.ErrorLevel) err = SetLevel("DEBUG") c.Assert(err, IsNil) - c.Assert(zaplog.GetLevel(), Equals, zap.DebugLevel) + c.Assert(log.GetLevel(), Equals, zap.DebugLevel) } diff --git a/util/logutil/slow_query_logger.go b/util/logutil/slow_query_logger.go new file mode 100644 index 0000000000000..3910b6ecd1192 --- /dev/null +++ b/util/logutil/slow_query_logger.go @@ -0,0 +1,76 @@ +package logutil + +import ( + "fmt" + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "go.uber.org/zap" + "go.uber.org/zap/buffer" + "go.uber.org/zap/zapcore" +) + +var _pool = buffer.NewPool() + +func newSlowQueryLogger(cfg *LogConfig) (*zap.Logger, error) { + + // reuse global config and override slow query log file + // if slow query log filename is empty, slow query log will behave the same as global log + sqConfig := &cfg.Config + if len(cfg.SlowQueryFile) != 0 { + sqConfig.File = log.FileLogConfig{ + MaxSize: cfg.File.MaxSize, + Filename: cfg.SlowQueryFile, + } + } + + // create the slow query logger + sqLogger, prop, err := log.InitLogger(sqConfig) + if err != nil { + return nil, errors.Trace(err) + } + + // replace 2018-12-19-unified-log-format text encoder with slow log encoder + sqLogger = sqLogger.WithOptions(zap.WrapCore(func(core zapcore.Core) zapcore.Core { + return log.NewTextCore(&slowLogEncoder{}, prop.Syncer, prop.Level) + })) + + return sqLogger, nil +} + +type slowLogEncoder struct{} + +func (e *slowLogEncoder) EncodeEntry(entry zapcore.Entry, _ []zapcore.Field) (*buffer.Buffer, error) { + b := _pool.Get() + fmt.Fprintf(b, "# Time: %s\n", entry.Time.Format(SlowLogTimeFormat)) + fmt.Fprintf(b, "%s\n", entry.Message) + return b, nil +} + +func (e *slowLogEncoder) Clone() zapcore.Encoder { return e } +func (e *slowLogEncoder) AddArray(string, zapcore.ArrayMarshaler) error { return nil } +func (e *slowLogEncoder) AddObject(string, zapcore.ObjectMarshaler) error { return nil } +func (e *slowLogEncoder) AddBinary(string, []byte) {} +func (e *slowLogEncoder) AddByteString(string, []byte) {} +func (e *slowLogEncoder) AddBool(string, bool) {} +func (e *slowLogEncoder) AddComplex128(string, complex128) {} +func (e *slowLogEncoder) AddComplex64(string, complex64) {} +func (e *slowLogEncoder) AddDuration(string, time.Duration) {} +func (e *slowLogEncoder) AddFloat64(string, float64) {} +func (e *slowLogEncoder) AddFloat32(string, float32) {} +func (e *slowLogEncoder) AddInt(string, int) {} +func (e *slowLogEncoder) AddInt64(string, int64) {} +func (e *slowLogEncoder) AddInt32(string, int32) {} +func (e *slowLogEncoder) AddInt16(string, int16) {} +func (e *slowLogEncoder) AddInt8(string, int8) {} +func (e *slowLogEncoder) AddString(string, string) {} +func (e *slowLogEncoder) AddTime(string, time.Time) {} +func (e *slowLogEncoder) AddUint(string, uint) {} +func (e *slowLogEncoder) AddUint64(string, uint64) {} +func (e *slowLogEncoder) AddUint32(string, uint32) {} +func (e *slowLogEncoder) AddUint16(string, uint16) {} +func (e *slowLogEncoder) AddUint8(string, uint8) {} +func (e *slowLogEncoder) AddUintptr(string, uintptr) {} +func (e *slowLogEncoder) AddReflected(string, interface{}) error { return nil } +func (e *slowLogEncoder) OpenNamespace(string) {} From 48e25cdebfaa35c71fced8f61b23b6936369022d Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 1 Jun 2021 11:05:37 +0800 Subject: [PATCH 222/343] executor: make the ParallelApply be safe to be called again after returning empty results (#24935) --- executor/parallel_apply.go | 7 +++++++ executor/parallel_apply_test.go | 11 +++++++++++ 2 files changed, 18 insertions(+) diff --git a/executor/parallel_apply.go b/executor/parallel_apply.go index d02ebac9e5349..636ec96ad2868 100644 --- a/executor/parallel_apply.go +++ b/executor/parallel_apply.go @@ -68,6 +68,7 @@ type ParallelNestedLoopApplyExec struct { // fields about concurrency control concurrency int started uint32 + drained uint32 // drained == true indicates there is no more data freeChkCh chan *chunk.Chunk resultChkCh chan result outerRowCh chan outerRow @@ -130,6 +131,11 @@ func (e *ParallelNestedLoopApplyExec) Open(ctx context.Context) error { // Next implements the Executor interface. func (e *ParallelNestedLoopApplyExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { + if atomic.LoadUint32(&e.drained) == 1 { + req.Reset() + return nil + } + if atomic.CompareAndSwapUint32(&e.started, 0, 1) { e.workerWg.Add(1) go e.outerWorker(ctx) @@ -147,6 +153,7 @@ func (e *ParallelNestedLoopApplyExec) Next(ctx context.Context, req *chunk.Chunk } if result.chk == nil { // no more data req.Reset() + atomic.StoreUint32(&e.drained, 1) return nil } req.SwapColumns(result.chk) diff --git a/executor/parallel_apply_test.go b/executor/parallel_apply_test.go index c0ecb19783273..a72944ebda507 100644 --- a/executor/parallel_apply_test.go +++ b/executor/parallel_apply_test.go @@ -596,3 +596,14 @@ func (s *testSuite) TestApplyGoroutinePanic(c *C) { c.Assert(failpoint.Disable(panicPath), IsNil) } } + +func (s *testSuite) TestIssue24930(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("set tidb_enable_parallel_apply=true") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2(a int)") + tk.MustQuery(`select case when t1.a is null + then (select t2.a from t2 where t2.a = t1.a limit 1) else t1.a end a + from t1 where t1.a=1 order by a limit 1`).Check(testkit.Rows()) // can return an empty result instead of hanging forever +} From 02b4f3bcc8fba02242b811f43d6346a430d5afef Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 1 Jun 2021 11:23:36 +0800 Subject: [PATCH 223/343] executor: add more cases about dynamic-mode with plan-cache and transaction (#24998) --- executor/partition_table_test.go | 20 +++- planner/core/common_plans.go | 1 + planner/core/prepare_test.go | 171 +++++++++++++++++++++++++++++++ 3 files changed, 191 insertions(+), 1 deletion(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index b04472d22da05..d8ae8abcd9476 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -694,7 +694,7 @@ func (s *partitionTableSuite) TestBatchGetforRangeandListPartitionTable(c *C) { tk.MustExec("set @@session.tidb_enable_list_partition = ON") // list partition table - tk.MustExec(`create table tlist(a int, b int, unique index idx_a(a), index idx_b(b)) partition by list(a)( + tk.MustExec(`create table tlist(a int, b int, unique index idx_a(a), index idx_b(b)) partition by list(a)( partition p0 values in (1, 2, 3, 4), partition p1 values in (5, 6, 7, 8), partition p2 values in (9, 10, 11, 12));`) @@ -1662,6 +1662,24 @@ func (s *partitionTableSuite) TestAddDropPartitions(c *C) { tk.MustPartition(`select * from t where a < 20`, "p1,p2,p3").Sort().Check(testkit.Rows("12", "15", "7")) } +func (s *partitionTableSuite) PartitionPruningInTransaction(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_pruning_transaction") + defer tk.MustExec(`drop database test_pruning_transaction`) + tk.MustExec("use test_pruning_transaction") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec(`create table t(a int, b int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11))`) + tk.MustExec(`begin`) + tk.MustPartition(`select * from t`, "all") + tk.MustPartition(`select * from t where a > 4`, "p1,p2") // partition pruning can work in transactions + tk.MustPartition(`select * from t where a > 7`, "p2") + tk.MustExec(`rollback`) +} + func (s *partitionTableSuite) TestDML(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index d13a2d3313288..46aa621f427b4 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -545,6 +545,7 @@ func (e *Execute) rebuildRange(p Plan) error { // The code should never run here as long as we're not using point get for partition table. // And if we change the logic one day, here work as defensive programming to cache the error. if x.PartitionInfo != nil { + // TODO: relocate the partition after rebuilding range to make PlanCache support PointGet return errors.New("point get for partition table can not use plan cache") } if x.HandleParam != nil { diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 7bb87c8736b74..1a8d8a496d38b 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -1335,3 +1335,174 @@ func (s *testPlanSerialSuite) TestPartitionTable(c *C) { } } } + +func (s *testPlanSerialSuite) TestPartitionWithVariedDatasources(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + // enable plan cache + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + orgEnable := core.PreparedPlanCacheEnabled() + defer func() { + dom.Close() + err = store.Close() + c.Assert(err, IsNil) + core.SetPreparedPlanCache(orgEnable) + }() + core.SetPreparedPlanCache(true) + tk.Se, err = session.CreateSession4TestWithOpt(store, &session.Opt{ + PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64), + }) + c.Assert(err, IsNil) + + // enable partition table dynamic mode + tk.MustExec("create database test_plan_cache2") + tk.MustExec("use test_plan_cache2") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + // prepare tables + tk.MustExec(`create table trangePK (a int primary key, b int) partition by range (a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + tk.MustExec(`create table thashPK (a int primary key, b int) partition by hash (a) partitions 4`) + tk.MustExec(`create table tnormalPK (a int primary key, b int)`) + tk.MustExec(`create table trangeIdx (a int unique key, b int) partition by range (a) ( + partition p0 values less than (10000), + partition p1 values less than (20000), + partition p2 values less than (30000), + partition p3 values less than (40000))`) + tk.MustExec(`create table thashIdx (a int unique key, b int) partition by hash (a) partitions 4`) + tk.MustExec(`create table tnormalIdx (a int unique key, b int)`) + uniqueVals := make(map[int]struct{}) + vals := make([]string, 0, 1000) + for len(vals) < 1000 { + a := rand.Intn(40000) + if _, ok := uniqueVals[a]; ok { + continue + } + uniqueVals[a] = struct{}{} + b := rand.Intn(40000) + vals = append(vals, fmt.Sprintf("(%v, %v)", a, b)) + } + for _, tbl := range []string{"trangePK", "thashPK", "tnormalPK", "trangeIdx", "thashIdx", "tnormalIdx"} { + tk.MustExec(fmt.Sprintf(`insert into %v values %v`, tbl, strings.Join(vals, ", "))) + } + + // TableReader, PointGet on PK, BatchGet on PK + for _, tbl := range []string{`trangePK`, `thashPK`, `tnormalPK`} { + tk.MustExec(fmt.Sprintf(`prepare stmt%v_tablescan from 'select * from %v use index(primary) where a > ? and a < ?'`, tbl, tbl)) + tk.MustExec(fmt.Sprintf(`prepare stmt%v_pointget from 'select * from %v use index(primary) where a = ?'`, tbl, tbl)) + tk.MustExec(fmt.Sprintf(`prepare stmt%v_batchget from 'select * from %v use index(primary) where a in (?, ?, ?)'`, tbl, tbl)) + } + for i := 0; i < 100; i++ { + mina, maxa := rand.Intn(40000), rand.Intn(40000) + if mina > maxa { + mina, maxa = maxa, mina + } + tk.MustExec(fmt.Sprintf(`set @mina=%v, @maxa=%v`, mina, maxa)) + tk.MustExec(fmt.Sprintf(`set @pointa=%v`, rand.Intn(40000))) + tk.MustExec(fmt.Sprintf(`set @a0=%v, @a1=%v, @a2=%v`, rand.Intn(40000), rand.Intn(40000), rand.Intn(40000))) + + var rscan, rpoint, rbatch [][]interface{} + for id, tbl := range []string{`trangePK`, `thashPK`, `tnormalPK`} { + scan := tk.MustQuery(fmt.Sprintf(`execute stmt%v_tablescan using @mina, @maxa`, tbl)).Sort() + if i > 0 { + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + } + if id == 0 { + rscan = scan.Rows() + } else { + scan.Check(rscan) + } + + point := tk.MustQuery(fmt.Sprintf(`execute stmt%v_pointget using @pointa`, tbl)).Sort() + if tbl == `tnormalPK` && i > 0 { + // PlanCache cannot support PointGet now since we haven't relocated partition after rebuilding range. + // Please see Execute.rebuildRange for more details. + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + } + if id == 0 { + rpoint = point.Rows() + } else { + point.Check(rpoint) + } + + batch := tk.MustQuery(fmt.Sprintf(`execute stmt%v_batchget using @a0, @a1, @a2`, tbl)).Sort() + if i > 0 { + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + } + if id == 0 { + rbatch = batch.Rows() + } else { + batch.Check(rbatch) + } + } + } + + // IndexReader, IndexLookUp, PointGet on Idx, BatchGet on Idx + for _, tbl := range []string{"trangeIdx", "thashIdx", "tnormalIdx"} { + tk.MustExec(fmt.Sprintf(`prepare stmt%v_indexscan from 'select a from %v use index(a) where a > ? and a < ?'`, tbl, tbl)) + tk.MustExec(fmt.Sprintf(`prepare stmt%v_indexlookup from 'select * from %v use index(a) where a > ? and a < ?'`, tbl, tbl)) + tk.MustExec(fmt.Sprintf(`prepare stmt%v_pointget_idx from 'select * from %v use index(a) where a = ?'`, tbl, tbl)) + tk.MustExec(fmt.Sprintf(`prepare stmt%v_batchget_idx from 'select * from %v use index(a) where a in (?, ?, ?)'`, tbl, tbl)) + } + for i := 0; i < 100; i++ { + mina, maxa := rand.Intn(40000), rand.Intn(40000) + if mina > maxa { + mina, maxa = maxa, mina + } + tk.MustExec(fmt.Sprintf(`set @mina=%v, @maxa=%v`, mina, maxa)) + tk.MustExec(fmt.Sprintf(`set @pointa=%v`, rand.Intn(40000))) + tk.MustExec(fmt.Sprintf(`set @a0=%v, @a1=%v, @a2=%v`, rand.Intn(40000), rand.Intn(40000), rand.Intn(40000))) + + var rscan, rlookup, rpoint, rbatch [][]interface{} + for id, tbl := range []string{"trangeIdx", "thashIdx", "tnormalIdx"} { + scan := tk.MustQuery(fmt.Sprintf(`execute stmt%v_indexscan using @mina, @maxa`, tbl)).Sort() + if i > 0 { + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + } + if id == 0 { + rscan = scan.Rows() + } else { + scan.Check(rscan) + } + + lookup := tk.MustQuery(fmt.Sprintf(`execute stmt%v_indexlookup using @mina, @maxa`, tbl)).Sort() + if i > 0 { + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + } + if id == 0 { + rlookup = lookup.Rows() + } else { + lookup.Check(rlookup) + } + + point := tk.MustQuery(fmt.Sprintf(`execute stmt%v_pointget_idx using @pointa`, tbl)).Sort() + if tbl == `tnormalPK` && i > 0 { + // PlanCache cannot support PointGet now since we haven't relocated partition after rebuilding range. + // Please see Execute.rebuildRange for more details. + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) + } + if id == 0 { + rpoint = point.Rows() + } else { + point.Check(rpoint) + } + + batch := tk.MustQuery(fmt.Sprintf(`execute stmt%v_batchget_idx using @a0, @a1, @a2`, tbl)).Sort() + if i > 0 { + tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) + } + if id == 0 { + rbatch = batch.Rows() + } else { + batch.Check(rbatch) + } + } + } +} From 8dc21198c9bd5f9e07e1d0e6ecd99244ece03b10 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 31 May 2021 21:49:37 -0600 Subject: [PATCH 224/343] *: fix sysvar inconsistencies/potential bugs (#24964) --- session/session.go | 2 +- sessionctx/variable/session_test.go | 14 ++--- sessionctx/variable/sysvar.go | 88 ++++++++++++++--------------- sessionctx/variable/tidb_vars.go | 18 +++--- tidb-server/main.go | 2 +- tidb-server/main_test.go | 4 +- 6 files changed, 64 insertions(+), 64 deletions(-) diff --git a/session/session.go b/session/session.go index dc64f5fa1dc9b..6e177f5cca368 100644 --- a/session/session.go +++ b/session/session.go @@ -2349,7 +2349,7 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { newCfg := *(config.GetGlobalConfig()) newCfg.MemQuotaQuery = newMemoryQuotaQuery config.StoreGlobalConfig(&newCfg) - variable.SetSysVar(variable.TIDBMemQuotaQuery, strconv.FormatInt(newCfg.MemQuotaQuery, 10)) + variable.SetSysVar(variable.TiDBMemQuotaQuery, strconv.FormatInt(newCfg.MemQuotaQuery, 10)) } newOOMAction, err := loadDefOOMAction(se) if err != nil { diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 00b728557d188..0ded1a362e9f5 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -46,13 +46,13 @@ func (*testSessionSuite) TestSetSystemVariable(c *C) { {variable.TimeZone, "xyz", true}, {variable.TiDBOptAggPushDown, "1", false}, {variable.TiDBOptDistinctAggPushDown, "1", false}, - {variable.TIDBMemQuotaQuery, "1024", false}, - {variable.TIDBMemQuotaHashJoin, "1024", false}, - {variable.TIDBMemQuotaMergeJoin, "1024", false}, - {variable.TIDBMemQuotaSort, "1024", false}, - {variable.TIDBMemQuotaTopn, "1024", false}, - {variable.TIDBMemQuotaIndexLookupReader, "1024", false}, - {variable.TIDBMemQuotaIndexLookupJoin, "1024", false}, + {variable.TiDBMemQuotaQuery, "1024", false}, + {variable.TiDBMemQuotaHashJoin, "1024", false}, + {variable.TiDBMemQuotaMergeJoin, "1024", false}, + {variable.TiDBMemQuotaSort, "1024", false}, + {variable.TiDBMemQuotaTopn, "1024", false}, + {variable.TiDBMemQuotaIndexLookupReader, "1024", false}, + {variable.TiDBMemQuotaIndexLookupJoin, "1024", false}, {variable.TiDBMemQuotaApplyCache, "1024", false}, {variable.TiDBEnableStmtSummary, "1", false}, } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index ff53d355e4443..bec4db52c1757 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -636,7 +636,7 @@ var defaultSysVars = []*SysVar{ s.SetStatusFlag(mysql.ServerStatusNoBackslashEscaped, sqlMode.HasNoBackslashEscapesMode()) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: MaxExecutionTime, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: MaxExecutionTime, Value: "0", Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, AutoConvertOutOfRange: true, IsHintUpdatable: true, SetSession: func(s *SessionVars, val string) error { timeoutMS := tidbOptPositiveInt32(val, 0) s.MaxExecutionTime = uint64(timeoutMS) return nil @@ -875,11 +875,11 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeStartTime, Value: DefAutoAnalyzeStartTime, Type: TypeTime}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeEndTime, Value: DefAutoAnalyzeEndTime, Type: TypeTime}, {Scope: ScopeSession, Name: TiDBChecksumTableConcurrency, skipInit: true, Value: strconv.Itoa(DefChecksumTableConcurrency)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBExecutorConcurrency, Value: strconv.Itoa(DefExecutorConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBExecutorConcurrency, Value: strconv.Itoa(DefExecutorConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.ExecutorConcurrency = tidbOptPositiveInt32(val, DefExecutorConcurrency) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDistSQLScanConcurrency, Value: strconv.Itoa(DefDistSQLScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBDistSQLScanConcurrency, Value: strconv.Itoa(DefDistSQLScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.distSQLScanConcurrency = tidbOptPositiveInt32(val, DefDistSQLScanConcurrency) return nil }}, @@ -895,7 +895,7 @@ var defaultSysVars = []*SysVar{ s.CorrelationThreshold = tidbOptFloat64(val, DefOptCorrelationThreshold) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationExpFactor, Value: strconv.Itoa(DefOptCorrelationExpFactor), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCorrelationExpFactor, Value: strconv.Itoa(DefOptCorrelationExpFactor), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.CorrelationExpFactor = int(tidbOptInt64(val, DefOptCorrelationExpFactor)) return nil }}, @@ -939,29 +939,29 @@ var defaultSysVars = []*SysVar{ s.ConcurrencyFactor = tidbOptFloat64(val, DefOptConcurrencyFactor) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexJoinBatchSize, Value: strconv.Itoa(DefIndexJoinBatchSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexJoinBatchSize, Value: strconv.Itoa(DefIndexJoinBatchSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.IndexJoinBatchSize = tidbOptPositiveInt32(val, DefIndexJoinBatchSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupSize, Value: strconv.Itoa(DefIndexLookupSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupSize, Value: strconv.Itoa(DefIndexLookupSize), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.IndexLookupSize = tidbOptPositiveInt32(val, DefIndexLookupSize) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupConcurrency, Value: strconv.Itoa(DefIndexLookupConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupConcurrency, Value: strconv.Itoa(DefIndexLookupConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.indexLookupConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TiDBIndexLookupConcurrency, TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupJoinConcurrency, Value: strconv.Itoa(DefIndexLookupJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexLookupJoinConcurrency, Value: strconv.Itoa(DefIndexLookupJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.indexLookupJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TiDBIndexLookupJoinConcurrency, TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexSerialScanConcurrency, Value: strconv.Itoa(DefIndexSerialScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBIndexSerialScanConcurrency, Value: strconv.Itoa(DefIndexSerialScanConcurrency), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.indexSerialScanConcurrency = tidbOptPositiveInt32(val, DefIndexSerialScanConcurrency) return nil }}, @@ -985,7 +985,7 @@ var defaultSysVars = []*SysVar{ s.BatchCommit = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBDMLBatchSize, Value: strconv.Itoa(DefDMLBatchSize), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBDMLBatchSize, Value: strconv.Itoa(DefDMLBatchSize), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.DMLBatchSize = int(tidbOptInt64(val, DefDMLBatchSize)) return nil }}, @@ -1002,7 +1002,7 @@ var defaultSysVars = []*SysVar{ } return string(info), nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxChunkSize, Value: strconv.Itoa(DefMaxChunkSize), Type: TypeUnsigned, MinValue: maxChunkSizeLowerBound, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBMaxChunkSize, Value: strconv.Itoa(DefMaxChunkSize), Type: TypeUnsigned, MinValue: maxChunkSizeLowerBound, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.MaxChunkSize = tidbOptPositiveInt32(val, DefMaxChunkSize) return nil }}, @@ -1027,50 +1027,50 @@ var defaultSysVars = []*SysVar{ s.SetEnableIndexMerge(TiDBOptOn(val)) return nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaQuery, Value: strconv.FormatInt(config.GetGlobalConfig().MemQuotaQuery, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBMemQuotaQuery, Value: strconv.FormatInt(config.GetGlobalConfig().MemQuotaQuery, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaQuery = tidbOptInt64(val, config.GetGlobalConfig().MemQuotaQuery) return nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaHashJoin, Value: strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBMemQuotaHashJoin, Value: strconv.FormatInt(DefTiDBMemQuotaHashJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaHashJoin = tidbOptInt64(val, DefTiDBMemQuotaHashJoin) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TIDBMemQuotaHashJoin, TIDBMemQuotaQuery) + appendDeprecationWarning(vars, TiDBMemQuotaHashJoin, TiDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaMergeJoin, Value: strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBMemQuotaMergeJoin, Value: strconv.FormatInt(DefTiDBMemQuotaMergeJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaMergeJoin = tidbOptInt64(val, DefTiDBMemQuotaMergeJoin) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TIDBMemQuotaMergeJoin, TIDBMemQuotaQuery) + appendDeprecationWarning(vars, TiDBMemQuotaMergeJoin, TiDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaSort, Value: strconv.FormatInt(DefTiDBMemQuotaSort, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBMemQuotaSort, Value: strconv.FormatInt(DefTiDBMemQuotaSort, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaSort = tidbOptInt64(val, DefTiDBMemQuotaSort) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TIDBMemQuotaSort, TIDBMemQuotaQuery) + appendDeprecationWarning(vars, TiDBMemQuotaSort, TiDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaTopn, Value: strconv.FormatInt(DefTiDBMemQuotaTopn, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBMemQuotaTopn, Value: strconv.FormatInt(DefTiDBMemQuotaTopn, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaTopn = tidbOptInt64(val, DefTiDBMemQuotaTopn) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TIDBMemQuotaTopn, TIDBMemQuotaQuery) + appendDeprecationWarning(vars, TiDBMemQuotaTopn, TiDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupReader, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupReader, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBMemQuotaIndexLookupReader, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupReader, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaIndexLookupReader = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupReader) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TIDBMemQuotaIndexLookupReader, TIDBMemQuotaQuery) + appendDeprecationWarning(vars, TiDBMemQuotaIndexLookupReader, TiDBMemQuotaQuery) return normalizedValue, nil }}, - {Scope: ScopeSession, Name: TIDBMemQuotaIndexLookupJoin, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBMemQuotaIndexLookupJoin, Value: strconv.FormatInt(DefTiDBMemQuotaIndexLookupJoin, 10), skipInit: true, Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaIndexLookupJoin = tidbOptInt64(val, DefTiDBMemQuotaIndexLookupJoin) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - appendDeprecationWarning(vars, TIDBMemQuotaIndexLookupJoin, TIDBMemQuotaQuery) + appendDeprecationWarning(vars, TiDBMemQuotaIndexLookupJoin, TiDBMemQuotaQuery) return normalizedValue, nil }}, {Scope: ScopeSession, Name: TiDBEnableStreaming, Value: Off, Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { @@ -1096,42 +1096,42 @@ var defaultSysVars = []*SysVar{ s.EnableListTablePartition = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashJoinConcurrency, Value: strconv.Itoa(DefTiDBHashJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashJoinConcurrency, Value: strconv.Itoa(DefTiDBHashJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.hashJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TiDBHashJoinConcurrency, TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBProjectionConcurrency, Value: strconv.Itoa(DefTiDBProjectionConcurrency), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBProjectionConcurrency, Value: strconv.Itoa(DefTiDBProjectionConcurrency), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.projectionConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TiDBProjectionConcurrency, TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggPartialConcurrency, Value: strconv.Itoa(DefTiDBHashAggPartialConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggPartialConcurrency, Value: strconv.Itoa(DefTiDBHashAggPartialConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.hashAggPartialConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TiDBHashAggPartialConcurrency, TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggFinalConcurrency, Value: strconv.Itoa(DefTiDBHashAggFinalConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBHashAggFinalConcurrency, Value: strconv.Itoa(DefTiDBHashAggFinalConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.hashAggFinalConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TiDBHashAggFinalConcurrency, TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBWindowConcurrency, Value: strconv.Itoa(DefTiDBWindowConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBWindowConcurrency, Value: strconv.Itoa(DefTiDBWindowConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.windowConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { appendDeprecationWarning(vars, TiDBWindowConcurrency, TiDBExecutorConcurrency) return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMergeJoinConcurrency, Value: strconv.Itoa(DefTiDBMergeJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBMergeJoinConcurrency, Value: strconv.Itoa(DefTiDBMergeJoinConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.mergeJoinConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { @@ -1139,7 +1139,7 @@ var defaultSysVars = []*SysVar{ return normalizedValue, nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStreamAggConcurrency, Value: strconv.Itoa(DefTiDBStreamAggConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStreamAggConcurrency, Value: strconv.Itoa(DefTiDBStreamAggConcurrency), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowAutoValue: true, SetSession: func(s *SessionVars, val string) error { s.streamAggConcurrency = tidbOptPositiveInt32(val, ConcurrencyUnset) return nil }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { @@ -1150,15 +1150,15 @@ var defaultSysVars = []*SysVar{ s.EnableParallelApply = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBMemQuotaApplyCache, Value: strconv.Itoa(DefTiDBMemQuotaApplyCache), SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBMemQuotaApplyCache, Value: strconv.Itoa(DefTiDBMemQuotaApplyCache), Type: TypeUnsigned, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { s.MemQuotaApplyCache = tidbOptInt64(val, DefTiDBMemQuotaApplyCache) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackoffLockFast, Value: strconv.Itoa(tikvstore.DefBackoffLockFast), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackoffLockFast, Value: strconv.Itoa(tikvstore.DefBackoffLockFast), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.KVVars.BackoffLockFast = tidbOptPositiveInt32(val, tikvstore.DefBackoffLockFast) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackOffWeight, Value: strconv.Itoa(tikvstore.DefBackOffWeight), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBBackOffWeight, Value: strconv.Itoa(tikvstore.DefBackOffWeight), Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.KVVars.BackOffWeight = tidbOptPositiveInt32(val, tikvstore.DefBackOffWeight) return nil }}, @@ -1188,7 +1188,7 @@ var defaultSysVars = []*SysVar{ SetDDLReorgRowFormat(tidbOptInt64(val, DefTiDBRowFormatV2)) return nil }}, - {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), skipInit: true, Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), skipInit: true, Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.OptimizerSelectivityLevel = tidbOptPositiveInt32(val, DefTiDBOptimizerSelectivityLevel) return nil }}, @@ -1286,7 +1286,7 @@ var defaultSysVars = []*SysVar{ SetDDLReorgBatchSize(int32(tidbOptPositiveInt32(val, DefTiDBDDLReorgBatchSize))) return nil }}, - {Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal, Name: TiDBDDLErrorCountLimit, Value: strconv.Itoa(DefTiDBDDLErrorCountLimit), Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt64, AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { SetDDLErrorCountLimit(tidbOptInt64(val, DefTiDBDDLErrorCountLimit)) return nil }}, @@ -1334,7 +1334,7 @@ var defaultSysVars = []*SysVar{ s.WaitSplitRegionFinish = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBWaitSplitRegionTimeout, Value: strconv.Itoa(DefWaitSplitRegionTimeout), skipInit: true, Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt64, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBWaitSplitRegionTimeout, Value: strconv.Itoa(DefWaitSplitRegionTimeout), skipInit: true, Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.WaitSplitRegionTimeout = uint64(tidbOptPositiveInt32(val, DefWaitSplitRegionTimeout)) return nil }}, @@ -1342,7 +1342,7 @@ var defaultSysVars = []*SysVar{ s.LowResolutionTSO = TiDBOptOn(val) return nil }}, - {Scope: ScopeSession, Name: TiDBExpensiveQueryTimeThreshold, Value: strconv.Itoa(DefTiDBExpensiveQueryTimeThreshold), Type: TypeUnsigned, MinValue: int64(MinExpensiveQueryTimeThreshold), MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBExpensiveQueryTimeThreshold, Value: strconv.Itoa(DefTiDBExpensiveQueryTimeThreshold), Type: TypeUnsigned, MinValue: int64(MinExpensiveQueryTimeThreshold), MaxValue: math.MaxInt32, AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { atomic.StoreUint64(&ExpensiveQueryTimeThreshold, uint64(tidbOptPositiveInt32(val, DefTiDBExpensiveQueryTimeThreshold))) return nil }, GetSession: func(s *SessionVars) (string, error) { @@ -1396,10 +1396,10 @@ var defaultSysVars = []*SysVar{ }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableStmtSummary, Value: BoolToOnOff(config.GetGlobalConfig().StmtSummary.Enable), Type: TypeBool, AllowEmpty: true}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryInternalQuery, Value: BoolToOnOff(config.GetGlobalConfig().StmtSummary.EnableInternalQuery), Type: TypeBool, AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryRefreshInterval, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.RefreshInterval), Type: TypeInt, MinValue: 1, MaxValue: uint64(math.MaxInt32), AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryHistorySize, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.HistorySize), Type: TypeInt, MinValue: 0, MaxValue: uint64(math.MaxUint8), AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxStmtCount, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxStmtCount), 10), Type: TypeInt, MinValue: 1, MaxValue: uint64(math.MaxInt16), AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxSQLLength, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxSQLLength), 10), Type: TypeInt, MinValue: 0, MaxValue: uint64(math.MaxInt32), AllowEmpty: true}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryRefreshInterval, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.RefreshInterval), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowEmpty: true}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryHistorySize, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.HistorySize), Type: TypeInt, MinValue: 0, MaxValue: math.MaxUint8, AllowEmpty: true}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxStmtCount, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxStmtCount), 10), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt16, AllowEmpty: true}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxSQLLength, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxSQLLength), 10), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, AllowEmpty: true}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBCapturePlanBaseline, Value: Off, Type: TypeBool, AllowEmptyAll: true, GetSession: func(s *SessionVars) (string, error) { return CapturePlanBaseline.GetVal(), nil }}, @@ -1452,7 +1452,7 @@ var defaultSysVars = []*SysVar{ } return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStoreLimit, Value: strconv.FormatInt(atomic.LoadInt64(&config.GetGlobalConfig().TiKVClient.StoreLimit), 10), Type: TypeInt, MinValue: 0, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStoreLimit, Value: strconv.FormatInt(atomic.LoadInt64(&config.GetGlobalConfig().TiKVClient.StoreLimit), 10), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt64, AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { tikvstore.StoreLimit.Store(tidbOptInt64(val, DefTiDBStoreLimit)) return nil }}, @@ -1556,7 +1556,7 @@ var defaultSysVars = []*SysVar{ errors.RedactLogEnabled.Store(s.EnableRedactLog) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBShardAllocateStep, Value: strconv.Itoa(DefTiDBShardAllocateStep), Type: TypeInt, MinValue: 1, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBShardAllocateStep, Value: strconv.Itoa(DefTiDBShardAllocateStep), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error { s.ShardAllocateStep = tidbOptInt64(val, DefTiDBShardAllocateStep) return nil }}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 63f03e4c14695..b262bfd69ff19 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -106,15 +106,15 @@ const ( // The following session variables controls the memory quota during query execution. // "tidb_mem_quota_query": control the memory quota of a query. - TIDBMemQuotaQuery = "tidb_mem_quota_query" // Bytes. + TiDBMemQuotaQuery = "tidb_mem_quota_query" // Bytes. TiDBMemQuotaApplyCache = "tidb_mem_quota_apply_cache" - // TODO: remove them below sometime, it should have only one Quota(TIDBMemQuotaQuery). - TIDBMemQuotaHashJoin = "tidb_mem_quota_hashjoin" // Bytes. - TIDBMemQuotaMergeJoin = "tidb_mem_quota_mergejoin" // Bytes. - TIDBMemQuotaSort = "tidb_mem_quota_sort" // Bytes. - TIDBMemQuotaTopn = "tidb_mem_quota_topn" // Bytes. - TIDBMemQuotaIndexLookupReader = "tidb_mem_quota_indexlookupreader" // Bytes. - TIDBMemQuotaIndexLookupJoin = "tidb_mem_quota_indexlookupjoin" // Bytes. + // TODO: remove them below sometime, it should have only one Quota(TiDBMemQuotaQuery). + TiDBMemQuotaHashJoin = "tidb_mem_quota_hashjoin" // Bytes. + TiDBMemQuotaMergeJoin = "tidb_mem_quota_mergejoin" // Bytes. + TiDBMemQuotaSort = "tidb_mem_quota_sort" // Bytes. + TiDBMemQuotaTopn = "tidb_mem_quota_topn" // Bytes. + TiDBMemQuotaIndexLookupReader = "tidb_mem_quota_indexlookupreader" // Bytes. + TiDBMemQuotaIndexLookupJoin = "tidb_mem_quota_indexlookupjoin" // Bytes. // tidb_general_log is used to log every query in the server in info level. TiDBGeneralLog = "tidb_general_log" @@ -409,7 +409,7 @@ const ( // tidb_enable_vectorized_expression is used to control whether to enable the vectorized expression evaluation. TiDBEnableVectorizedExpression = "tidb_enable_vectorized_expression" - // TIDBOptJoinReorderThreshold defines the threshold less than which + // TiDBOptJoinReorderThreshold defines the threshold less than which // we'll choose a rather time consuming algorithm to calculate the join order. TiDBOptJoinReorderThreshold = "tidb_opt_join_reorder_threshold" diff --git a/tidb-server/main.go b/tidb-server/main.go index a4e81f51862af..3aa21808f97c3 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -526,7 +526,7 @@ func setGlobalVars() { variable.SetSysVar(variable.TiDBForcePriority, mysql.Priority2Str[priority]) variable.SetSysVar(variable.TiDBOptDistinctAggPushDown, variable.BoolToOnOff(cfg.Performance.DistinctAggPushDown)) - variable.SetSysVar(variable.TIDBMemQuotaQuery, strconv.FormatInt(cfg.MemQuotaQuery, 10)) + variable.SetSysVar(variable.TiDBMemQuotaQuery, strconv.FormatInt(cfg.MemQuotaQuery, 10)) variable.SetSysVar(variable.LowerCaseTableNames, strconv.Itoa(cfg.LowerCaseTableNames)) variable.SetSysVar(variable.LogBin, variable.BoolToOnOff(cfg.Binlog.Enable)) variable.SetSysVar(variable.Port, fmt.Sprintf("%d", cfg.Port)) diff --git a/tidb-server/main_test.go b/tidb-server/main_test.go index b36166c6ed0a4..def1048adb719 100644 --- a/tidb-server/main_test.go +++ b/tidb-server/main_test.go @@ -41,7 +41,7 @@ type testMainSuite struct{} func (t *testMainSuite) TestSetGlobalVars(c *C) { c.Assert(variable.GetSysVar(variable.TiDBIsolationReadEngines).Value, Equals, "tikv, tiflash, tidb") - c.Assert(variable.GetSysVar(variable.TIDBMemQuotaQuery).Value, Equals, "1073741824") + c.Assert(variable.GetSysVar(variable.TiDBMemQuotaQuery).Value, Equals, "1073741824") config.UpdateGlobal(func(conf *config.Config) { conf.IsolationRead.Engines = []string{"tikv", "tidb"} conf.MemQuotaQuery = 9999999 @@ -49,5 +49,5 @@ func (t *testMainSuite) TestSetGlobalVars(c *C) { setGlobalVars() c.Assert(variable.GetSysVar(variable.TiDBIsolationReadEngines).Value, Equals, "tikv, tidb") - c.Assert(variable.GetSysVar(variable.TIDBMemQuotaQuery).Value, Equals, "9999999") + c.Assert(variable.GetSysVar(variable.TiDBMemQuotaQuery).Value, Equals, "9999999") } From ccaefa2a3ddf8adb4cf303edcf0795387e7fc6cf Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Tue, 1 Jun 2021 11:59:37 +0800 Subject: [PATCH 225/343] executor: add CTEExec and CTETableReaderExec (#24809) --- executor/builder.go | 100 +++++++ executor/cte.go | 490 ++++++++++++++++++++++++++++++++++ executor/cte_table_reader.go | 78 ++++++ executor/cte_test.go | 244 +++++++++++++++++ executor/executor.go | 9 +- session/session.go | 36 ++- sessionctx/stmtctx/stmtctx.go | 3 + util/chunk/chunk.go | 15 ++ util/chunk/disk.go | 3 +- util/cteutil/storage.go | 9 +- util/cteutil/storage_test.go | 14 +- util/memory/tracker.go | 2 + 12 files changed, 984 insertions(+), 19 deletions(-) create mode 100644 executor/cte.go create mode 100644 executor/cte_table_reader.go create mode 100644 executor/cte_test.go diff --git a/executor/builder.go b/executor/builder.go index 037123cc1fa96..3ef6d96358e0b 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -49,6 +49,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/cteutil" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" @@ -83,6 +84,14 @@ type executorBuilder struct { hasLock bool } +// CTEStorages stores resTbl and iterInTbl for CTEExec. +// There will be a map[CTEStorageID]*CTEStorages in StmtCtx, +// which will store all CTEStorages to make all shared CTEs use same the CTEStorages. +type CTEStorages struct { + ResTbl cteutil.Storage + IterInTbl cteutil.Storage +} + func newExecutorBuilder(ctx sessionctx.Context, is infoschema.InfoSchema) *executorBuilder { return &executorBuilder{ ctx: ctx, @@ -235,6 +244,10 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor { return b.buildAdminShowTelemetry(v) case *plannercore.AdminResetTelemetryID: return b.buildAdminResetTelemetryID(v) + case *plannercore.PhysicalCTE: + return b.buildCTE(v) + case *plannercore.PhysicalCTETable: + return b.buildCTETableReader(v) default: if mp, ok := p.(MockPhysicalPlan); ok { return mp.GetExecutor() @@ -4072,3 +4085,90 @@ func (b *executorBuilder) buildTableSample(v *plannercore.PhysicalTableSample) * } return e } + +func (b *executorBuilder) buildCTE(v *plannercore.PhysicalCTE) Executor { + // 1. Build seedPlan. + seedExec := b.build(v.SeedPlan) + if b.err != nil { + return nil + } + + // 2. Build iterInTbl. + chkSize := b.ctx.GetSessionVars().MaxChunkSize + tps := seedExec.base().retFieldTypes + iterOutTbl := cteutil.NewStorageRowContainer(tps, chkSize) + if err := iterOutTbl.OpenAndRef(); err != nil { + b.err = err + return nil + } + + var resTbl cteutil.Storage + var iterInTbl cteutil.Storage + storageMap, ok := b.ctx.GetSessionVars().StmtCtx.CTEStorageMap.(map[int]*CTEStorages) + if !ok { + b.err = errors.New("type assertion for CTEStorageMap failed") + return nil + } + storages, ok := storageMap[v.CTE.IDForStorage] + if ok { + // Storage already setup. + resTbl = storages.ResTbl + iterInTbl = storages.IterInTbl + } else { + resTbl = cteutil.NewStorageRowContainer(tps, chkSize) + if err := resTbl.OpenAndRef(); err != nil { + b.err = err + return nil + } + iterInTbl = cteutil.NewStorageRowContainer(tps, chkSize) + if err := iterInTbl.OpenAndRef(); err != nil { + b.err = err + return nil + } + storageMap[v.CTE.IDForStorage] = &CTEStorages{ResTbl: resTbl, IterInTbl: iterInTbl} + } + + // 3. Build recursive part. + recursiveExec := b.build(v.RecurPlan) + if b.err != nil { + return nil + } + + var sel []int + if v.CTE.IsDistinct { + sel = make([]int, chkSize) + for i := 0; i < chkSize; i++ { + sel[i] = i + } + } + + return &CTEExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), + seedExec: seedExec, + recursiveExec: recursiveExec, + resTbl: resTbl, + iterInTbl: iterInTbl, + iterOutTbl: iterOutTbl, + chkIdx: 0, + isDistinct: v.CTE.IsDistinct, + sel: sel, + } +} + +func (b *executorBuilder) buildCTETableReader(v *plannercore.PhysicalCTETable) Executor { + storageMap, ok := b.ctx.GetSessionVars().StmtCtx.CTEStorageMap.(map[int]*CTEStorages) + if !ok { + b.err = errors.New("type assertion for CTEStorageMap failed") + return nil + } + storages, ok := storageMap[v.IDForStorage] + if !ok { + b.err = errors.Errorf("iterInTbl should already be set up by CTEExec(id: %d)", v.IDForStorage) + return nil + } + return &CTETableReaderExec{ + baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()), + iterInTbl: storages.IterInTbl, + chkIdx: 0, + } +} diff --git a/executor/cte.go b/executor/cte.go new file mode 100644 index 0000000000000..a5e063e9dc9ee --- /dev/null +++ b/executor/cte.go @@ -0,0 +1,490 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor + +import ( + "context" + + "github.com/pingcap/errors" + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/cteutil" + "github.com/pingcap/tidb/util/memory" +) + +var _ Executor = &CTEExec{} + +// CTEExec implements CTE. +// Following diagram describes how CTEExec works. +// +// `iterInTbl` is shared by `CTEExec` and `CTETableReaderExec`. +// `CTETableReaderExec` reads data from `iterInTbl`, +// and its output will be stored `iterOutTbl` by `CTEExec`. +// +// When an iteration ends, `CTEExec` will move all data from `iterOutTbl` into `iterInTbl`, +// which will be the input for new iteration. +// At the end of each iteration, data in `iterOutTbl` will also be added into `resTbl`. +// `resTbl` stores data of all iteration. +// +----------+ +// write |iterOutTbl| +// CTEExec ------------------->| | +// | +----+-----+ +// ------------- | write +// | | v +// other op other op +----------+ +// (seed) (recursive) | resTbl | +// ^ | | +// | +----------+ +// CTETableReaderExec +// ^ +// | read +----------+ +// +---------------+iterInTbl | +// | | +// +----------+ +type CTEExec struct { + baseExecutor + + seedExec Executor + recursiveExec Executor + + // `resTbl` and `iterInTbl` are shared by all CTEExec which reference to same the CTE. + // `iterInTbl` is also shared by CTETableReaderExec. + resTbl cteutil.Storage + iterInTbl cteutil.Storage + iterOutTbl cteutil.Storage + + hashTbl baseHashTable + + // Index of chunk to read from `resTbl`. + chkIdx int + + // UNION ALL or UNION DISTINCT. + isDistinct bool + curIter int + hCtx *hashContext + sel []int +} + +// Open implements the Executor interface. +func (e *CTEExec) Open(ctx context.Context) (err error) { + e.reset() + if err := e.baseExecutor.Open(ctx); err != nil { + return err + } + + if e.seedExec == nil { + return errors.New("seedExec for CTEExec is nil") + } + if err = e.seedExec.Open(ctx); err != nil { + return err + } + + if e.recursiveExec != nil { + if err = e.recursiveExec.Open(ctx); err != nil { + return err + } + recursiveTypes := e.recursiveExec.base().retFieldTypes + e.iterOutTbl = cteutil.NewStorageRowContainer(recursiveTypes, e.maxChunkSize) + if err = e.iterOutTbl.OpenAndRef(); err != nil { + return err + } + + setupCTEStorageTracker(e.iterOutTbl, e.ctx) + } + + if e.isDistinct { + e.hashTbl = newConcurrentMapHashTable() + e.hCtx = &hashContext{ + allTypes: e.base().retFieldTypes, + } + // We use all columns to compute hash. + e.hCtx.keyColIdx = make([]int, len(e.hCtx.allTypes)) + for i := range e.hCtx.keyColIdx { + e.hCtx.keyColIdx[i] = i + } + } + return nil +} + +// Next implements the Executor interface. +func (e *CTEExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { + req.Reset() + e.resTbl.Lock() + if !e.resTbl.Done() { + defer e.resTbl.Unlock() + resAction := setupCTEStorageTracker(e.resTbl, e.ctx) + iterInAction := setupCTEStorageTracker(e.iterInTbl, e.ctx) + + failpoint.Inject("testCTEStorageSpill", func(val failpoint.Value) { + if val.(bool) && config.GetGlobalConfig().OOMUseTmpStorage { + defer resAction.WaitForTest() + defer iterInAction.WaitForTest() + } + }) + + if err = e.computeSeedPart(ctx); err != nil { + // Don't put it in defer. + // Because it should be called only when the filling process is not completed. + if err1 := e.reopenTbls(); err1 != nil { + return err1 + } + return err + } + if err = e.computeRecursivePart(ctx); err != nil { + if err1 := e.reopenTbls(); err1 != nil { + return err1 + } + return err + } + e.resTbl.SetDone() + } else { + e.resTbl.Unlock() + } + + if e.chkIdx < e.resTbl.NumChunks() { + res, err := e.resTbl.GetChunk(e.chkIdx) + if err != nil { + return err + } + // Need to copy chunk to make sure upper operator will not change chunk in resTbl. + // Also we ignore copying rows not selected, because some operators like Projection + // doesn't support swap column if chunk.sel is no nil. + req.SwapColumns(res.CopyConstructSel()) + e.chkIdx++ + } + return nil +} + +// Close implements the Executor interface. +func (e *CTEExec) Close() (err error) { + e.reset() + if err = e.seedExec.Close(); err != nil { + return err + } + if e.recursiveExec != nil { + if err = e.recursiveExec.Close(); err != nil { + return err + } + } + + // `iterInTbl` and `resTbl` are shared by multiple operators, + // so will be closed when the SQL finishes. + if err = e.iterOutTbl.DerefAndClose(); err != nil { + return err + } + return e.baseExecutor.Close() +} + +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 { + chk := newFirstChunk(e.seedExec) + if err = Next(ctx, e.seedExec, chk); err != nil { + return err + } + if chk.NumRows() == 0 { + break + } + if chk, err = e.tryDedupAndAdd(chk, e.iterInTbl, e.hashTbl); err != nil { + return err + } + chks = append(chks, chk) + } + // Initial resTbl is empty, so no need to deduplicate chk using resTbl. + // Just adding is ok. + for _, chk := range chks { + if err = e.resTbl.Add(chk); err != nil { + return err + } + } + e.curIter++ + e.iterInTbl.SetIter(e.curIter) + + return nil +} + +func (e *CTEExec) computeRecursivePart(ctx context.Context) (err error) { + if e.recursiveExec == nil || e.iterInTbl.NumChunks() == 0 { + return nil + } + + if e.curIter > e.ctx.GetSessionVars().CTEMaxRecursionDepth { + return ErrCTEMaxRecursionDepth.GenWithStackByArgs(e.curIter) + } + + for { + chk := newFirstChunk(e.recursiveExec) + if err = Next(ctx, e.recursiveExec, chk); err != nil { + return err + } + if chk.NumRows() == 0 { + if err = e.setupTblsForNewIteration(); err != nil { + return err + } + if e.iterInTbl.NumChunks() == 0 { + break + } + // Next iteration begins. Need use iterOutTbl as input of next iteration. + e.curIter++ + e.iterInTbl.SetIter(e.curIter) + if e.curIter > e.ctx.GetSessionVars().CTEMaxRecursionDepth { + return ErrCTEMaxRecursionDepth.GenWithStackByArgs(e.curIter) + } + // Make sure iterInTbl is setup before Close/Open, + // because some executors will read iterInTbl in Open() (like IndexLookupJoin). + if err = e.recursiveExec.Close(); err != nil { + return err + } + if err = e.recursiveExec.Open(ctx); err != nil { + return err + } + } else { + if err = e.iterOutTbl.Add(chk); err != nil { + return err + } + } + } + return nil +} + +func (e *CTEExec) setupTblsForNewIteration() (err error) { + num := e.iterOutTbl.NumChunks() + chks := make([]*chunk.Chunk, 0, num) + // Setup resTbl's data. + for i := 0; i < num; i++ { + chk, err := e.iterOutTbl.GetChunk(i) + if err != nil { + return err + } + // Data should be copied in UNION DISTINCT. + // Because deduplicate() will change data in iterOutTbl, + // which will cause panic when spilling data into disk concurrently. + if e.isDistinct { + chk = chk.CopyConstruct() + } + chk, err = e.tryDedupAndAdd(chk, e.resTbl, e.hashTbl) + if err != nil { + return err + } + chks = append(chks, chk) + } + + // Setup new iteration data in iterInTbl. + 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 { + if err = e.iterInTbl.Add(chk); err != nil { + return err + } + } + } else { + if err = e.iterInTbl.SwapData(e.iterOutTbl); err != nil { + return err + } + } + + // Clear data in iterOutTbl. + return e.iterOutTbl.Reopen() +} + +func (e *CTEExec) reset() { + e.curIter = 0 + e.chkIdx = 0 + e.hashTbl = nil +} + +func (e *CTEExec) reopenTbls() (err error) { + e.hashTbl = newConcurrentMapHashTable() + if err := e.resTbl.Reopen(); err != nil { + return err + } + return e.iterInTbl.Reopen() +} + +func setupCTEStorageTracker(tbl cteutil.Storage, ctx sessionctx.Context) (actionSpill *chunk.SpillDiskAction) { + memTracker := tbl.GetMemTracker() + memTracker.SetLabel(memory.LabelForCTEStorage) + memTracker.AttachTo(ctx.GetSessionVars().StmtCtx.MemTracker) + + diskTracker := tbl.GetDiskTracker() + diskTracker.SetLabel(memory.LabelForCTEStorage) + diskTracker.AttachTo(ctx.GetSessionVars().StmtCtx.DiskTracker) + + if config.GetGlobalConfig().OOMUseTmpStorage { + actionSpill = tbl.ActionSpill() + failpoint.Inject("testCTEStorageSpill", func(val failpoint.Value) { + if val.(bool) { + actionSpill = tbl.(*cteutil.StorageRC).ActionSpillForTest() + } + }) + ctx.GetSessionVars().StmtCtx.MemTracker.FallbackOldAndSetNewAction(actionSpill) + } + return actionSpill +} + +func (e *CTEExec) tryDedupAndAdd(chk *chunk.Chunk, + storage cteutil.Storage, + hashTbl baseHashTable) (res *chunk.Chunk, err error) { + if e.isDistinct { + if chk, err = e.deduplicate(chk, storage, hashTbl); err != nil { + return nil, err + } + } + return chk, storage.Add(chk) +} + +// Compute hash values in chk and put it in hCtx.hashVals. +// Use the returned sel to choose the computed hash values. +func (e *CTEExec) computeChunkHash(chk *chunk.Chunk) (sel []int, err error) { + numRows := chk.NumRows() + e.hCtx.initHash(numRows) + // Continue to reset to make sure all hasher is new. + for i := numRows; i < len(e.hCtx.hashVals); i++ { + e.hCtx.hashVals[i].Reset() + } + sel = chk.Sel() + var hashBitMap []bool + if sel != nil { + hashBitMap = make([]bool, chk.Capacity()) + for _, val := range sel { + hashBitMap[val] = true + } + } else { + // All rows is selected, sel will be [0....numRows). + // e.sel is setup when building executor. + sel = e.sel + } + + for i := 0; i < chk.NumCols(); i++ { + if err = codec.HashChunkSelected(e.ctx.GetSessionVars().StmtCtx, e.hCtx.hashVals, + chk, e.hCtx.allTypes[i], i, e.hCtx.buf, e.hCtx.hasNull, + hashBitMap, false); err != nil { + return nil, err + } + } + return sel, nil +} + +// Use hashTbl to deduplicate rows, and unique rows will be added to hashTbl. +// Duplicated rows are only marked to be removed by sel in Chunk, instead of really deleted. +func (e *CTEExec) deduplicate(chk *chunk.Chunk, + storage cteutil.Storage, + hashTbl baseHashTable) (chkNoDup *chunk.Chunk, err error) { + numRows := chk.NumRows() + if numRows == 0 { + return chk, nil + } + + // 1. Compute hash values for chunk. + chkHashTbl := newConcurrentMapHashTable() + selOri, err := e.computeChunkHash(chk) + if err != nil { + return nil, err + } + + // 2. Filter rows duplicated in input chunk. + // This sel is for filtering rows duplicated in cur chk. + selChk := make([]int, 0, numRows) + for i := 0; i < numRows; i++ { + key := e.hCtx.hashVals[selOri[i]].Sum64() + row := chk.GetRow(i) + + hasDup, err := e.checkHasDup(key, row, chk, storage, chkHashTbl) + if err != nil { + return nil, err + } + if hasDup { + continue + } + + selChk = append(selChk, selOri[i]) + + rowPtr := chunk.RowPtr{ChkIdx: uint32(0), RowIdx: uint32(i)} + chkHashTbl.Put(key, rowPtr) + } + chk.SetSel(selChk) + chkIdx := storage.NumChunks() + + // 3. Filter rows duplicated in RowContainer. + // This sel is for filtering rows duplicated in cteutil.Storage. + selStorage := make([]int, 0, len(selChk)) + for i := 0; i < len(selChk); i++ { + key := e.hCtx.hashVals[selChk[i]].Sum64() + row := chk.GetRow(i) + + hasDup, err := e.checkHasDup(key, row, nil, storage, hashTbl) + if err != nil { + return nil, err + } + if hasDup { + continue + } + + rowIdx := len(selStorage) + selStorage = append(selStorage, selChk[i]) + + rowPtr := chunk.RowPtr{ChkIdx: uint32(chkIdx), RowIdx: uint32(rowIdx)} + hashTbl.Put(key, rowPtr) + } + + chk.SetSel(selStorage) + return chk, nil +} + +// Use the row's probe key to check if it already exists in chk or storage. +// We also need to compare the row's real encoding value to avoid hash collision. +func (e *CTEExec) checkHasDup(probeKey uint64, + row chunk.Row, + curChk *chunk.Chunk, + storage cteutil.Storage, + hashTbl baseHashTable) (hasDup bool, err error) { + ptrs := hashTbl.Get(probeKey) + + if len(ptrs) == 0 { + return false, nil + } + + for _, ptr := range ptrs { + var matchedRow chunk.Row + if curChk != nil { + matchedRow = curChk.GetRow(int(ptr.RowIdx)) + } else { + matchedRow, err = storage.GetRow(ptr) + } + if err != nil { + return false, err + } + isEqual, err := codec.EqualChunkRow(e.ctx.GetSessionVars().StmtCtx, + row, e.hCtx.allTypes, e.hCtx.keyColIdx, + matchedRow, e.hCtx.allTypes, e.hCtx.keyColIdx) + if err != nil { + return false, err + } + if isEqual { + return true, nil + } + } + return false, nil +} diff --git a/executor/cte_table_reader.go b/executor/cte_table_reader.go new file mode 100644 index 0000000000000..94fedf01fd93e --- /dev/null +++ b/executor/cte_table_reader.go @@ -0,0 +1,78 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor + +import ( + "context" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/cteutil" +) + +// CTETableReaderExec scans data in iterInTbl, which is filled by corresponding CTEExec. +type CTETableReaderExec struct { + baseExecutor + + iterInTbl cteutil.Storage + chkIdx int + curIter int +} + +// Open implements the Executor interface. +func (e *CTETableReaderExec) Open(ctx context.Context) error { + e.reset() + return e.baseExecutor.Open(ctx) +} + +// Next implements the Executor interface. +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`. + if e.curIter != e.iterInTbl.GetIter() { + if e.curIter > e.iterInTbl.GetIter() { + return errors.Errorf("invalid iteration for CTETableReaderExec (e.curIter: %d, e.iterInTbl.GetIter(): %d)", + e.curIter, e.iterInTbl.GetIter()) + } + e.chkIdx = 0 + e.curIter = e.iterInTbl.GetIter() + } + if e.chkIdx < e.iterInTbl.NumChunks() { + res, err := e.iterInTbl.GetChunk(e.chkIdx) + if err != nil { + return err + } + // Need to copy chunk to make sure upper operators will not change chunk in iterInTbl. + req.SwapColumns(res.CopyConstructSel()) + e.chkIdx++ + } + return nil +} + +// Close implements the Executor interface. +func (e *CTETableReaderExec) Close() (err error) { + e.reset() + return e.baseExecutor.Close() +} + +func (e *CTETableReaderExec) reset() { + e.chkIdx = 0 + e.curIter = 0 +} diff --git a/executor/cte_test.go b/executor/cte_test.go new file mode 100644 index 0000000000000..e5789170627f7 --- /dev/null +++ b/executor/cte_test.go @@ -0,0 +1,244 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor_test + +import ( + "context" + "fmt" + "math/rand" + "sort" + + "github.com/pingcap/check" + + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/sessionctx" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/testkit" +) + +var _ = check.Suite(&CTETestSuite{}) + +type CTETestSuite struct { + store kv.Storage + dom *domain.Domain + sessionCtx sessionctx.Context + session session.Session + ctx context.Context +} + +func (test *CTETestSuite) SetUpSuite(c *check.C) { + var err error + test.store, err = mockstore.NewMockStore() + c.Assert(err, check.IsNil) + + test.dom, err = session.BootstrapSession(test.store) + c.Assert(err, check.IsNil) + + test.sessionCtx = mock.NewContext() + + test.session, err = session.CreateSession4Test(test.store) + c.Assert(err, check.IsNil) + test.session.SetConnectionID(0) + + test.ctx = context.Background() +} + +func (test *CTETestSuite) TearDownSuite(c *check.C) { + test.dom.Close() + test.store.Close() +} + +func (test *CTETestSuite) TestBasicCTE(c *check.C) { + tk := testkit.NewTestKit(c, test.store) + tk.MustExec("use test") + + rows := tk.MustQuery("with recursive cte1 as (" + + "select 1 c1 " + + "union all " + + "select c1 + 1 c1 from cte1 where c1 < 5) " + + "select * from cte1") + rows.Check(testkit.Rows("1", "2", "3", "4", "5")) + + // Two seed parts. + rows = tk.MustQuery("with recursive cte1 as (" + + "select 1 c1 " + + "union all " + + "select 2 c1 " + + "union all " + + "select c1 + 1 c1 from cte1 where c1 < 10) " + + "select * from cte1 order by c1") + rows.Check(testkit.Rows("1", "2", "2", "3", "3", "4", "4", "5", "5", "6", "6", "7", "7", "8", "8", "9", "9", "10", "10")) + + // Two recursive parts. + rows = tk.MustQuery("with recursive cte1 as (" + + "select 1 c1 " + + "union all " + + "select 2 c1 " + + "union all " + + "select c1 + 1 c1 from cte1 where c1 < 3 " + + "union all " + + "select c1 + 2 c1 from cte1 where c1 < 5) " + + "select * from cte1 order by c1") + rows.Check(testkit.Rows("1", "2", "2", "3", "3", "3", "4", "4", "5", "5", "5", "6", "6")) + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(a int);") + tk.MustExec("insert into t1 values(1);") + tk.MustExec("insert into t1 values(2);") + rows = tk.MustQuery("SELECT * FROM t1 dt WHERE EXISTS(WITH RECURSIVE qn AS (SELECT a*0 AS b UNION ALL SELECT b+1 FROM qn WHERE b=0) SELECT * FROM qn WHERE b=a);") + rows.Check(testkit.Rows("1")) + rows = tk.MustQuery("SELECT * FROM t1 dt WHERE EXISTS( WITH RECURSIVE qn AS (SELECT a*0 AS b UNION ALL SELECT b+1 FROM qn WHERE b=0 or b = 1) SELECT * FROM qn WHERE b=a );") + rows.Check(testkit.Rows("1", "2")) +} + +func (test *CTETestSuite) TestSpillToDisk(c *check.C) { + tk := testkit.NewTestKit(c, test.store) + tk.MustExec("use test;") + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testCTEStorageSpill", "return(true)"), check.IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testCTEStorageSpill"), check.IsNil) + }() + + insertStr := "insert into t1 values(0, 0)" + for i := 1; i < 5000; i++ { + insertStr += fmt.Sprintf(", (%d, %d)", i, i) + } + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int, c2 int);") + tk.MustExec(insertStr) + tk.MustExec("set tidb_mem_quota_query = 80000;") + rows := tk.MustQuery("with recursive cte1 as ( " + + "select c1 from t1 " + + "union " + + "select c1 + 1 c1 from cte1 where c1 < 5000) " + + "select c1 from cte1;") + + memTracker := tk.Se.GetSessionVars().StmtCtx.MemTracker + diskTracker := tk.Se.GetSessionVars().StmtCtx.DiskTracker + c.Assert(memTracker.MaxConsumed(), check.Greater, int64(0)) + c.Assert(diskTracker.MaxConsumed(), check.Greater, int64(0)) + + rowNum := 5000 + var resRows []string + for i := 0; i <= rowNum; i++ { + resRows = append(resRows, fmt.Sprintf("%d", i)) + } + rows.Check(testkit.Rows(resRows...)) + + // Use duplicated rows to test UNION DISTINCT. + tk.MustExec("set tidb_mem_quota_query = 1073741824;") + insertStr = "insert into t1 values(0, 0)" + vals := make([]int, rowNum) + vals[0] = 0 + for i := 1; i < rowNum; i++ { + v := rand.Intn(100) + vals[i] = v + insertStr += fmt.Sprintf(", (%d, %d)", v, v) + } + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int, c2 int);") + tk.MustExec(insertStr) + tk.MustExec("set tidb_mem_quota_query = 80000;") + tk.MustExec("set cte_max_recursion_depth = 500000;") + rows = tk.MustQuery("with recursive cte1 as ( " + + "select c1 from t1 " + + "union " + + "select c1 + 1 c1 from cte1 where c1 < 5000) " + + "select c1 from cte1 order by c1;") + + memTracker = tk.Se.GetSessionVars().StmtCtx.MemTracker + diskTracker = tk.Se.GetSessionVars().StmtCtx.DiskTracker + c.Assert(memTracker.MaxConsumed(), check.Greater, int64(0)) + c.Assert(diskTracker.MaxConsumed(), check.Greater, int64(0)) + + sort.Ints(vals) + resRows = make([]string, 0, rowNum) + for i := vals[0]; i <= rowNum; i++ { + resRows = append(resRows, fmt.Sprintf("%d", i)) + } + rows.Check(testkit.Rows(resRows...)) +} + +func (test *CTETestSuite) TestUnionDistinct(c *check.C) { + tk := testkit.NewTestKit(c, test.store) + tk.MustExec("use test;") + + // Basic test. UNION/UNION ALL intersects. + rows := tk.MustQuery("with recursive cte1(c1) as (select 1 union select 1 union select 1 union all select c1 + 1 from cte1 where c1 < 3) select * from cte1 order by c1;") + rows.Check(testkit.Rows("1", "2", "3")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union all select 1 union select 1 union all select c1 + 1 from cte1 where c1 < 3) select * from cte1 order by c1;") + rows.Check(testkit.Rows("1", "2", "3")) + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int, c2 int);") + tk.MustExec("insert into t1 values(1, 1), (1, 2), (2, 2);") + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from t1) select * from cte1 order by c1;") + rows.Check(testkit.Rows("1", "2", "3")) + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int);") + tk.MustExec("insert into t1 values(1), (1), (1), (2), (2), (2);") + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 where c1 < 4) select * from cte1 order by c1;") + rows.Check(testkit.Rows("1", "2", "3", "4")) +} + +func (test *CTETestSuite) TestCTEMaxRecursionDepth(c *check.C) { + tk := testkit.NewTestKit(c, test.store) + tk.MustExec("use test;") + + tk.MustExec("set @@cte_max_recursion_depth = -1;") + err := tk.QueryToErr("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 100) select * from cte1;") + c.Assert(err, check.NotNil) + c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + // If there is no recursive part, query runs ok. + rows := tk.MustQuery("with recursive cte1(c1) as (select 1 union select 2) select * from cte1 order by c1;") + rows.Check(testkit.Rows("1", "2")) + rows = tk.MustQuery("with cte1(c1) as (select 1 union select 2) select * from cte1 order by c1;") + rows.Check(testkit.Rows("1", "2")) + + tk.MustExec("set @@cte_max_recursion_depth = 0;") + err = tk.QueryToErr("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 0) select * from cte1;") + c.Assert(err, check.NotNil) + c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + err = tk.QueryToErr("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 1) select * from cte1;") + c.Assert(err, check.NotNil) + c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + // If there is no recursive part, query runs ok. + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select 2) select * from cte1 order by c1;") + rows.Check(testkit.Rows("1", "2")) + rows = tk.MustQuery("with cte1(c1) as (select 1 union select 2) select * from cte1 order by c1;") + rows.Check(testkit.Rows("1", "2")) + + tk.MustExec("set @@cte_max_recursion_depth = 1;") + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 0) select * from cte1;") + rows.Check(testkit.Rows("1")) + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 1) select * from cte1;") + rows.Check(testkit.Rows("1")) + err = tk.QueryToErr("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 2) select * from cte1;") + c.Assert(err, check.NotNil) + c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 2 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + // If there is no recursive part, query runs ok. + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select 2) select * from cte1 order by c1;") + rows.Check(testkit.Rows("1", "2")) + rows = tk.MustQuery("with cte1(c1) as (select 1 union select 2) select * from cte1 order by c1;") + rows.Check(testkit.Rows("1", "2")) +} diff --git a/executor/executor.go b/executor/executor.go index f9cffcdfdba54..ecb9102ba2f39 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1629,10 +1629,11 @@ func (e *UnionExec) Close() error { func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars := ctx.GetSessionVars() sc := &stmtctx.StatementContext{ - TimeZone: vars.Location(), - MemTracker: memory.NewTracker(memory.LabelForSQLText, vars.MemQuotaQuery), - DiskTracker: disk.NewTracker(memory.LabelForSQLText, -1), - TaskID: stmtctx.AllocateTaskID(), + TimeZone: vars.Location(), + MemTracker: memory.NewTracker(memory.LabelForSQLText, vars.MemQuotaQuery), + DiskTracker: disk.NewTracker(memory.LabelForSQLText, -1), + TaskID: stmtctx.AllocateTaskID(), + CTEStorageMap: map[int]*CTEStorages{}, } sc.MemTracker.AttachToGlobalTracker(GlobalMemoryUsageTracker) globalConfig := config.GetGlobalConfig() diff --git a/session/session.go b/session/session.go index 6e177f5cca368..0b8dce4ef9a22 100644 --- a/session/session.go +++ b/session/session.go @@ -1680,8 +1680,40 @@ type execStmtResult struct { func (rs *execStmtResult) Close() error { se := rs.se - err := rs.RecordSet.Close() - return finishStmt(context.Background(), se, err, rs.sql) + if err := resetCTEStorageMap(se); err != nil { + return finishStmt(context.Background(), se, err, rs.sql) + } + if err := rs.RecordSet.Close(); err != nil { + return finishStmt(context.Background(), se, err, rs.sql) + } + return finishStmt(context.Background(), se, nil, rs.sql) +} + +func resetCTEStorageMap(se *session) error { + tmp := se.GetSessionVars().StmtCtx.CTEStorageMap + if tmp == nil { + // Close() is already called, so no need to reset. Such as TraceExec. + return nil + } + storageMap, ok := tmp.(map[int]*executor.CTEStorages) + if !ok { + return errors.New("type assertion for CTEStorageMap failed") + } + for _, v := range storageMap { + // No need to lock IterInTbl. + v.ResTbl.Lock() + defer v.ResTbl.Unlock() + err1 := v.ResTbl.DerefAndClose() + err2 := v.IterInTbl.DerefAndClose() + if err1 != nil { + return err1 + } + if err2 != nil { + return err2 + } + } + se.GetSessionVars().StmtCtx.CTEStorageMap = nil + return nil } // rollbackOnError makes sure the next statement starts a new transaction with the latest InfoSchema. diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index ea8bd70b8c0f2..23fc0f52664f6 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -168,6 +168,9 @@ type StatementContext struct { stmtCache map[StmtCacheKey]interface{} // resourceGroupTag cache for the current statement resource group tag. resourceGroupTag atomic.Value + // Map to store all CTE storages of current SQL. + // Will clean up at the end of the execution. + CTEStorageMap interface{} } // StmtHints are SessionVars related sql hints. diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index e91cff2559d79..a15279e1a7ada 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -291,6 +291,21 @@ func (c *Chunk) CopyConstruct() *Chunk { return newChk } +// CopyConstructSel is just like CopyConstruct, +// but ignore the rows that was not selected. +func (c *Chunk) CopyConstructSel() *Chunk { + if c.sel == nil { + return c.CopyConstruct() + } + newChk := renewWithCapacity(c, c.capacity, c.requiredRows) + for colIdx, dstCol := range newChk.columns { + for _, rowIdx := range c.sel { + appendCellByCell(dstCol, c.columns[colIdx], rowIdx) + } + } + return newChk +} + // GrowAndReset resets the Chunk and doubles the capacity of the Chunk. // The doubled capacity should not be larger than maxChunkSize. // TODO: this method will be used in following PR. diff --git a/util/chunk/disk.go b/util/chunk/disk.go index 47e6b25c0b492..a9f7eecec3641 100644 --- a/util/chunk/disk.go +++ b/util/chunk/disk.go @@ -14,7 +14,6 @@ package chunk import ( - "errors" "io" "os" "strconv" @@ -132,7 +131,7 @@ func (l *ListInDisk) flush() (err error) { // Warning: do not mix Add and GetRow (always use GetRow after you have added all the chunks), and do not use Add concurrently. func (l *ListInDisk) Add(chk *Chunk) (err error) { if chk.NumRows() == 0 { - return errors.New("chunk appended to List should have at least 1 row") + return errors2.New("chunk appended to List should have at least 1 row") } if l.disk == nil { err = l.initDiskFile() diff --git a/util/cteutil/storage.go b/util/cteutil/storage.go index 3397e81fa7c05..9d42b1a11c015 100644 --- a/util/cteutil/storage.go +++ b/util/cteutil/storage.go @@ -51,6 +51,7 @@ type Storage interface { Reopen() error // Add chunk into underlying storage. + // Should return directly if chk is empty. Add(chk *chunk.Chunk) error // Get Chunk by index. @@ -84,7 +85,7 @@ type Storage interface { GetMemTracker() *memory.Tracker GetDiskTracker() *disk.Tracker - ActionSpill() memory.ActionOnExceed + ActionSpill() *chunk.SpillDiskAction } // StorageRC implements Storage interface using RowContainer. @@ -101,8 +102,8 @@ type StorageRC struct { rc *chunk.RowContainer } -// NewStorageRC create a new StorageRC. -func NewStorageRC(tp []*types.FieldType, chkSize int) *StorageRC { +// NewStorageRowContainer create a new StorageRC. +func NewStorageRowContainer(tp []*types.FieldType, chkSize int) *StorageRC { return &StorageRC{tp: tp, chkSize: chkSize} } @@ -245,7 +246,7 @@ func (s *StorageRC) GetDiskTracker() *memory.Tracker { } // ActionSpill impls Storage ActionSpill interface. -func (s *StorageRC) ActionSpill() memory.ActionOnExceed { +func (s *StorageRC) ActionSpill() *chunk.SpillDiskAction { return s.rc.ActionSpill() } diff --git a/util/cteutil/storage_test.go b/util/cteutil/storage_test.go index 89376fc8580b8..0e494978f2f84 100644 --- a/util/cteutil/storage_test.go +++ b/util/cteutil/storage_test.go @@ -35,7 +35,7 @@ type StorageRCTestSuite struct{} func (test *StorageRCTestSuite) TestStorageBasic(c *check.C) { fields := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} chkSize := 1 - storage := NewStorageRC(fields, chkSize) + storage := NewStorageRowContainer(fields, chkSize) c.Assert(storage, check.NotNil) // Close before open. @@ -67,7 +67,7 @@ func (test *StorageRCTestSuite) TestStorageBasic(c *check.C) { func (test *StorageRCTestSuite) TestOpenAndClose(c *check.C) { fields := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} chkSize := 1 - storage := NewStorageRC(fields, chkSize) + storage := NewStorageRowContainer(fields, chkSize) for i := 0; i < 10; i++ { err := storage.OpenAndRef() @@ -89,7 +89,7 @@ func (test *StorageRCTestSuite) TestAddAndGetChunk(c *check.C) { fields := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} chkSize := 10 - storage := NewStorageRC(fields, chkSize) + storage := NewStorageRowContainer(fields, chkSize) inChk := chunk.NewChunkWithCapacity(fields, chkSize) for i := 0; i < chkSize; i++ { @@ -117,7 +117,7 @@ func (test *StorageRCTestSuite) TestAddAndGetChunk(c *check.C) { func (test *StorageRCTestSuite) TestSpillToDisk(c *check.C) { fields := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} chkSize := 10 - storage := NewStorageRC(fields, chkSize) + storage := NewStorageRowContainer(fields, chkSize) var tmp interface{} = storage inChk := chunk.NewChunkWithCapacity(fields, chkSize) @@ -171,7 +171,7 @@ func (test *StorageRCTestSuite) TestSpillToDisk(c *check.C) { func (test *StorageRCTestSuite) TestReopen(c *check.C) { fields := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} chkSize := 10 - storage := NewStorageRC(fields, chkSize) + storage := NewStorageRowContainer(fields, chkSize) err := storage.OpenAndRef() c.Assert(err, check.IsNil) @@ -216,7 +216,7 @@ func (test *StorageRCTestSuite) TestReopen(c *check.C) { func (test *StorageRCTestSuite) TestSwapData(c *check.C) { tp1 := []*types.FieldType{types.NewFieldType(mysql.TypeLong)} chkSize := 10 - storage1 := NewStorageRC(tp1, chkSize) + storage1 := NewStorageRowContainer(tp1, chkSize) err := storage1.OpenAndRef() c.Assert(err, check.IsNil) inChk1 := chunk.NewChunkWithCapacity(tp1, chkSize) @@ -228,7 +228,7 @@ func (test *StorageRCTestSuite) TestSwapData(c *check.C) { c.Assert(err, check.IsNil) tp2 := []*types.FieldType{types.NewFieldType(mysql.TypeVarString)} - storage2 := NewStorageRC(tp2, chkSize) + storage2 := NewStorageRowContainer(tp2, chkSize) err = storage2.OpenAndRef() c.Assert(err, check.IsNil) diff --git a/util/memory/tracker.go b/util/memory/tracker.go index 3c369724c229e..2525ee76c2e0a 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -491,4 +491,6 @@ const ( LabelForApplyCache int = -17 // LabelForSimpleTask represents the label of the simple task LabelForSimpleTask int = -18 + // LabelForCTEStorage represents the label of CTE storage + LabelForCTEStorage int = -19 ) From d435129d015463088e5b37ea1ca2e8e6e738a9a2 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Tue, 1 Jun 2021 16:29:38 +0800 Subject: [PATCH 226/343] executor: make set transaction as of timestamp supports select statement (#24984) --- executor/set.go | 41 ++++++++---- executor/simple.go | 22 ++----- executor/stale_txn_test.go | 110 +++++++++++++++++++++++++++++--- planner/core/errors.go | 1 + planner/core/planbuilder.go | 3 + planner/core/preprocess.go | 9 +++ session/session.go | 4 +- sessionctx/variable/session.go | 53 ++++++++++++++- sessionctx/variable/varsutil.go | 7 +- 9 files changed, 206 insertions(+), 44 deletions(-) diff --git a/executor/set.go b/executor/set.go index 1223f973f5e39..4ab6878a21d08 100644 --- a/executor/set.go +++ b/executor/set.go @@ -134,7 +134,22 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e if err != nil { return err } - oldSnapshotTS := sessionVars.SnapshotTS + getSnapshotTSByName := func() uint64 { + if name == variable.TiDBSnapshot { + return sessionVars.SnapshotTS + } else if name == variable.TiDBTxnReadTS { + return sessionVars.TxnReadTS.PeakTxnReadTS() + } + return 0 + } + oldSnapshotTS := getSnapshotTSByName() + fallbackOldSnapshotTS := func() { + if name == variable.TiDBSnapshot { + sessionVars.SnapshotTS = oldSnapshotTS + } else if name == variable.TiDBTxnReadTS { + sessionVars.TxnReadTS.SetTxnReadTS(oldSnapshotTS) + } + } if name == variable.TxnIsolationOneShot && sessionVars.InTxn() { return errors.Trace(ErrCantChangeTxCharacteristics) } @@ -142,17 +157,18 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e if err != nil { return err } - newSnapshotIsSet := sessionVars.SnapshotTS > 0 && sessionVars.SnapshotTS != oldSnapshotTS + newSnapshotTS := getSnapshotTSByName() + newSnapshotIsSet := newSnapshotTS > 0 && newSnapshotTS != oldSnapshotTS if newSnapshotIsSet { - err = gcutil.ValidateSnapshot(e.ctx, sessionVars.SnapshotTS) + err = gcutil.ValidateSnapshot(e.ctx, newSnapshotTS) if err != nil { - sessionVars.SnapshotTS = oldSnapshotTS + fallbackOldSnapshotTS() return err } } - err = e.loadSnapshotInfoSchemaIfNeeded(name) + err = e.loadSnapshotInfoSchemaIfNeeded(newSnapshotTS) if err != nil { - sessionVars.SnapshotTS = oldSnapshotTS + fallbackOldSnapshotTS() return err } // Clients are often noisy in setting session variables such as @@ -248,18 +264,17 @@ func (e *SetExecutor) getVarValue(v *expression.VarAssignment, sysVar *variable. return nativeVal.ToString() } -func (e *SetExecutor) loadSnapshotInfoSchemaIfNeeded(name string) error { - if name != variable.TiDBSnapshot { - return nil - } +func (e *SetExecutor) loadSnapshotInfoSchemaIfNeeded(snapshotTS uint64) error { vars := e.ctx.GetSessionVars() - if vars.SnapshotTS == 0 { + if snapshotTS == 0 { vars.SnapshotInfoschema = nil return nil } - logutil.BgLogger().Info("load snapshot info schema", zap.Uint64("conn", vars.ConnectionID), zap.Uint64("SnapshotTS", vars.SnapshotTS)) + logutil.BgLogger().Info("load snapshot info schema", + zap.Uint64("conn", vars.ConnectionID), + zap.Uint64("SnapshotTS", snapshotTS)) dom := domain.GetDomain(e.ctx) - snapInfo, err := dom.GetSnapshotInfoSchema(vars.SnapshotTS) + snapInfo, err := dom.GetSnapshotInfoSchema(snapshotTS) if err != nil { return err } diff --git a/executor/simple.go b/executor/simple.go index 4bbb2c3903b0a..95067f42621d0 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -575,31 +575,21 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { } if s.AsOf != nil { // start transaction read only as of failed due to we set tx_read_ts before - if e.ctx.GetSessionVars().TxnReadTS > 0 { + if e.ctx.GetSessionVars().TxnReadTS.PeakTxnReadTS() > 0 { return errors.New("start transaction read only as of is forbidden after set transaction read only as of") } - if err := e.ctx.NewStaleTxnWithStartTS(ctx, e.staleTxnStartTS); err != nil { - return err - } - // With START TRANSACTION, autocommit remains disabled until you end - // the transaction with COMMIT or ROLLBACK. The autocommit mode then - // reverts to its previous state. - e.ctx.GetSessionVars().SetInTxn(true) - return nil } } - // When TxnReadTS is not 0, it indicates the transaction is staleness transaction - if e.ctx.GetSessionVars().TxnReadTS > 0 { - startTS := e.ctx.GetSessionVars().TxnReadTS - // clear TxnReadTS after we used it. - e.ctx.GetSessionVars().TxnReadTS = 0 - if err := e.ctx.NewStaleTxnWithStartTS(ctx, startTS); err != nil { + if e.staleTxnStartTS > 0 { + if err := e.ctx.NewStaleTxnWithStartTS(ctx, e.staleTxnStartTS); err != nil { return err } + // With START TRANSACTION, autocommit remains disabled until you end + // the transaction with COMMIT or ROLLBACK. The autocommit mode then + // reverts to its previous state. e.ctx.GetSessionVars().SetInTxn(true) return nil } - // If BEGIN is the first statement in TxnCtx, we can reuse the existing transaction, without the // need to call NewTxn, which commits the existing transaction and begins a new one. // If the last un-committed/un-rollback transaction is a time-bounded read-only transaction, we should diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index b90ece5b77224..9fd0a9f83d6f3 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -97,6 +97,14 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { func (s *testStaleTxnSerialSuite) TestSelectAsOf(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") + // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. + safePointName := "tikv_gc_safe_point" + safePointValue := "20160102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) tk.MustExec("drop table if exists t") tk.MustExec(`drop table if exists b`) tk.MustExec("create table t (id int primary key);") @@ -110,6 +118,7 @@ func (s *testStaleTxnSerialSuite) TestSelectAsOf(c *C) { time.Sleep(2 * time.Second) testcases := []struct { + setTxnSQL string name string sql string expectPhysicalTS int64 @@ -117,13 +126,25 @@ func (s *testStaleTxnSerialSuite) TestSelectAsOf(c *C) { // IsStaleness is auto cleanup in select stmt. errorStr string }{ + { + name: "set transaction as of", + setTxnSQL: fmt.Sprintf("set transaction read only as of timestamp '%s';", now.Format("2006-1-2 15:04:05")), + sql: "select * from t;", + expectPhysicalTS: now.Unix(), + }, + { + name: "set transaction as of, expect error", + setTxnSQL: fmt.Sprintf("set transaction read only as of timestamp '%s';", now.Format("2006-1-2 15:04:05")), + sql: fmt.Sprintf("select * from t as of timestamp '%s';", now.Format("2006-1-2 15:04:05")), + errorStr: ".*can't use select as of while already set transaction as of.*", + }, { name: "TimestampExactRead1", sql: fmt.Sprintf("select * from t as of timestamp '%s';", now.Format("2006-1-2 15:04:05")), expectPhysicalTS: now.Unix(), }, { - name: "NomalRead", + name: "NormalRead", sql: `select * from b;`, preSec: 0, }, @@ -187,6 +208,9 @@ func (s *testStaleTxnSerialSuite) TestSelectAsOf(c *C) { for _, testcase := range testcases { c.Log(testcase.name) + if len(testcase.setTxnSQL) > 0 { + tk.MustExec(testcase.setTxnSQL) + } if testcase.expectPhysicalTS > 0 { c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/assertStaleTSO", fmt.Sprintf(`return(%d)`, testcase.expectPhysicalTS)), IsNil) } else if testcase.preSec > 0 { @@ -203,6 +227,9 @@ func (s *testStaleTxnSerialSuite) TestSelectAsOf(c *C) { } else if testcase.preSec > 0 { c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/assertStaleTSOWithTolerance"), IsNil) } + if len(testcase.setTxnSQL) > 0 { + c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) + } } } @@ -281,14 +308,13 @@ func (s *testStaleTxnSuite) TestStalenessAndHistoryRead(c *C) { c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, NotNil) tk.MustExec("SET TRANSACTION READ ONLY AS OF TIMESTAMP '2020-10-08 16:46:26'") c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(0)) - c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, IsNil) - c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, uint64(419993167069184000)) + c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(419993167069184000)) tk.MustExec("SET TRANSACTION READ ONLY AS OF TIMESTAMP '2020-10-08 16:46:26'") - c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, uint64(419993167069184000)) + c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(419993167069184000)) tk.MustExec(`set @@tidb_snapshot="2020-10-08 16:45:26";`) c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(419993151340544000)) - c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, uint64(0)) + c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) c.Assert(tk.Se.GetSessionVars().SnapshotInfoschema, NotNil) } @@ -388,6 +414,14 @@ func (s *testStaleTxnSerialSuite) TestSetTransactionReadOnlyAsOf(c *C) { t1, err := time.Parse(types.TimeFormat, "2016-09-21 09:53:04") c.Assert(err, IsNil) tk := testkit.NewTestKit(c, s.store) + // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. + safePointName := "tikv_gc_safe_point" + safePointValue := "20160102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) testcases := []struct { sql string expectedTS uint64 @@ -410,11 +444,11 @@ func (s *testStaleTxnSerialSuite) TestSetTransactionReadOnlyAsOf(c *C) { fmt.Sprintf("return(%v)", testcase.injectSafeTS)), IsNil) } tk.MustExec(testcase.sql) - c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, testcase.expectedTS) + c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, testcase.expectedTS) tk.MustExec("begin") - c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, uint64(0)) c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Equals, testcase.expectedTS) tk.MustExec("commit") + c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) tk.MustExec("begin") c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Not(Equals), testcase.expectedTS) @@ -423,12 +457,13 @@ func (s *testStaleTxnSerialSuite) TestSetTransactionReadOnlyAsOf(c *C) { err = tk.ExecToErr(`SET TRANSACTION READ ONLY as of timestamp tidb_bounded_staleness(invalid1, invalid2')`) c.Assert(err, NotNil) - c.Assert(tk.Se.GetSessionVars().TxnReadTS, Equals, uint64(0)) + c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) tk.MustExec(`SET TRANSACTION READ ONLY as of timestamp '2021-04-21 00:42:12'`) err = tk.ExecToErr(`START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00'`) c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "start transaction read only as of is forbidden after set transaction read only as of") + tk.MustExec(`SET TRANSACTION READ ONLY as of timestamp '2021-04-21 00:42:12'`) } func (s *testStaleTxnSerialSuite) TestValidateReadOnlyInStalenessTransaction(c *C) { @@ -554,6 +589,14 @@ func (s *testStaleTxnSerialSuite) TestValidateReadOnlyInStalenessTransaction(c * }, } tk := testkit.NewTestKit(c, s.store) + // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. + safePointName := "tikv_gc_safe_point" + safePointValue := "20160102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) tk.MustExec("use test") tk.MustExec("create table t (id int);") tk.MustExec("create table t1 (id int);") @@ -566,12 +609,22 @@ func (s *testStaleTxnSerialSuite) TestValidateReadOnlyInStalenessTransaction(c * if testcase.isValidate { _, err := tk.Exec(testcase.sql) c.Assert(err, IsNil) - tk.MustExec("commit") } else { err := tk.ExecToErr(testcase.sql) c.Assert(err, NotNil) c.Assert(err.Error(), Matches, `.*only support read-only statement during read-only staleness transactions.*`) } + tk.MustExec("commit") + tk.MustExec("set transaction read only as of timestamp NOW(3);") + if testcase.isValidate { + _, err := tk.Exec(testcase.sql) + c.Assert(err, IsNil) + } else { + err := tk.ExecToErr(testcase.sql) + c.Assert(err, NotNil) + c.Assert(err.Error(), Matches, `.*only support read-only statement during read-only staleness transactions.*`) + } + tk.MustExec("set transaction read only as of timestamp ''") } } @@ -628,3 +681,42 @@ func (s *testStaleTxnSuite) TestSpecialSQLInStalenessTxn(c *C) { c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.sameSession, comment) } } + +func (s *testStaleTxnSuite) TestSetTransactionInfoSchema(c *C) { + tk := testkit.NewTestKit(c, s.store) + // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. + safePointName := "tikv_gc_safe_point" + safePointValue := "20160102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + defer tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id int primary key);") + + schemaVer1 := tk.Se.GetInfoSchema().SchemaMetaVersion() + time1 := time.Now() + tk.MustExec("alter table t add c int") + + // confirm schema changed + schemaVer2 := tk.Se.GetInfoSchema().SchemaMetaVersion() + time2 := time.Now() + c.Assert(schemaVer1, Less, schemaVer2) + tk.MustExec(fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) + c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) + tk.MustExec("select * from t;") + tk.MustExec("alter table t add d int") + schemaVer3 := tk.Se.GetInfoSchema().SchemaMetaVersion() + tk.MustExec(fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) + tk.MustExec("begin;") + c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) + tk.MustExec("commit") + tk.MustExec(fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time2.Format("2006-1-2 15:04:05.000"))) + tk.MustExec("begin;") + c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) + tk.MustExec("commit") + c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer3) +} diff --git a/planner/core/errors.go b/planner/core/errors.go index c1bb613cda005..dd457e0ca4efe 100644 --- a/planner/core/errors.go +++ b/planner/core/errors.go @@ -98,5 +98,6 @@ var ( ErrBadNull = dbterror.ClassOptimizer.NewStd(mysql.ErrBadNull) ErrNotSupportedWithSem = dbterror.ClassOptimizer.NewStd(mysql.ErrNotSupportedWithSem) ErrDifferentAsOf = dbterror.ClassOptimizer.NewStd(mysql.ErrUnknown) + ErrAsOf = dbterror.ClassOptimizer.NewStd(mysql.ErrUnknown) ErrOptOnTemporaryTable = dbterror.ClassOptimizer.NewStd(mysql.ErrOptOnTemporaryTable) ) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 2189f645e32b1..f2de374321f25 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2370,12 +2370,15 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, case *ast.ShutdownStmt: b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShutdownPriv, "", "", "", nil) case *ast.BeginStmt: + readTS := b.ctx.GetSessionVars().TxnReadTS.UseTxnReadTS() if raw.AsOf != nil { startTS, err := calculateTsExpr(b.ctx, raw.AsOf) if err != nil { return nil, err } p.StaleTxnStartTS = startTS + } else if readTS > 0 { + p.StaleTxnStartTS = readTS } } return p, nil diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 29ad5c848dbeb..635a53d72da72 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -101,6 +101,10 @@ func Preprocess(ctx sessionctx.Context, node ast.Node, preprocessOpt ...Preproce v.PreprocessorReturn = &PreprocessorReturn{} } node.Accept(&v) + readTS := ctx.GetSessionVars().TxnReadTS.UseTxnReadTS() + if readTS > 0 { + v.PreprocessorReturn.SnapshotTS = readTS + } // InfoSchema must be non-nil after preprocessing if v.InfoSchema == nil { v.ensureInfoSchema() @@ -1377,6 +1381,11 @@ func (p *preprocessor) checkFuncCastExpr(node *ast.FuncCastExpr) { // handleAsOf tries to validate the timestamp. // If it is not nil, timestamp is used to get the history infoschema from the infocache. func (p *preprocessor) handleAsOf(node *ast.AsOfClause) { + readTS := p.ctx.GetSessionVars().TxnReadTS.PeakTxnReadTS() + if readTS > 0 && node != nil { + p.err = ErrAsOf.FastGenWithCause("can't use select as of while already set transaction as of") + return + } dom := domain.GetDomain(p.ctx) ts := uint64(0) if node != nil { diff --git a/session/session.go b/session/session.go index 0b8dce4ef9a22..b24461ba653b6 100644 --- a/session/session.go +++ b/session/session.go @@ -670,6 +670,7 @@ func (s *session) CommitTxn(ctx context.Context) error { } }) s.sessionVars.TxnCtx.Cleanup() + s.sessionVars.CleanupTxnReadTSIfUsed() return err } @@ -687,6 +688,7 @@ func (s *session) RollbackTxn(ctx context.Context) { } s.txn.changeToInvalid() s.sessionVars.TxnCtx.Cleanup() + s.sessionVars.CleanupTxnReadTSIfUsed() s.sessionVars.SetInTxn(false) } @@ -1554,7 +1556,7 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex func (s *session) validateStatementReadOnlyInStaleness(stmtNode ast.StmtNode) error { vars := s.GetSessionVars() - if !vars.TxnCtx.IsStaleness { + if !vars.TxnCtx.IsStaleness && vars.TxnReadTS.PeakTxnReadTS() == 0 { return nil } errMsg := "only support read-only statement during read-only staleness transactions" diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 2a8be21adec07..6e6d766bd8bf5 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -463,7 +463,7 @@ type SessionVars struct { SnapshotTS uint64 // TxnReadTS is used for staleness transaction, it provides next staleness transaction startTS. - TxnReadTS uint64 + TxnReadTS *TxnReadTS // SnapshotInfoschema is used with SnapshotTS, when the schema version at snapshotTS less than current schema // version, we load an old version schema for query. @@ -2055,3 +2055,54 @@ type QueryInfo struct { ForUpdateTS uint64 `json:"for_update_ts"` ErrMsg string `json:"error,omitempty"` } + +// TxnReadTS indicates the value and used situation for tx_read_ts +type TxnReadTS struct { + readTS uint64 + used bool +} + +// NewTxnReadTS creates TxnReadTS +func NewTxnReadTS(ts uint64) *TxnReadTS { + return &TxnReadTS{ + readTS: ts, + used: false, + } +} + +// UseTxnReadTS returns readTS, and mark used as true +func (t *TxnReadTS) UseTxnReadTS() uint64 { + if t == nil { + return 0 + } + t.used = true + return t.readTS +} + +// SetTxnReadTS update readTS, and refresh used +func (t *TxnReadTS) SetTxnReadTS(ts uint64) { + if t == nil { + return + } + t.used = false + t.readTS = ts +} + +// PeakTxnReadTS returns readTS +func (t *TxnReadTS) PeakTxnReadTS() uint64 { + if t == nil { + return 0 + } + return t.readTS +} + +// CleanupTxnReadTSIfUsed cleans txnReadTS if used +func (s *SessionVars) CleanupTxnReadTSIfUsed() { + if s.TxnReadTS == nil { + return + } + if s.TxnReadTS.used && s.TxnReadTS.readTS > 0 { + s.TxnReadTS = NewTxnReadTS(0) + s.SnapshotInfoschema = nil + } +} diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 094f3cbfb2ed5..4447606508beb 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -375,13 +375,13 @@ func setSnapshotTS(s *SessionVars, sVal string) error { t1, err := t.GoTime(s.TimeZone) s.SnapshotTS = oracle.GoTimeToTS(t1) // tx_read_ts should be mutual exclusive with tidb_snapshot - s.TxnReadTS = 0 + s.TxnReadTS = NewTxnReadTS(0) return err } func setTxnReadTS(s *SessionVars, sVal string) error { if sVal == "" { - s.TxnReadTS = 0 + s.TxnReadTS = NewTxnReadTS(0) return nil } t, err := types.ParseTime(s.StmtCtx, sVal, mysql.TypeTimestamp, types.MaxFsp) @@ -392,10 +392,9 @@ func setTxnReadTS(s *SessionVars, sVal string) error { if err != nil { return err } - s.TxnReadTS = oracle.GoTimeToTS(t1) + s.TxnReadTS = NewTxnReadTS(oracle.GoTimeToTS(t1)) // tx_read_ts should be mutual exclusive with tidb_snapshot s.SnapshotTS = 0 - s.SnapshotInfoschema = nil return err } From 5d7c852770eb319df0eff575463c2f055d967555 Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Tue, 1 Jun 2021 16:55:37 +0800 Subject: [PATCH 227/343] planner: keep the original join schema in predicate pushdown (#24862) --- cmd/explaintest/r/subquery.result | 5 +++++ cmd/explaintest/t/subquery.test | 5 +++++ planner/core/integration_test.go | 9 +++++++++ planner/core/logical_plan_test.go | 2 +- planner/core/rule_predicate_push_down.go | 1 - 5 files changed, 20 insertions(+), 2 deletions(-) diff --git a/cmd/explaintest/r/subquery.result b/cmd/explaintest/r/subquery.result index 0b054d45ba2a3..2bf71b91088cb 100644 --- a/cmd/explaintest/r/subquery.result +++ b/cmd/explaintest/r/subquery.result @@ -41,3 +41,8 @@ HashJoin 7992.00 root inner join, equal:[eq(test.t.b, test.t.b) eq(test.t.c, te └─TableReader(Probe) 9970.03 root data:Selection └─Selection 9970.03 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.b)), not(isnull(test.t.c)) └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +drop table if exists t1, t2; +create table t1(a int(11)); +create table t2(a decimal(40,20) unsigned, b decimal(40,20)); +select count(*) as x from t1 group by a having x not in (select a from t2 where x = t2.b); +x diff --git a/cmd/explaintest/t/subquery.test b/cmd/explaintest/t/subquery.test index 8d9bd730b767c..6a3aa13e7e95a 100644 --- a/cmd/explaintest/t/subquery.test +++ b/cmd/explaintest/t/subquery.test @@ -15,3 +15,8 @@ explain format = 'brief' select t.c in (select count(*) from t s use index(idx), drop table if exists t; create table t(a int, b int, c int); explain format = 'brief' select a from t t1 where t1.a = (select max(t2.a) from t t2 where t1.b=t2.b and t1.c=t2.b); + +drop table if exists t1, t2; +create table t1(a int(11)); +create table t2(a decimal(40,20) unsigned, b decimal(40,20)); +select count(*) as x from t1 group by a having x not in (select a from t2 where x = t2.b); diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 3a2489a803cc8..1e13d28b96e1c 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3429,6 +3429,15 @@ func (s *testIntegrationSuite) TestIssue22850(c *C) { tk.MustQuery("SELECT @v:=(SELECT 1 FROM t1 t2 LEFT JOIN t1 ON t1.a GROUP BY t1.a) FROM t1").Check(testkit.Rows()) // work fine } +func (s *testIntegrationSuite) TestJoinSchemaChange(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2") + tk.MustExec("create table t1(a int(11))") + tk.MustExec("create table t2(a decimal(40,20) unsigned, b decimal(40,20))") + tk.MustQuery("select count(*) as x from t1 group by a having x not in (select a from t2 where x = t2.b)").Check(testkit.Rows()) +} + // #22949: test HexLiteral Used in GetVar expr func (s *testIntegrationSuite) TestGetVarExprWithHexLiteral(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 05f51d517393f..f59aceb40ee86 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1794,7 +1794,7 @@ func (s *testPlanSuite) TestUpdateEQCond(c *C) { }{ { sql: "select t1.a from t t1, t t2 where t1.a = t2.a+1", - best: "Join{DataScan(t1)->DataScan(t2)->Projection}(test.t.a,Column#25)->Projection", + best: "Join{DataScan(t1)->DataScan(t2)->Projection}(test.t.a,Column#25)->Projection->Projection", }, } ctx := context.TODO() diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index a3c6737db74d7..b591343870ce4 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -208,7 +208,6 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression) (ret addSelection(p, lCh, leftRet, 0) addSelection(p, rCh, rightRet, 1) p.updateEQCond() - p.mergeSchema() buildKeyInfo(p) return ret, p.self } From 0158f71cda6ccbbd382c6e8759904234baca259c Mon Sep 17 00:00:00 2001 From: tangenta Date: Tue, 1 Jun 2021 18:59:37 +0800 Subject: [PATCH 228/343] ddl: support type conversion between non-varchar and varchar (#24959) --- ddl/column.go | 14 +++ ddl/column_test.go | 2 + ddl/column_type_change_test.go | 213 +++++++++++++++++++++------------ ddl/db_test.go | 8 +- ddl/ddl_api.go | 10 +- executor/ddl_test.go | 6 +- 6 files changed, 168 insertions(+), 85 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 17abe6aefbe08..7d93c4e15668d 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" decoder "github.com/pingcap/tidb/util/rowDecoder" "github.com/pingcap/tidb/util/sqlexec" @@ -712,6 +713,10 @@ func needChangeColumnData(oldCol, newCol *model.ColumnInfo) bool { return needTruncationOrToggleSign() } + if convertBetweenCharAndVarchar(oldCol.Tp, newCol.Tp) { + return true + } + // Deal with the different type. switch oldCol.Tp { case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeBlob, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: @@ -734,6 +739,15 @@ func needChangeColumnData(oldCol, newCol *model.ColumnInfo) bool { return true } +// Column type conversion between varchar to char need reorganization because +// 1. varchar -> char: char type is stored with the padding removed. All the indexes need to be rewritten. +// 2. char -> varchar: the index value encoding of secondary index on clustered primary key tables is different. +// These secondary indexes need to be rewritten. +func convertBetweenCharAndVarchar(oldCol, newCol byte) bool { + return (types.IsTypeVarchar(oldCol) && newCol == mysql.TypeString) || + (oldCol == mysql.TypeString && types.IsTypeVarchar(newCol) && collate.NewCollationEnabled()) +} + func isElemsChangedToModifyColumn(oldElems, newElems []string) bool { if len(newElems) < len(oldElems) { return true diff --git a/ddl/column_test.go b/ddl/column_test.go index f3eaa26d22385..10b883a76935a 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -1170,6 +1170,8 @@ func (s *testColumnSuite) TestModifyColumn(c *C) { }{ {"int", "bigint", nil}, {"int", "int unsigned", errUnsupportedModifyColumn.GenWithStackByArgs("can't change unsigned integer to signed or vice versa, and tidb_enable_change_column_type is false")}, + {"varchar(10)", "text", nil}, + {"varbinary(10)", "blob", nil}, {"text", "blob", errUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8mb4 to binary")}, {"varchar(10)", "varchar(8)", errUnsupportedModifyColumn.GenWithStackByArgs("length 8 is less than origin 10, and tidb_enable_change_column_type is false")}, {"varchar(10)", "varchar(11)", nil}, diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 0b6234a21d924..32418dd031c29 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/testkit" ) @@ -317,10 +318,14 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromIntegerToOthers(c *C // integer to string prepare(tk) - tk.MustGetErrCode("alter table t modify a varchar(10)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify a varchar(10)") + modifiedColumn := getModifyColumn(c, tk.Se, "test", "t", "a", false) + c.Assert(modifiedColumn, NotNil) + c.Assert(modifiedColumn.Tp, Equals, parser_mysql.TypeVarchar) + tk.MustQuery("select a from t").Check(testkit.Rows("1")) tk.MustExec("alter table t modify b char(10)") - modifiedColumn := getModifyColumn(c, tk.Se, "test", "t", "b", false) + modifiedColumn = getModifyColumn(c, tk.Se, "test", "t", "b", false) c.Assert(modifiedColumn, NotNil) c.Assert(modifiedColumn.Tp, Equals, parser_mysql.TypeString) tk.MustQuery("select b from t").Check(testkit.Rows("11")) @@ -331,7 +336,11 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromIntegerToOthers(c *C c.Assert(modifiedColumn.Tp, Equals, parser_mysql.TypeString) tk.MustQuery("select c from t").Check(testkit.Rows("111\x00\x00\x00\x00\x00\x00\x00")) - tk.MustGetErrCode("alter table t modify d varbinary(10)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify d varbinary(10)") + modifiedColumn = getModifyColumn(c, tk.Se, "test", "t", "d", false) + c.Assert(modifiedColumn, NotNil) + c.Assert(modifiedColumn.Tp, Equals, parser_mysql.TypeVarchar) + tk.MustQuery("select d from t").Check(testkit.Rows("1111")) tk.MustExec("alter table t modify e blob(10)") modifiedColumn = getModifyColumn(c, tk.Se, "test", "t", "e", false) @@ -445,7 +454,9 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromIntegerToOthers(c *C func (s *testColumnTypeChangeSuite) TestColumnTypeChangeBetweenVarcharAndNonVarchar(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - + tk.Se.GetSessionVars().EnableChangeColumnType = false + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) tk.MustExec("drop database if exists col_type_change_char;") tk.MustExec("create database col_type_change_char;") tk.MustExec("use col_type_change_char;") @@ -461,6 +472,26 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeBetweenVarcharAndNonVarc tk.MustGetErrCode("alter table t change column a a varchar(10);", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t change column b b char(10);", mysql.ErrUnsupportedDDLOperation) tk.MustExec("admin check table t;") + + tk.Se.GetSessionVars().EnableChangeColumnType = true + // https://github.com/pingcap/tidb/issues/23624 + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(b varchar(10));") + tk.MustExec("insert into t values ('aaa ');") + tk.MustExec("alter table t change column b b char(10);") + tk.MustExec("alter table t add index idx(b);") + tk.MustExec("alter table t change column b b varchar(10);") + tk.MustQuery("select b from t use index(idx);").Check(testkit.Rows("aaa")) + tk.MustQuery("select b from t ignore index(idx);").Check(testkit.Rows("aaa")) + tk.MustExec("admin check table t;") + + // https://github.com/pingcap/tidb/pull/23688#issuecomment-810166597 + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a varchar(10));") + tk.MustExec("insert into t values ('aaa ');") + tk.MustQuery("select a from t;").Check(testkit.Rows("aaa ")) + tk.MustExec("alter table t modify column a char(10);") + tk.MustQuery("select a from t;").Check(testkit.Rows("aaa")) } func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromStringToOthers(c *C) { @@ -480,11 +511,12 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromStringToOthers(c *C) // Init string date type table. reset := func(tk *testkit.TestKit) { tk.MustExec("drop table if exists t") - // FIXME(tangenta): not support changing from varchar/varbinary to other types. tk.MustExec(` create table t ( c char(8), + vc varchar(8), bny binary(8), + vbny varbinary(8), bb blob, txt text, e enum('123', '2020-07-15 18:32:17.888', 'str', '{"k1": "value"}'), @@ -496,147 +528,181 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromStringToOthers(c *C) // To numeric data types. // tinyint reset(tk) - tk.MustExec("insert into t values ('123', '123', '123', '123', '123', '123')") + tk.MustExec("insert into t values ('123', '123', '123', '123', '123', '123', '123', '123')") tk.MustExec("alter table t modify c tinyint") + tk.MustExec("alter table t modify vc tinyint") tk.MustExec("alter table t modify bny tinyint") + tk.MustExec("alter table t modify vbny tinyint") tk.MustExec("alter table t modify bb tinyint") tk.MustExec("alter table t modify txt tinyint") tk.MustExec("alter table t modify e tinyint") tk.MustExec("alter table t modify s tinyint") - tk.MustQuery("select * from t").Check(testkit.Rows("123 123 123 123 1 1")) + tk.MustQuery("select * from t").Check(testkit.Rows("123 123 123 123 123 123 1 1")) // int reset(tk) - tk.MustExec("insert into t values ('17305', '17305', '17305', '17305', '123', '123')") + tk.MustExec("insert into t values ('17305', '17305', '17305', '17305', '17305', '17305', '123', '123')") tk.MustExec("alter table t modify c int") + tk.MustExec("alter table t modify vc int") tk.MustExec("alter table t modify bny int") + tk.MustExec("alter table t modify vbny int") tk.MustExec("alter table t modify bb int") tk.MustExec("alter table t modify txt int") tk.MustExec("alter table t modify e int") tk.MustExec("alter table t modify s int") - tk.MustQuery("select * from t").Check(testkit.Rows("17305 17305 17305 17305 1 1")) + tk.MustQuery("select * from t").Check(testkit.Rows("17305 17305 17305 17305 17305 17305 1 1")) // bigint reset(tk) - tk.MustExec("insert into t values ('17305867', '17305867', '17305867', '17305867', '123', '123')") + tk.MustExec("insert into t values ('17305867', '17305867', '17305867', '17305867', '17305867', '17305867', '123', '123')") tk.MustExec("alter table t modify c bigint") + tk.MustExec("alter table t modify vc bigint") tk.MustExec("alter table t modify bny bigint") + tk.MustExec("alter table t modify vbny bigint") tk.MustExec("alter table t modify bb bigint") tk.MustExec("alter table t modify txt bigint") tk.MustExec("alter table t modify e bigint") tk.MustExec("alter table t modify s bigint") - tk.MustQuery("select * from t").Check(testkit.Rows("17305867 17305867 17305867 17305867 1 1")) + tk.MustQuery("select * from t").Check(testkit.Rows("17305867 17305867 17305867 17305867 17305867 17305867 1 1")) // bit reset(tk) - tk.MustExec("insert into t values ('1', '1', '1', '1', '123', '123')") + tk.MustExec("insert into t values ('1', '1', '1', '1', '1', '1', '123', '123')") tk.MustGetErrCode("alter table t modify c bit", mysql.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("alter table t modify vc bit", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify bny bit", mysql.ErrUnsupportedDDLOperation) + tk.MustGetErrCode("alter table t modify vbny bit", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify bb bit", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify txt bit", mysql.ErrUnsupportedDDLOperation) tk.MustExec("alter table t modify e bit") tk.MustExec("alter table t modify s bit") - tk.MustQuery("select * from t").Check(testkit.Rows("1 1\x00\x00\x00\x00\x00\x00\x00 1 1 \x01 \x01")) + tk.MustQuery("select * from t").Check(testkit.Rows("1 1 1\x00\x00\x00\x00\x00\x00\x00 1 1 1 \x01 \x01")) // decimal reset(tk) - tk.MustExec("insert into t values ('123.45', '123.45', '123.45', '123.45', '123', '123')") + tk.MustExec("insert into t values ('123.45', '123.45', '123.45', '123.45', '123.45', '123.45', '123', '123')") tk.MustExec("alter table t modify c decimal(7, 4)") + tk.MustExec("alter table t modify vc decimal(7, 4)") tk.MustExec("alter table t modify bny decimal(7, 4)") + tk.MustExec("alter table t modify vbny decimal(7, 4)") tk.MustExec("alter table t modify bb decimal(7, 4)") tk.MustExec("alter table t modify txt decimal(7, 4)") tk.MustExec("alter table t modify e decimal(7, 4)") tk.MustExec("alter table t modify s decimal(7, 4)") - tk.MustQuery("select * from t").Check(testkit.Rows("123.4500 123.4500 123.4500 123.4500 1.0000 1.0000")) + tk.MustQuery("select * from t").Check(testkit.Rows("123.4500 123.4500 123.4500 123.4500 123.4500 123.4500 1.0000 1.0000")) // double reset(tk) - tk.MustExec("insert into t values ('123.45', '123.45', '123.45', '123.45', '123', '123')") + tk.MustExec("insert into t values ('123.45', '123.45', '123.45', '123.45', '123.45', '123.45', '123', '123')") tk.MustExec("alter table t modify c double(7, 4)") + tk.MustExec("alter table t modify vc double(7, 4)") tk.MustExec("alter table t modify bny double(7, 4)") + tk.MustExec("alter table t modify vbny double(7, 4)") tk.MustExec("alter table t modify bb double(7, 4)") tk.MustExec("alter table t modify txt double(7, 4)") tk.MustExec("alter table t modify e double(7, 4)") tk.MustExec("alter table t modify s double(7, 4)") - tk.MustQuery("select * from t").Check(testkit.Rows("123.45 123.45 123.45 123.45 1 1")) + tk.MustQuery("select * from t").Check(testkit.Rows("123.45 123.45 123.45 123.45 123.45 123.45 1 1")) // To date and time data types. // date reset(tk) - tk.MustExec("insert into t values ('20200826', '20200826', '2020-08-26', '08-26 19:35:41', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888')") + tk.MustExec("insert into t values ('20200826', '2008261', '20200826', '200826', '2020-08-26', '08-26 19:35:41', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888')") tk.MustExec("alter table t modify c date") + tk.MustExec("alter table t modify vc date") tk.MustExec("alter table t modify bny date") + tk.MustExec("alter table t modify vbny date") tk.MustExec("alter table t modify bb date") // Alter text '08-26 19:35:41' to date will error. (same as mysql does) tk.MustGetErrCode("alter table t modify txt date", mysql.ErrTruncatedWrongValue) tk.MustGetErrCode("alter table t modify e date", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify s date", mysql.ErrUnsupportedDDLOperation) - tk.MustQuery("select * from t").Check(testkit.Rows("2020-08-26 2020-08-26 2020-08-26 08-26 19:35:41 2020-07-15 18:32:17.888 2020-07-15 18:32:17.888")) + tk.MustQuery("select * from t").Check(testkit.Rows("2020-08-26 2020-08-26 2020-08-26 2020-08-26 2020-08-26 08-26 19:35:41 2020-07-15 18:32:17.888 2020-07-15 18:32:17.888")) // time reset(tk) - tk.MustExec("insert into t values ('19:35:41', '19:35:41', '19:35:41.45678', '19:35:41.45678', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888')") + tk.MustExec("insert into t values ('19:35:41', '19:35:41', '19:35:41', '19:35:41', '19:35:41.45678', '19:35:41.45678', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888')") tk.MustExec("alter table t modify c time") + tk.MustExec("alter table t modify vc time") tk.MustExec("alter table t modify bny time") + tk.MustExec("alter table t modify vbny time") tk.MustExec("alter table t modify bb time") tk.MustExec("alter table t modify txt time") tk.MustGetErrCode("alter table t modify e time", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify s time", mysql.ErrUnsupportedDDLOperation) - tk.MustQuery("select * from t").Check(testkit.Rows("19:35:41 19:35:41 19:35:41 19:35:41 2020-07-15 18:32:17.888 2020-07-15 18:32:17.888")) + tk.MustQuery("select * from t").Check(testkit.Rows("19:35:41 19:35:41 19:35:41 19:35:41 19:35:41 19:35:41 2020-07-15 18:32:17.888 2020-07-15 18:32:17.888")) // datetime reset(tk) tk.MustExec("alter table t modify c char(23)") + tk.MustExec("alter table t modify vc varchar(23)") tk.MustExec("alter table t modify bny binary(23)") - tk.MustExec("insert into t values ('2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888')") + tk.MustExec("alter table t modify vbny varbinary(23)") + tk.MustExec("insert into t values ('2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888')") tk.MustExec("alter table t modify c datetime") + tk.MustExec("alter table t modify vc datetime") tk.MustExec("alter table t modify bny datetime") + tk.MustExec("alter table t modify vbny datetime") tk.MustExec("alter table t modify bb datetime") tk.MustExec("alter table t modify txt datetime") tk.MustGetErrCode("alter table t modify e datetime", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify s datetime", mysql.ErrUnsupportedDDLOperation) - tk.MustQuery("select * from t").Check(testkit.Rows("2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:17.888 2020-07-15 18:32:17.888")) + tk.MustQuery("select * from t").Check(testkit.Rows("2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:17.888 2020-07-15 18:32:17.888")) // timestamp reset(tk) tk.MustExec("alter table t modify c char(23)") + tk.MustExec("alter table t modify vc varchar(23)") tk.MustExec("alter table t modify bny binary(23)") - tk.MustExec("insert into t values ('2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888')") + tk.MustExec("alter table t modify vbny varbinary(23)") + tk.MustExec("insert into t values ('2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888')") tk.MustExec("alter table t modify c timestamp") + tk.MustExec("alter table t modify vc timestamp") tk.MustExec("alter table t modify bny timestamp") + tk.MustExec("alter table t modify vbny timestamp") tk.MustExec("alter table t modify bb timestamp") tk.MustExec("alter table t modify txt timestamp") tk.MustGetErrCode("alter table t modify e timestamp", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify s timestamp", mysql.ErrUnsupportedDDLOperation) - tk.MustQuery("select * from t").Check(testkit.Rows("2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:17.888 2020-07-15 18:32:17.888")) + tk.MustQuery("select * from t").Check(testkit.Rows("2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:18 2020-07-15 18:32:17.888 2020-07-15 18:32:17.888")) // year reset(tk) - tk.MustExec("insert into t values ('2020', '2', '20', '99', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888')") + tk.MustExec("insert into t values ('2020', '91', '2', '2020', '20', '99', '2020-07-15 18:32:17.888', '2020-07-15 18:32:17.888')") tk.MustExec("alter table t modify c year") + tk.MustExec("alter table t modify vc year") tk.MustExec("alter table t modify bny year") + tk.MustExec("alter table t modify vbny year") tk.MustExec("alter table t modify bb year") tk.MustExec("alter table t modify txt year") tk.MustExec("alter table t modify e year") tk.MustExec("alter table t modify s year") - tk.MustQuery("select * from t").Check(testkit.Rows("2020 2002 2020 1999 2002 2002")) + tk.MustQuery("select * from t").Check(testkit.Rows("2020 1991 2002 2020 2020 1999 2002 2002")) // To json data type. reset(tk) tk.MustExec("alter table t modify c char(15)") + tk.MustExec("alter table t modify vc varchar(15)") tk.MustExec("alter table t modify bny binary(15)") - tk.MustExec("insert into t values ('{\"k1\": \"value\"}', '{\"k1\": \"value\"}', '{\"k1\": \"value\"}', '{\"k1\": \"value\"}', '{\"k1\": \"value\"}', '{\"k1\": \"value\"}')") + tk.MustExec("alter table t modify vbny varbinary(15)") + tk.MustExec("insert into t values ('{\"k1\": \"value\"}', '{\"k1\": \"value\"}', '{\"k1\": \"value\"}', '{\"k1\": \"value\"}', '{\"k1\": \"value\"}', '{\"k1\": \"value\"}', '{\"k1\": \"value\"}', '{\"k1\": \"value\"}')") tk.MustExec("alter table t modify c json") + tk.MustExec("alter table t modify vc json") tk.MustExec("alter table t modify bny json") + tk.MustExec("alter table t modify vbny json") tk.MustExec("alter table t modify bb json") tk.MustExec("alter table t modify txt json") tk.MustExec("alter table t modify e json") tk.MustExec("alter table t modify s json") - tk.MustQuery("select * from t").Check(testkit.Rows("{\"k1\": \"value\"} {\"k1\": \"value\"} {\"k1\": \"value\"} {\"k1\": \"value\"} \"{\\\"k1\\\": \\\"value\\\"}\" \"{\\\"k1\\\": \\\"value\\\"}\"")) + tk.MustQuery("select * from t").Check(testkit.Rows("{\"k1\": \"value\"} {\"k1\": \"value\"} {\"k1\": \"value\"} {\"k1\": \"value\"} {\"k1\": \"value\"} {\"k1\": \"value\"} \"{\\\"k1\\\": \\\"value\\\"}\" \"{\\\"k1\\\": \\\"value\\\"}\"")) reset(tk) - tk.MustExec("insert into t values ('123x', 'abc', 'timestamp', 'date', '123', '123')") + tk.MustExec("insert into t values ('123x', 'x123', 'abc', 'datetime', 'timestamp', 'date', '123', '123')") tk.MustGetErrCode("alter table t modify c int", mysql.ErrTruncatedWrongValue) + tk.MustGetErrCode("alter table t modify vc smallint", mysql.ErrTruncatedWrongValue) + tk.MustGetErrCode("alter table t modify bny bigint", mysql.ErrTruncatedWrongValue) + tk.MustGetErrCode("alter table t modify vbny datetime", mysql.ErrTruncatedWrongValue) + tk.MustGetErrCode("alter table t modify bb timestamp", mysql.ErrTruncatedWrongValue) tk.MustGetErrCode("alter table t modify txt date", mysql.ErrTruncatedWrongValue) reset(tk) - tk.MustExec("alter table t add vc char(20)") + tk.MustExec("alter table t modify vc varchar(20)") tk.MustExec("insert into t(c, vc) values ('1x', '20200915110836')") tk.MustGetErrCode("alter table t modify c year", mysql.ErrTruncatedWrongValue) @@ -649,7 +715,6 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromStringToOthers(c *C) // Both error but different error message. // MySQL will get "ERROR 3140 (22032): Invalid JSON text: "The document root must not be followed by other values." at position 1 in value for column '#sql-5b_42.c'." error. reset(tk) - tk.MustExec("alter table t add vc char(20)") tk.MustExec("alter table t modify c char(15)") tk.MustExec("insert into t(c) values ('{\"k1\": \"value\"')") tk.MustGetErrCode("alter table t modify c json", mysql.ErrInvalidJSONText) @@ -757,15 +822,15 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromNumericToOthers(c *C // varchar reset(tk) tk.MustExec("insert into t values (-258.12345, 333.33, 2000000.20000002, 323232323.3232323232, -111.11111111, -222222222222.222222222222222, b'10101')") - tk.MustGetErrCode("alter table t modify d varchar(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify n varchar(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify r varchar(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify db varchar(30)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify d varchar(30)") + tk.MustExec("alter table t modify n varchar(30)") + tk.MustExec("alter table t modify r varchar(30)") + tk.MustExec("alter table t modify db varchar(30)") // MySQL will get "-111.111" rather than "-111.111115" at TiDB. - tk.MustGetErrCode("alter table t modify f32 varchar(30)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify f32 varchar(30)") // MySQL will get "ERROR 1406 (22001): Data truncation: Data too long for column 'f64' at row 1". - tk.MustGetErrCode("alter table t modify f64 varchar(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify b varchar(30)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify f64 varchar(30)") + tk.MustExec("alter table t modify b varchar(30)") tk.MustQuery("select * from t").Check(testkit.Rows("-258.1234500 333.33 2000000.20000002 323232323.32323235 -111.111115 -222222222222.22223 \x15")) // binary @@ -784,15 +849,15 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromNumericToOthers(c *C // varbinary reset(tk) tk.MustExec("insert into t values (-258.12345, 333.33, 2000000.20000002, 323232323.3232323232, -111.11111111, -222222222222.222222222222222, b'10101')") - tk.MustGetErrCode("alter table t modify d varbinary(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify n varbinary(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify r varbinary(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify db varbinary(30)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify d varbinary(30)") + tk.MustExec("alter table t modify n varbinary(30)") + tk.MustExec("alter table t modify r varbinary(30)") + tk.MustExec("alter table t modify db varbinary(30)") // MySQL will get "-111.111" rather than "-111.111115" at TiDB. - tk.MustGetErrCode("alter table t modify f32 varbinary(30)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify f32 varbinary(30)") // MySQL will get "ERROR 1406 (22001): Data truncation: Data too long for column 'f64' at row 1". - tk.MustGetErrCode("alter table t modify f64 varbinary(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify b varbinary(30)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify f64 varbinary(30)") + tk.MustExec("alter table t modify b varbinary(30)") tk.MustQuery("select * from t").Check(testkit.Rows("-258.1234500 333.33 2000000.20000002 323232323.32323235 -111.111115 -222222222222.22223 \x15")) // blob @@ -1070,11 +1135,11 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromDateTimeTypeToOthers // varchar reset(tk) tk.MustExec("insert into t values ('2020-10-30', '19:38:25.001', 20201030082133.455555, 20201030082133.455555, 2020)") - tk.MustGetErrCode("alter table t modify d varchar(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify t varchar(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify dt varchar(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify tmp varchar(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify y varchar(30)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify d varchar(30)") + tk.MustExec("alter table t modify t varchar(30)") + tk.MustExec("alter table t modify dt varchar(30)") + tk.MustExec("alter table t modify tmp varchar(30)") + tk.MustExec("alter table t modify y varchar(30)") tk.MustQuery("select * from t").Check(testkit.Rows("2020-10-30 19:38:25.001 2020-10-30 08:21:33.455555 2020-10-30 08:21:33.455555 2020")) // binary @@ -1094,11 +1159,11 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromDateTimeTypeToOthers // varbinary reset(tk) tk.MustExec("insert into t values ('2020-10-30', '19:38:25.001', 20201030082133.455555, 20201030082133.455555, 2020)") - tk.MustGetErrCode("alter table t modify d varbinary(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify t varbinary(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify dt varbinary(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify tmp varbinary(30)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify y varbinary(30)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify d varbinary(30)") + tk.MustExec("alter table t modify t varbinary(30)") + tk.MustExec("alter table t modify dt varbinary(30)") + tk.MustExec("alter table t modify tmp varbinary(30)") + tk.MustExec("alter table t modify y varbinary(30)") tk.MustQuery("select * from t").Check(testkit.Rows("2020-10-30 19:38:25.001 2020-10-30 08:21:33.455555 2020-10-30 08:21:33.455555 2020")) // text @@ -1336,15 +1401,15 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromJsonToOthers(c *C) { // varchar reset(tk) tk.MustExec("insert into t values ('{\"obj\": 100}', '[-1, 0, 1]', 'null', 'true', 'false', '-22', '22', '323232323.3232323232', '\"json string\"')") - tk.MustGetErrCode("alter table t modify obj varchar(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify arr varchar(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify nil varchar(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify t varchar(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify f varchar(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify i varchar(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify ui varchar(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify f64 varchar(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify str varchar(20)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify obj varchar(20)") + tk.MustExec("alter table t modify arr varchar(20)") + tk.MustExec("alter table t modify nil varchar(20)") + tk.MustExec("alter table t modify t varchar(20)") + tk.MustExec("alter table t modify f varchar(20)") + tk.MustExec("alter table t modify i varchar(20)") + tk.MustExec("alter table t modify ui varchar(20)") + tk.MustExec("alter table t modify f64 varchar(20)") + tk.MustExec("alter table t modify str varchar(20)") tk.MustQuery("select * from t").Check(testkit.Rows("{\"obj\": 100} [-1, 0, 1] null true false -22 22 323232323.32323235 \"json string\"")) // binary @@ -1372,15 +1437,15 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromJsonToOthers(c *C) { // varbinary reset(tk) tk.MustExec("insert into t values ('{\"obj\": 100}', '[-1, 0, 1]', 'null', 'true', 'false', '-22', '22', '323232323.3232323232', '\"json string\"')") - tk.MustGetErrCode("alter table t modify obj varbinary(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify arr varbinary(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify nil varbinary(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify t varbinary(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify f varbinary(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify i varbinary(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify ui varbinary(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify f64 varbinary(20)", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t modify str varbinary(20)", mysql.ErrUnsupportedDDLOperation) + tk.MustExec("alter table t modify obj varbinary(20)") + tk.MustExec("alter table t modify arr varbinary(20)") + tk.MustExec("alter table t modify nil varbinary(20)") + tk.MustExec("alter table t modify t varbinary(20)") + tk.MustExec("alter table t modify f varbinary(20)") + tk.MustExec("alter table t modify i varbinary(20)") + tk.MustExec("alter table t modify ui varbinary(20)") + tk.MustExec("alter table t modify f64 varbinary(20)") + tk.MustExec("alter table t modify str varbinary(20)") tk.MustQuery("select * from t").Check(testkit.Rows("{\"obj\": 100} [-1, 0, 1] null true false -22 22 323232323.32323235 \"json string\"")) // blob diff --git a/ddl/db_test.go b/ddl/db_test.go index 80a821a40e387..00cf3fc42b6fc 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -4102,9 +4102,11 @@ func (s *testSerialDBSuite) TestModifyColumnBetweenStringTypes(c *C) { tk.MustGetErrMsg("alter table tt change a a varchar(4);", "[types:1406]Data Too Long, field len 4, data len 5") tk.MustExec("alter table tt change a a varchar(100);") - tk.MustExec("drop table if exists tt;") - tk.MustExec("create table tt (a char(10));") - tk.MustExec("insert into tt values ('111'),('10000');") + // varchar to char + tk.MustExec("alter table tt change a a char(10);") + c2 = getModifyColumn(c, s.s.(sessionctx.Context), "test", "tt", "a", false) + c.Assert(c2.FieldType.Tp, Equals, mysql.TypeString) + c.Assert(c2.FieldType.Flen, Equals, 10) tk.MustQuery("select * from tt").Check(testkit.Rows("111", "10000")) tk.MustGetErrMsg("alter table tt change a a char(4);", "[types:1406]Data Too Long, field len 4, data len 5") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 9c3eca13c7fee..e5871311d78a9 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3537,7 +3537,7 @@ func CheckModifyTypeCompatible(origin *types.FieldType, to *types.FieldType) (ca return true, notCompatibleMsg, errUnsupportedModifyColumn.GenWithStackByArgs(notCompatibleMsg) } -func needReorgToChange(origin *types.FieldType, to *types.FieldType) (needOreg bool, reasonMsg string) { +func needReorgToChange(origin *types.FieldType, to *types.FieldType) (needReorg bool, reasonMsg string) { toFlen := to.Flen originFlen := origin.Flen if mysql.IsIntegerType(to.Tp) && mysql.IsIntegerType(origin.Tp) { @@ -3547,6 +3547,10 @@ func needReorgToChange(origin *types.FieldType, to *types.FieldType) (needOreg b toFlen, _ = mysql.GetDefaultFieldLengthAndDecimal(to.Tp) } + if convertBetweenCharAndVarchar(origin.Tp, to.Tp) { + return true, "conversion between char and varchar string needs reorganization" + } + if toFlen > 0 && toFlen < originFlen { return true, fmt.Sprintf("length %d is less than origin %d", toFlen, originFlen) } @@ -3621,10 +3625,6 @@ func checkModifyTypes(ctx sessionctx.Context, origin *types.FieldType, to *types return errUnsupportedModifyColumn.GenWithStackByArgs(msg) } } - if types.IsTypeVarchar(origin.Tp) != types.IsTypeVarchar(to.Tp) { - unsupportedMsg := "column type conversion between 'varchar' and 'non-varchar' is currently unsupported yet" - return errUnsupportedModifyColumn.GenWithStackByArgs(unsupportedMsg) - } err = checkModifyCharsetAndCollation(to.Charset, to.Collate, origin.Charset, origin.Collate, needRewriteCollationData) // column type change can handle the charset change between these two types in the process of the reorg. diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 7d50b66e5a65c..67c8b8978caa8 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -546,12 +546,12 @@ func (s *testSuite6) TestAlterTableModifyColumn(c *C) { _, err = tk.Exec("alter table mc modify column c2 varchar(8)") c.Assert(err, NotNil) tk.MustExec("alter table mc modify column c2 varchar(11)") - tk.MustGetErrCode("alter table mc modify column c2 text(13)", errno.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table mc modify column c2 text", errno.ErrUnsupportedDDLOperation) + tk.MustExec("alter table mc modify column c2 text(13)") + tk.MustExec("alter table mc modify column c2 text") tk.MustExec("alter table mc modify column c3 bit") result := tk.MustQuery("show create table mc") createSQL := result.Rows()[0][1] - expected := "CREATE TABLE `mc` (\n `c1` bigint(20) DEFAULT NULL,\n `c2` varchar(11) DEFAULT NULL,\n `c3` bit(1) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin" + expected := "CREATE TABLE `mc` (\n `c1` bigint(20) DEFAULT NULL,\n `c2` text DEFAULT NULL,\n `c3` bit(1) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin" c.Assert(createSQL, Equals, expected) tk.MustExec("create or replace view alter_view as select c1,c2 from mc") _, err = tk.Exec("alter table alter_view modify column c2 text") From c8c0dd0bb94d13f526547dae1448b6ee4f877fb0 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Wed, 2 Jun 2021 11:29:38 +0800 Subject: [PATCH 229/343] executor: supports as of timestamp compatibility (#25019) --- executor/stale_txn_test.go | 52 +++++++++++++++++++++++++++++++++++ planner/core/preprocess.go | 4 +++ sessionctx/variable/sysvar.go | 5 ++++ 3 files changed, 61 insertions(+) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 9fd0a9f83d6f3..44def0981bed0 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -451,6 +451,7 @@ func (s *testStaleTxnSerialSuite) TestSetTransactionReadOnlyAsOf(c *C) { c.Assert(tk.Se.GetSessionVars().TxnReadTS.PeakTxnReadTS(), Equals, uint64(0)) tk.MustExec("begin") c.Assert(tk.Se.GetSessionVars().TxnCtx.StartTS, Not(Equals), testcase.expectedTS) + tk.MustExec("commit") failpoint.Disable("github.com/pingcap/tidb/expression/injectSafeTS") } @@ -682,6 +683,57 @@ func (s *testStaleTxnSuite) TestSpecialSQLInStalenessTxn(c *C) { } } +func (s *testStaleTxnSuite) TestAsOfTimestampCompatibility(c *C) { + tk := testkit.NewTestKit(c, s.store) + // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. + safePointName := "tikv_gc_safe_point" + safePointValue := "20160102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) + tk.MustExec("use test") + tk.MustExec("create table t5(id int);") + time1 := time.Now() + testcases := []struct { + beginSQL string + sql string + }{ + { + beginSQL: fmt.Sprintf("START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'", time1.Format("2006-1-2 15:04:05.000")), + sql: fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000")), + }, + { + beginSQL: "begin", + sql: fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000")), + }, + { + beginSQL: "start transaction", + sql: fmt.Sprintf(`SET TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000")), + }, + { + beginSQL: fmt.Sprintf("START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'", time1.Format("2006-1-2 15:04:05.000")), + sql: fmt.Sprintf("select * from t5 as of timestamp '%s'", time1.Format("2006-1-2 15:04:05.000")), + }, + { + beginSQL: "begin", + sql: fmt.Sprintf("select * from t5 as of timestamp '%s'", time1.Format("2006-1-2 15:04:05.000")), + }, + { + beginSQL: "start transaction", + sql: fmt.Sprintf("select * from t5 as of timestamp '%s'", time1.Format("2006-1-2 15:04:05.000")), + }, + } + for _, testcase := range testcases { + tk.MustExec(testcase.beginSQL) + err := tk.ExecToErr(testcase.sql) + c.Assert(err, NotNil) + c.Assert(err.Error(), Matches, ".*as of timestamp can't be set in transaction.*") + tk.MustExec("commit") + } +} + func (s *testStaleTxnSuite) TestSetTransactionInfoSchema(c *C) { tk := testkit.NewTestKit(c, s.store) // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 635a53d72da72..947b1fdcef1ec 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -1389,6 +1389,10 @@ func (p *preprocessor) handleAsOf(node *ast.AsOfClause) { dom := domain.GetDomain(p.ctx) ts := uint64(0) if node != nil { + if p.ctx.GetSessionVars().InTxn() { + p.err = ErrAsOf.FastGenWithCause("as of timestamp can't be set in transaction.") + return + } ts, p.err = calculateTsExpr(p.ctx, node) if p.err != nil { return diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index bec4db52c1757..d49e4b1b19bda 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -829,6 +829,11 @@ var defaultSysVars = []*SysVar{ }}, {Scope: ScopeSession, Name: TiDBTxnReadTS, Value: "", Hidden: true, SetSession: func(s *SessionVars, val string) error { return setTxnReadTS(s, val) + }, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + if vars.InTxn() { + return "", errors.New("as of timestamp can't be set in transaction") + } + return normalizedValue, nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBAllowMPPExecution, Value: On, Type: TypeEnum, PossibleValues: []string{"OFF", "ON", "ENFORCE"}, SetSession: func(s *SessionVars, val string) error { s.allowMPPExecution = val From f24dee95fe195ddee2f95956e9ec6d5841778bba Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Wed, 2 Jun 2021 11:43:38 +0800 Subject: [PATCH 230/343] ddl: fix column type change won't cast the default value to new one (#25025) --- ddl/column_type_change_test.go | 30 ++++++++++++++++++++++++++++++ util/rowDecoder/decoder.go | 2 ++ 2 files changed, 32 insertions(+) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 32418dd031c29..7febe2209f62c 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -1883,3 +1883,33 @@ func (s *testColumnTypeChangeSuite) TestChangeIntToBitWillPanicInBackfillIndexes ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) tk.MustQuery("select * from t").Check(testkit.Rows("\x13 1 1.00", "\x11 2 2.00")) } + +// Close issue #24971, #24973, #24971 +func (s *testColumnTypeChangeSuite) TestCTCShouldCastTheDefaultValue(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + // Enable column change variable. + tk.Se.GetSessionVars().EnableChangeColumnType = true + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(1)") + tk.MustExec("alter table t add column b bit(51) default 1512687856625472") // virtual fill the column data + tk.MustGetErrCode("alter table t modify column b decimal(30,18)", mysql.ErrDataOutOfRange) // because 1512687856625472 is out of range. + + tk.MustExec("drop table if exists t") + tk.MustExec("create table tbl_1 (col int)") + tk.MustExec("insert into tbl_1 values (9790)") + tk.MustExec("alter table tbl_1 add column col1 blob(6) collate binary not null") + tk.MustQuery("select col1 from tbl_1").Check(testkit.Rows("")) + tk.MustGetErrCode("alter table tbl_1 change column col1 col2 int", mysql.ErrTruncatedWrongValue) + tk.MustQuery("select col1 from tbl_1").Check(testkit.Rows("")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table tbl(col_214 decimal(30,8))") + tk.MustExec("replace into tbl values (89687.448)") + tk.MustExec("alter table tbl add column col_279 binary(197) collate binary default 'RAWTdm' not null") + tk.MustQuery("select col_279 from tbl").Check(testkit.Rows("RAWTdm\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00")) + tk.MustGetErrCode("alter table tbl change column col_279 col_287 int", mysql.ErrTruncatedWrongValue) + tk.MustQuery("select col_279 from tbl").Check(testkit.Rows("RAWTdm\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00")) +} diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index 3fd63c05fae96..35413904fe3d8 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -164,6 +164,8 @@ func (rd *RowDecoder) DecodeTheExistedColumnMap(ctx sessionctx.Context, handle k if err != nil { return nil, err } + // Fill the default value into map. + row[colInfo.ID] = val rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) } // return the existed column map here. From 7811bf9a37f42f91fc2e8673e4f12bd64ce56139 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 2 Jun 2021 12:51:39 +0800 Subject: [PATCH 231/343] *: use pprof profile to collect CPU time group by SQL and plan digest (#24892) --- executor/adapter.go | 40 ++- executor/executor.go | 17 +- executor/prepared.go | 9 +- planner/optimize.go | 8 +- server/conn.go | 20 +- server/conn_stmt.go | 39 ++- server/http_status.go | 11 +- server/sql_info_fetcher.go | 8 +- server/tidb_test.go | 232 ++++++++++++++++ session/session.go | 14 +- statistics/handle/handle.go | 5 + tidb-server/main.go | 2 + util/misc.go | 9 + util/topsql/collector/collector.go | 24 ++ util/topsql/topsql.go | 72 +++++ util/topsql/tracecpu/mock/mock.go | 173 ++++++++++++ util/topsql/tracecpu/profile.go | 424 +++++++++++++++++++++++++++++ util/topsql/tracecpu_test.go | 143 ++++++++++ 18 files changed, 1204 insertions(+), 46 deletions(-) create mode 100644 util/topsql/collector/collector.go create mode 100644 util/topsql/topsql.go create mode 100644 util/topsql/tracecpu/mock/mock.go create mode 100644 util/topsql/tracecpu/profile.go create mode 100644 util/topsql/tracecpu_test.go diff --git a/executor/adapter.go b/executor/adapter.go index df6e62751f90a..f1d6a8b8e6040 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -56,7 +57,7 @@ import ( "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/stringutil" - + "github.com/pingcap/tidb/util/topsql" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -213,6 +214,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec defer span1.Finish() ctx = opentracing.ContextWithSpan(ctx, span1) } + ctx = a.setPlanLabelForTopSQL(ctx) startTs := uint64(math.MaxUint64) err := a.Ctx.InitTxnWithStartTS(startTs) if err != nil { @@ -288,6 +290,15 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { return a.InfoSchema.SchemaMetaVersion(), nil } +func (a *ExecStmt) setPlanLabelForTopSQL(ctx context.Context) context.Context { + if a.Plan == nil || !variable.TopSQLEnabled() { + return ctx + } + normalizedSQL, sqlDigest := a.Ctx.GetSessionVars().StmtCtx.SQLDigest() + normalizedPlan, planDigest := getPlanDigest(a.Ctx, a.Plan) + return topsql.AttachSQLInfo(ctx, normalizedSQL, sqlDigest, normalizedPlan, planDigest) +} + // Exec builds an Executor from a plan. If the Executor doesn't return result, // like the INSERT, UPDATE statements, it executes in this function, if the Executor returns // result, execution is done after this function returns, in the returned sqlexec.RecordSet Next method. @@ -357,8 +368,8 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { if err != nil { return nil, err } - - getPlanDigest(a.Ctx, a.Plan) + // ExecuteExec will rewrite `a.Plan`, so set plan label should be executed after `a.buildExecutor`. + ctx = a.setPlanLabelForTopSQL(ctx) if err = e.Open(ctx); err != nil { terror.Call(e.Close) @@ -951,7 +962,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { statsInfos := plannercore.GetStatsInfo(a.Plan) memMax := sessVars.StmtCtx.MemTracker.MaxConsumed() diskMax := sessVars.StmtCtx.DiskTracker.MaxConsumed() - planDigest := getPlanDigest(a.Ctx, a.Plan) + _, planDigest := getPlanDigest(a.Ctx, a.Plan) slowItems := &variable.SlowQueryLogItems{ TxnTS: txnTS, SQL: sql.String(), @@ -969,7 +980,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { DiskMax: diskMax, Succ: succ, Plan: getPlanTree(a.Ctx, a.Plan), - PlanDigest: planDigest, + PlanDigest: planDigest.String(), Prepared: a.isPreparedStmt, HasMoreResults: hasMoreResults, PlanFromCache: sessVars.FoundInPlanCache, @@ -1043,15 +1054,15 @@ func getPlanTree(sctx sessionctx.Context, p plannercore.Plan) string { } // getPlanDigest will try to get the select plan tree if the plan is select or the select plan of delete/update/insert statement. -func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) string { +func getPlanDigest(sctx sessionctx.Context, p plannercore.Plan) (string, *parser.Digest) { sc := sctx.GetSessionVars().StmtCtx - _, planDigest := sc.GetPlanDigest() - if planDigest != nil { - return planDigest.String() + normalized, planDigest := sc.GetPlanDigest() + if len(normalized) > 0 && planDigest != nil { + return normalized, planDigest } - normalized, planDigest := plannercore.NormalizePlan(p) + normalized, planDigest = plannercore.NormalizePlan(p) sc.SetPlanDigest(normalized, planDigest) - return planDigest.String() + return normalized, planDigest } // getEncodedPlan gets the encoded plan, and generates the hint string if indicated. @@ -1125,11 +1136,12 @@ func (a *ExecStmt) SummaryStmt(succ bool) { var planDigestGen func() string if a.Plan.TP() == plancodec.TypePointGet { planDigestGen = func() string { - planDigest := getPlanDigest(a.Ctx, a.Plan) - return planDigest + _, planDigest := getPlanDigest(a.Ctx, a.Plan) + return planDigest.String() } } else { - planDigest = getPlanDigest(a.Ctx, a.Plan) + _, tmp := getPlanDigest(a.Ctx, a.Plan) + planDigest = tmp.String() } execDetail := stmtCtx.GetExecDetails() diff --git a/executor/executor.go b/executor/executor.go index ecb9102ba2f39..c444c0920069e 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -18,6 +18,7 @@ import ( "fmt" "math" "runtime" + "runtime/pprof" "runtime/trace" "strconv" "strings" @@ -66,6 +67,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/resourcegrouptag" + "github.com/pingcap/tidb/util/topsql" "go.uber.org/zap" ) @@ -1653,9 +1655,20 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.MemTracker.SetActionOnExceed(action) } if execStmt, ok := s.(*ast.ExecuteStmt); ok { - s, err = planner.GetPreparedStmt(execStmt, vars) + prepareStmt, err := planner.GetPreparedStmt(execStmt, vars) if err != nil { - return + return err + } + s = prepareStmt.PreparedAst.Stmt + sc.InitSQLDigest(prepareStmt.NormalizedSQL, prepareStmt.SQLDigest) + // For `execute stmt` SQL, should reset the SQL digest with the prepare SQL digest. + goCtx := context.Background() + if variable.EnablePProfSQLCPU.Load() && len(prepareStmt.NormalizedSQL) > 0 { + goCtx = pprof.WithLabels(goCtx, pprof.Labels("sql", util.QueryStrForLog(prepareStmt.NormalizedSQL))) + pprof.SetGoroutineLabels(goCtx) + } + if variable.TopSQLEnabled() && prepareStmt.SQLDigest != nil { + goCtx = topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest, "", nil) } } // execute missed stmtID uses empty sql diff --git a/executor/prepared.go b/executor/prepared.go index f494ebac3dc9d..2a11977ef607e 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -29,12 +29,14 @@ import ( "github.com/pingcap/tidb/planner" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/topsql" "go.uber.org/zap" ) @@ -178,6 +180,10 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { Params: sorter.markers, SchemaVersion: ret.InfoSchema.SchemaMetaVersion(), } + normalizedSQL, digest := parser.NormalizeDigest(prepared.Stmt.Text()) + if variable.TopSQLEnabled() { + ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil) + } if !plannercore.PreparedPlanCacheEnabled() { prepared.UseCache = false @@ -213,11 +219,10 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { vars.PreparedStmtNameToID[e.name] = e.ID } - normalized, digest := parser.NormalizeDigest(prepared.Stmt.Text()) preparedObj := &plannercore.CachedPrepareStmt{ PreparedAst: prepared, VisitInfos: destBuilder.GetVisitInfo(), - NormalizedSQL: normalized, + NormalizedSQL: normalizedSQL, SQLDigest: digest, ForUpdateRead: destBuilder.GetIsForUpdateRead(), } diff --git a/planner/optimize.go b/planner/optimize.go index c667d6b124ef9..6d87d6ffac5ed 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -44,7 +44,7 @@ import ( ) // GetPreparedStmt extract the prepared statement from the execute statement. -func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (ast.StmtNode, error) { +func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (*plannercore.CachedPrepareStmt, error) { var ok bool execID := stmt.ExecID if stmt.Name != "" { @@ -57,7 +57,7 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (ast.Stm if !ok { return nil, errors.Errorf("invalid CachedPrepareStmt type") } - return preparedObj.PreparedAst.Stmt, nil + return preparedObj, nil } return nil, plannercore.ErrStmtNotFound } @@ -65,12 +65,12 @@ func GetPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (ast.Stm // IsReadOnly check whether the ast.Node is a read only statement. func IsReadOnly(node ast.Node, vars *variable.SessionVars) bool { if execStmt, isExecStmt := node.(*ast.ExecuteStmt); isExecStmt { - s, err := GetPreparedStmt(execStmt, vars) + prepareStmt, err := GetPreparedStmt(execStmt, vars) if err != nil { logutil.BgLogger().Warn("GetPreparedStmt failed", zap.Error(err)) return false } - return ast.IsReadOnly(s) + return ast.IsReadOnly(prepareStmt.PreparedAst.Stmt) } return ast.IsReadOnly(node) } diff --git a/server/conn.go b/server/conn.go index 46d0a7c023a27..bba2a41a6a8c1 100644 --- a/server/conn.go +++ b/server/conn.go @@ -78,6 +78,7 @@ import ( storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" + tidbutil "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/arena" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" @@ -906,14 +907,6 @@ func (cc *clientConn) ShutdownOrNotify() bool { return false } -func queryStrForLog(query string) string { - const size = 4096 - if len(query) > size { - return query[:size] + fmt.Sprintf("(len: %d)", len(query)) - } - return query -} - func errStrForLog(err error, enableRedactLog bool) string { if enableRedactLog { // currently, only ErrParse is considered when enableRedactLog because it may contain sensitive information like @@ -1025,6 +1018,9 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { cc.lastPacket = data cmd := data[0] data = data[1:] + if variable.TopSQLEnabled() { + defer pprof.SetGoroutineLabels(ctx) + } if variable.EnablePProfSQLCPU.Load() { label := getLastStmtInConn{cc}.PProfLabel() if len(label) > 0 { @@ -2125,10 +2121,10 @@ func (cc getLastStmtInConn) String() string { if cc.ctx.GetSessionVars().EnableRedactLog { sql = parser.Normalize(sql) } - return queryStrForLog(sql) + return tidbutil.QueryStrForLog(sql) case mysql.ComStmtExecute, mysql.ComStmtFetch: stmtID := binary.LittleEndian.Uint32(data[0:4]) - return queryStrForLog(cc.preparedStmt2String(stmtID)) + return tidbutil.QueryStrForLog(cc.preparedStmt2String(stmtID)) case mysql.ComStmtClose, mysql.ComStmtReset: stmtID := binary.LittleEndian.Uint32(data[0:4]) return mysql.Command2Str[cmd] + " " + strconv.Itoa(int(stmtID)) @@ -2156,10 +2152,10 @@ func (cc getLastStmtInConn) PProfLabel() string { case mysql.ComStmtReset: return "ResetStmt" case mysql.ComQuery, mysql.ComStmtPrepare: - return parser.Normalize(queryStrForLog(string(hack.String(data)))) + return parser.Normalize(tidbutil.QueryStrForLog(string(hack.String(data)))) case mysql.ComStmtExecute, mysql.ComStmtFetch: stmtID := binary.LittleEndian.Uint32(data[0:4]) - return queryStrForLog(cc.preparedStmt2StringNoArgs(stmtID)) + return tidbutil.QueryStrForLog(cc.preparedStmt2StringNoArgs(stmtID)) default: return "" } diff --git a/server/conn_stmt.go b/server/conn_stmt.go index e9f56306d9800..df85f7ce45f52 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -50,11 +50,13 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/topsql" ) func (cc *clientConn) handleStmtPrepare(ctx context.Context, sql string) error { @@ -128,6 +130,13 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e stmtID := binary.LittleEndian.Uint32(data[0:4]) pos += 4 + if variable.TopSQLEnabled() { + preparedStmt, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) + if preparedStmt != nil && preparedStmt.SQLDigest != nil { + ctx = topsql.AttachSQLInfo(ctx, preparedStmt.NormalizedSQL, preparedStmt.SQLDigest, "", nil) + } + } + stmt := cc.ctx.GetStatement(int(stmtID)) if stmt == nil { return mysql.NewErr(mysql.ErrUnknownStmtHandler, @@ -265,6 +274,12 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err return errors.Annotate(mysql.NewErr(mysql.ErrUnknownStmtHandler, strconv.FormatUint(uint64(stmtID), 10), "stmt_fetch"), cc.preparedStmt2String(stmtID)) } + if variable.TopSQLEnabled() { + prepareObj, _ := cc.preparedStmtID2CachePreparedStmt(stmtID) + if prepareObj != nil && prepareObj.SQLDigest != nil { + ctx = topsql.AttachSQLInfo(ctx, prepareObj.NormalizedSQL, prepareObj.SQLDigest, "", nil) + } + } sql := "" if prepared, ok := cc.ctx.GetStatement(int(stmtID)).(*TiDBStatement); ok { sql = prepared.sql @@ -680,14 +695,30 @@ func (cc *clientConn) preparedStmt2StringNoArgs(stmtID uint32) string { if sv == nil { return "" } + preparedObj, invalid := cc.preparedStmtID2CachePreparedStmt(stmtID) + if invalid { + return "invalidate CachedPrepareStmt type, ID: " + strconv.FormatUint(uint64(stmtID), 10) + } + if preparedObj == nil { + return "prepared statement not found, ID: " + strconv.FormatUint(uint64(stmtID), 10) + } + return preparedObj.PreparedAst.Stmt.Text() +} + +func (cc *clientConn) preparedStmtID2CachePreparedStmt(stmtID uint32) (_ *plannercore.CachedPrepareStmt, invalid bool) { + sv := cc.ctx.GetSessionVars() + if sv == nil { + return nil, false + } preparedPointer, ok := sv.PreparedStmts[stmtID] if !ok { - return "prepared statement not found, ID: " + strconv.FormatUint(uint64(stmtID), 10) + // not found + return nil, false } preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt) if !ok { - return "invalidate CachedPrepareStmt type, ID: " + strconv.FormatUint(uint64(stmtID), 10) + // invalid cache. should never happen. + return nil, true } - preparedAst := preparedObj.PreparedAst - return preparedAst.Stmt.Text() + return preparedObj, false } diff --git a/server/http_status.go b/server/http_status.go index b0fc7615caf49..67eace56562fe 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/printer" + "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tidb/util/versioninfo" "github.com/prometheus/client_golang/prometheus/promhttp" "github.com/soheilhy/cmux" @@ -184,7 +185,7 @@ func (s *Server) startHTTPServer() { serverMux.HandleFunc("/debug/pprof/", pprof.Index) serverMux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) - serverMux.HandleFunc("/debug/pprof/profile", pprof.Profile) + serverMux.HandleFunc("/debug/pprof/profile", tracecpu.ProfileHTTPHandler) serverMux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) serverMux.HandleFunc("/debug/pprof/trace", pprof.Trace) serverMux.HandleFunc("/debug/gogc", func(w http.ResponseWriter, r *http.Request) { @@ -251,7 +252,7 @@ func (s *Server) startHTTPServer() { serveError(w, http.StatusInternalServerError, fmt.Sprintf("Create zipped %s fail: %v", "profile", err)) return } - if err := rpprof.StartCPUProfile(fw); err != nil { + if err := tracecpu.StartCPUProfile(fw); err != nil { serveError(w, http.StatusInternalServerError, fmt.Sprintf("Could not enable CPU profiling: %s", err)) return @@ -261,7 +262,11 @@ func (s *Server) startHTTPServer() { sec = 10 } sleepWithCtx(r.Context(), time.Duration(sec)*time.Second) - rpprof.StopCPUProfile() + err = tracecpu.StopCPUProfile() + if err != nil { + serveError(w, http.StatusInternalServerError, fmt.Sprintf("Create zipped %s fail: %v", "config", err)) + return + } // dump config fw, err = zw.Create("config") diff --git a/server/sql_info_fetcher.go b/server/sql_info_fetcher.go index 6fc80daf506d6..57f51f544b90b 100644 --- a/server/sql_info_fetcher.go +++ b/server/sql_info_fetcher.go @@ -20,7 +20,6 @@ import ( "encoding/json" "fmt" "net/http" - "runtime/pprof" "strconv" "strings" "time" @@ -35,6 +34,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/topsql/tracecpu" ) type sqlInfoFetcher struct { @@ -275,13 +275,13 @@ func (sh *sqlInfoFetcher) getExplainAnalyze(ctx context.Context, sql string, res } func (sh *sqlInfoFetcher) catchCPUProfile(ctx context.Context, sec int, buf *bytes.Buffer, errChan chan<- error) { - if err := pprof.StartCPUProfile(buf); err != nil { + if err := tracecpu.StartCPUProfile(buf); err != nil { errChan <- err return } sleepWithCtx(ctx, time.Duration(sec)*time.Second) - pprof.StopCPUProfile() - errChan <- nil + err := tracecpu.StopCPUProfile() + errChan <- err } func (sh *sqlInfoFetcher) getStatsForTable(pair tableNamePair) (*handle.JSONTable, error) { diff --git a/server/tidb_test.go b/server/tidb_test.go index 426e45cd0c57b..09025e8bec95b 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -15,23 +15,29 @@ package server import ( + "bytes" "context" "crypto/rand" "crypto/rsa" "crypto/tls" "crypto/x509" "crypto/x509/pkix" + "database/sql" "encoding/pem" + "fmt" "math/big" "net/http" "os" "path/filepath" + "regexp" + "strings" "sync/atomic" "time" "github.com/go-sql-driver/mysql" . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/parser" tmysql "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" @@ -43,7 +49,10 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/topsql/tracecpu/mock" ) type tidbTestSuite struct { @@ -54,6 +63,10 @@ type tidbTestSerialSuite struct { *tidbTestSuiteBase } +type tidbTestTopSQLSuite struct { + *tidbTestSuiteBase +} + type tidbTestSuiteBase struct { *testServerClient tidbdrv *TiDBDriver @@ -70,12 +83,18 @@ func newTiDBTestSuiteBase() *tidbTestSuiteBase { var _ = Suite(&tidbTestSuite{newTiDBTestSuiteBase()}) var _ = SerialSuites(&tidbTestSerialSuite{newTiDBTestSuiteBase()}) +var _ = SerialSuites(&tidbTestTopSQLSuite{newTiDBTestSuiteBase()}) func (ts *tidbTestSuite) SetUpSuite(c *C) { metrics.RegisterMetrics() ts.tidbTestSuiteBase.SetUpSuite(c) } +func (ts *tidbTestTopSQLSuite) SetUpSuite(c *C) { + ts.tidbTestSuiteBase.SetUpSuite(c) + tracecpu.GlobalSQLCPUProfiler.Run() +} + func (ts *tidbTestSuiteBase) SetUpSuite(c *C) { var err error ts.store, err = mockstore.NewMockStore() @@ -1153,3 +1172,216 @@ func (ts *tidbTestSerialSuite) TestPrepareCount(c *C) { c.Assert(err, IsNil) c.Assert(atomic.LoadInt64(&variable.PreparedStmtCount), Equals, prepareCnt) } + +func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { + db, err := sql.Open("mysql", ts.getDSN()) + c.Assert(err, IsNil, Commentf("Error connecting")) + defer func() { + err := db.Close() + c.Assert(err, IsNil) + }() + collector := mock.NewTopSQLCollector() + tracecpu.GlobalSQLCPUProfiler.SetCollector(collector) + + dbt := &DBTest{c, db} + dbt.mustExec("drop database if exists topsql") + dbt.mustExec("create database topsql") + dbt.mustExec("use topsql;") + dbt.mustExec("create table t (a int auto_increment, b int, unique index idx(a));") + dbt.mustExec("create table t1 (a int auto_increment, b int, unique index idx(a));") + dbt.mustExec("create table t2 (a int auto_increment, b int, unique index idx(a));") + dbt.mustExec("set @@global.tidb_enable_top_sql='On';") + dbt.mustExec("set @@global.tidb_top_sql_agent_address='127.0.0.1:4001';") + dbt.mustExec("set @@global.tidb_top_sql_precision_seconds=1;") + + // Test case 1: DML query: insert/update/replace/delete/select + cases1 := []struct { + sql string + planRegexp string + cancel func() + }{ + {sql: "insert into t () values (),(),(),(),(),(),();", planRegexp: ""}, + {sql: "insert into t (b) values (1),(1),(1),(1),(1),(1),(1),(1);", planRegexp: ""}, + {sql: "replace into t (b) values (1),(1),(1),(1),(1),(1),(1),(1);", planRegexp: ""}, + {sql: "update t set b=a where b is null limit 1;", planRegexp: ".*Limit.*TableReader.*"}, + {sql: "delete from t where b is null limit 2;", planRegexp: ".*Limit.*TableReader.*"}, + {sql: "select * from t use index(idx) where a>0;", planRegexp: ".*IndexLookUp.*"}, + {sql: "select * from t ignore index(idx) where a>0;", planRegexp: ".*TableReader.*"}, + {sql: "select /*+ HASH_JOIN(t1, t2) */ * from t t1 join t t2 on t1.a=t2.a where t1.b is not null;", planRegexp: ".*HashJoin.*"}, + {sql: "select /*+ INL_HASH_JOIN(t1, t2) */ * from t t1 join t t2 on t1.a=t2.a where t1.b is not null;", planRegexp: ".*IndexHashJoin.*"}, + {sql: "select * from t where a=1;", planRegexp: ".*Point_Get.*"}, + {sql: "select * from t where a in (1,2,3,4)", planRegexp: ".*Batch_Point_Get.*"}, + } + for i, ca := range cases1 { + ctx, cancel := context.WithCancel(context.Background()) + cases1[i].cancel = cancel + sqlStr := ca.sql + go ts.loopExec(ctx, c, func(db *sql.DB) { + dbt := &DBTest{c, db} + if strings.HasPrefix(sqlStr, "select") { + rows := dbt.mustQuery(sqlStr) + for rows.Next() { + } + } else { + // Ignore error here since the error may be write conflict. + db.Exec(sqlStr) + } + }) + } + + // Test case 2: prepare/execute sql + cases2 := []struct { + prepare string + args []interface{} + planRegexp string + cancel func() + }{ + {prepare: "insert into t1 (b) values (?);", args: []interface{}{1}, planRegexp: ""}, + {prepare: "replace into t1 (b) values (?);", args: []interface{}{1}, planRegexp: ""}, + {prepare: "update t1 set b=a where b is null limit ?;", args: []interface{}{1}, planRegexp: ".*Limit.*TableReader.*"}, + {prepare: "delete from t1 where b is null limit ?;", args: []interface{}{1}, planRegexp: ".*Limit.*TableReader.*"}, + {prepare: "select * from t1 use index(idx) where a>?;", args: []interface{}{1}, planRegexp: ".*IndexLookUp.*"}, + {prepare: "select * from t1 ignore index(idx) where a>?;", args: []interface{}{1}, planRegexp: ".*TableReader.*"}, + {prepare: "select /*+ HASH_JOIN(t1, t2) */ * from t1 t1 join t1 t2 on t1.a=t2.a where t1.b is not null;", args: nil, planRegexp: ".*HashJoin.*"}, + {prepare: "select /*+ INL_HASH_JOIN(t1, t2) */ * from t1 t1 join t1 t2 on t1.a=t2.a where t1.b is not null;", args: nil, planRegexp: ".*IndexHashJoin.*"}, + {prepare: "select * from t1 where a=?;", args: []interface{}{1}, planRegexp: ".*Point_Get.*"}, + {prepare: "select * from t1 where a in (?,?,?,?)", args: []interface{}{1, 2, 3, 4}, planRegexp: ".*Batch_Point_Get.*"}, + } + for i, ca := range cases2 { + ctx, cancel := context.WithCancel(context.Background()) + cases2[i].cancel = cancel + prepare, args := ca.prepare, ca.args + go ts.loopExec(ctx, c, func(db *sql.DB) { + stmt, err := db.Prepare(prepare) + c.Assert(err, IsNil) + if strings.HasPrefix(prepare, "select") { + rows, err := stmt.Query(args...) + c.Assert(err, IsNil) + for rows.Next() { + } + } else { + // Ignore error here since the error may be write conflict. + stmt.Exec(args...) + } + }) + } + + // Test case 3: prepare, execute stmt using @val... + cases3 := []struct { + prepare string + args []interface{} + planRegexp string + cancel func() + }{ + {prepare: "insert into t2 (b) values (?);", args: []interface{}{1}, planRegexp: ""}, + {prepare: "replace into t2 (b) values (?);", args: []interface{}{1}, planRegexp: ""}, + {prepare: "update t2 set b=a where b is null limit ?;", args: []interface{}{1}, planRegexp: ".*Limit.*TableReader.*"}, + {prepare: "delete from t2 where b is null limit ?;", args: []interface{}{1}, planRegexp: ".*Limit.*TableReader.*"}, + {prepare: "select * from t2 use index(idx) where a>?;", args: []interface{}{1}, planRegexp: ".*IndexLookUp.*"}, + {prepare: "select * from t2 ignore index(idx) where a>?;", args: []interface{}{1}, planRegexp: ".*TableReader.*"}, + {prepare: "select /*+ HASH_JOIN(t1, t2) */ * from t2 t1 join t2 t2 on t1.a=t2.a where t1.b is not null;", args: nil, planRegexp: ".*HashJoin.*"}, + {prepare: "select /*+ INL_HASH_JOIN(t1, t2) */ * from t2 t1 join t2 t2 on t1.a=t2.a where t1.b is not null;", args: nil, planRegexp: ".*IndexHashJoin.*"}, + {prepare: "select * from t2 where a=?;", args: []interface{}{1}, planRegexp: ".*Point_Get.*"}, + {prepare: "select * from t2 where a in (?,?,?,?)", args: []interface{}{1, 2, 3, 4}, planRegexp: ".*Batch_Point_Get.*"}, + } + for i, ca := range cases3 { + ctx, cancel := context.WithCancel(context.Background()) + cases3[i].cancel = cancel + prepare, args := ca.prepare, ca.args + go ts.loopExec(ctx, c, func(db *sql.DB) { + _, err := db.Exec(fmt.Sprintf("prepare stmt from '%v'", prepare)) + c.Assert(err, IsNil) + sqlBuf := bytes.NewBuffer(nil) + sqlBuf.WriteString("execute stmt ") + for i := range args { + _, err = db.Exec(fmt.Sprintf("set @%c=%v", 'a'+i, args[i])) + c.Assert(err, IsNil) + if i == 0 { + sqlBuf.WriteString("using ") + } else { + sqlBuf.WriteByte(',') + } + sqlBuf.WriteByte('@') + sqlBuf.WriteByte('a' + byte(i)) + } + if strings.HasPrefix(prepare, "select") { + rows, err := db.Query(sqlBuf.String()) + c.Assert(err, IsNil, Commentf("%v", sqlBuf.String())) + for rows.Next() { + } + } else { + // Ignore error here since the error may be write conflict. + db.Exec(sqlBuf.String()) + } + }) + } + + // Wait the top sql collector to collect profile data. + collector.WaitCollectCnt(1) + + checkFn := func(sql, planRegexp string) { + commentf := Commentf("sql: %v", sql) + stats := collector.GetSQLStatsBySQLWithRetry(sql, len(planRegexp) > 0) + // since 1 sql may has many plan, check `len(stats) > 0` instead of `len(stats) == 1`. + c.Assert(len(stats) > 0, IsTrue, commentf) + + match := false + for _, s := range stats { + sqlStr := collector.GetSQL(s.SQLDigest) + encodedPlan := collector.GetPlan(s.PlanDigest) + // Normalize the user SQL before check. + normalizedSQL := parser.Normalize(sql) + c.Assert(sqlStr, Equals, normalizedSQL, commentf) + // decode plan before check. + normalizedPlan, err := plancodec.DecodeNormalizedPlan(encodedPlan) + c.Assert(err, IsNil) + // remove '\n' '\t' before do regexp match. + normalizedPlan = strings.Replace(normalizedPlan, "\n", " ", -1) + normalizedPlan = strings.Replace(normalizedPlan, "\t", " ", -1) + ok, err := regexp.MatchString(planRegexp, normalizedPlan) + c.Assert(err, IsNil, commentf) + if ok { + match = true + break + } + } + c.Assert(match, IsTrue, commentf) + } + + // Check result of test case 1. + for _, ca := range cases1 { + checkFn(ca.sql, ca.planRegexp) + ca.cancel() + } + + // Check result of test case 2. + for _, ca := range cases2 { + checkFn(ca.prepare, ca.planRegexp) + ca.cancel() + } + + // Check result of test case 3. + for _, ca := range cases3 { + checkFn(ca.prepare, ca.planRegexp) + ca.cancel() + } +} + +func (ts *tidbTestTopSQLSuite) loopExec(ctx context.Context, c *C, fn func(db *sql.DB)) { + db, err := sql.Open("mysql", ts.getDSN()) + c.Assert(err, IsNil, Commentf("Error connecting")) + defer func() { + err := db.Close() + c.Assert(err, IsNil) + }() + dbt := &DBTest{c, db} + dbt.mustExec("use topsql;") + for { + select { + case <-ctx.Done(): + return + default: + } + fn(db) + } +} diff --git a/session/session.go b/session/session.go index b24461ba653b6..c4b9c77587790 100644 --- a/session/session.go +++ b/session/session.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/util/topsql" "github.com/pingcap/tipb/go-binlog" "go.uber.org/zap" @@ -1386,6 +1387,13 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter for _, warn := range warns { s.sessionVars.StmtCtx.AppendWarning(util.SyntaxWarn(warn)) } + if variable.TopSQLEnabled() { + normalized, digest := parser.NormalizeDigest(sql) + if digest != nil { + // Fixme: reset/clean the label when internal sql execute finish. + ctx = topsql.AttachSQLInfo(ctx, normalized, digest, "", nil) + } + } return stmts[0], nil } @@ -1496,6 +1504,11 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex if err := executor.ResetContextOfStmt(s, stmtNode); err != nil { return nil, err } + normalizedSQL, digest := s.sessionVars.StmtCtx.SQLDigest() + if variable.TopSQLEnabled() { + ctx = topsql.AttachSQLInfo(ctx, normalizedSQL, digest, "", nil) + } + if err := s.validateStatementReadOnlyInStaleness(stmtNode); err != nil { return nil, err } @@ -1503,7 +1516,6 @@ func (s *session) ExecuteStmt(ctx context.Context, stmtNode ast.StmtNode) (sqlex // Uncorrelated subqueries will execute once when building plan, so we reset process info before building plan. cmd32 := atomic.LoadUint32(&s.GetSessionVars().CommandValue) s.SetProcessInfo(stmtNode.Text(), time.Now(), byte(cmd32), 0) - _, digest := s.sessionVars.StmtCtx.SQLDigest() s.txn.onStmtStart(digest.String()) defer s.txn.onStmtEnd() diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 0026c97bb0c7c..8d4ca1bc678bb 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -17,6 +17,7 @@ import ( "context" "encoding/json" "fmt" + "runtime/pprof" "sort" "strconv" "sync" @@ -121,6 +122,10 @@ func (h *Handle) withRestrictedSQLExecutor(ctx context.Context, fn func(context. func (h *Handle) execRestrictedSQL(ctx context.Context, sql string, params ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { return h.withRestrictedSQLExecutor(ctx, func(ctx context.Context, exec sqlexec.RestrictedSQLExecutor) ([]chunk.Row, []*ast.ResultField, error) { + if variable.TopSQLEnabled() { + // Restore the goroutine label by using the original ctx after execution is finished. + defer pprof.SetGoroutineLabels(ctx) + } stmt, err := exec.ParseWithParams(ctx, sql, params...) if err != nil { return nil, nil, errors.Trace(err) diff --git a/tidb-server/main.go b/tidb-server/main.go index 3aa21808f97c3..f1b16397046f9 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -63,6 +63,7 @@ import ( "github.com/pingcap/tidb/util/sys/linux" storageSys "github.com/pingcap/tidb/util/sys/storage" "github.com/pingcap/tidb/util/systimemon" + "github.com/pingcap/tidb/util/topsql" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/push" pd "github.com/tikv/pd/client" @@ -176,6 +177,7 @@ func main() { printInfo() setupBinlogClient() setupMetrics() + topsql.SetupTopSQL() storage, dom := createStoreAndDomain() svr := createServer(storage, dom) diff --git a/util/misc.go b/util/misc.go index 817c5715af818..ccba554bb07d0 100644 --- a/util/misc.go +++ b/util/misc.go @@ -535,3 +535,12 @@ func GetLocalIP() string { } return "" } + +// QueryStrForLog trim the query if the query length more than 4096 +func QueryStrForLog(query string) string { + const size = 4096 + if len(query) > size { + return query[:size] + fmt.Sprintf("(len: %d)", len(query)) + } + return query +} diff --git a/util/topsql/collector/collector.go b/util/topsql/collector/collector.go new file mode 100644 index 0000000000000..9cc2a429ea990 --- /dev/null +++ b/util/topsql/collector/collector.go @@ -0,0 +1,24 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package collector + +import "github.com/pingcap/tidb/util/topsql/tracecpu" + +// TopSQLCollector uses to collect SQL stats. +// TODO: add a collector to collect and store the SQL stats. +type TopSQLCollector interface { + tracecpu.Collector + RegisterSQL(sqlDigest []byte, normalizedSQL string) + RegisterPlan(planDigest []byte, normalizedPlan string) +} diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go new file mode 100644 index 0000000000000..255ece7256c09 --- /dev/null +++ b/util/topsql/topsql.go @@ -0,0 +1,72 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package topsql + +import ( + "context" + "runtime/pprof" + + "github.com/pingcap/parser" + "github.com/pingcap/tidb/util/topsql/collector" + "github.com/pingcap/tidb/util/topsql/tracecpu" +) + +// SetupTopSQL sets up the top-sql worker. +func SetupTopSQL() { + tracecpu.GlobalSQLCPUProfiler.Run() +} + +// AttachSQLInfo attach the sql information info top sql. +func AttachSQLInfo(ctx context.Context, normalizedSQL string, sqlDigest *parser.Digest, normalizedPlan string, planDigest *parser.Digest) context.Context { + if len(normalizedSQL) == 0 || sqlDigest == nil || len(sqlDigest.Bytes()) == 0 { + return ctx + } + var sqlDigestBytes, planDigestBytes []byte + sqlDigestBytes = sqlDigest.Bytes() + if planDigest != nil { + planDigestBytes = planDigest.Bytes() + } + ctx = tracecpu.CtxWithDigest(ctx, sqlDigestBytes, planDigestBytes) + pprof.SetGoroutineLabels(ctx) + + if len(normalizedPlan) == 0 || len(planDigestBytes) == 0 { + // If plan digest is '', indicate it is the first time to attach the SQL info, since it only know the sql digest. + linkSQLTextWithDigest(sqlDigestBytes, normalizedSQL) + } else { + linkPlanTextWithDigest(planDigestBytes, normalizedPlan) + } + return ctx +} + +func linkSQLTextWithDigest(sqlDigest []byte, normalizedSQL string) { + c := tracecpu.GlobalSQLCPUProfiler.GetCollector() + if c == nil { + return + } + topc, ok := c.(collector.TopSQLCollector) + if ok { + topc.RegisterSQL(sqlDigest, normalizedSQL) + } +} + +func linkPlanTextWithDigest(planDigest []byte, normalizedPlan string) { + c := tracecpu.GlobalSQLCPUProfiler.GetCollector() + if c == nil { + return + } + topc, ok := c.(collector.TopSQLCollector) + if ok { + topc.RegisterPlan(planDigest, normalizedPlan) + } +} diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go new file mode 100644 index 0000000000000..2737306f1a096 --- /dev/null +++ b/util/topsql/tracecpu/mock/mock.go @@ -0,0 +1,173 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package mock + +import ( + "bytes" + "sync" + "time" + + "github.com/pingcap/parser" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/uber-go/atomic" +) + +// TopSQLCollector uses for testing. +type TopSQLCollector struct { + sync.Mutex + // sql_digest -> normalized SQL + sqlMap map[string]string + // plan_digest -> normalized plan + planMap map[string]string + // (sql + plan_digest) -> sql stats + sqlStatsMap map[string]*tracecpu.SQLCPUResult + collectCnt atomic.Int64 +} + +// NewTopSQLCollector uses for testing. +func NewTopSQLCollector() *TopSQLCollector { + return &TopSQLCollector{ + sqlMap: make(map[string]string), + planMap: make(map[string]string), + sqlStatsMap: make(map[string]*tracecpu.SQLCPUResult), + } +} + +// Collect uses for testing. +func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.SQLCPUResult) { + defer c.collectCnt.Inc() + if len(stats) == 0 { + return + } + c.Lock() + defer c.Unlock() + for _, stmt := range stats { + hash := c.hash(stmt) + stats, ok := c.sqlStatsMap[hash] + if !ok { + stats = &tracecpu.SQLCPUResult{ + SQLDigest: stmt.SQLDigest, + PlanDigest: stmt.PlanDigest, + } + c.sqlStatsMap[hash] = stats + } + stats.CPUTimeMs += stmt.CPUTimeMs + } +} + +// GetSQLStatsBySQLWithRetry uses for testing. +func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.SQLCPUResult { + after := time.After(time.Second * 10) + for { + select { + case <-after: + return nil + default: + } + stats := c.GetSQLStatsBySQL(sql, planIsNotNull) + if len(stats) > 0 { + return stats + } + c.WaitCollectCnt(1) + } +} + +// GetSQLStatsBySQL uses for testing. +func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*tracecpu.SQLCPUResult { + stats := make([]*tracecpu.SQLCPUResult, 0, 2) + sqlDigest := GenSQLDigest(sql) + c.Lock() + for _, stmt := range c.sqlStatsMap { + if bytes.Equal(stmt.SQLDigest, sqlDigest.Bytes()) { + if planIsNotNull { + plan := c.planMap[string(stmt.PlanDigest)] + if len(plan) > 0 { + stats = append(stats, stmt) + } + } else { + stats = append(stats, stmt) + } + } + } + c.Unlock() + return stats +} + +// GetSQL uses for testing. +func (c *TopSQLCollector) GetSQL(sqlDigest []byte) string { + c.Lock() + sql := c.sqlMap[string(sqlDigest)] + c.Unlock() + return sql +} + +// GetPlan uses for testing. +func (c *TopSQLCollector) GetPlan(planDigest []byte) string { + c.Lock() + plan := c.planMap[string(planDigest)] + c.Unlock() + return plan +} + +// RegisterSQL uses for testing. +func (c *TopSQLCollector) RegisterSQL(sqlDigest []byte, normalizedSQL string) { + digestStr := string(hack.String(sqlDigest)) + c.Lock() + _, ok := c.sqlMap[digestStr] + if !ok { + c.sqlMap[digestStr] = normalizedSQL + } + c.Unlock() + +} + +// RegisterPlan uses for testing. +func (c *TopSQLCollector) RegisterPlan(planDigest []byte, normalizedPlan string) { + digestStr := string(hack.String(planDigest)) + c.Lock() + _, ok := c.planMap[digestStr] + if !ok { + c.planMap[digestStr] = normalizedPlan + } + c.Unlock() +} + +// WaitCollectCnt uses for testing. +func (c *TopSQLCollector) WaitCollectCnt(count int64) { + timeout := time.After(time.Second * 10) + end := c.collectCnt.Load() + count + for { + // Wait for collector collect sql stats count >= expected count + if c.collectCnt.Load() >= end { + break + } + select { + case <-timeout: + break + default: + time.Sleep(time.Millisecond * 10) + } + } +} + +func (c *TopSQLCollector) hash(stat tracecpu.SQLCPUResult) string { + return string(stat.SQLDigest) + string(stat.PlanDigest) +} + +// GenSQLDigest uses for testing. +func GenSQLDigest(sql string) *parser.Digest { + _, digest := parser.NormalizeDigest(sql) + return digest +} diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go new file mode 100644 index 0000000000000..c0fdbe5fee52f --- /dev/null +++ b/util/topsql/tracecpu/profile.go @@ -0,0 +1,424 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tracecpu + +import ( + "bytes" + "context" + "errors" + "fmt" + "io" + "net/http" + "runtime/pprof" + "strconv" + "sync" + "sync/atomic" + "time" + + "github.com/google/pprof/profile" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + +const ( + labelSQL = "sql" + labelSQLDigest = "sql_digest" + labelPlanDigest = "plan_digest" +) + +// GlobalSQLCPUProfiler is the global SQL stats profiler. +var GlobalSQLCPUProfiler = newSQLCPUProfiler() + +// Collector uses to collect SQL execution cpu time. +type Collector interface { + // Collect uses to collect the SQL execution cpu time. + // ts is a Unix time, unit is second. + Collect(ts int64, stats []SQLCPUResult) +} + +// SQLCPUResult contains the SQL meta and cpu time. +type SQLCPUResult struct { + SQLDigest []byte + PlanDigest []byte + CPUTimeMs uint32 +} + +type sqlCPUProfiler struct { + taskCh chan *profileData + + mu struct { + sync.Mutex + ept *exportProfileTask + } + collector atomic.Value +} + +var ( + defaultProfileBufSize = 100 * 1024 + profileBufPool = sync.Pool{ + New: func() interface{} { + return bytes.NewBuffer(make([]byte, 0, defaultProfileBufSize)) + }, + } +) + +// newSQLCPUProfiler create a sqlCPUProfiler. +func newSQLCPUProfiler() *sqlCPUProfiler { + return &sqlCPUProfiler{ + taskCh: make(chan *profileData, 128), + } +} + +func (sp *sqlCPUProfiler) Run() { + logutil.BgLogger().Info("cpu profiler started") + go sp.startCPUProfileWorker() + go sp.startAnalyzeProfileWorker() +} + +func (sp *sqlCPUProfiler) SetCollector(c Collector) { + sp.collector.Store(c) +} + +func (sp *sqlCPUProfiler) GetCollector() Collector { + c, ok := sp.collector.Load().(Collector) + if !ok || c == nil { + return nil + } + return c +} + +func (sp *sqlCPUProfiler) startCPUProfileWorker() { + defer util.Recover("top-sql", "profileWorker", nil, false) + for { + if sp.IsEnabled() { + sp.doCPUProfile() + } else { + time.Sleep(time.Second) + } + } +} + +func (sp *sqlCPUProfiler) doCPUProfile() { + intervalSecond := variable.TopSQLVariable.PrecisionSeconds.Load() + task := sp.newProfileTask() + if err := pprof.StartCPUProfile(task.buf); err != nil { + // Sleep a while before retry. + time.Sleep(time.Second) + sp.putTaskToBuffer(task) + return + } + ns := int64(time.Second)*intervalSecond - int64(time.Now().Nanosecond()) + time.Sleep(time.Nanosecond * time.Duration(ns)) + pprof.StopCPUProfile() + task.end = time.Now().Unix() + sp.taskCh <- task +} + +func (sp *sqlCPUProfiler) startAnalyzeProfileWorker() { + defer util.Recover("top-sql", "analyzeProfileWorker", nil, false) + for { + task := <-sp.taskCh + p, err := profile.ParseData(task.buf.Bytes()) + if err != nil { + logutil.BgLogger().Error("parse profile error", zap.Error(err)) + sp.putTaskToBuffer(task) + continue + } + stats := sp.parseCPUProfileBySQLLabels(p) + sp.handleExportProfileTask(p) + if c := sp.GetCollector(); c != nil { + c.Collect(task.end, stats) + } + sp.putTaskToBuffer(task) + } +} + +type profileData struct { + buf *bytes.Buffer + end int64 +} + +func (sp *sqlCPUProfiler) newProfileTask() *profileData { + buf := profileBufPool.Get().(*bytes.Buffer) + return &profileData{ + buf: buf, + } +} + +func (sp *sqlCPUProfiler) putTaskToBuffer(task *profileData) { + task.buf.Reset() + profileBufPool.Put(task.buf) +} + +// parseCPUProfileBySQLLabels uses to aggregate the cpu-profile sample data by sql_digest and plan_digest labels, +// output the SQLCPUResult slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ +// The sql_digest label is been set by `SetSQLLabels` function after parse the SQL. +// The plan_digest label is been set by `SetSQLAndPlanLabels` function after build the SQL plan. +// Since `sqlCPUProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data +// without those label will be ignore. +func (sp *sqlCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLCPUResult { + sqlMap := make(map[string]*sqlStats) + idx := len(p.SampleType) - 1 + for _, s := range p.Sample { + digests, ok := s.Label[labelSQLDigest] + if !ok || len(digests) == 0 { + continue + } + for _, digest := range digests { + stmt, ok := sqlMap[digest] + if !ok { + stmt = &sqlStats{ + plans: make(map[string]int64), + total: 0, + } + sqlMap[digest] = stmt + } + stmt.total += s.Value[idx] + + plans := s.Label[labelPlanDigest] + for _, plan := range plans { + stmt.plans[plan] += s.Value[idx] + } + } + } + return sp.createSQLStats(sqlMap) +} + +func (sp *sqlCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []SQLCPUResult { + stats := make([]SQLCPUResult, 0, len(sqlMap)) + for sqlDigest, stmt := range sqlMap { + stmt.tune() + for planDigest, val := range stmt.plans { + stats = append(stats, SQLCPUResult{ + SQLDigest: []byte(sqlDigest), + PlanDigest: []byte(planDigest), + CPUTimeMs: uint32(time.Duration(val).Milliseconds()), + }) + } + } + return stats +} + +type sqlStats struct { + plans map[string]int64 + total int64 +} + +// tune use to adjust sql stats. Consider following situation: +// The `sqlStats` maybe: +// plans: { +// "table_scan": 200ms, // The cpu time of the sql that plan with `table_scan` is 200ms. +// "index_scan": 300ms, // The cpu time of the sql that plan with `table_scan` is 300ms. +// }, +// total: 600ms, // The total cpu time of the sql is 600ms. +// total_time - table_scan_time - index_scan_time = 100ms, and this 100ms means those sample data only contain the +// sql_digest label, doesn't contain the plan_digest label. This is cause by the `pprof profile` is base on sample, +// and the plan digest can only be set after optimizer generated execution plan. So the remain 100ms means the plan +// optimizer takes time to generated plan. +// After this tune function, the `sqlStats` become to: +// plans: { +// "" : 100ms, // 600 - 200 - 300 = 100ms, indicate the optimizer generated plan time cost. +// "table_scan": 200ms, +// "index_scan": 300ms, +// }, +// total: 600ms, +func (s *sqlStats) tune() { + if len(s.plans) == 0 { + s.plans[""] = s.total + return + } + planTotal := int64(0) + for _, v := range s.plans { + planTotal += v + } + optimize := s.total - planTotal + if optimize <= 0 { + return + } + s.plans[""] += optimize +} + +func (sp *sqlCPUProfiler) handleExportProfileTask(p *profile.Profile) { + sp.mu.Lock() + defer sp.mu.Unlock() + if sp.mu.ept == nil { + return + } + sp.mu.ept.mergeProfile(p) +} + +func (sp *sqlCPUProfiler) hasExportProfileTask() bool { + sp.mu.Lock() + has := sp.mu.ept != nil + sp.mu.Unlock() + return has +} + +// IsEnabled return true if it is(should be) enabled. It exports for tests. +func (sp *sqlCPUProfiler) IsEnabled() bool { + return variable.TopSQLEnabled() || sp.hasExportProfileTask() +} + +// StartCPUProfile same like pprof.StartCPUProfile. +// Because the GlobalSQLCPUProfiler keep calling pprof.StartCPUProfile to fetch SQL cpu stats, other place (such pprof profile HTTP API handler) call pprof.StartCPUProfile will be failed, +// other place should call tracecpu.StartCPUProfile instead of pprof.StartCPUProfile. +func StartCPUProfile(w io.Writer) error { + if GlobalSQLCPUProfiler.IsEnabled() { + return GlobalSQLCPUProfiler.startExportCPUProfile(w) + } + return pprof.StartCPUProfile(w) +} + +// StopCPUProfile same like pprof.StopCPUProfile. +// other place should call tracecpu.StopCPUProfile instead of pprof.StopCPUProfile. +func StopCPUProfile() error { + if GlobalSQLCPUProfiler.IsEnabled() { + return GlobalSQLCPUProfiler.stopExportCPUProfile() + } + pprof.StopCPUProfile() + return nil +} + +// CtxWithDigest wrap the ctx with sql digest, if plan digest is not null, wrap with plan digest too. +func CtxWithDigest(ctx context.Context, sqlDigest, planDigest []byte) context.Context { + if len(planDigest) == 0 { + return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, string(hack.String(sqlDigest)))) + } + return pprof.WithLabels(ctx, pprof.Labels(labelSQLDigest, string(hack.String(sqlDigest)), + labelPlanDigest, string(hack.String(planDigest)))) +} + +func (sp *sqlCPUProfiler) startExportCPUProfile(w io.Writer) error { + sp.mu.Lock() + defer sp.mu.Unlock() + if sp.mu.ept != nil { + return errors.New("cpu profiling already in use") + } + sp.mu.ept = &exportProfileTask{w: w} + return nil +} + +func (sp *sqlCPUProfiler) stopExportCPUProfile() error { + sp.mu.Lock() + ept := sp.mu.ept + sp.mu.ept = nil + sp.mu.Unlock() + if ept.err != nil { + return ept.err + } + if w := ept.w; w != nil && ept.cpuProfile != nil { + sp.removeLabel(ept.cpuProfile) + return ept.cpuProfile.Write(w) + } + return nil +} + +// removeLabel uses to remove labels for export cpu profile data. +// Since the sql_digest and plan_digest label is strange for other users. +// If `variable.EnablePProfSQLCPU` is true means wanto keep the `sql` label, otherwise, remove the `sql` label too. +func (sp *sqlCPUProfiler) removeLabel(p *profile.Profile) { + if p == nil { + return + } + keepLabelSQL := variable.EnablePProfSQLCPU.Load() + for _, s := range p.Sample { + for k := range s.Label { + switch k { + case labelSQL: + if !keepLabelSQL { + delete(s.Label, k) + } + case labelSQLDigest, labelPlanDigest: + delete(s.Label, k) + } + } + } +} + +type exportProfileTask struct { + cpuProfile *profile.Profile + err error + w io.Writer +} + +func (t *exportProfileTask) mergeProfile(p *profile.Profile) { + if t.err != nil || p == nil { + return + } + ps := make([]*profile.Profile, 0, 2) + if t.cpuProfile != nil { + ps = append(ps, t.cpuProfile) + } + ps = append(ps, p) + t.cpuProfile, t.err = profile.Merge(ps) +} + +// ProfileHTTPHandler is same as pprof.Profile. +// The difference is ProfileHTTPHandler uses tracecpu.StartCPUProfile/StopCPUProfile to fetch profile data. +func ProfileHTTPHandler(w http.ResponseWriter, r *http.Request) { + w.Header().Set("X-Content-Type-Options", "nosniff") + sec, err := strconv.ParseInt(r.FormValue("seconds"), 10, 64) + if sec <= 0 || err != nil { + sec = 30 + } + + if durationExceedsWriteTimeout(r, float64(sec)) { + serveError(w, http.StatusBadRequest, "profile duration exceeds server's WriteTimeout") + return + } + + // Set Content Type assuming StartCPUProfile will work, + // because if it does it starts writing. + w.Header().Set("Content-Type", "application/octet-stream") + w.Header().Set("Content-Disposition", `attachment; filename="profile"`) + + err = StartCPUProfile(w) + if err != nil { + serveError(w, http.StatusInternalServerError, "Could not enable CPU profiling: "+err.Error()) + return + } + // TODO: fix me. + // This can be fixed by always starts a 1 second profiling one by one, + // but to aggregate (merge) multiple profiles into one according to the precision. + // |<-- 1s -->| + // -|----------|----------|----------|----------|----------|-----------|-----> Background profile task timeline. + // |________________________________| + // (start cpu profile) v v (stop cpu profile) // expected profile timeline + // |________________________________| // actual profile timeline + time.Sleep(time.Second * time.Duration(sec)) + err = StopCPUProfile() + if err != nil { + serveError(w, http.StatusInternalServerError, "Could not enable CPU profiling: "+err.Error()) + return + } +} + +func durationExceedsWriteTimeout(r *http.Request, seconds float64) bool { + srv, ok := r.Context().Value(http.ServerContextKey).(*http.Server) + return ok && srv.WriteTimeout != 0 && seconds >= srv.WriteTimeout.Seconds() +} + +func serveError(w http.ResponseWriter, status int, txt string) { + w.Header().Set("Content-Type", "text/plain; charset=utf-8") + w.Header().Set("X-Go-Pprof", "1") + w.Header().Del("Content-Disposition") + w.WriteHeader(status) + _, err := fmt.Fprintln(w, txt) + if err != nil { + logutil.BgLogger().Info("write http response error", zap.Error(err)) + } +} diff --git a/util/topsql/tracecpu_test.go b/util/topsql/tracecpu_test.go new file mode 100644 index 0000000000000..6091457a9db07 --- /dev/null +++ b/util/topsql/tracecpu_test.go @@ -0,0 +1,143 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package topsql_test + +import ( + "bytes" + "context" + "testing" + "time" + + "github.com/google/pprof/profile" + . "github.com/pingcap/check" + "github.com/pingcap/parser" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/topsql" + "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tidb/util/topsql/tracecpu/mock" +) + +func TestT(t *testing.T) { + CustomVerboseFlag = true + TestingT(t) +} + +var _ = SerialSuites(&testSuite{}) + +type testSuite struct{} + +func (s *testSuite) SetUpSuite(c *C) { + variable.TopSQLVariable.Enable.Store(true) + variable.TopSQLVariable.AgentAddress.Store("mock") + variable.TopSQLVariable.PrecisionSeconds.Store(1) + tracecpu.GlobalSQLCPUProfiler.Run() +} + +func (s *testSuite) TestTopSQLCPUProfile(c *C) { + collector := mock.NewTopSQLCollector() + tracecpu.GlobalSQLCPUProfiler.SetCollector(collector) + reqs := []struct { + sql string + plan string + }{ + {"select * from t where a=?", "point-get"}, + {"select * from t where a>?", "table-scan"}, + {"insert into t values (?)", ""}, + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + for _, req := range reqs { + go func(sql, plan string) { + for { + select { + case <-ctx.Done(): + return + default: + s.mockExecuteSQL(sql, plan) + } + } + }(req.sql, req.plan) + } + + // test for StartCPUProfile. + buf := bytes.NewBuffer(nil) + err := tracecpu.StartCPUProfile(buf) + c.Assert(err, IsNil) + collector.WaitCollectCnt(2) + err = tracecpu.StopCPUProfile() + c.Assert(err, IsNil) + _, err = profile.Parse(buf) + c.Assert(err, IsNil) + + for _, req := range reqs { + stats := collector.GetSQLStatsBySQLWithRetry(req.sql, len(req.plan) > 0) + c.Assert(len(stats), Equals, 1) + sql := collector.GetSQL(stats[0].SQLDigest) + plan := collector.GetPlan(stats[0].PlanDigest) + c.Assert(sql, Equals, req.sql) + c.Assert(plan, Equals, req.plan) + } +} + +func (s *testSuite) TestIsEnabled(c *C) { + s.setTopSQLEnable(false) + c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsFalse) + + s.setTopSQLEnable(true) + err := tracecpu.StartCPUProfile(bytes.NewBuffer(nil)) + c.Assert(err, IsNil) + c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsTrue) + s.setTopSQLEnable(false) + c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsTrue) + err = tracecpu.StopCPUProfile() + c.Assert(err, IsNil) + + s.setTopSQLEnable(false) + c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsFalse) + s.setTopSQLEnable(true) + c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsTrue) +} + +func (s *testSuite) setTopSQLEnable(enabled bool) { + variable.TopSQLVariable.Enable.Store(enabled) +} + +func (s *testSuite) mockExecuteSQL(sql, plan string) { + ctx := context.Background() + sqlDigest := mock.GenSQLDigest(sql) + topsql.AttachSQLInfo(ctx, sql, sqlDigest, "", nil) + s.mockExecute(time.Millisecond * 100) + planDigest := genDigest(plan) + topsql.AttachSQLInfo(ctx, sql, sqlDigest, plan, planDigest) + s.mockExecute(time.Millisecond * 300) +} + +func genDigest(str string) *parser.Digest { + if str == "" { + return parser.NewDigest(nil) + } + return parser.DigestNormalized(str) +} + +func (s *testSuite) mockExecute(d time.Duration) { + start := time.Now() + for { + for i := 0; i < 10e5; i++ { + } + if time.Since(start) > d { + return + } + } +} From 6c44ec29d88c9102799ab76f706097f2a70e75b5 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Wed, 2 Jun 2021 13:01:47 +0800 Subject: [PATCH 232/343] execdetails: make `ConcurrencyInfo` only appear once in explain analyze (#24514) --- util/execdetails/execdetails.go | 8 ++------ util/execdetails/execdetails_test.go | 5 +---- 2 files changed, 3 insertions(+), 10 deletions(-) diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 16a17b656c1cc..97f9e9611513d 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -734,12 +734,8 @@ func (e *RuntimeStatsWithConcurrencyInfo) String() string { } // Merge implements the RuntimeStats interface. -func (e *RuntimeStatsWithConcurrencyInfo) Merge(rs RuntimeStats) { - tmp, ok := rs.(*RuntimeStatsWithConcurrencyInfo) - if !ok { - return - } - e.concurrency = append(e.concurrency, tmp.concurrency...) +func (e *RuntimeStatsWithConcurrencyInfo) Merge(_ RuntimeStats) { + return } // RuntimeStatsWithCommit is the RuntimeStats with commit detail. diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index 827410cb04350..3d3a70959c180 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -272,11 +272,8 @@ func TestRootRuntimeStats(t *testing.T) { stmtStats.RegisterStats(pid, &RuntimeStatsWithCommit{ Commit: commitDetail, }) - concurrency = &RuntimeStatsWithConcurrencyInfo{} - concurrency.SetConcurrencyInfo(NewConcurrencyInfo("concurrent", 0)) - stmtStats.RegisterStats(pid, concurrency) stats := stmtStats.GetRootStats(1) - expect := "time:3s, loops:2, worker:15, concurrent:OFF, commit_txn: {prewrite:1s, get_commit_ts:1s, commit:1s, region_num:5, write_keys:3, write_byte:66, txn_retry:2}" + expect := "time:3s, loops:2, worker:15, commit_txn: {prewrite:1s, get_commit_ts:1s, commit:1s, region_num:5, write_keys:3, write_byte:66, txn_retry:2}" if stats.String() != expect { t.Fatalf("%v != %v", stats.String(), expect) } From 7c3e0361fb78fb0dd71f85fcdcfb9f5e302fb564 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Wed, 2 Jun 2021 13:31:37 +0800 Subject: [PATCH 233/343] executor: support explain analyze for CTE statement (#25023) --- executor/cte.go | 22 ++++++++++++++++------ executor/explain_test.go | 25 +++++++++++++++++++++++++ 2 files changed, 41 insertions(+), 6 deletions(-) diff --git a/executor/cte.go b/executor/cte.go index a5e063e9dc9ee..95921670cb5cf 100644 --- a/executor/cte.go +++ b/executor/cte.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/cteutil" + "github.com/pingcap/tidb/util/disk" "github.com/pingcap/tidb/util/memory" ) @@ -77,6 +78,9 @@ type CTEExec struct { curIter int hCtx *hashContext sel []int + + memTracker *memory.Tracker + diskTracker *disk.Tracker } // Open implements the Executor interface. @@ -93,6 +97,11 @@ func (e *CTEExec) Open(ctx context.Context) (err error) { return err } + e.memTracker = memory.NewTracker(e.id, -1) + e.diskTracker = disk.NewTracker(e.id, -1) + e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) + e.diskTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.DiskTracker) + if e.recursiveExec != nil { if err = e.recursiveExec.Open(ctx); err != nil { return err @@ -103,7 +112,7 @@ func (e *CTEExec) Open(ctx context.Context) (err error) { return err } - setupCTEStorageTracker(e.iterOutTbl, e.ctx) + setupCTEStorageTracker(e.iterOutTbl, e.ctx, e.memTracker, e.diskTracker) } if e.isDistinct { @@ -126,8 +135,8 @@ func (e *CTEExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { e.resTbl.Lock() if !e.resTbl.Done() { defer e.resTbl.Unlock() - resAction := setupCTEStorageTracker(e.resTbl, e.ctx) - iterInAction := setupCTEStorageTracker(e.iterInTbl, e.ctx) + resAction := setupCTEStorageTracker(e.resTbl, e.ctx, e.memTracker, e.diskTracker) + iterInAction := setupCTEStorageTracker(e.iterInTbl, e.ctx, e.memTracker, e.diskTracker) failpoint.Inject("testCTEStorageSpill", func(val failpoint.Value) { if val.(bool) && config.GetGlobalConfig().OOMUseTmpStorage { @@ -323,14 +332,15 @@ func (e *CTEExec) reopenTbls() (err error) { return e.iterInTbl.Reopen() } -func setupCTEStorageTracker(tbl cteutil.Storage, ctx sessionctx.Context) (actionSpill *chunk.SpillDiskAction) { +func setupCTEStorageTracker(tbl cteutil.Storage, ctx sessionctx.Context, parentMemTracker *memory.Tracker, + parentDiskTracker *disk.Tracker) (actionSpill *chunk.SpillDiskAction) { memTracker := tbl.GetMemTracker() memTracker.SetLabel(memory.LabelForCTEStorage) - memTracker.AttachTo(ctx.GetSessionVars().StmtCtx.MemTracker) + memTracker.AttachTo(parentMemTracker) diskTracker := tbl.GetDiskTracker() diskTracker.SetLabel(memory.LabelForCTEStorage) - diskTracker.AttachTo(ctx.GetSessionVars().StmtCtx.DiskTracker) + diskTracker.AttachTo(parentDiskTracker) if config.GetGlobalConfig().OOMUseTmpStorage { actionSpill = tbl.ActionSpill() diff --git a/executor/explain_test.go b/executor/explain_test.go index 8a0b062ef68cb..a0ce1a1eb2e0c 100644 --- a/executor/explain_test.go +++ b/executor/explain_test.go @@ -207,6 +207,7 @@ func (s *testSuite2) TestExplainAnalyzeExecutionInfo(c *C) { s.checkExecutionInfo(c, tk, "explain analyze select * from t") s.checkExecutionInfo(c, tk, "explain analyze select k from t use index(k)") s.checkExecutionInfo(c, tk, "explain analyze select * from t use index(k)") + s.checkExecutionInfo(c, tk, "explain analyze with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000) select * from cte;") tk.MustExec("CREATE TABLE IF NOT EXISTS nation ( N_NATIONKEY BIGINT NOT NULL,N_NAME CHAR(25) NOT NULL,N_REGIONKEY BIGINT NOT NULL,N_COMMENT VARCHAR(152),PRIMARY KEY (N_NATIONKEY));") tk.MustExec("CREATE TABLE IF NOT EXISTS part ( P_PARTKEY BIGINT NOT NULL,P_NAME VARCHAR(55) NOT NULL,P_MFGR CHAR(25) NOT NULL,P_BRAND CHAR(10) NOT NULL,P_TYPE VARCHAR(25) NOT NULL,P_SIZE BIGINT NOT NULL,P_CONTAINER CHAR(10) NOT NULL,P_RETAILPRICE DECIMAL(15,2) NOT NULL,P_COMMENT VARCHAR(23) NOT NULL,PRIMARY KEY (P_PARTKEY));") @@ -320,9 +321,33 @@ func (s *testSuite1) TestCheckActRowsWithUnistore(c *C) { sql: "select count(*) from t_unistore_act_rows group by b", expected: []string{"2", "2", "2", "4"}, }, + { + sql: "with cte(a) as (select a from t_unistore_act_rows) select (select 1 from cte limit 1) from cte;", + expected: []string{"4", "4", "4", "4", "4"}, + }, } for _, test := range tests { checkActRows(c, tk, test.sql, test.expected) } } + +func (s *testSuite2) TestExplainAnalyzeCTEMemoryAndDiskInfo(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int)") + tk.MustExec("insert into t with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000) select * from cte;") + + rows := tk.MustQuery("explain analyze with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000)" + + " select * from cte, t;").Rows() + + c.Assert(rows[4][7].(string), Not(Equals), "N/A") + c.Assert(rows[4][8].(string), Equals, "0 Bytes") + + tk.MustExec("set @@tidb_mem_quota_query=10240;") + rows = tk.MustQuery("explain analyze with recursive cte(a) as (select 1 union select a + 1 from cte where a < 1000)" + + " select * from cte, t;").Rows() + + c.Assert(rows[4][7].(string), Not(Equals), "N/A") + c.Assert(rows[4][8].(string), Not(Equals), "N/A") +} From 52e89cb8bfba9acf5e75397b32d898726d7dcab7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 2 Jun 2021 14:15:38 +0800 Subject: [PATCH 234/343] planner/core: support union all for mpp. (#24287) --- executor/mpp_gather.go | 24 ++-- executor/tiflash_test.go | 42 ++++++ expression/column.go | 3 +- planner/core/exhaust_physical_plans.go | 34 ++++- planner/core/fragment.go | 139 ++++++++++++++++--- planner/core/initialize.go | 2 +- planner/core/logical_plan_builder.go | 4 + planner/core/physical_plans.go | 45 +++++- planner/core/plan.go | 3 + planner/core/rule_inject_extra_projection.go | 33 +++++ planner/core/task.go | 24 ++++ store/copr/mpp.go | 1 + 12 files changed, 314 insertions(+), 40 deletions(-) diff --git a/executor/mpp_gather.go b/executor/mpp_gather.go index 7cfeb613c40f6..e517a0130ec4f 100644 --- a/executor/mpp_gather.go +++ b/executor/mpp_gather.go @@ -50,7 +50,7 @@ type MPPGather struct { respIter distsql.SelectResult } -func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment, tasks []*kv.MPPTask, isRoot bool) error { +func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error { dagReq, _, err := constructDAGReq(e.ctx, []plannercore.PhysicalPlan{pf.ExchangeSender}, kv.TiFlash) if err != nil { return errors.Trace(err) @@ -58,12 +58,12 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment, tasks []*kv.M for i := range pf.ExchangeSender.Schema().Columns { dagReq.OutputOffsets = append(dagReq.OutputOffsets, uint32(i)) } - if !isRoot { + if !pf.IsRoot { dagReq.EncodeType = tipb.EncodeType_TypeCHBlock } else { dagReq.EncodeType = tipb.EncodeType_TypeChunk } - for _, mppTask := range tasks { + for _, mppTask := range pf.ExchangeSender.Tasks { err := updateExecutorTableID(context.Background(), dagReq.RootExecutor, mppTask.TableID, true) if err != nil { return errors.Trace(err) @@ -77,7 +77,7 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment, tasks []*kv.M Data: pbData, Meta: mppTask.Meta, ID: mppTask.ID, - IsRoot: isRoot, + IsRoot: pf.IsRoot, Timeout: 10, SchemaVar: e.is.SchemaMetaVersion(), StartTs: e.startTS, @@ -85,12 +85,6 @@ func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment, tasks []*kv.M } e.mppReqs = append(e.mppReqs, req) } - for _, r := range pf.ExchangeReceivers { - err = e.appendMPPDispatchReq(r.GetExchangeSender().Fragment, r.Tasks, false) - if err != nil { - return errors.Trace(err) - } - } return nil } @@ -108,13 +102,15 @@ func (e *MPPGather) Open(ctx context.Context) (err error) { // TODO: Move the construct tasks logic to planner, so we can see the explain results. sender := e.originalPlan.(*plannercore.PhysicalExchangeSender) planIDs := collectPlanIDS(e.originalPlan, nil) - rootTasks, err := plannercore.GenerateRootMPPTasks(e.ctx, e.startTS, sender, e.is) + frags, err := plannercore.GenerateRootMPPTasks(e.ctx, e.startTS, sender, e.is) if err != nil { return errors.Trace(err) } - err = e.appendMPPDispatchReq(sender.Fragment, rootTasks, true) - if err != nil { - return errors.Trace(err) + for _, frag := range frags { + err = e.appendMPPDispatchReq(frag) + if err != nil { + return errors.Trace(err) + } } failpoint.Inject("checkTotalMPPTasks", func(val failpoint.Value) { if val.(int) != len(e.mppReqs) { diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index 690106a9a38e5..e442a219ac1d2 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -425,6 +425,48 @@ func (s *tiflashTestSuite) TestMppGoroutinesExitFromErrors(c *C) { c.Assert(failpoint.Disable(hang), IsNil) } +func (s *tiflashTestSuite) TestMppUnionAll(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists x1") + tk.MustExec("create table x1(a int , b int);") + tk.MustExec("alter table x1 set tiflash replica 1") + tk.MustExec("drop table if exists x2") + tk.MustExec("create table x2(a int , b int);") + tk.MustExec("alter table x2 set tiflash replica 1") + tb := testGetTableByName(c, tk.Se, "test", "x1") + err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + tb = testGetTableByName(c, tk.Se, "test", "x2") + err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + + tk.MustExec("insert into x1 values (1, 1), (2, 2), (3, 3), (4, 4)") + tk.MustExec("insert into x2 values (5, 1), (2, 2), (3, 3), (4, 4)") + + // test join + union (join + select) + tk.MustQuery("select x1.a, x.a from x1 left join (select x2.b a, x1.b from x1 join x2 on x1.a = x2.b union all select * from x1 ) x on x1.a = x.a order by x1.a").Check(testkit.Rows("1 1", "1 1", "2 2", "2 2", "3 3", "3 3", "4 4", "4 4")) + tk.MustQuery("select x1.a, x.a from x1 left join (select count(*) a, sum(b) b from x1 group by a union all select * from x2 ) x on x1.a = x.a order by x1.a").Check(testkit.Rows("1 1", "1 1", "1 1", "1 1", "2 2", "3 3", "4 4")) + + tk.MustExec("drop table if exists x3") + tk.MustExec("create table x3(a int , b int);") + tk.MustExec("alter table x3 set tiflash replica 1") + tb = testGetTableByName(c, tk.Se, "test", "x3") + err = domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + + tk.MustExec("insert into x3 values (2, 2), (2, 3), (2, 4)") + // test nested union all + tk.MustQuery("select count(*) from (select a, b from x1 union all select a, b from x3 union all (select x1.a, x3.b from (select * from x3 union all select * from x2) x3 left join x1 on x3.a = x1.b))").Check(testkit.Rows("14")) + // test union all join union all + tk.MustQuery("select count(*) from (select * from x1 union all select * from x2 union all select * from x3) x join (select * from x1 union all select * from x2 union all select * from x3) y on x.a = y.b").Check(testkit.Rows("29")) + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count=100000") + failpoint.Enable("github.com/pingcap/tidb/executor/checkTotalMPPTasks", `return(6)`) + tk.MustQuery("select count(*) from (select * from x1 union all select * from x2 union all select * from x3) x join (select * from x1 union all select * from x2 union all select * from x3) y on x.a = y.b").Check(testkit.Rows("29")) + failpoint.Disable("github.com/pingcap/tidb/executor/checkTotalMPPTasks") + +} + func (s *tiflashTestSuite) TestMppApply(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/expression/column.go b/expression/column.go index 006b9a3867cda..ebc0feaf93a06 100644 --- a/expression/column.go +++ b/expression/column.go @@ -38,10 +38,9 @@ type CorrelatedColumn struct { // Clone implements Expression interface. func (col *CorrelatedColumn) Clone() Expression { - var d types.Datum return &CorrelatedColumn{ Column: col.Column, - Data: &d, + Data: col.Data, } } diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index dcb4e991ebe33..b2ba91ff99dcd 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2126,7 +2126,7 @@ func (p *baseLogicalPlan) canPushToCop(storeTp kv.StoreType) bool { } } ret = ret && validDs - case *LogicalAggregation, *LogicalProjection, *LogicalSelection, *LogicalJoin: + case *LogicalAggregation, *LogicalProjection, *LogicalSelection, *LogicalJoin, *LogicalUnionAll: if storeTp == kv.TiFlash { ret = ret && c.canPushToCop(storeTp) } else { @@ -2494,15 +2494,41 @@ func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { // TODO: UnionAll can not pass any order, but we can change it to sort merge to keep order. - if !prop.IsEmpty() || prop.IsFlashProp() { + if !prop.IsEmpty() || (prop.IsFlashProp() && prop.TaskTp != property.MppTaskType) { + return nil, true, nil + } + // TODO: UnionAll can pass partition info, but for briefness, we prevent it from pushing down. + if prop.TaskTp == property.MppTaskType && prop.PartitionTp != property.AnyType { return nil, true, nil } + canUseMpp := p.ctx.GetSessionVars().IsMPPAllowed() && p.canPushToCop(kv.TiFlash) chReqProps := make([]*property.PhysicalProperty, 0, len(p.children)) for range p.children { - chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: prop.ExpectedCnt}) + if canUseMpp && prop.TaskTp == property.MppTaskType { + chReqProps = append(chReqProps, &property.PhysicalProperty{ + ExpectedCnt: prop.ExpectedCnt, + TaskTp: property.MppTaskType, + }) + } else { + chReqProps = append(chReqProps, &property.PhysicalProperty{ExpectedCnt: prop.ExpectedCnt}) + } } - ua := PhysicalUnionAll{}.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) + ua := PhysicalUnionAll{ + mpp: canUseMpp && prop.TaskTp == property.MppTaskType, + }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) ua.SetSchema(p.Schema()) + if canUseMpp && prop.TaskTp == property.RootTaskType { + chReqProps = make([]*property.PhysicalProperty, 0, len(p.children)) + for range p.children { + chReqProps = append(chReqProps, &property.PhysicalProperty{ + ExpectedCnt: prop.ExpectedCnt, + TaskTp: property.MppTaskType, + }) + } + mppUA := PhysicalUnionAll{mpp: true}.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) + mppUA.SetSchema(p.Schema()) + return []PhysicalPlan{ua, mppUA}, true, nil + } return []PhysicalPlan{ua}, true, nil } diff --git a/planner/core/fragment.go b/planner/core/fragment.go index d0ab808c742e7..7315da176e6b8 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -38,32 +38,49 @@ type Fragment struct { // following fields are filled after scheduling. ExchangeSender *PhysicalExchangeSender // data exporter + + IsRoot bool +} + +type tasksAndFrags struct { + tasks []*kv.MPPTask + frags []*Fragment } type mppTaskGenerator struct { ctx sessionctx.Context startTS uint64 is infoschema.InfoSchema + frags []*Fragment + cache map[int]tasksAndFrags } // GenerateRootMPPTasks generate all mpp tasks and return root ones. -func GenerateRootMPPTasks(ctx sessionctx.Context, startTs uint64, sender *PhysicalExchangeSender, is infoschema.InfoSchema) ([]*kv.MPPTask, error) { - g := &mppTaskGenerator{ctx: ctx, startTS: startTs, is: is} +func GenerateRootMPPTasks(ctx sessionctx.Context, startTs uint64, sender *PhysicalExchangeSender, is infoschema.InfoSchema) ([]*Fragment, error) { + g := &mppTaskGenerator{ + ctx: ctx, + startTS: startTs, + is: is, + cache: make(map[int]tasksAndFrags), + } return g.generateMPPTasks(sender) } -func (e *mppTaskGenerator) generateMPPTasks(s *PhysicalExchangeSender) ([]*kv.MPPTask, error) { +func (e *mppTaskGenerator) generateMPPTasks(s *PhysicalExchangeSender) ([]*Fragment, error) { logutil.BgLogger().Info("Mpp will generate tasks", zap.String("plan", ToString(s))) tidbTask := &kv.MPPTask{ StartTs: e.startTS, ID: -1, } - rootTasks, err := e.generateMPPTasksForFragment(s) + _, frags, err := e.generateMPPTasksForExchangeSender(s) if err != nil { return nil, errors.Trace(err) } - s.TargetTasks = []*kv.MPPTask{tidbTask} - return rootTasks, nil + for _, frag := range frags { + frag.ExchangeSender.TargetTasks = []*kv.MPPTask{tidbTask} + frag.IsRoot = true + } + return e.frags, nil } type mppAddr struct { @@ -105,6 +122,8 @@ func (f *Fragment) init(p PhysicalPlan) error { f.TableScan = x case *PhysicalExchangeReceiver: f.ExchangeReceivers = append(f.ExchangeReceivers, x) + case *PhysicalUnionAll: + return errors.New("unexpected union all detected") default: for _, ch := range p.Children() { if err := f.init(ch); err != nil { @@ -115,20 +134,107 @@ func (f *Fragment) init(p PhysicalPlan) error { return nil } -func newFragment(s *PhysicalExchangeSender) (*Fragment, error) { - f := &Fragment{ExchangeSender: s} - s.Fragment = f - err := f.init(s) - return f, errors.Trace(err) +// We would remove all the union-all operators by 'untwist'ing and copying the plans above union-all. +// This will make every route from root (ExchangeSender) to leaf nodes (ExchangeReceiver and TableScan) +// a new ioslated tree (and also a fragment) without union all. These trees (fragments then tasks) will +// finally be gathered to TiDB or be exchanged to upper tasks again. +// For instance, given a plan "select c1 from t union all select c1 from s" +// after untwist, there will be two plans in `forest` slice: +// - ExchangeSender -> Projection (c1) -> TableScan(t) +// - ExchangeSender -> Projection (c2) -> TableScan(s) +func untwistPlanAndRemoveUnionAll(stack []PhysicalPlan, forest *[]*PhysicalExchangeSender) error { + cur := stack[len(stack)-1] + switch x := cur.(type) { + case *PhysicalTableScan, *PhysicalExchangeReceiver: // This should be the leave node. + p, err := stack[0].Clone() + if err != nil { + return errors.Trace(err) + } + *forest = append(*forest, p.(*PhysicalExchangeSender)) + for i := 1; i < len(stack); i++ { + if _, ok := stack[i].(*PhysicalUnionAll); ok { + continue + } + ch, err := stack[i].Clone() + if err != nil { + return errors.Trace(err) + } + if join, ok := p.(*PhysicalHashJoin); ok { + join.SetChild(1-join.InnerChildIdx, ch) + } else { + p.SetChildren(ch) + } + p = ch + } + case *PhysicalHashJoin: + stack = append(stack, x.children[1-x.InnerChildIdx]) + err := untwistPlanAndRemoveUnionAll(stack, forest) + stack = stack[:len(stack)-1] + return errors.Trace(err) + case *PhysicalUnionAll: + for _, ch := range x.children { + stack = append(stack, ch) + err := untwistPlanAndRemoveUnionAll(stack, forest) + stack = stack[:len(stack)-1] + if err != nil { + return errors.Trace(err) + } + } + default: + if len(cur.Children()) != 1 { + return errors.Trace(errors.New("unexpected plan " + cur.ExplainID().String())) + } + ch := cur.Children()[0] + stack = append(stack, ch) + err := untwistPlanAndRemoveUnionAll(stack, forest) + stack = stack[:len(stack)-1] + return errors.Trace(err) + } + return nil } -func (e *mppTaskGenerator) generateMPPTasksForFragment(s *PhysicalExchangeSender) (tasks []*kv.MPPTask, err error) { - f, err := newFragment(s) +func buildFragments(s *PhysicalExchangeSender) ([]*Fragment, error) { + forest := make([]*PhysicalExchangeSender, 0, 1) + err := untwistPlanAndRemoveUnionAll([]PhysicalPlan{s}, &forest) if err != nil { return nil, errors.Trace(err) } + fragments := make([]*Fragment, 0, len(forest)) + for _, s := range forest { + f := &Fragment{ExchangeSender: s} + err = f.init(s) + if err != nil { + return nil, errors.Trace(err) + } + fragments = append(fragments, f) + } + return fragments, nil +} + +func (e *mppTaskGenerator) generateMPPTasksForExchangeSender(s *PhysicalExchangeSender) ([]*kv.MPPTask, []*Fragment, error) { + if cached, ok := e.cache[s.ID()]; ok { + return cached.tasks, cached.frags, nil + } + frags, err := buildFragments(s) + if err != nil { + return nil, nil, errors.Trace(err) + } + results := make([]*kv.MPPTask, 0, len(frags)) + for _, f := range frags { + tasks, err := e.generateMPPTasksForFragment(f) + if err != nil { + return nil, nil, errors.Trace(err) + } + results = append(results, tasks...) + } + e.frags = append(e.frags, frags...) + e.cache[s.ID()] = tasksAndFrags{results, frags} + return results, frags, nil +} + +func (e *mppTaskGenerator) generateMPPTasksForFragment(f *Fragment) (tasks []*kv.MPPTask, err error) { for _, r := range f.ExchangeReceivers { - r.Tasks, err = e.generateMPPTasksForFragment(r.GetExchangeSender()) + r.Tasks, r.frags, err = e.generateMPPTasksForExchangeSender(r.GetExchangeSender()) if err != nil { return nil, errors.Trace(err) } @@ -149,8 +255,9 @@ func (e *mppTaskGenerator) generateMPPTasksForFragment(s *PhysicalExchangeSender return nil, errors.New("cannot find mpp task") } for _, r := range f.ExchangeReceivers { - s := r.GetExchangeSender() - s.TargetTasks = tasks + for _, frag := range r.frags { + frag.ExchangeSender.TargetTasks = append(frag.ExchangeSender.TargetTasks, tasks...) + } } f.ExchangeSender.Tasks = tasks return tasks, nil diff --git a/planner/core/initialize.go b/planner/core/initialize.go index f41340147abfe..cbf099baf4113 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -419,7 +419,7 @@ func (p PhysicalTableReader) Init(ctx sessionctx.Context, offset int) *PhysicalT if p.tablePlan != nil { p.TablePlans = flattenPushDownPlan(p.tablePlan) p.schema = p.tablePlan.Schema() - if p.StoreType == kv.TiFlash && !p.GetTableScan().KeepOrder { + if p.StoreType == kv.TiFlash && p.GetTableScan() != nil && !p.GetTableScan().KeepOrder { // When allow batch cop is 1, only agg / topN uses batch cop. // When allow batch cop is 2, every query uses batch cop. switch ctx.GetSessionVars().AllowBatchCop { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index e6b2c6b6ef20b..ccb93029edd25 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -1378,6 +1378,10 @@ func (b *PlanBuilder) buildProjection4Union(ctx context.Context, u *LogicalUnion b.optFlag |= flagEliminateProjection proj := LogicalProjection{Exprs: exprs, AvoidColumnEvaluator: true}.Init(b.ctx, b.getSelectOffset()) proj.SetSchema(u.schema.Clone()) + // reset the schema type to make the "not null" flag right. + for i, expr := range exprs { + proj.schema.Columns[i].RetType = expr.GetType() + } proj.SetChildren(child) u.children[childID] = proj } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 5602399a63b74..74c70e1fce3c9 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -111,7 +111,10 @@ func (p *PhysicalTableReader) GetTableScan() *PhysicalTableScan { } else if chCnt == 1 { curPlan = curPlan.Children()[0] } else { - join := curPlan.(*PhysicalHashJoin) + join, ok := curPlan.(*PhysicalHashJoin) + if !ok { + return nil + } curPlan = join.children[1-join.globalChildIndex] } } @@ -886,6 +889,18 @@ type PhysicalExchangeReceiver struct { basePhysicalPlan Tasks []*kv.MPPTask + frags []*Fragment +} + +// Clone implment PhysicalPlan interface. +func (p *PhysicalExchangeReceiver) Clone() (PhysicalPlan, error) { + np := new(PhysicalExchangeReceiver) + base, err := p.basePhysicalPlan.cloneWithSelf(np) + if err != nil { + return nil, errors.Trace(err) + } + np.basePhysicalPlan = *base + return np, nil } // GetExchangeSender return the connected sender of this receiver. We assume that its child must be a receiver. @@ -900,10 +915,21 @@ type PhysicalExchangeSender struct { TargetTasks []*kv.MPPTask ExchangeType tipb.ExchangeType HashCols []*expression.Column - // Tasks is the mpp task for current PhysicalExchangeSender + // Tasks is the mpp task for current PhysicalExchangeSender. Tasks []*kv.MPPTask +} - Fragment *Fragment +// Clone implment PhysicalPlan interface. +func (p *PhysicalExchangeSender) Clone() (PhysicalPlan, error) { + np := new(PhysicalExchangeSender) + base, err := p.basePhysicalPlan.cloneWithSelf(np) + if err != nil { + return nil, errors.Trace(err) + } + np.basePhysicalPlan = *base + np.ExchangeType = p.ExchangeType + np.HashCols = p.HashCols + return np, nil } // Clone implements PhysicalPlan interface. @@ -952,6 +978,19 @@ func (p *PhysicalLimit) Clone() (PhysicalPlan, error) { // PhysicalUnionAll is the physical operator of UnionAll. type PhysicalUnionAll struct { physicalSchemaProducer + + mpp bool +} + +// Clone implements PhysicalPlan interface. +func (p *PhysicalUnionAll) Clone() (PhysicalPlan, error) { + cloned := new(PhysicalUnionAll) + base, err := p.physicalSchemaProducer.cloneWithSelf(cloned) + if err != nil { + return nil, err + } + cloned.physicalSchemaProducer = *base + return cloned, nil } // AggMppRunMode defines the running mode of aggregation in MPP diff --git a/planner/core/plan.go b/planner/core/plan.go index dd7e41b77f7fa..2a62afa960b63 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -408,6 +408,9 @@ func (p *basePhysicalPlan) cloneWithSelf(newSelf PhysicalPlan) (*basePhysicalPla base.children = append(base.children, cloned) } for _, prop := range p.childrenReqProps { + if prop == nil { + continue + } base.childrenReqProps = append(base.childrenReqProps, prop.CloneEssentialFields()) } return base, nil diff --git a/planner/core/rule_inject_extra_projection.go b/planner/core/rule_inject_extra_projection.go index 2896a1dade0ff..968917a4fef2e 100644 --- a/planner/core/rule_inject_extra_projection.go +++ b/planner/core/rule_inject_extra_projection.go @@ -14,6 +14,7 @@ package core import ( + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/kv" @@ -62,10 +63,42 @@ func (pe *projInjector) inject(plan PhysicalPlan) PhysicalPlan { plan = InjectProjBelowSort(p, p.ByItems) case *NominalSort: plan = TurnNominalSortIntoProj(p, p.OnlyColumn, p.ByItems) + case *PhysicalUnionAll: + plan = injectProjBelowUnion(p) } return plan } +func injectProjBelowUnion(un *PhysicalUnionAll) *PhysicalUnionAll { + if !un.mpp { + return un + } + for i, ch := range un.children { + exprs := make([]expression.Expression, len(ch.Schema().Columns)) + needChange := false + for i, dstCol := range un.schema.Columns { + dstType := dstCol.RetType + srcCol := ch.Schema().Columns[i] + srcType := srcCol.RetType + if !srcType.Equal(dstType) || !(mysql.HasNotNullFlag(dstType.Flag) == mysql.HasNotNullFlag(srcType.Flag)) { + exprs[i] = expression.BuildCastFunction4Union(un.ctx, srcCol, dstType) + needChange = true + } else { + exprs[i] = srcCol + } + } + if needChange { + proj := PhysicalProjection{ + Exprs: exprs, + }.Init(un.ctx, ch.statsInfo(), 0) + proj.SetSchema(un.schema.Clone()) + proj.SetChildren(ch) + un.children[i] = proj + } + } + return un +} + // wrapCastForAggFunc wraps the args of an aggregate function with a cast function. // If the mode is FinalMode or Partial2Mode, we do not need to wrap cast upon the args, // since the types of the args are already the expected. diff --git a/planner/core/task.go b/planner/core/task.go index fa6855503dd0e..f4c1d1ca72c65 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1300,7 +1300,31 @@ func (p *PhysicalProjection) attach2Task(tasks ...task) task { return t } +func (p *PhysicalUnionAll) attach2MppTasks(tasks ...task) task { + t := &mppTask{p: p} + childPlans := make([]PhysicalPlan, 0, len(tasks)) + var childMaxCost float64 + for _, tk := range tasks { + if mpp, ok := tk.(*mppTask); ok && !tk.invalid() { + childCost := mpp.cost() + if childCost > childMaxCost { + childMaxCost = childCost + } + childPlans = append(childPlans, mpp.plan()) + } else { + return invalidTask + } + } + p.SetChildren(childPlans...) + t.cst = childMaxCost + p.cost = t.cost() + return t +} + func (p *PhysicalUnionAll) attach2Task(tasks ...task) task { + if _, ok := tasks[0].(*mppTask); ok { + return p.attach2MppTasks(tasks...) + } t := &rootTask{p: p} childPlans := make([]PhysicalPlan, 0, len(tasks)) var childMaxCost float64 diff --git a/store/copr/mpp.go b/store/copr/mpp.go index db0aff7e22696..4be45c3288e23 100644 --- a/store/copr/mpp.go +++ b/store/copr/mpp.go @@ -239,6 +239,7 @@ func (m *mppIterator) handleDispatchReq(ctx context.Context, bo *Backoffer, req realResp := rpcResp.Resp.(*mpp.DispatchTaskResponse) if realResp.Error != nil { + logutil.BgLogger().Error("mpp dispatch response meet error", zap.String("error", realResp.Error.Msg)) m.sendError(errors.New(realResp.Error.Msg)) return } From fb81a7dfd81a2b2841edbc4ddbc56b47c45f330f Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Wed, 2 Jun 2021 14:51:38 +0800 Subject: [PATCH 235/343] planner: warn for incremental analyze in version 3 stats (#24689) --- planner/core/integration_test.go | 25 +++++++++++++++++++++++++ planner/core/planbuilder.go | 5 ++++- 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 1e13d28b96e1c..c691e5341b0a5 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3637,6 +3637,31 @@ func (s *testIntegrationSuite) TestIssue24281(c *C) { "UNION select 1 as v1, 2 as v2") } +func (s *testIntegrationSuite) TestIncrementalAnalyzeStatsVer3(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key, b int, index idx_b(b))") + tk.MustExec("insert into t values(1,1),(2,2),(3,3)") + tk.MustExec("set @@session.tidb_analyze_version = 3") + tk.MustExec("analyze table t") + is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tblID := tbl.Meta().ID + rows := tk.MustQuery(fmt.Sprintf("select distinct_count from mysql.stats_histograms where table_id = %d and is_index = 1", tblID)).Rows() + c.Assert(len(rows), Equals, 1) + c.Assert(rows[0][0], Equals, "3") + tk.MustExec("insert into t values(4,4),(5,5),(6,6)") + tk.MustExec("analyze incremental table t index idx_b") + c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 2) + c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, "The version 3 would collect all statistics not only the selected indexes") + c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[1].Err.Error(), Equals, "The version 3 stats would ignore the INCREMENTAL keyword and do full sampling") + rows = tk.MustQuery(fmt.Sprintf("select distinct_count from mysql.stats_histograms where table_id = %d and is_index = 1", tblID)).Rows() + c.Assert(len(rows), Equals, 1) + c.Assert(rows[0][0], Equals, "6") +} + func (s *testIntegrationSuite) TestConflictReadFromStorage(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index f2de374321f25..3990cade06511 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1683,6 +1683,9 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( } idxInfos = append(idxInfos, idx) } + if as.Incremental { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 3 stats would ignore the INCREMENTAL keyword and do full sampling")) + } for i, id := range physicalIDs { if id == tbl.TableInfo.ID { id = -1 @@ -1692,7 +1695,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( TableName: tbl.Name.O, PartitionName: names[i], TableID: AnalyzeTableID{TableID: tbl.TableInfo.ID, PartitionID: id}, - Incremental: as.Incremental, + Incremental: false, StatsVersion: version, } newTask := AnalyzeColumnsTask{ From b4f3ea6f6b9d9f87a7df5e38876d78c81e792a1b Mon Sep 17 00:00:00 2001 From: wjHuang Date: Wed, 2 Jun 2021 15:07:37 +0800 Subject: [PATCH 236/343] cmd: add tests for CTE (#24907) --- cmd/explaintest/r/cte.result | 149 ++++++++++++++++++++++++++++++ cmd/explaintest/t/cte.test | 170 +++++++++++++++++++++++++++++++++++ 2 files changed, 319 insertions(+) create mode 100644 cmd/explaintest/r/cte.result create mode 100644 cmd/explaintest/t/cte.test diff --git a/cmd/explaintest/r/cte.result b/cmd/explaintest/r/cte.result new file mode 100644 index 0000000000000..af8209d6c9e48 --- /dev/null +++ b/cmd/explaintest/r/cte.result @@ -0,0 +1,149 @@ +use test; +drop table if exists tbl_0; +create table tbl_0(a int); +with recursive cte_0 (col_10,col_11,col_12) AS ( select 1, 2,3 from tbl_0 UNION select col_10 + 1,col_11 + 1,col_12 + 1 from cte_0 where col_10 < 10 ) select * from cte_0; +drop table if exists tbl_1; +CREATE TABLE `tbl_1` ( +`col_5` decimal(47,21) NOT NULL DEFAULT '5308.880000000000000000000', +`col_6` enum('Alice','Bob','Charlie','David') DEFAULT NULL, +`col_7` float NOT NULL, +`col_8` bigint NOT NULL DEFAULT '-688199144806783096', +`col_9` varchar(248) NOT NULL, +PRIMARY KEY (`col_5`,`col_7`,`col_9`,`col_8`), +UNIQUE KEY `idx_4` (`col_8`), +UNIQUE KEY `idx_7` (`col_5`,`col_7`,`col_8`), +UNIQUE KEY `idx_9` (`col_9`,`col_8`), +UNIQUE KEY `idx_3` (`col_9`(3),`col_8`), +UNIQUE KEY `idx_8` (`col_7`,`col_6`,`col_8`,`col_5`), +KEY `idx_5` (`col_7`), +KEY `idx_6` (`col_7`), +KEY `idx_10` (`col_9`,`col_5`), +KEY `idx_11` (`col_5`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci +/*!50100 PARTITION BY HASH (`col_8`) +PARTITIONS 4 */; +with recursive cte_1 (col_13,col_14,col_15,col_16,col_17) AS ( with recursive cte_2 (col_18,col_19,col_20,col_21,col_22,col_23,col_24) AS ( select 1, 2,col_8,4,5,6,7 from tbl_1 ) select col_19,col_18,col_22,col_23,col_21 from cte_2 UNION ALL select col_13 + 1,col_14 + 1,col_15 + 1,col_16 + 1,col_17 + 1 from cte_1 where col_13 < 10 ) select * from cte_1; +with recursive cte_256 (col_969,col_970,col_971) AS ( with recursive cte_257 (col_972,col_973,col_974,col_975) AS ( select 1, 2,col_8,4 from tbl_1 UNION select col_972 + 1,col_973 + 1,col_974 + 1,col_975 + 1 from cte_257 where col_972 < 10 ) select col_975,col_974,col_973 from cte_257 UNION DISTINCT select col_969 + 1,col_970 + 1,col_971 + 1 from cte_256 where col_969 < 10 ) select * from cte_256; +drop table if exists tbl_2, tbl_3; +create table tbl_2 ( col_4 char(246) collate utf8_unicode_ci not null , col_5 char(253) collate utf8mb4_unicode_ci ) ; +create table tbl_3 ( col_6 char(207) collate utf8mb4_unicode_ci , col_7 int unsigned not null ) ; +insert into tbl_2 values ( "0",null ) ; +insert into tbl_2 values ( "1","0" ) ; +insert into tbl_2 values ( "1","1" ) ; +insert into tbl_2 values ( "0","0" ) ; +insert into tbl_2 values ( "0","1" ) ; +insert into tbl_3 values ( "1",0 ) ; +insert into tbl_3 values ( "1",1 ) ; +insert into tbl_3 values ( "0",0 ) ; +insert into tbl_3 values ( "0",1 ) ; +with recursive tbl_2 (col_64,col_65,col_66,col_67) AS ( select 1, col_6,col_6,4 from tbl_3 UNION DISTINCT select col_64 + 1,concat(col_65, 1),col_66 + 1,concat(col_67, 1) from tbl_2 where col_64 < 5 ) select * from tbl_2 order by col_64; +drop table if exists tbl_3, tbl_4; +create table tbl_3 ( col_6 int not null , col_7 char(95) collate utf8_general_ci ) ; +create table tbl_4 ( col_8 char collate utf8_unicode_ci , col_9 char collate utf8mb4_bin ) ; +insert into tbl_3 values ( 0,"1" ) ; +insert into tbl_4 values ( "1","0" ) ; +with recursive cte_2245 (col_8692,col_8693) AS ( select 1, col_7 from tbl_3 UNION select col_8692 + 1,concat(col_8693, 1) from cte_2245 where col_8692 < 5 ) , cte_2246 (col_8694,col_8695,col_8696,col_8697) AS ( with recursive cte_2247 (col_8698,col_8699,col_8700,col_8701) AS ( select 1, cast("2" as char(20)),3,col_8 from tbl_4 ) select col_8698,col_8699,col_8700,col_8701 from cte_2247 UNION select col_8694 + 1,col_8695 + 1,col_8696 + 1,col_8697 + 1 from cte_2246 where col_8694 < 5 ) select * from cte_2245,cte_2246 order by col_8692,col_8693,col_8696,col_8695,col_8697,col_8694; +with recursive cte2 as (select 1 as col_1, 2 as col_2) select c1.col_1, c2.col_2 from cte2 as c1, cte2 as c2 where c2.col_2 = 1; +with recursive cte (c1) as (select 1), cte1 (c2) as (select 1 union select c1 + 1 from cte, cte1) select * from cte, cte1; +with recursive tbl_0 (col_943,col_944,col_945,col_946,col_947) AS ( with recursive tbl_0 (col_948,col_949,col_950,col_951,col_952) AS ( select 1, 2,3,4,5 UNION ALL select col_948 + 1,col_949 + 1,col_950 + 1,col_951 + 1,col_952 + 1 from tbl_0 where col_948 < 5 ) select col_948,col_949,col_951,col_950,col_952 from tbl_0 UNION ALL select col_943 + 1,col_944 + 1,col_945 + 1,col_946 + 1,col_947 + 1 from tbl_0 where col_943 < 5 ) select * from tbl_0; +Error 1054: Unknown column 'col_943' in 'where clause' +with recursive cte1 (c1, c2) as (select 1, '1' union select concat(c1, 1), c2 + 1 from cte1 where c1 < 100) select * from cte1; +with recursive cte_8 (col_51,col_52,col_53,col_54) AS ( with recursive cte_9 (col_55,col_56,col_57,col_58) AS ( select 1, 2,3,4 UNION ALL select col_55 + 1,col_56 + 1,col_57 + 1,col_58 + 1 from cte_9 where col_55 < 5 ) select col_55,col_57,col_56,col_58 from cte_9 UNION DISTINCT select col_51 + 1,col_52 + 1,col_53 + 1,col_54 + 1 from cte_8 where col_51 < 5 ) select * from cte_8; +with recursive qn as (select 1 from dual union all select 1 from dual) select * from qn; +with recursive qn as (select 1 as a from dual group by a union all select a+1 from qn where a<3) select * from qn; +with recursive qn as ((select 1 as a from dual order by a) union all select a+1 from qn where a<3) select * from qn; +drop table if exists employees; +CREATE TABLE employees ( +ID INT PRIMARY KEY, +NAME VARCHAR(100), +MANAGER_ID INT, +INDEX (MANAGER_ID), +FOREIGN KEY (MANAGER_ID) REFERENCES employees(ID) +); +INSERT INTO employees VALUES +(333, "Yasmina", NULL), +(198, "John", 333), +(692, "Tarek", 333), +(29, "Pedro", 198), +(4610, "Sarah", 29), +(72, "Pierre", 29), +(123, "Adil", 692); +WITH RECURSIVE employees_extended AS (SELECT ID, NAME, MANAGER_ID, CAST(ID AS CHAR(200)) AS PATH FROM employees WHERE NAME='Pierre' UNION ALL SELECT S.ID, S.NAME, S.MANAGER_ID, CONCAT(M.PATH, ",", S.ID) FROM employees_extended M JOIN employees S ON M.MANAGER_ID=S.ID) SELECT * FROM employees_extended; +with recursive cte (c1) as (select 1), cte1 (c2) as (select 1 union select c1 + 1 from cte where c1 < 10) select * from cte where c1 < 5; +with recursive cte_581 (col_2343,col_2344,col_2345) AS ( select 1, '2',cast('3' as char(20))) , cte_582 (col_2346,col_2347,col_2348) AS ( select 1, 2, 3) select * from cte_581 as cte_as_583,cte_582 as cte_as_584,cte_582 as cte_as_585 order by cte_as_583.col_2343,cte_as_585.col_2348,cte_as_584.col_2346,cte_as_584.col_2348,cte_as_583.col_2344,cte_as_584.col_2347,cte_as_585.col_2346,cte_as_585.col_2347,cte_as_583.col_2345; +with recursive tbl_3 (col_19,col_20,col_21,col_22) AS ( select 1, 2,3,4 UNION select col_19 + 1,col_20 + 1,col_21 + 1,concat(col_22, 1) from tbl_3 where col_19 < 5 ) , cte_4 (col_23,col_24,col_25,col_26) AS ( select 1, 2,cast("3" as char(20)),4 UNION DISTINCT select col_23 + 1,col_24 + 1,concat(col_25, 1),col_26 + 1 from cte_4 where col_23 < 5 ) select * from tbl_3 as cte_as_3,cte_4 as cte_as_4,tbl_3 as cte_as_5 order by cte_as_3.col_19,cte_as_4.col_23,cte_as_4.col_25,cte_as_4.col_24,cte_as_4.col_26,cte_as_3.col_20,cte_as_5.col_22,cte_as_3.col_21,cte_as_5.col_20,cte_as_3.col_22,cte_as_5.col_19,cte_as_5.col_21; +with cte1 (c1) as (select 1) select * from cte1 as b, cte1 as a; +WITH RECURSIVE qn AS +( +select 1 +union all +select 3, 0 from qn +) +select * from qn; +Error 1222: The used SELECT statements have a different number of columns +with recursive cte1 as (select 1 union all (select 1 from cte1 limit 10)) select * from cte1; +Error 1235: This version of TiDB doesn't yet support 'ORDER BY / LIMIT / SELECT DISTINCT in recursive query block of Common Table Expression' +with recursive qn as (select 123 as a union all select null from qn where a is not null) select * from qn; +with recursive q (b) as (select 1, 1 union all select 1, 1 from q) select b from q; +Error 1353: In definition of view, derived table or common table expression, SELECT list and column names list have different column counts +drop table if exists t1; +create table t1(a int); +insert into t1 values(1); +insert into t1 values(2); +SELECT * +FROM +t1 dt +WHERE +EXISTS( +WITH RECURSIVE qn AS (SELECT a*0 AS b UNION ALL SELECT b+1 FROM qn WHERE b=0) +SELECT * FROM qn WHERE b=a +); +a +1 +drop table if exists t1; +create table t1 (a int); +insert into t1 values (1); +SELECT (WITH qn AS (SELECT 10*a as a FROM t1), +qn2 AS (SELECT 3*a AS b FROM qn) SELECT * from qn2 LIMIT 1) +FROM t1; +(WITH qn AS (SELECT 10*a as a FROM t1), +qn2 AS (SELECT 3*a AS b FROM qn) +30 +select (with qn as (select "with") select * from qn) as scal_subq +from dual; +scal_subq +with +drop table if exists t1; +create table t1 (a int); + insert into t1 values(1), (2), (3); +with q as (select * from t1) +select /*+ merge(q) no_merge(q1) */ * from q, q q1 where q.a=1 and q1.a=2; +drop table if exists t1; + create table t1 (a int, b int); +with qn as (select a, b from t1) select b from qn group by a; +drop table if exists t1; +create table t1(a int); +insert into t1 values(1); +insert into t1 values(2); +SELECT * +FROM +t1 dt +WHERE +EXISTS( +WITH RECURSIVE qn AS (SELECT a*0+1 AS b UNION ALL SELECT b+1 FROM qn WHERE b=0) +SELECT * FROM qn WHERE b=1 +); +a +1 +2 +drop table if exists tbl_1; +CREATE TABLE `tbl_1` ( +`col_2` char(65) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, +`col_3` int(11) NOT NULL +); +with recursive cte_8932 (col_34891,col_34892) AS ( with recursive cte_8932 (col_34893,col_34894,col_34895) AS ( with tbl_1 (col_34896,col_34897,col_34898,col_34899) AS ( select 1, "2",3,col_3 from tbl_1 ) select cte_as_8958.col_34896,cte_as_8958.col_34898,cte_as_8958.col_34899 from tbl_1 as cte_as_8958 UNION DISTINCT select col_34893 + 1,concat(col_34894, 1),col_34895 + 1 from cte_8932 where col_34893 < 5 ) select cte_as_8959.col_34893,cte_as_8959.col_34895 from cte_8932 as cte_as_8959 ) select * from cte_8932 as cte_as_8960 order by cte_as_8960.col_34891,cte_as_8960.col_34892; +drop table if exists t1; +create table t1(c1 bigint unsigned); +insert into t1 values(0); +with recursive cte1 as (select c1 - 1 c1 from t1 union all select c1 - 1 c1 from cte1 where c1 != 0) select * from cte1 dt1, cte1 dt2; +Error 1690: BIGINT UNSIGNED value is out of range in '(test.t1.c1 - 1)' diff --git a/cmd/explaintest/t/cte.test b/cmd/explaintest/t/cte.test new file mode 100644 index 0000000000000..b45695e641135 --- /dev/null +++ b/cmd/explaintest/t/cte.test @@ -0,0 +1,170 @@ +use test; +# case 1 +drop table if exists tbl_0; +create table tbl_0(a int); +with recursive cte_0 (col_10,col_11,col_12) AS ( select 1, 2,3 from tbl_0 UNION select col_10 + 1,col_11 + 1,col_12 + 1 from cte_0 where col_10 < 10 ) select * from cte_0; +# case 2 +drop table if exists tbl_1; +CREATE TABLE `tbl_1` ( + `col_5` decimal(47,21) NOT NULL DEFAULT '5308.880000000000000000000', + `col_6` enum('Alice','Bob','Charlie','David') DEFAULT NULL, + `col_7` float NOT NULL, + `col_8` bigint NOT NULL DEFAULT '-688199144806783096', + `col_9` varchar(248) NOT NULL, + PRIMARY KEY (`col_5`,`col_7`,`col_9`,`col_8`), + UNIQUE KEY `idx_4` (`col_8`), + UNIQUE KEY `idx_7` (`col_5`,`col_7`,`col_8`), + UNIQUE KEY `idx_9` (`col_9`,`col_8`), + UNIQUE KEY `idx_3` (`col_9`(3),`col_8`), + UNIQUE KEY `idx_8` (`col_7`,`col_6`,`col_8`,`col_5`), + KEY `idx_5` (`col_7`), + KEY `idx_6` (`col_7`), + KEY `idx_10` (`col_9`,`col_5`), + KEY `idx_11` (`col_5`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci +/*!50100 PARTITION BY HASH (`col_8`) +PARTITIONS 4 */; +with recursive cte_1 (col_13,col_14,col_15,col_16,col_17) AS ( with recursive cte_2 (col_18,col_19,col_20,col_21,col_22,col_23,col_24) AS ( select 1, 2,col_8,4,5,6,7 from tbl_1 ) select col_19,col_18,col_22,col_23,col_21 from cte_2 UNION ALL select col_13 + 1,col_14 + 1,col_15 + 1,col_16 + 1,col_17 + 1 from cte_1 where col_13 < 10 ) select * from cte_1; +# case 3 +with recursive cte_256 (col_969,col_970,col_971) AS ( with recursive cte_257 (col_972,col_973,col_974,col_975) AS ( select 1, 2,col_8,4 from tbl_1 UNION select col_972 + 1,col_973 + 1,col_974 + 1,col_975 + 1 from cte_257 where col_972 < 10 ) select col_975,col_974,col_973 from cte_257 UNION DISTINCT select col_969 + 1,col_970 + 1,col_971 + 1 from cte_256 where col_969 < 10 ) select * from cte_256; +# case 4 +drop table if exists tbl_2, tbl_3; +create table tbl_2 ( col_4 char(246) collate utf8_unicode_ci not null , col_5 char(253) collate utf8mb4_unicode_ci ) ; +create table tbl_3 ( col_6 char(207) collate utf8mb4_unicode_ci , col_7 int unsigned not null ) ; +insert into tbl_2 values ( "0",null ) ; +insert into tbl_2 values ( "1","0" ) ; +insert into tbl_2 values ( "1","1" ) ; +insert into tbl_2 values ( "0","0" ) ; +insert into tbl_2 values ( "0","1" ) ; +insert into tbl_3 values ( "1",0 ) ; +insert into tbl_3 values ( "1",1 ) ; +insert into tbl_3 values ( "0",0 ) ; +insert into tbl_3 values ( "0",1 ) ; +with recursive tbl_2 (col_64,col_65,col_66,col_67) AS ( select 1, col_6,col_6,4 from tbl_3 UNION DISTINCT select col_64 + 1,concat(col_65, 1),col_66 + 1,concat(col_67, 1) from tbl_2 where col_64 < 5 ) select * from tbl_2 order by col_64; +# case 5 +drop table if exists tbl_3, tbl_4; +create table tbl_3 ( col_6 int not null , col_7 char(95) collate utf8_general_ci ) ; +create table tbl_4 ( col_8 char collate utf8_unicode_ci , col_9 char collate utf8mb4_bin ) ; +insert into tbl_3 values ( 0,"1" ) ; +insert into tbl_4 values ( "1","0" ) ; +with recursive cte_2245 (col_8692,col_8693) AS ( select 1, col_7 from tbl_3 UNION select col_8692 + 1,concat(col_8693, 1) from cte_2245 where col_8692 < 5 ) , cte_2246 (col_8694,col_8695,col_8696,col_8697) AS ( with recursive cte_2247 (col_8698,col_8699,col_8700,col_8701) AS ( select 1, cast("2" as char(20)),3,col_8 from tbl_4 ) select col_8698,col_8699,col_8700,col_8701 from cte_2247 UNION select col_8694 + 1,col_8695 + 1,col_8696 + 1,col_8697 + 1 from cte_2246 where col_8694 < 5 ) select * from cte_2245,cte_2246 order by col_8692,col_8693,col_8696,col_8695,col_8697,col_8694; +# case 6 +with recursive cte2 as (select 1 as col_1, 2 as col_2) select c1.col_1, c2.col_2 from cte2 as c1, cte2 as c2 where c2.col_2 = 1; +# case 7 +with recursive cte (c1) as (select 1), cte1 (c2) as (select 1 union select c1 + 1 from cte, cte1) select * from cte, cte1; +# case 8 +--error 1054 +with recursive tbl_0 (col_943,col_944,col_945,col_946,col_947) AS ( with recursive tbl_0 (col_948,col_949,col_950,col_951,col_952) AS ( select 1, 2,3,4,5 UNION ALL select col_948 + 1,col_949 + 1,col_950 + 1,col_951 + 1,col_952 + 1 from tbl_0 where col_948 < 5 ) select col_948,col_949,col_951,col_950,col_952 from tbl_0 UNION ALL select col_943 + 1,col_944 + 1,col_945 + 1,col_946 + 1,col_947 + 1 from tbl_0 where col_943 < 5 ) select * from tbl_0; +# case 9 +with recursive cte1 (c1, c2) as (select 1, '1' union select concat(c1, 1), c2 + 1 from cte1 where c1 < 100) select * from cte1; +# case 10 +with recursive cte_8 (col_51,col_52,col_53,col_54) AS ( with recursive cte_9 (col_55,col_56,col_57,col_58) AS ( select 1, 2,3,4 UNION ALL select col_55 + 1,col_56 + 1,col_57 + 1,col_58 + 1 from cte_9 where col_55 < 5 ) select col_55,col_57,col_56,col_58 from cte_9 UNION DISTINCT select col_51 + 1,col_52 + 1,col_53 + 1,col_54 + 1 from cte_8 where col_51 < 5 ) select * from cte_8; +# case 11 +with recursive qn as (select 1 from dual union all select 1 from dual) select * from qn; +# case 12 +with recursive qn as (select 1 as a from dual group by a union all select a+1 from qn where a<3) select * from qn; +# case 13 +with recursive qn as ((select 1 as a from dual order by a) union all select a+1 from qn where a<3) select * from qn; +# case 14 +drop table if exists employees; +CREATE TABLE employees ( +ID INT PRIMARY KEY, +NAME VARCHAR(100), +MANAGER_ID INT, +INDEX (MANAGER_ID), +FOREIGN KEY (MANAGER_ID) REFERENCES employees(ID) +); +INSERT INTO employees VALUES +(333, "Yasmina", NULL), +(198, "John", 333), +(692, "Tarek", 333), +(29, "Pedro", 198), +(4610, "Sarah", 29), +(72, "Pierre", 29), +(123, "Adil", 692); +WITH RECURSIVE employees_extended AS (SELECT ID, NAME, MANAGER_ID, CAST(ID AS CHAR(200)) AS PATH FROM employees WHERE NAME='Pierre' UNION ALL SELECT S.ID, S.NAME, S.MANAGER_ID, CONCAT(M.PATH, ",", S.ID) FROM employees_extended M JOIN employees S ON M.MANAGER_ID=S.ID) SELECT * FROM employees_extended; +# case 15 +with recursive cte (c1) as (select 1), cte1 (c2) as (select 1 union select c1 + 1 from cte where c1 < 10) select * from cte where c1 < 5; +# case 16 +with recursive cte_581 (col_2343,col_2344,col_2345) AS ( select 1, '2',cast('3' as char(20))) , cte_582 (col_2346,col_2347,col_2348) AS ( select 1, 2, 3) select * from cte_581 as cte_as_583,cte_582 as cte_as_584,cte_582 as cte_as_585 order by cte_as_583.col_2343,cte_as_585.col_2348,cte_as_584.col_2346,cte_as_584.col_2348,cte_as_583.col_2344,cte_as_584.col_2347,cte_as_585.col_2346,cte_as_585.col_2347,cte_as_583.col_2345; +# case 17 +with recursive tbl_3 (col_19,col_20,col_21,col_22) AS ( select 1, 2,3,4 UNION select col_19 + 1,col_20 + 1,col_21 + 1,concat(col_22, 1) from tbl_3 where col_19 < 5 ) , cte_4 (col_23,col_24,col_25,col_26) AS ( select 1, 2,cast("3" as char(20)),4 UNION DISTINCT select col_23 + 1,col_24 + 1,concat(col_25, 1),col_26 + 1 from cte_4 where col_23 < 5 ) select * from tbl_3 as cte_as_3,cte_4 as cte_as_4,tbl_3 as cte_as_5 order by cte_as_3.col_19,cte_as_4.col_23,cte_as_4.col_25,cte_as_4.col_24,cte_as_4.col_26,cte_as_3.col_20,cte_as_5.col_22,cte_as_3.col_21,cte_as_5.col_20,cte_as_3.col_22,cte_as_5.col_19,cte_as_5.col_21; +# case 18 +with cte1 (c1) as (select 1) select * from cte1 as b, cte1 as a; +# case 19 +--error 1222 +WITH RECURSIVE qn AS +( +select 1 +union all +select 3, 0 from qn +) +select * from qn; +# case 20 +--error 1235 +with recursive cte1 as (select 1 union all (select 1 from cte1 limit 10)) select * from cte1; +# case 21 +# TODO: uncomment this case after we support limit +# with recursive cte1 as (select 1 union all select 1 from cte1 limit 10) select * from cte1; +# case 22 +with recursive qn as (select 123 as a union all select null from qn where a is not null) select * from qn; +# case 23 +--error 1353 +with recursive q (b) as (select 1, 1 union all select 1, 1 from q) select b from q; +# case 24 +drop table if exists t1; +create table t1(a int); +insert into t1 values(1); +insert into t1 values(2); +SELECT * +FROM +t1 dt +WHERE +EXISTS( + WITH RECURSIVE qn AS (SELECT a*0 AS b UNION ALL SELECT b+1 FROM qn WHERE b=0) + SELECT * FROM qn WHERE b=a + ); +# case 25 +drop table if exists t1; +create table t1 (a int); +insert into t1 values (1); +SELECT (WITH qn AS (SELECT 10*a as a FROM t1), + qn2 AS (SELECT 3*a AS b FROM qn) SELECT * from qn2 LIMIT 1) +FROM t1; +# case 26 +select (with qn as (select "with") select * from qn) as scal_subq +from dual; +# case 27 +drop table if exists t1; +create table t1 (a int); insert into t1 values(1), (2), (3); +with q as (select * from t1) +select /*+ merge(q) no_merge(q1) */ * from q, q q1 where q.a=1 and q1.a=2; +# case 28 +drop table if exists t1; create table t1 (a int, b int); +with qn as (select a, b from t1) select b from qn group by a; +# case 29 +drop table if exists t1; +create table t1(a int); +insert into t1 values(1); +insert into t1 values(2); +SELECT * +FROM +t1 dt +WHERE +EXISTS( + WITH RECURSIVE qn AS (SELECT a*0+1 AS b UNION ALL SELECT b+1 FROM qn WHERE b=0) + SELECT * FROM qn WHERE b=1 + ); +# case 30 +drop table if exists tbl_1; +CREATE TABLE `tbl_1` ( + `col_2` char(65) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL, + `col_3` int(11) NOT NULL +); +with recursive cte_8932 (col_34891,col_34892) AS ( with recursive cte_8932 (col_34893,col_34894,col_34895) AS ( with tbl_1 (col_34896,col_34897,col_34898,col_34899) AS ( select 1, "2",3,col_3 from tbl_1 ) select cte_as_8958.col_34896,cte_as_8958.col_34898,cte_as_8958.col_34899 from tbl_1 as cte_as_8958 UNION DISTINCT select col_34893 + 1,concat(col_34894, 1),col_34895 + 1 from cte_8932 where col_34893 < 5 ) select cte_as_8959.col_34893,cte_as_8959.col_34895 from cte_8932 as cte_as_8959 ) select * from cte_8932 as cte_as_8960 order by cte_as_8960.col_34891,cte_as_8960.col_34892; +# case 31 +drop table if exists t1; +create table t1(c1 bigint unsigned); +insert into t1 values(0); +--error 1690 +with recursive cte1 as (select c1 - 1 c1 from t1 union all select c1 - 1 c1 from cte1 where c1 != 0) select * from cte1 dt1, cte1 dt2; From e5b0389d8754fd453f64863cf00551b5a4faf3dd Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Wed, 2 Jun 2021 15:39:38 +0800 Subject: [PATCH 237/343] test: fix a name typo error in test file (#25054) --- ddl/column_type_change_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 7febe2209f62c..38e2cc83ebc5a 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -1884,7 +1884,7 @@ func (s *testColumnTypeChangeSuite) TestChangeIntToBitWillPanicInBackfillIndexes tk.MustQuery("select * from t").Check(testkit.Rows("\x13 1 1.00", "\x11 2 2.00")) } -// Close issue #24971, #24973, #24971 +// Close issue #24971, #24973, #24974 func (s *testColumnTypeChangeSuite) TestCTCShouldCastTheDefaultValue(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") From b7e448aa109c4ebf819ad0ca9fd6d3864d401ad5 Mon Sep 17 00:00:00 2001 From: Howie Date: Wed, 2 Jun 2021 15:56:26 +0800 Subject: [PATCH 238/343] ddl: add auto random && shard_row_id_bits compatibility for temporary table (#24940) --- ddl/db_test.go | 13 +++++++++++++ ddl/ddl_api.go | 3 +++ ddl/serial_test.go | 11 +++++++++++ planner/core/planbuilder.go | 12 ++++++++---- planner/core/preprocess.go | 16 ++++++++++++++-- 5 files changed, 49 insertions(+), 6 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 00cf3fc42b6fc..7ef7d23a82196 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" @@ -5360,6 +5361,18 @@ func (s *testSerialDBSuite) TestAlterShardRowIDBits(c *C) { c.Assert(err.Error(), Equals, "[autoid:1467]Failed to read auto-increment value from storage engine") } +func (s *testSerialDBSuite) TestShardRowIDBitsOnTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists shard_row_id_temporary") + _, err := tk.Exec("create global temporary table shard_row_id_temporary (a int) shard_row_id_bits = 5 on commit delete rows;") + c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("shard_row_id_bits").Error()) + tk.MustExec("create global temporary table shard_row_id_temporary (a int) on commit delete rows;") + defer tk.MustExec("drop table if exists shard_row_id_temporary") + _, err = tk.Exec("alter table shard_row_id_temporary shard_row_id_bits = 4;") + c.Assert(err.Error(), Equals, ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("shard_row_id_bits").Error()) +} + // port from mysql // https://github.com/mysql/mysql-server/blob/124c7ab1d6f914637521fd4463a993aa73403513/mysql-test/t/lock.test func (s *testDBSuite2) TestLock(c *C) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e5871311d78a9..4684d5d4cc7ef 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2647,6 +2647,9 @@ func (d *ddl) ShardRowID(ctx sessionctx.Context, tableIdent ast.Ident, uVal uint if err != nil { return errors.Trace(err) } + if t.Meta().TempTableType != model.TempTableNone { + return ErrOptOnTemporaryTable.GenWithStackByArgs("shard_row_id_bits") + } if uVal == t.Meta().ShardRowIDBits { // Nothing need to do. return nil diff --git a/ddl/serial_test.go b/ddl/serial_test.go index 8d9217e1df8e3..62610e3279cf6 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -942,6 +942,17 @@ func (s *testSerialSuite) TestTableLocksEnable(c *C) { checkTableLock(c, tk.Se, "test", "t1", model.TableLockNone) } +func (s *testSerialDBSuite) TestAutoRandomOnTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists auto_random_temporary") + _, err := tk.Exec("create global temporary table auto_random_temporary (a bigint primary key auto_random(3), b varchar(255)) on commit delete rows;") + c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("auto_random").Error()) + tk.MustExec("set @@tidb_enable_noop_functions = 1") + _, err = tk.Exec("create temporary table t(a bigint key auto_random);") + c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("auto_random").Error()) +} + func (s *testSerialDBSuite) TestAutoRandom(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists auto_random_db") diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 3990cade06511..4df8572b19447 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -3399,7 +3399,11 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err authErr = ErrTableaccessDenied.GenWithStackByArgs("ALTER", b.ctx.GetSessionVars().User.AuthUsername, b.ctx.GetSessionVars().User.AuthHostname, v.Table.Name.L) } - b.visitInfo = appendVisitInfo(b.visitInfo, mysql.AlterPriv, v.Table.Schema.L, + dbName := v.Table.Schema.L + if dbName == "" { + dbName = b.ctx.GetSessionVars().CurrentDB + } + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.AlterPriv, dbName, v.Table.Name.L, "", authErr) for _, spec := range v.Specs { if spec.Tp == ast.AlterTableRenameTable || spec.Tp == ast.AlterTableExchangePartition { @@ -3407,21 +3411,21 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err authErr = ErrTableaccessDenied.GenWithStackByArgs("DROP", b.ctx.GetSessionVars().User.AuthUsername, b.ctx.GetSessionVars().User.AuthHostname, v.Table.Name.L) } - b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, v.Table.Schema.L, + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.DropPriv, dbName, v.Table.Name.L, "", authErr) if b.ctx.GetSessionVars().User != nil { authErr = ErrTableaccessDenied.GenWithStackByArgs("CREATE", b.ctx.GetSessionVars().User.AuthUsername, b.ctx.GetSessionVars().User.AuthHostname, spec.NewTable.Name.L) } - b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreatePriv, spec.NewTable.Schema.L, + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.CreatePriv, dbName, spec.NewTable.Name.L, "", authErr) if b.ctx.GetSessionVars().User != nil { authErr = ErrTableaccessDenied.GenWithStackByArgs("INSERT", b.ctx.GetSessionVars().User.AuthUsername, b.ctx.GetSessionVars().User.AuthHostname, spec.NewTable.Name.L) } - b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, spec.NewTable.Schema.L, + b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, dbName, spec.NewTable.Name.L, "", authErr) } else if spec.Tp == ast.AlterTableDropPartition { if b.ctx.GetSessionVars().User != nil { diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 947b1fdcef1ec..07d7a485ab04e 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -640,6 +640,14 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { return } } + if stmt.TemporaryKeyword != ast.TemporaryNone { + for _, opt := range stmt.Options { + if opt.Tp == ast.TableOptionShardRowID { + p.err = ErrOptOnTemporaryTable.GenWithStackByArgs("shard_row_id_bits") + return + } + } + } tName := stmt.Table.Name.String() if isIncorrectName(tName) { p.err = ddl.ErrWrongTableName.GenWithStackByArgs(tName) @@ -656,7 +664,7 @@ func (p *preprocessor) checkCreateTableGrammar(stmt *ast.CreateTableStmt) { p.err = err return } - isPrimary, err := checkColumnOptions(colDef.Options) + isPrimary, err := checkColumnOptions(stmt.TemporaryKeyword != ast.TemporaryNone, colDef.Options) if err != nil { p.err = err return @@ -813,7 +821,7 @@ func isTableAliasDuplicate(node ast.ResultSetNode, tableAliases map[string]inter return nil } -func checkColumnOptions(ops []*ast.ColumnOption) (int, error) { +func checkColumnOptions(isTempTable bool, ops []*ast.ColumnOption) (int, error) { isPrimary, isGenerated, isStored := 0, 0, false for _, op := range ops { @@ -823,6 +831,10 @@ func checkColumnOptions(ops []*ast.ColumnOption) (int, error) { case ast.ColumnOptionGenerated: isGenerated = 1 isStored = op.Stored + case ast.ColumnOptionAutoRandom: + if isTempTable { + return isPrimary, ErrOptOnTemporaryTable.GenWithStackByArgs("auto_random") + } } } From ca3d88eba5c67c17ed0f56faa10dcef8af6236d3 Mon Sep 17 00:00:00 2001 From: Shirly Date: Wed, 2 Jun 2021 16:08:26 +0800 Subject: [PATCH 239/343] store/tikv: make pub vars related to region to private and add functions to set & get (#25044) --- store/copr/batch_request_sender.go | 3 +-- store/tikv/region_cache.go | 31 ++++++++++++++++++++++-------- store/tikv/region_request.go | 13 ++++++++++++- tidb-server/main.go | 6 +++--- 4 files changed, 39 insertions(+), 14 deletions(-) diff --git a/store/copr/batch_request_sender.go b/store/copr/batch_request_sender.go index dea0c98148aa0..b12ced3f65c12 100644 --- a/store/copr/batch_request_sender.go +++ b/store/copr/batch_request_sender.go @@ -15,7 +15,6 @@ package copr import ( "context" - "sync/atomic" "time" "github.com/pingcap/errors" @@ -80,7 +79,7 @@ func (ss *RegionBatchRequestSender) onSendFailForBatchRegions(bo *Backoffer, ctx // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled || status.Code(errors.Cause(err)) == codes.Canceled { return errors.Trace(err) - } else if atomic.LoadUint32(&tikv.ShuttingDown) > 0 { + } else if tikv.LoadShuttingDown() > 0 { return tikverr.ErrTiDBShuttingDown } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 2fb006138dff5..50eb437545024 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -55,8 +55,13 @@ const ( defaultRegionsPerBatch = 128 ) -// RegionCacheTTLSec is the max idle time for regions in the region cache. -var RegionCacheTTLSec int64 = 600 +// regionCacheTTLSec is the max idle time for regions in the region cache. +var regionCacheTTLSec int64 = 600 + +// SetRegionCacheTTLSec sets regionCacheTTLSec to t. +func SetRegionCacheTTLSec(t int64) { + regionCacheTTLSec = t +} const ( updated int32 = iota // region is updated and no need to reload. @@ -254,7 +259,7 @@ func (r *Region) checkRegionCacheTTL(ts int64) bool { }) for { lastAccess := atomic.LoadInt64(&r.lastAccess) - if ts-lastAccess > RegionCacheTTLSec { + if ts-lastAccess > regionCacheTTLSec { return false } if atomic.CompareAndSwapInt64(&r.lastAccess, lastAccess, ts) { @@ -1180,7 +1185,7 @@ func (c *RegionCache) getRegionByIDFromCache(regionID uint64) *Region { return nil } lastAccess := atomic.LoadInt64(&latestRegion.lastAccess) - if ts-lastAccess > RegionCacheTTLSec { + if ts-lastAccess > regionCacheTTLSec { return nil } if latestRegion != nil { @@ -2070,10 +2075,20 @@ type livenessState uint32 var ( livenessSf singleflight.Group - // StoreLivenessTimeout is the max duration of resolving liveness of a TiKV instance. - StoreLivenessTimeout time.Duration + // storeLivenessTimeout is the max duration of resolving liveness of a TiKV instance. + storeLivenessTimeout time.Duration ) +// SetStoreLivenessTimeout sets storeLivenessTimeout to t. +func SetStoreLivenessTimeout(t time.Duration) { + storeLivenessTimeout = t +} + +// GetStoreLivenessTimeout returns storeLivenessTimeout. +func GetStoreLivenessTimeout() time.Duration { + return storeLivenessTimeout +} + const ( unknown livenessState = iota reachable @@ -2136,7 +2151,7 @@ func (s *Store) requestLiveness(bo *Backoffer, c *RegionCache) (l livenessState) return c.testingKnobs.mockRequestLiveness(s, bo) } - if StoreLivenessTimeout == 0 { + if storeLivenessTimeout == 0 { return unreachable } @@ -2146,7 +2161,7 @@ func (s *Store) requestLiveness(bo *Backoffer, c *RegionCache) (l livenessState) } addr := s.addr rsCh := livenessSf.DoChan(addr, func() (interface{}, error) { - return invokeKVStatusAPI(addr, StoreLivenessTimeout), nil + return invokeKVStatusAPI(addr, storeLivenessTimeout), nil }) var ctx context.Context if bo != nil { diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index b8fcc837c1bc6..a90d638233578 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -44,8 +44,19 @@ import ( // ShuttingDown is a flag to indicate tidb-server is exiting (Ctrl+C signal // receved for example). If this flag is set, tikv client should not retry on // network error because tidb-server expect tikv client to exit as soon as possible. +// TODO: make it private when br is ready. var ShuttingDown uint32 +// StoreShuttingDown atomically stores ShuttingDown into v. +func StoreShuttingDown(v uint32) { + atomic.StoreUint32(&ShuttingDown, v) +} + +// LoadShuttingDown atomically loads ShuttingDown. +func LoadShuttingDown() uint32 { + return atomic.LoadUint32(&ShuttingDown) +} + // RegionRequestSender sends KV/Cop requests to tikv server. It handles network // errors and some region errors internally. // @@ -555,7 +566,7 @@ func (s *RegionRequestSender) onSendFail(bo *Backoffer, ctx *RPCContext, err err // If it failed because the context is cancelled by ourself, don't retry. if errors.Cause(err) == context.Canceled { return errors.Trace(err) - } else if atomic.LoadUint32(&ShuttingDown) > 0 { + } else if LoadShuttingDown() > 0 { return tikverr.ErrTiDBShuttingDown } if status.Code(errors.Cause(err)) == codes.Canceled { diff --git a/tidb-server/main.go b/tidb-server/main.go index f1b16397046f9..f99722e8bfd13 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -562,7 +562,7 @@ func setGlobalVars() { } atomic.StoreUint64(&tikv.CommitMaxBackoff, uint64(parseDuration(cfg.TiKVClient.CommitTimeout).Seconds()*1000)) - tikv.RegionCacheTTLSec = int64(cfg.TiKVClient.RegionCacheTTL) + tikv.SetRegionCacheTTLSec(int64(cfg.TiKVClient.RegionCacheTTL)) domainutil.RepairInfo.SetRepairMode(cfg.RepairMode) domainutil.RepairInfo.SetRepairTableList(cfg.RepairTableList) executor.GlobalDiskUsageTracker.SetBytesLimit(cfg.TempStorageQuota) @@ -579,7 +579,7 @@ func setGlobalVars() { logutil.BgLogger().Fatal("invalid duration value for store-liveness-timeout", zap.String("currentValue", cfg.TiKVClient.StoreLivenessTimeout)) } - tikv.StoreLivenessTimeout = t + tikv.SetStoreLivenessTimeout(t) parsertypes.TiDBStrictIntegerDisplayWidth = cfg.DeprecateIntegerDisplayWidth } @@ -649,7 +649,7 @@ func setupTracing() { } func closeDomainAndStorage(storage kv.Storage, dom *domain.Domain) { - atomic.StoreUint32(&tikv.ShuttingDown, 1) + tikv.StoreShuttingDown(1) dom.Close() err := storage.Close() terror.Log(errors.Trace(err)) From f966a3d9fb41a893674b4ca59121b35d54300b35 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 2 Jun 2021 17:30:43 +0800 Subject: [PATCH 240/343] executor: fix the wrong KVRange for partition tables in TableReader (#25047) --- executor/builder.go | 6 +- executor/partition_table_test.go | 149 +++++++++++++++++++++++++++++++ executor/table_reader.go | 4 +- 3 files changed, 153 insertions(+), 6 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 3ef6d96358e0b..66f20c000a92b 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2762,7 +2762,6 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E ret.kvRangeBuilder = kvRangeBuilderFromRangeAndPartition{ sctx: b.ctx, partitions: partitions, - ranges: ts.Ranges, } return ret @@ -3447,15 +3446,14 @@ func dedupHandles(lookUpContents []*indexJoinLookUpContent) ([]kv.Handle, []*ind type kvRangeBuilderFromRangeAndPartition struct { sctx sessionctx.Context partitions []table.PhysicalTable - ranges []*ranger.Range } -func (h kvRangeBuilderFromRangeAndPartition) buildKeyRange(int64) ([]kv.KeyRange, error) { +func (h kvRangeBuilderFromRangeAndPartition) buildKeyRange(_ int64, ranges []*ranger.Range) ([]kv.KeyRange, error) { var ret []kv.KeyRange for _, p := range h.partitions { pid := p.GetPhysicalID() meta := p.Meta() - kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, h.ranges, nil) + kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges, nil) if err != nil { return nil, err } diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index d8ae8abcd9476..34066a301bff4 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -2110,6 +2110,155 @@ func (s *partitionTableSuite) TestDirectReadingWithUnionScan(c *C) { tk.MustExec(`rollback`) } +func (s *partitionTableSuite) TestIssue25030(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_issue_25030") + tk.MustExec("use test_issue_25030") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`CREATE TABLE tbl_936 ( + col_5410 smallint NOT NULL, + col_5411 double, + col_5412 boolean NOT NULL DEFAULT 1, + col_5413 set('Alice', 'Bob', 'Charlie', 'David') NOT NULL DEFAULT 'Charlie', + col_5414 varbinary(147) COLLATE 'binary' DEFAULT 'bvpKgYWLfyuTiOYSkj', + col_5415 timestamp NOT NULL DEFAULT '2021-07-06', + col_5416 decimal(6, 6) DEFAULT 0.49, + col_5417 text COLLATE utf8_bin, + col_5418 float DEFAULT 2048.0762299371554, + col_5419 int UNSIGNED NOT NULL DEFAULT 3152326370, + PRIMARY KEY (col_5419) ) + PARTITION BY HASH (col_5419) PARTITIONS 3`) + tk.MustQuery(`SELECT last_value(col_5414) OVER w FROM tbl_936 + WINDOW w AS (ORDER BY col_5410, col_5411, col_5412, col_5413, col_5414, col_5415, col_5416, col_5417, col_5418, col_5419) + ORDER BY col_5410, col_5411, col_5412, col_5413, col_5414, col_5415, col_5416, col_5417, col_5418, col_5419, nth_value(col_5412, 5) OVER w`). + Check(testkit.Rows()) // can work properly without any error or panic +} + +func (s *partitionTableSuite) TestUnsignedPartitionColumn(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_unsigned_partition") + tk.MustExec("use test_unsigned_partition") + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + + tk.MustExec(`create table thash_pk (a int unsigned, b int, primary key(a)) partition by hash (a) partitions 3`) + tk.MustExec(`create table trange_pk (a int unsigned, b int, primary key(a)) partition by range (a) ( + partition p1 values less than (100000), + partition p2 values less than (200000), + partition p3 values less than (300000), + partition p4 values less than (400000))`) + tk.MustExec(`create table tnormal_pk (a int unsigned, b int, primary key(a))`) + tk.MustExec(`create table thash_uniq (a int unsigned, b int, unique key(a)) partition by hash (a) partitions 3`) + tk.MustExec(`create table trange_uniq (a int unsigned, b int, unique key(a)) partition by range (a) ( + partition p1 values less than (100000), + partition p2 values less than (200000), + partition p3 values less than (300000), + partition p4 values less than (400000))`) + tk.MustExec(`create table tnormal_uniq (a int unsigned, b int, unique key(a))`) + + valColA := make(map[int]struct{}, 1000) + vals := make([]string, 0, 1000) + for len(vals) < 1000 { + a := rand.Intn(400000) + if _, ok := valColA[a]; ok { + continue + } + valColA[a] = struct{}{} + vals = append(vals, fmt.Sprintf("(%v, %v)", a, rand.Intn(400000))) + } + valStr := strings.Join(vals, ", ") + for _, tbl := range []string{"thash_pk", "trange_pk", "tnormal_pk", "thash_uniq", "trange_uniq", "tnormal_uniq"} { + tk.MustExec(fmt.Sprintf("insert into %v values %v", tbl, valStr)) + } + + for i := 0; i < 100; i++ { + scanCond := fmt.Sprintf("a %v %v", []string{">", "<"}[rand.Intn(2)], rand.Intn(400000)) + pointCond := fmt.Sprintf("a = %v", rand.Intn(400000)) + batchCond := fmt.Sprintf("a in (%v, %v, %v)", rand.Intn(400000), rand.Intn(400000), rand.Intn(400000)) + + var rScan, rPoint, rBatch [][]interface{} + for tid, tbl := range []string{"tnormal_pk", "trange_pk", "thash_pk"} { + // unsigned + TableReader + scanSQL := fmt.Sprintf("select * from %v use index(primary) where %v", tbl, scanCond) + c.Assert(tk.HasPlan(scanSQL, "TableReader"), IsTrue) + r := tk.MustQuery(scanSQL).Sort() + if tid == 0 { + rScan = r.Rows() + } else { + r.Check(rScan) + } + + // unsigned + PointGet on PK + pointSQL := fmt.Sprintf("select * from %v use index(primary) where %v", tbl, pointCond) + tk.MustPointGet(pointSQL) + r = tk.MustQuery(pointSQL).Sort() + if tid == 0 { + rPoint = r.Rows() + } else { + r.Check(rPoint) + } + + // unsigned + BatchGet on PK + batchSQL := fmt.Sprintf("select * from %v where %v", tbl, batchCond) + c.Assert(tk.HasPlan(batchSQL, "Batch_Point_Get"), IsTrue) + r = tk.MustQuery(batchSQL).Sort() + if tid == 0 { + rBatch = r.Rows() + } else { + r.Check(rBatch) + } + } + + lookupCond := fmt.Sprintf("a %v %v", []string{">", "<"}[rand.Intn(2)], rand.Intn(400000)) + var rLookup [][]interface{} + for tid, tbl := range []string{"tnormal_uniq", "trange_uniq", "thash_uniq"} { + // unsigned + IndexReader + scanSQL := fmt.Sprintf("select a from %v use index(a) where %v", tbl, scanCond) + c.Assert(tk.HasPlan(scanSQL, "IndexReader"), IsTrue) + r := tk.MustQuery(scanSQL).Sort() + if tid == 0 { + rScan = r.Rows() + } else { + r.Check(rScan) + } + + // unsigned + IndexLookUp + lookupSQL := fmt.Sprintf("select * from %v use index(a) where %v", tbl, lookupCond) + tk.MustIndexLookup(lookupSQL) + r = tk.MustQuery(lookupSQL).Sort() + if tid == 0 { + rLookup = r.Rows() + } else { + r.Check(rLookup) + } + + // unsigned + PointGet on UniqueIndex + pointSQL := fmt.Sprintf("select * from %v use index(a) where %v", tbl, pointCond) + tk.MustPointGet(pointSQL) + r = tk.MustQuery(pointSQL).Sort() + if tid == 0 { + rPoint = r.Rows() + } else { + r.Check(rPoint) + } + + // unsigned + BatchGet on UniqueIndex + batchSQL := fmt.Sprintf("select * from %v where %v", tbl, batchCond) + c.Assert(tk.HasPlan(batchSQL, "Batch_Point_Get"), IsTrue) + r = tk.MustQuery(batchSQL).Sort() + if tid == 0 { + rBatch = r.Rows() + } else { + r.Check(rBatch) + } + } + } +} + func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") diff --git a/executor/table_reader.go b/executor/table_reader.go index 29d9e4e6908c5..a4459ee920291 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -53,7 +53,7 @@ func (sr selectResultHook) SelectResult(ctx context.Context, sctx sessionctx.Con } type kvRangeBuilder interface { - buildKeyRange(pid int64) ([]kv.KeyRange, error) + buildKeyRange(pid int64, ranges []*ranger.Range) ([]kv.KeyRange, error) } // TableReaderExecutor sends DAG request and reads table data from kv layer. @@ -212,7 +212,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra var builder distsql.RequestBuilder var reqBuilder *distsql.RequestBuilder if e.kvRangeBuilder != nil { - kvRange, err := e.kvRangeBuilder.buildKeyRange(getPhysicalTableID(e.table)) + kvRange, err := e.kvRangeBuilder.buildKeyRange(getPhysicalTableID(e.table), ranges) if err != nil { return nil, err } From 1f79bfe63778d557d25a1f7258f6d5ff7fda58d6 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 2 Jun 2021 17:58:54 +0800 Subject: [PATCH 241/343] *: fix ci lint (#25057) --- executor/executor.go | 2 +- session/session.go | 2 +- util/execdetails/execdetails.go | 1 - 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index c444c0920069e..bb5ba3b89fcac 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1668,7 +1668,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { pprof.SetGoroutineLabels(goCtx) } if variable.TopSQLEnabled() && prepareStmt.SQLDigest != nil { - goCtx = topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest, "", nil) + topsql.AttachSQLInfo(goCtx, prepareStmt.NormalizedSQL, prepareStmt.SQLDigest, "", nil) } } // execute missed stmtID uses empty sql diff --git a/session/session.go b/session/session.go index c4b9c77587790..0c10d2edbcc2d 100644 --- a/session/session.go +++ b/session/session.go @@ -1391,7 +1391,7 @@ func (s *session) ParseWithParams(ctx context.Context, sql string, args ...inter normalized, digest := parser.NormalizeDigest(sql) if digest != nil { // Fixme: reset/clean the label when internal sql execute finish. - ctx = topsql.AttachSQLInfo(ctx, normalized, digest, "", nil) + topsql.AttachSQLInfo(ctx, normalized, digest, "", nil) } } return stmts[0], nil diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 97f9e9611513d..5f19b30158284 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -735,7 +735,6 @@ func (e *RuntimeStatsWithConcurrencyInfo) String() string { // Merge implements the RuntimeStats interface. func (e *RuntimeStatsWithConcurrencyInfo) Merge(_ RuntimeStats) { - return } // RuntimeStatsWithCommit is the RuntimeStats with commit detail. From ad7102cdeedfcf5f014003f2c1579fc13d82e467 Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Wed, 2 Jun 2021 18:20:26 +0800 Subject: [PATCH 242/343] planner: generate correct number of rows when all agg funcs are pruned (#24937) --- cmd/explaintest/r/explain_easy.result | 21 +++--- planner/core/integration_test.go | 23 +++++++ planner/core/rule_column_pruning.go | 21 ++++-- .../core/testdata/integration_suite_in.json | 15 +++++ .../core/testdata/integration_suite_out.json | 65 +++++++++++++++++++ 5 files changed, 131 insertions(+), 14 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 927e25c8024f2..214e51b366de1 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -194,31 +194,32 @@ test t4 1 expr_idx 1 NULL NULL (`a` + `b` + 1) 2 YES NO explain format = 'brief' select count(1) from (select count(1) from (select * from t1 where c3 = 100) k) k2; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#5 -└─StreamAgg 1.00 root funcs:firstrow(Column#9)->Column#7 +└─StreamAgg 1.00 root funcs:count(Column#9)->Column#7 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#9 + └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#9 └─Selection 10.00 cop[tikv] eq(test.t1.c3, 100) └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select 1 from (select count(c2), count(c3) from t1) k; id estRows task access object operator info Projection 1.00 root 1->Column#6 -└─StreamAgg 1.00 root funcs:firstrow(Column#14)->Column#9 +└─StreamAgg 1.00 root funcs:count(Column#14)->Column#9 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#14 + └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#14 └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select count(1) from (select max(c2), count(c3) as m from t1) k; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#6 -└─StreamAgg 1.00 root funcs:firstrow(Column#13)->Column#8 +└─StreamAgg 1.00 root funcs:count(Column#13)->Column#8 └─TableReader 1.00 root data:StreamAgg - └─StreamAgg 1.00 cop[tikv] funcs:firstrow(1)->Column#13 + └─StreamAgg 1.00 cop[tikv] funcs:count(1)->Column#13 └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain format = 'brief' select count(1) from (select count(c2) from t1 group by c3) k; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#5 -└─HashAgg 8000.00 root group by:test.t1.c3, funcs:firstrow(1)->Column#7 - └─TableReader 10000.00 root data:TableFullScan - └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─HashAgg 8000.00 root group by:test.t1.c3, funcs:count(Column#9)->Column#7 + └─TableReader 8000.00 root data:HashAgg + └─HashAgg 8000.00 cop[tikv] group by:test.t1.c3, funcs:count(1)->Column#9 + └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo set @@session.tidb_opt_insubq_to_join_and_agg=0; explain format = 'brief' select sum(t1.c1 in (select c1 from t2)) from t1; id estRows task access object operator info @@ -498,7 +499,7 @@ PRIMARY KEY (`id`) explain format = 'brief' SELECT COUNT(1) FROM (SELECT COALESCE(b.region_name, '不详') region_name, SUM(a.registration_num) registration_num FROM (SELECT stat_date, show_date, region_id, 0 registration_num FROM test01 WHERE period = 1 AND stat_date >= 20191202 AND stat_date <= 20191202 UNION ALL SELECT stat_date, show_date, region_id, registration_num registration_num FROM test01 WHERE period = 1 AND stat_date >= 20191202 AND stat_date <= 20191202) a LEFT JOIN test02 b ON a.region_id = b.id WHERE registration_num > 0 AND a.stat_date >= '20191202' AND a.stat_date <= '20191202' GROUP BY a.stat_date , a.show_date , COALESCE(b.region_name, '不详') ) JLS; id estRows task access object operator info StreamAgg 1.00 root funcs:count(1)->Column#22 -└─HashAgg 8000.00 root group by:Column#32, Column#33, Column#34, funcs:firstrow(1)->Column#31 +└─HashAgg 8000.00 root group by:Column#32, Column#33, Column#34, funcs:count(1)->Column#31 └─Projection 10000.01 root Column#14, Column#15, coalesce(test.test02.region_name, 不详)->Column#34 └─HashJoin 10000.01 root left outer join, equal:[eq(Column#16, test.test02.id)] ├─TableReader(Build) 10000.00 root data:TableFullScan diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index c691e5341b0a5..bbfe52a0ee710 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -180,6 +180,29 @@ func (s *testIntegrationSuite) TestPushLimitDownIndexLookUpReader(c *C) { } } +func (s *testIntegrationSuite) TestAggColumnPrune(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(1),(2)") + + var input []string + var output []struct { + SQL string + Res []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Res = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + tk.MustQuery(tt).Check(testkit.Rows(output[i].Res...)) + } +} + func (s *testIntegrationSuite) TestIsFromUnixtimeNullRejective(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 4b31853c138d0..8a627792ecc7f 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -88,7 +88,11 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) child := la.children[0] used := expression.GetUsedList(parentUsedCols, la.Schema()) + allFirstRow := true for i := len(used) - 1; i >= 0; i-- { + if la.AggFuncs[i].Name != ast.AggFuncFirstRow { + allFirstRow = false + } if !used[i] { la.schema.Columns = append(la.schema.Columns[:i], la.schema.Columns[i+1:]...) la.AggFuncs = append(la.AggFuncs[:i], la.AggFuncs[i+1:]...) @@ -103,15 +107,24 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) selfUsedCols = append(selfUsedCols, cols...) } if len(la.AggFuncs) == 0 { - // If all the aggregate functions are pruned, we should add an aggregate function to keep the correctness. - one, err := aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncFirstRow, []expression.Expression{expression.NewOne()}, false) + // If all the aggregate functions are pruned, we should add an aggregate function to maintain the info of row numbers. + // For all the aggregate functions except `first_row`, if we have an empty table defined as t(a,b), + // `select agg(a) from t` would always return one row, while `select agg(a) from t group by b` would return empty. + // For `first_row` which is only used internally by tidb, `first_row(a)` would always return empty for empty input now. + var err error + var newAgg *aggregation.AggFuncDesc + if allFirstRow { + newAgg, err = aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncFirstRow, []expression.Expression{expression.NewOne()}, false) + } else { + newAgg, err = aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncCount, []expression.Expression{expression.NewOne()}, false) + } if err != nil { return err } - la.AggFuncs = []*aggregation.AggFuncDesc{one} + la.AggFuncs = []*aggregation.AggFuncDesc{newAgg} col := &expression.Column{ UniqueID: la.ctx.GetSessionVars().AllocPlanColumnID(), - RetType: one.RetTp, + RetType: newAgg.RetTp, } la.schema.Columns = []*expression.Column{col} } diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 087b32110e18f..63b866ad3badd 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -19,6 +19,21 @@ "explain format = 'brief' select * from t t1 left join t t2 on t1.a=t2.a where from_unixtime(t2.b);" ] }, + { + "name": "TestAggColumnPrune", + "cases": [ + "select count(1) from t join (select count(1) from t where false) as tmp", + "select count(1) from t join (select max(a) from t where false) as tmp", + "select count(1) from t join (select min(a) from t where false) as tmp", + "select count(1) from t join (select sum(a) from t where false) as tmp", + "select count(1) from t join (select avg(a) from t where false) as tmp", + "select count(1) from t join (select count(1) from t where false group by a) as tmp", + "select count(1) from t join (select max(a) from t where false group by a) as tmp", + "select count(1) from t join (select min(a) from t where false group by a) as tmp", + "select count(1) from t join (select sum(a) from t where false group by a) as tmp", + "select count(1) from t join (select avg(a) from t where false group by a) as tmp" + ] + }, { "name": "TestIndexJoinInnerIndexNDV", "cases": [ diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 7c735fcb5657c..77aa5b1494da7 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -63,6 +63,71 @@ } ] }, + { + "Name": "TestAggColumnPrune", + "Cases": [ + { + "SQL": "select count(1) from t join (select count(1) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select max(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select min(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select sum(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select avg(a) from t where false) as tmp", + "Res": [ + "2" + ] + }, + { + "SQL": "select count(1) from t join (select count(1) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select max(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select min(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select sum(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + }, + { + "SQL": "select count(1) from t join (select avg(a) from t where false group by a) as tmp", + "Res": [ + "0" + ] + } + ] + }, { "Name": "TestIndexJoinInnerIndexNDV", "Cases": [ From 8656b5d39668fb1511bc2cdaecf7f6ccb757c9bd Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 2 Jun 2021 18:32:26 +0800 Subject: [PATCH 243/343] executor: avoid distsql request for TableReader/IndexReader/IndexLookup on temporary table (#24769) --- executor/distsql.go | 29 +++++++++++++++++++++ executor/executor_test.go | 51 ++++++++++++++++++++++++++++++++++++ executor/table_reader.go | 55 +++++++++++++++++++++++++++++++-------- 3 files changed, 124 insertions(+), 11 deletions(-) diff --git a/executor/distsql.go b/executor/distsql.go index 23fa8bd4ce58e..a4d71f45a6a9e 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -196,6 +196,10 @@ type IndexReaderExecutor struct { // Close clears all resources hold by current object. func (e *IndexReaderExecutor) Close() error { + if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { + return nil + } + err := e.result.Close() e.result = nil e.ctx.StoreQueryFeedback(e.feedback) @@ -204,6 +208,11 @@ func (e *IndexReaderExecutor) Close() error { // Next implements the Executor Next interface. func (e *IndexReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error { + if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { + req.Reset() + return nil + } + err := e.result.Next(ctx, req) if err != nil { e.feedback.Invalidate() @@ -266,6 +275,11 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) e.dagPB.CollectExecutionSummaries = &collExec } e.kvRanges = kvRanges + // Treat temporary table as dummy table, avoid sending distsql request to TiKV. + // In a test case IndexReaderExecutor is mocked and e.table is nil. + if e.table != nil && e.table.Meta().TempTableType != model.TempTableNone { + return nil + } e.memTracker = memory.NewTracker(e.id, -1) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) @@ -381,6 +395,12 @@ func (e *IndexLookUpExecutor) Open(ctx context.Context) error { e.feedback.Invalidate() return err } + + // Treat temporary table as dummy table, avoid sending distsql request to TiKV. + if e.table.Meta().TempTableType == model.TempTableGlobal { + return nil + } + err = e.open(ctx) if err != nil { e.feedback.Invalidate() @@ -639,6 +659,10 @@ func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, task *lookup // Close implements Exec Close interface. func (e *IndexLookUpExecutor) Close() error { + if e.table.Meta().TempTableType != model.TempTableNone { + return nil + } + if !e.workerStarted || e.finished == nil { return nil } @@ -659,6 +683,11 @@ func (e *IndexLookUpExecutor) Close() error { // Next implements Exec Next interface. func (e *IndexLookUpExecutor) Next(ctx context.Context, req *chunk.Chunk) error { + if e.table.Meta().TempTableType == model.TempTableGlobal { + req.Reset() + return nil + } + if !e.workerStarted { if err := e.startWorkers(ctx, req.RequiredRows()); err != nil { return err diff --git a/executor/executor_test.go b/executor/executor_test.go index c41b6f2a7773e..8d70d7fa16468 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8356,6 +8356,57 @@ func (s testSerialSuite) TestExprBlackListForEnum(c *C) { c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue) } +func (s testSerialSuite) TestTemporaryTableNoNetwork(c *C) { + // Test that table reader/index reader/index lookup on the temporary table do not need to visit TiKV. + tk := testkit.NewTestKit(c, s.store) + tk1 := testkit.NewTestKit(c, s.store) + + tk.MustExec("use test") + tk1.MustExec("use test") + tk.MustExec("create table normal (id int, a int, index(a))") + tk.MustExec("create global temporary table tmp_t (id int, a int, index(a)) on commit delete rows") + + tk.MustExec("begin") + tk.MustExec("insert into tmp_t values (1, 1)") + tk.MustExec("insert into tmp_t values (2, 2)") + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy", "return(true)"), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy"), IsNil) + }() + + // Make sure the fail point works. + // With that failpoint, all requests to the TiKV is discard. + rs, err := tk1.Exec("select * from normal") + c.Assert(err, IsNil) + blocked := make(chan struct{}) + ctx, cancelFunc := context.WithCancel(context.Background()) + go func() { + _, err := session.ResultSetToStringSlice(ctx, tk1.Se, rs) + blocked <- struct{}{} + c.Assert(err, NotNil) + }() + select { + case <-blocked: + c.Error("The query should block when the failpoint is enabled") + case <-time.After(200 * time.Millisecond): + } + cancelFunc() + + // Check the temporary table do not send request to TiKV. + // Table reader + tk.HasPlan("select * from tmp_t", "TableReader") + tk.MustQuery("select * from tmp_t").Check(testkit.Rows("1 1", "2 2")) + // Index reader + tk.HasPlan("select /*+ USE_INDEX(tmp_t, a) */ a from tmp_t", "IndexReader") + tk.MustQuery("select /*+ USE_INDEX(tmp_t, a) */ a from tmp_t").Check(testkit.Rows("1", "2")) + // Index lookup + tk.HasPlan("select id from tmp_t where a = 1", "IndexLookUp") + tk.MustQuery("select id from tmp_t where a = 1").Check(testkit.Rows("1")) + + tk.MustExec("rollback") +} + func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/table_reader.go b/executor/table_reader.go index a4459ee920291..fc5e5a2c11096 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -153,6 +153,25 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { } } firstPartRanges, secondPartRanges := distsql.SplitRangesAcrossInt64Boundary(e.ranges, e.keepOrder, e.desc, e.table.Meta() != nil && e.table.Meta().IsCommonHandle) + + // Treat temporary table as dummy table, avoid sending distsql request to TiKV. + // Calculate the kv ranges here, UnionScan rely on this kv ranges. + if e.table.Meta() != nil && e.table.Meta().TempTableType != model.TempTableNone { + kvReq, err := e.buildKVReq(ctx, firstPartRanges) + if err != nil { + return err + } + e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...) + if len(secondPartRanges) != 0 { + kvReq, err = e.buildKVReq(ctx, secondPartRanges) + if err != nil { + return err + } + e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...) + } + return nil + } + firstResult, err := e.buildResp(ctx, firstPartRanges) if err != nil { e.feedback.Invalidate() @@ -175,6 +194,12 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error { // Next fills data into the chunk passed by its caller. // The task was actually done by tableReaderHandler. func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error { + if e.table.Meta() != nil && e.table.Meta().TempTableType != model.TempTableNone { + // Treat temporary table as dummy table, avoid sending distsql request to TiKV. + req.Reset() + return nil + } + logutil.Eventf(ctx, "table scan table: %s, range: %v", stringutil.MemoizeStr(func() string { var tableName string if meta := e.table.Meta(); meta != nil { @@ -197,6 +222,10 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.Chunk) error // 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() @@ -209,6 +238,20 @@ func (e *TableReaderExecutor) Close() error { // buildResp first builds request and sends it to tikv using distsql.Select. It uses SelectResult returned by the callee // to fetch all results. func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (distsql.SelectResult, error) { + kvReq, err := e.buildKVReq(ctx, ranges) + if err != nil { + return nil, err + } + e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...) + + result, err := e.SelectResult(ctx, e.ctx, kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.id) + if err != nil { + return nil, err + } + return result, nil +} + +func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.Range) (*kv.Request, error) { var builder distsql.RequestBuilder var reqBuilder *distsql.RequestBuilder if e.kvRangeBuilder != nil { @@ -231,17 +274,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra SetMemTracker(e.memTracker). SetStoreType(e.storeType). SetAllowBatchCop(e.batchCop) - kvReq, err := reqBuilder.Build() - if err != nil { - return nil, err - } - e.kvRanges = append(e.kvRanges, kvReq.KeyRanges...) - - result, err := e.SelectResult(ctx, e.ctx, kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.id) - if err != nil { - return nil, err - } - return result, nil + return reqBuilder.Build() } func buildVirtualColumnIndex(schema *expression.Schema, columns []*model.ColumnInfo) []int { From debf8c76a37ab659594f81337b72d4f177c69814 Mon Sep 17 00:00:00 2001 From: Raywill Date: Wed, 2 Jun 2021 19:26:25 +0800 Subject: [PATCH 244/343] docs: fix typo (#25048) --- docs/design/2020-06-24-placement-rules-in-sql.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/design/2020-06-24-placement-rules-in-sql.md b/docs/design/2020-06-24-placement-rules-in-sql.md index a07c7d5b5e171..d5eb56f690e1e 100644 --- a/docs/design/2020-06-24-placement-rules-in-sql.md +++ b/docs/design/2020-06-24-placement-rules-in-sql.md @@ -14,7 +14,7 @@ The scenarios of defining placement rules in SQL include: - Place data across regions to improve access locality - Add a TiFlash replica for a table -- Limit data within its national border to gaurantee data sovereignty +- Limit data within its national border to guarantee data sovereignty - Place latest data to SSD and history data to HDD - Place the leader of hot data to a high-performance TiKV instance - Increase the replica count of more important data @@ -233,7 +233,7 @@ For example, `CONSTRAINTS="{+zone=sh:1,-zone=bj:2}"` indicates to place 1 replic In the list format, `count` is not specified. The number of replicas for each constraint is not limited, but the total number of replicas should still conform to the `REPLICAS` option. -For example, `CONSTRAINTS="[+zone=sh,+zone=bj]" REPLICAS=3` indicates to place 3 repicas on either `sh` or `bj`. There may be 2 replicas on `sh` and 1 in `bj`, or 2 in `bj` and 1 in `sh`. It's up to PD. +For example, `CONSTRAINTS="[+zone=sh,+zone=bj]" REPLICAS=3` indicates to place 3 replicas on either `sh` or `bj`. There may be 2 replicas on `sh` and 1 in `bj`, or 2 in `bj` and 1 in `sh`. It's up to PD. Label constraints can be implemented by defining `label_constraints` field in PD placement rule configuration. `+` and `-` correspond to property `op`. Specifically, `+` is equivalent to `in` and `-` is equivalent to `notIn`. @@ -553,7 +553,7 @@ However, TiDB also uses placement rules in some cases, as discussed in section " Before choosing the solution, transactional requirements need to be noticed: -- Defining placement rules may fail, and users will probably retry it. As retrying `ADD PLACEMENT POLICY` will add more replicas than expected, the atomacity of the opertion needs to be gauranteed. +- Defining placement rules may fail, and users will probably retry it. As retrying `ADD PLACEMENT POLICY` will add more replicas than expected, the atomicity of the opertion needs to be guaranteed. - `ADD PLACEMENT POLICY` needs to read the original placement rules, combine the 2 rules and then store them to PD, so linearizability should be gauranteed. If the placement rules are stored on both TiKV and PD, the approaches to keep atomicity are as follows: @@ -583,7 +583,7 @@ The comparison shows that both solutions are possible, but storing placement rul The scenarios where TiDB queries placement rules are as follows: 1. The optimizer uses placement rules to decide to route cop request to TiKV or TiFlash. It's already implemented and the TiFlash information is written into table information, which is stored on TiKV. -2. It will be probably used in locality-aware features in the furture, such as follower-read. Follower-read is always used when TiDB wants to read the nearest replica to reduce multi-region latency. In some distributed databases, it’s implemented by labelling data nodes and selecting the nearest replica according to the labels. +2. It will be probably used in locality-aware features in the future, such as follower-read. Follower-read is always used when TiDB wants to read the nearest replica to reduce multi-region latency. In some distributed databases, it’s implemented by labelling data nodes and selecting the nearest replica according to the labels. 3. Local transactions need to know the binding relationship between Raft leader and region, which is also defined by placement rules. 4. Once a rule is defined on a table, all the subsequent partitions added to the table should also inherit the rule. So the `ADD PARTITION` operation should query the rules on the table. The same is true for creating tables and indices. 5. `SHOW PLACEMENT POLICY` statement should output the placement rules correctly. @@ -615,7 +615,7 @@ The fact that the DDL procedure in TiDB is mature helps to achieve some features - Placement rules are defined in serial as there's only one DDL owner at the same time - DDL is capable of disaster recovery as the middle states are persistent in TiKV - DDL is rollbackable as the middle states can transform from one to another -- Updating schema version guarantees all active transactions are based on the same version of placement ruels +- Updating schema version guarantees all active transactions are based on the same version of placement rules ### Rule priorities @@ -712,7 +712,7 @@ ALTER TABLE t ALTER PLACEMENT POLICY CONSTRAINTS="{+zone=bj:2,+zone=sh:1}" ROLE=voter; ``` -It needs 2 placement rules for `voter` in the PD placment rule configuration, because each rule can only specify one `count`. To make `id` unique, a unique identifier must be appended to `id`. DDL job ID plus an index in the job is a good choice. +It needs 2 placement rules for `voter` in the PD placement rule configuration, because each rule can only specify one `count`. To make `id` unique, a unique identifier must be appended to `id`. DDL job ID plus an index in the job is a good choice. Take the case above for example, assuming the table ID of `t` is 100, the ID of the DDL job executing this statement is 200, then `id` of the placement rules are `100-200-1` and `100-200-2`. From da8dbe6085b4e60ddafe4836aa1f325b5cc44f13 Mon Sep 17 00:00:00 2001 From: Howie Date: Wed, 2 Jun 2021 19:34:25 +0800 Subject: [PATCH 245/343] ddl: add foreign key compatibility for temporary table (#24961) --- ddl/db_test.go | 20 +++++++++++++++++++- ddl/ddl_api.go | 3 +++ ddl/serial_test.go | 7 ------- planner/core/planbuilder.go | 8 ++++++-- 4 files changed, 28 insertions(+), 10 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 7ef7d23a82196..0bfc654941727 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -3012,7 +3012,7 @@ func (s *testDBSuite2) TestTableForeignKey(c *C) { tk.MustExec("create table t3 (a int, b int);") failSQL = "alter table t1 add foreign key (c) REFERENCES t3(a);" tk.MustGetErrCode(failSQL, errno.ErrKeyColumnDoesNotExits) - // test oreign key not match error + // test origin key not match error failSQL = "alter table t1 add foreign key (a) REFERENCES t3(a, b);" tk.MustGetErrCode(failSQL, errno.ErrWrongFkDef) // Test drop column with foreign key. @@ -3031,6 +3031,24 @@ func (s *testDBSuite2) TestTableForeignKey(c *C) { tk.MustExec("drop table if exists t1,t2,t3,t4;") } +func (s *testDBSuite2) TestTemporaryTableForeignKey(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1 (a int, b int);") + tk.MustExec("drop table if exists t1_tmp;") + tk.MustExec("create global temporary table t1_tmp (a int, b int) on commit delete rows;") + // test add foreign key. + tk.MustExec("drop table if exists t2;") + tk.MustExec("create table t2 (a int, b int);") + failSQL := "alter table t1_tmp add foreign key (c) REFERENCES t2(a);" + tk.MustGetErrCode(failSQL, mysql.ErrCannotAddForeign) + // Test drop column with foreign key. + failSQL = "create global temporary table t3 (c int,d int,foreign key (d) references t1 (b)) on commit delete rows;" + tk.MustGetErrCode(failSQL, mysql.ErrCannotAddForeign) + tk.MustExec("drop table if exists t1,t2,t3,t1_tmp;") +} + func (s *testDBSuite8) TestFKOnGeneratedColumns(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 4684d5d4cc7ef..640823b23305a 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5267,6 +5267,9 @@ func (d *ddl) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName mode if err != nil { return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) } + if t.Meta().TempTableType != model.TempTableNone { + return infoschema.ErrCannotAddForeign + } fkInfo, err := buildFKInfo(fkName, keys, refer, t.Cols(), t.Meta()) if err != nil { diff --git a/ddl/serial_test.go b/ddl/serial_test.go index 62610e3279cf6..30c81bf2f2e7b 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -534,13 +534,6 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) { _, err = tk.Exec("create table temporary_table_t1 like temporary_table") c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error()) tk.MustExec("drop table if exists temporary_table;") - - tk.MustExec("drop table if exists temporary_table_like;") - tk.MustExec("create table temporary_table (a int, b int,index(a))") - tk.MustExec("drop table if exists temporary_table_like_t1;") - _, err = tk.Exec("create global temporary table temporary_table_like_t1 like temporary_table on commit delete rows;") - c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("create table like").Error()) - tk.MustExec("drop table if exists temporary_table_like;") } // TestCancelAddIndex1 tests canceling ddl job when the add index worker is not started. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 4df8572b19447..b743204a1db91 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -3481,8 +3481,12 @@ func (b *PlanBuilder) buildDDL(ctx context.Context, node ast.DDLNode) (Plan, err b.visitInfo = appendVisitInfo(b.visitInfo, mysql.IndexPriv, v.Table.Schema.L, v.Table.Name.L, "", authErr) case *ast.CreateTableStmt: - if v.TemporaryKeyword != ast.TemporaryNone && v.ReferTable != nil { - return nil, ErrOptOnTemporaryTable.GenWithStackByArgs("create table like") + if v.TemporaryKeyword != ast.TemporaryNone { + for _, cons := range v.Constraints { + if cons.Tp == ast.ConstraintForeignKey { + return nil, infoschema.ErrCannotAddForeign + } + } } if b.ctx.GetSessionVars().User != nil { authErr = ErrTableaccessDenied.GenWithStackByArgs("CREATE", b.ctx.GetSessionVars().User.AuthUsername, From 32cf14bd7dafdc23409fd0ee289c3742063f6478 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Wed, 2 Jun 2021 22:16:25 +0800 Subject: [PATCH 246/343] statistics: relax the check of the OutOfRange (#24958) --- statistics/cmsketch.go | 8 +++ statistics/histogram.go | 92 ++++++++++++------------ statistics/selectivity_test.go | 8 +-- statistics/statistics_test.go | 2 +- statistics/table.go | 21 ++++-- statistics/testdata/stats_suite_in.json | 6 +- statistics/testdata/stats_suite_out.json | 46 ++++++++---- 7 files changed, 113 insertions(+), 70 deletions(-) diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index c510186b16c40..07d90434a6cc7 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -530,6 +530,14 @@ func (c *TopN) Num() int { return len(c.TopN) } +// outOfRange checks whether the the given value falls back in [TopN.LowestOne, TopN.HighestOne]. +func (c *TopN) outOfRange(val []byte) bool { + if c == nil || len(c.TopN) == 0 { + return true + } + return bytes.Compare(c.TopN[0].Encoded, val) > 0 || bytes.Compare(val, c.TopN[c.Num()-1].Encoded) > 0 +} + // DecodedString returns the value with decoded result. func (c *TopN) DecodedString(ctx sessionctx.Context, colTypes []byte) (string, error) { builder := &strings.Builder{} diff --git a/statistics/histogram.go b/statistics/histogram.go index 027950b8326c7..f377b14c0c067 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -506,20 +506,12 @@ func (hg *Histogram) BetweenRowCount(a, b types.Datum) float64 { } // BetweenRowCount estimates the row count for interval [l, r). -func (c *Column) BetweenRowCount(sc *stmtctx.StatementContext, l, r types.Datum) (float64, error) { +func (c *Column) BetweenRowCount(sc *stmtctx.StatementContext, l, r types.Datum, lowEncoded, highEncoded []byte) float64 { histBetweenCnt := c.Histogram.BetweenRowCount(l, r) if c.StatsVer <= Version1 { - return histBetweenCnt, nil - } - lBytes, err := codec.EncodeKey(sc, nil, l) - if err != nil { - return 0, errors.Trace(err) - } - rBytes, err := codec.EncodeKey(sc, nil, r) - if err != nil { - return 0, errors.Trace(err) + return histBetweenCnt } - return float64(c.TopN.BetweenCount(lBytes, rBytes)) + histBetweenCnt, nil + return float64(c.TopN.BetweenCount(lowEncoded, highEncoded)) + histBetweenCnt } // TotalRowCount returns the total count of this histogram. @@ -978,7 +970,7 @@ func (c *Column) IsInvalid(sc *stmtctx.StatementContext, collPseudo bool) bool { return c.TotalRowCount() == 0 || (c.Histogram.NDV > 0 && c.notNullCount() == 0) } -func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, modifyCount int64) (float64, error) { +func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, encodedVal []byte, modifyCount int64) (float64, error) { if val.IsNull() { return float64(c.NullCount), nil } @@ -987,7 +979,7 @@ func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, mo if c.Histogram.Bounds.NumRows() == 0 { return 0.0, nil } - if c.Histogram.NDV > 0 && c.outOfRange(val) { + if c.Histogram.NDV > 0 && c.outOfRange(val, encodedVal) { return outOfRangeEQSelectivity(c.Histogram.NDV, modifyCount, int64(c.TotalRowCount())) * c.TotalRowCount(), nil } if c.CMSketch != nil { @@ -996,14 +988,17 @@ func (c *Column) equalRowCount(sc *stmtctx.StatementContext, val types.Datum, mo } return c.Histogram.equalRowCount(val, false), nil } + // All the values are null. + if c.Histogram.Bounds.NumRows() == 0 && c.TopN.Num() == 0 { + return 0, nil + } + if c.Histogram.NDV+int64(c.TopN.Num()) > 0 && c.outOfRange(val, encodedVal) { + return outOfRangeEQSelectivity(c.Histogram.NDV, modifyCount, int64(c.TotalRowCount())) * c.TotalRowCount(), nil + } // Stats version == 2 // 1. try to find this value in TopN if c.TopN != nil { - valBytes, err := codec.EncodeKey(sc, nil, val) - if err != nil { - return 0, errors.Trace(err) - } - rowcount, ok := c.QueryTopN(valBytes) + rowcount, ok := c.QueryTopN(encodedVal) if ok { return float64(rowcount), nil } @@ -1054,6 +1049,14 @@ func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*range if err != nil { return 0, errors.Trace(err) } + lowEncoded, err := codec.EncodeKey(sc, nil, lowVal) + if err != nil { + return 0, err + } + highEncoded, err := codec.EncodeKey(sc, nil, highVal) + if err != nil { + return 0, err + } if cmp == 0 { // the point case. if !rg.LowExclude && !rg.HighExclude { @@ -1063,7 +1066,7 @@ func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*range continue } var cnt float64 - cnt, err = c.equalRowCount(sc, lowVal, modifyCount) + cnt, err = c.equalRowCount(sc, lowVal, lowEncoded, modifyCount) if err != nil { return 0, errors.Trace(err) } @@ -1075,7 +1078,7 @@ func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*range // The small range case. if rangeVals != nil { for _, val := range rangeVals { - cnt, err := c.equalRowCount(sc, val, modifyCount) + cnt, err := c.equalRowCount(sc, val, lowEncoded, modifyCount) if err != nil { return 0, err } @@ -1084,18 +1087,15 @@ func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*range continue } // The interval case. - cnt, err := c.BetweenRowCount(sc, lowVal, highVal) - if err != nil { - return 0, err - } - if (c.outOfRange(lowVal) && !lowVal.IsNull()) || c.outOfRange(highVal) { + cnt := c.BetweenRowCount(sc, lowVal, highVal, lowEncoded, highEncoded) + if (c.outOfRange(lowVal, lowEncoded) && !lowVal.IsNull()) || c.outOfRange(highVal, highEncoded) { cnt += outOfRangeEQSelectivity(outOfRangeBetweenRate, modifyCount, int64(c.TotalRowCount())) * c.TotalRowCount() } // `betweenRowCount` returns count for [l, h) range, we adjust cnt for boudaries here. // Note that, `cnt` does not include null values, we need specially handle cases // where null is the lower bound. if rg.LowExclude && !lowVal.IsNull() { - lowCnt, err := c.equalRowCount(sc, lowVal, modifyCount) + lowCnt, err := c.equalRowCount(sc, lowVal, lowEncoded, modifyCount) if err != nil { return 0, errors.Trace(err) } @@ -1105,7 +1105,7 @@ func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*range cnt += float64(c.NullCount) } if !rg.HighExclude { - highCnt, err := c.equalRowCount(sc, highVal, modifyCount) + highCnt, err := c.equalRowCount(sc, highVal, highEncoded, modifyCount) if err != nil { return 0, errors.Trace(err) } @@ -1121,6 +1121,15 @@ func (c *Column) GetColumnRowCount(sc *stmtctx.StatementContext, ranges []*range return rowCount, nil } +func (c *Column) outOfRange(val types.Datum, encodedVal []byte) bool { + outOfHist := c.Histogram.outOfRange(val) + if !outOfHist { + return false + } + // Already out of hist. + return c.TopN.outOfRange(encodedVal) +} + // Index represents an index histogram. type Index struct { Histogram @@ -1504,26 +1513,21 @@ func (coll *HistColl) NewHistCollBySelectivity(sc *stmtctx.StatementContext, sta } func (idx *Index) outOfRange(val types.Datum) bool { - histEmpty, topNEmpty := idx.Histogram.Len() == 0, idx.TopN.Num() == 0 - // All empty. - if histEmpty && topNEmpty { - return true - } - // TopN is not empty. Record found. - if !topNEmpty && idx.TopN.findTopN(val.GetBytes()) >= 0 { + outOfTopN := idx.TopN.outOfRange(val.GetBytes()) + // The val is in TopN, return false. + if !outOfTopN { return false } - if !histEmpty { - withInLowBoundOrPrefixMatch := chunk.Compare(idx.Bounds.GetRow(0), 0, &val) <= 0 || - matchPrefix(idx.Bounds.GetRow(0), 0, &val) - withInHighBound := chunk.Compare(idx.Bounds.GetRow(idx.Bounds.NumRows()-1), 0, &val) >= 0 - // Hist is not empty. Record found. - if withInLowBoundOrPrefixMatch && withInHighBound { - return false - } + + histEmpty := idx.Histogram.Len() == 0 + // HistEmpty->Hist out of range. + if histEmpty { + return true } - // No record found. Is out of range. - return true + withInLowBoundOrPrefixMatch := chunk.Compare(idx.Bounds.GetRow(0), 0, &val) <= 0 || + matchPrefix(idx.Bounds.GetRow(0), 0, &val) + withInHighBound := chunk.Compare(idx.Bounds.GetRow(idx.Bounds.NumRows()-1), 0, &val) >= 0 + return !withInLowBoundOrPrefixMatch || !withInHighBound } // matchPrefix checks whether ad is the prefix of value diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 359e1d2db9585..ac12be22442b9 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -649,19 +649,19 @@ func (s *testStatsSuite) TestTopNOutOfHist(c *C) { testKit.MustExec("drop table if exists topn_before_hist") testKit.MustExec("create table topn_before_hist(a int, index idx(a))") - testKit.MustExec("insert into topn_before_hist values(1), (1), (1), (1), (2), (2), (3), (4), (5)") + testKit.MustExec("insert into topn_before_hist values(1), (1), (1), (1), (3), (3), (4), (5), (6)") testKit.MustExec("analyze table topn_before_hist with 2 topn, 3 buckets") testKit.MustExec("create table topn_after_hist(a int, index idx(a))") - testKit.MustExec("insert into topn_after_hist values(2), (2), (3), (4), (5), (6), (6), (6), (6)") + testKit.MustExec("insert into topn_after_hist values(2), (2), (3), (4), (5), (7), (7), (7), (7)") testKit.MustExec("analyze table topn_after_hist with 2 topn, 3 buckets") testKit.MustExec("create table topn_before_hist_no_index(a int)") - testKit.MustExec("insert into topn_before_hist_no_index values(1), (1), (1), (1), (2), (2), (3), (4), (5)") + testKit.MustExec("insert into topn_before_hist_no_index values(1), (1), (1), (1), (3), (3), (4), (5), (6)") testKit.MustExec("analyze table topn_before_hist_no_index with 2 topn, 3 buckets") testKit.MustExec("create table topn_after_hist_no_index(a int)") - testKit.MustExec("insert into topn_after_hist_no_index values(2), (2), (3), (4), (5), (6), (6), (6), (6)") + testKit.MustExec("insert into topn_after_hist_no_index values(2), (2), (3), (4), (5), (7), (7), (7), (7)") testKit.MustExec("analyze table topn_after_hist_no_index with 2 topn, 3 buckets") var ( diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index 7fd0bf64b0bf5..c352c3576c89b 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -458,7 +458,7 @@ func (s *testStatisticsSuite) TestPseudoTable(c *C) { count, err := tbl.ColumnEqualRowCount(sc, types.NewIntDatum(1000), colInfo.ID) c.Assert(err, IsNil) c.Assert(int(count), Equals, 10) - count = tbl.ColumnBetweenRowCount(sc, types.NewIntDatum(1000), types.NewIntDatum(5000), colInfo.ID) + count, _ = tbl.ColumnBetweenRowCount(sc, types.NewIntDatum(1000), types.NewIntDatum(5000), colInfo.ID) c.Assert(int(count), Equals, 250) } diff --git a/statistics/table.go b/statistics/table.go index 7628e018e25a5..85807fbefc67f 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -277,19 +277,24 @@ func (t *Table) ColumnLessRowCount(sc *stmtctx.StatementContext, value types.Dat } // ColumnBetweenRowCount estimates the row count where column greater or equal to a and less than b. -func (t *Table) ColumnBetweenRowCount(sc *stmtctx.StatementContext, a, b types.Datum, colID int64) float64 { +func (t *Table) ColumnBetweenRowCount(sc *stmtctx.StatementContext, a, b types.Datum, colID int64) (float64, error) { c, ok := t.Columns[colID] if !ok || c.IsInvalid(sc, t.Pseudo) { - return float64(t.Count) / pseudoBetweenRate + return float64(t.Count) / pseudoBetweenRate, nil } - count, err := c.BetweenRowCount(sc, a, b) + aEncoded, err := codec.EncodeKey(sc, nil, a) if err != nil { - return 0 + return 0, err } + bEncoded, err := codec.EncodeKey(sc, nil, b) + if err != nil { + return 0, err + } + count := c.BetweenRowCount(sc, a, b, aEncoded, bEncoded) if a.IsNull() { count += float64(c.NullCount) } - return count * c.GetIncreaseFactor(t.Count) + return count * c.GetIncreaseFactor(t.Count), nil } // ColumnEqualRowCount estimates the row count where the column equals to value. @@ -298,7 +303,11 @@ func (t *Table) ColumnEqualRowCount(sc *stmtctx.StatementContext, value types.Da if !ok || c.IsInvalid(sc, t.Pseudo) { return float64(t.Count) / pseudoEqualRate, nil } - result, err := c.equalRowCount(sc, value, t.ModifyCount) + encodedVal, err := codec.EncodeKey(sc, nil, value) + if err != nil { + return 0, err + } + result, err := c.equalRowCount(sc, value, encodedVal, t.ModifyCount) result *= c.GetIncreaseFactor(t.Count) return result, errors.Trace(err) } diff --git a/statistics/testdata/stats_suite_in.json b/statistics/testdata/stats_suite_in.json index 631e2aa6c60e2..b20b6d8300433 100644 --- a/statistics/testdata/stats_suite_in.json +++ b/statistics/testdata/stats_suite_in.json @@ -72,9 +72,13 @@ "show stats_topn", "show stats_buckets", "explain select * from topn_before_hist where a = 1", + "explain select * from topn_before_hist where a = 2", + "explain select * from topn_after_hist where a = 7", "explain select * from topn_after_hist where a = 6", + "explain select * from topn_after_hist_no_index where a = 7", "explain select * from topn_after_hist_no_index where a = 6", - "explain select * from topn_before_hist_no_index where a = 1" + "explain select * from topn_before_hist_no_index where a = 1", + "explain select * from topn_before_hist_no_index where a = 2" ] }, { diff --git a/statistics/testdata/stats_suite_out.json b/statistics/testdata/stats_suite_out.json index c25f082455c2f..b60a351b6cead 100644 --- a/statistics/testdata/stats_suite_out.json +++ b/statistics/testdata/stats_suite_out.json @@ -427,29 +427,29 @@ "Cases": [ [ "test topn_before_hist a 0 1 4", - "test topn_before_hist a 0 2 2", + "test topn_before_hist a 0 3 2", "test topn_before_hist idx 1 1 4", - "test topn_before_hist idx 1 2 2", + "test topn_before_hist idx 1 3 2", "test topn_after_hist a 0 2 2", - "test topn_after_hist a 0 6 4", + "test topn_after_hist a 0 7 4", "test topn_after_hist idx 1 2 2", - "test topn_after_hist idx 1 6 4", + "test topn_after_hist idx 1 7 4", "test topn_before_hist_no_index a 0 1 4", - "test topn_before_hist_no_index a 0 2 2", + "test topn_before_hist_no_index a 0 3 2", "test topn_after_hist_no_index a 0 2 2", - "test topn_after_hist_no_index a 0 6 4" + "test topn_after_hist_no_index a 0 7 4" ], [ - "test topn_before_hist a 0 0 2 1 3 4 0", - "test topn_before_hist a 0 1 3 1 5 5 0", - "test topn_before_hist idx 1 0 2 1 3 4 0", - "test topn_before_hist idx 1 1 3 1 5 5 0", + "test topn_before_hist a 0 0 2 1 4 5 0", + "test topn_before_hist a 0 1 3 1 6 6 0", + "test topn_before_hist idx 1 0 2 1 4 5 0", + "test topn_before_hist idx 1 1 3 1 6 6 0", "test topn_after_hist a 0 0 2 1 3 4 0", "test topn_after_hist a 0 1 3 1 5 5 0", "test topn_after_hist idx 1 0 2 1 3 4 0", "test topn_after_hist idx 1 1 3 1 5 5 0", - "test topn_before_hist_no_index a 0 0 2 1 3 4 0", - "test topn_before_hist_no_index a 0 1 3 1 5 5 0", + "test topn_before_hist_no_index a 0 0 2 1 4 5 0", + "test topn_before_hist_no_index a 0 1 3 1 6 6 0", "test topn_after_hist_no_index a 0 0 2 1 3 4 0", "test topn_after_hist_no_index a 0 1 3 1 5 5 0" ], @@ -457,19 +457,37 @@ "IndexReader_6 4.00 root index:IndexRangeScan_5", "└─IndexRangeScan_5 4.00 cop[tikv] table:topn_before_hist, index:idx(a) range:[1,1], keep order:false" ], + [ + "IndexReader_6 0.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.00 cop[tikv] table:topn_before_hist, index:idx(a) range:[2,2], keep order:false" + ], [ "IndexReader_6 4.00 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 4.00 cop[tikv] table:topn_after_hist, index:idx(a) range:[6,6], keep order:false" + "└─IndexRangeScan_5 4.00 cop[tikv] table:topn_after_hist, index:idx(a) range:[7,7], keep order:false" + ], + [ + "IndexReader_6 0.00 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 0.00 cop[tikv] table:topn_after_hist, index:idx(a) range:[6,6], keep order:false" ], [ "TableReader_7 4.00 root data:Selection_6", - "└─Selection_6 4.00 cop[tikv] eq(test.topn_after_hist_no_index.a, 6)", + "└─Selection_6 4.00 cop[tikv] eq(test.topn_after_hist_no_index.a, 7)", + " └─TableFullScan_5 9.00 cop[tikv] table:topn_after_hist_no_index keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.topn_after_hist_no_index.a, 6)", " └─TableFullScan_5 9.00 cop[tikv] table:topn_after_hist_no_index keep order:false" ], [ "TableReader_7 4.00 root data:Selection_6", "└─Selection_6 4.00 cop[tikv] eq(test.topn_before_hist_no_index.a, 1)", " └─TableFullScan_5 9.00 cop[tikv] table:topn_before_hist_no_index keep order:false" + ], + [ + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.topn_before_hist_no_index.a, 2)", + " └─TableFullScan_5 9.00 cop[tikv] table:topn_before_hist_no_index keep order:false" ] ] }, From 9fad132d1d892ade178d0614687660c29cc5af54 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 2 Jun 2021 23:26:26 +0800 Subject: [PATCH 247/343] case: make CTE case be stable (#25035) --- executor/cte.go | 4 +-- executor/cte_test.go | 85 +++++++++++++++++++++----------------------- 2 files changed, 42 insertions(+), 47 deletions(-) diff --git a/executor/cte.go b/executor/cte.go index 95921670cb5cf..055163dc17e4f 100644 --- a/executor/cte.go +++ b/executor/cte.go @@ -111,8 +111,6 @@ func (e *CTEExec) Open(ctx context.Context) (err error) { if err = e.iterOutTbl.OpenAndRef(); err != nil { return err } - - setupCTEStorageTracker(e.iterOutTbl, e.ctx, e.memTracker, e.diskTracker) } if e.isDistinct { @@ -137,11 +135,13 @@ func (e *CTEExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { defer e.resTbl.Unlock() resAction := setupCTEStorageTracker(e.resTbl, e.ctx, e.memTracker, e.diskTracker) iterInAction := setupCTEStorageTracker(e.iterInTbl, e.ctx, e.memTracker, e.diskTracker) + iterOutAction := setupCTEStorageTracker(e.iterOutTbl, e.ctx, e.memTracker, e.diskTracker) failpoint.Inject("testCTEStorageSpill", func(val failpoint.Value) { if val.(bool) && config.GetGlobalConfig().OOMUseTmpStorage { defer resAction.WaitForTest() defer iterInAction.WaitForTest() + defer iterOutAction.WaitForTest() } }) diff --git a/executor/cte_test.go b/executor/cte_test.go index e5789170627f7..d6e212484ad29 100644 --- a/executor/cte_test.go +++ b/executor/cte_test.go @@ -27,14 +27,16 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" ) -var _ = check.Suite(&CTETestSuite{}) +var _ = check.Suite(&CTETestSuite{&baseCTETestSuite{}}) +var _ = check.SerialSuites(&CTESerialTestSuite{&baseCTETestSuite{}}) -type CTETestSuite struct { +type baseCTETestSuite struct { store kv.Storage dom *domain.Domain sessionCtx sessionctx.Context @@ -42,7 +44,15 @@ type CTETestSuite struct { ctx context.Context } -func (test *CTETestSuite) SetUpSuite(c *check.C) { +type CTETestSuite struct { + *baseCTETestSuite +} + +type CTESerialTestSuite struct { + *baseCTETestSuite +} + +func (test *baseCTETestSuite) SetUpSuite(c *check.C) { var err error test.store, err = mockstore.NewMockStore() c.Assert(err, check.IsNil) @@ -59,7 +69,7 @@ func (test *CTETestSuite) SetUpSuite(c *check.C) { test.ctx = context.Background() } -func (test *CTETestSuite) TearDownSuite(c *check.C) { +func (test *baseCTETestSuite) TearDownSuite(c *check.C) { test.dom.Close() test.store.Close() } @@ -107,70 +117,55 @@ func (test *CTETestSuite) TestBasicCTE(c *check.C) { rows.Check(testkit.Rows("1", "2")) } -func (test *CTETestSuite) TestSpillToDisk(c *check.C) { +func (test *CTESerialTestSuite) TestSpillToDisk(c *check.C) { + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.OOMUseTmpStorage = true + }) + tk := testkit.NewTestKit(c, test.store) tk.MustExec("use test;") c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testCTEStorageSpill", "return(true)"), check.IsNil) defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testCTEStorageSpill"), check.IsNil) + tk.MustExec("set tidb_mem_quota_query = 1073741824;") + }() + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill", "return(true)"), check.IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill"), check.IsNil) }() - - insertStr := "insert into t1 values(0, 0)" - for i := 1; i < 5000; i++ { - insertStr += fmt.Sprintf(", (%d, %d)", i, i) - } - - tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t1(c1 int, c2 int);") - tk.MustExec(insertStr) - tk.MustExec("set tidb_mem_quota_query = 80000;") - rows := tk.MustQuery("with recursive cte1 as ( " + - "select c1 from t1 " + - "union " + - "select c1 + 1 c1 from cte1 where c1 < 5000) " + - "select c1 from cte1;") - - memTracker := tk.Se.GetSessionVars().StmtCtx.MemTracker - diskTracker := tk.Se.GetSessionVars().StmtCtx.DiskTracker - c.Assert(memTracker.MaxConsumed(), check.Greater, int64(0)) - c.Assert(diskTracker.MaxConsumed(), check.Greater, int64(0)) - - rowNum := 5000 - var resRows []string - for i := 0; i <= rowNum; i++ { - resRows = append(resRows, fmt.Sprintf("%d", i)) - } - rows.Check(testkit.Rows(resRows...)) // Use duplicated rows to test UNION DISTINCT. tk.MustExec("set tidb_mem_quota_query = 1073741824;") - insertStr = "insert into t1 values(0, 0)" + insertStr := "insert into t1 values(0)" + rowNum := 1000 vals := make([]int, rowNum) vals[0] = 0 for i := 1; i < rowNum; i++ { v := rand.Intn(100) vals[i] = v - insertStr += fmt.Sprintf(", (%d, %d)", v, v) + insertStr += fmt.Sprintf(", (%d)", v) } tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t1(c1 int, c2 int);") + tk.MustExec("create table t1(c1 int);") tk.MustExec(insertStr) - tk.MustExec("set tidb_mem_quota_query = 80000;") + tk.MustExec("set tidb_mem_quota_query = 40000;") tk.MustExec("set cte_max_recursion_depth = 500000;") - rows = tk.MustQuery("with recursive cte1 as ( " + - "select c1 from t1 " + - "union " + - "select c1 + 1 c1 from cte1 where c1 < 5000) " + - "select c1 from cte1 order by c1;") - - memTracker = tk.Se.GetSessionVars().StmtCtx.MemTracker - diskTracker = tk.Se.GetSessionVars().StmtCtx.DiskTracker + sql := fmt.Sprintf("with recursive cte1 as ( "+ + "select c1 from t1 "+ + "union "+ + "select c1 + 1 c1 from cte1 where c1 < %d) "+ + "select c1 from cte1 order by c1;", rowNum) + rows := tk.MustQuery(sql) + + memTracker := tk.Se.GetSessionVars().StmtCtx.MemTracker + diskTracker := tk.Se.GetSessionVars().StmtCtx.DiskTracker c.Assert(memTracker.MaxConsumed(), check.Greater, int64(0)) c.Assert(diskTracker.MaxConsumed(), check.Greater, int64(0)) sort.Ints(vals) - resRows = make([]string, 0, rowNum) + resRows := make([]string, 0, rowNum) for i := vals[0]; i <= rowNum; i++ { resRows = append(resRows, fmt.Sprintf("%d", i)) } From c44b8d118bfc13c9d8632f01332e4e4fb46a87fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Thu, 3 Jun 2021 12:26:26 +0800 Subject: [PATCH 248/343] config: make deadlock history's max size configuable (#24925) --- config/config.go | 7 +-- executor/executor_test.go | 1 + server/conn.go | 3 +- server/http_handler.go | 15 ++++++ server/http_handler_test.go | 28 +++++++++++ session/pessimistic_test.go | 1 + tidb-server/main.go | 2 + util/deadlockhistory/deadlock_history.go | 33 +++++++++++-- util/deadlockhistory/deadlock_history_test.go | 46 +++++++++++++++++++ 9 files changed, 127 insertions(+), 9 deletions(-) diff --git a/config/config.go b/config/config.go index 8390ba050b98f..3b4bda54436bd 100644 --- a/config/config.go +++ b/config/config.go @@ -381,7 +381,6 @@ func (e *ErrConfigValidationFailed) Error() string { "TiDB manual to make sure this option has not been deprecated and removed from your TiDB "+ "version if the option does not appear to be a typo", e.confFile, strings.Join( e.UndecodedItems, ", ")) - } // ClusterSecurity returns Security info for cluster @@ -494,13 +493,15 @@ type Binlog struct { // PessimisticTxn is the config for pessimistic transaction. type PessimisticTxn struct { // The max count of retry for a single statement in a pessimistic transaction. - MaxRetryCount uint `toml:"max-retry-count" json:"max-retry-count"` + MaxRetryCount uint `toml:"max-retry-count" json:"max-retry-count"` + DeadlockHistoryCapacity uint `toml:"deadlock-history-capacity" json:"deadlock-history-capacity"` } // DefaultPessimisticTxn returns the default configuration for PessimisticTxn func DefaultPessimisticTxn() PessimisticTxn { return PessimisticTxn{ - MaxRetryCount: 256, + MaxRetryCount: 256, + DeadlockHistoryCapacity: 10, } } diff --git a/executor/executor_test.go b/executor/executor_test.go index 8d70d7fa16468..1f5cc0b6b7b3d 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8179,6 +8179,7 @@ func (s *testSerialSuite) TestIssue24210(c *C) { func (s *testSerialSuite) TestDeadlockTable(c *C) { deadlockhistory.GlobalDeadlockHistory.Clear() + deadlockhistory.GlobalDeadlockHistory.Resize(10) occurTime := time.Date(2021, 5, 10, 1, 2, 3, 456789000, time.Local) rec := &deadlockhistory.DeadlockRecord{ diff --git a/server/conn.go b/server/conn.go index bba2a41a6a8c1..76b4f8d32dd1d 100644 --- a/server/conn.go +++ b/server/conn.go @@ -39,6 +39,7 @@ import ( "context" "crypto/tls" "encoding/binary" + goerr "errors" "fmt" "io" "net" @@ -52,8 +53,6 @@ import ( "time" "unsafe" - goerr "errors" - "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" diff --git a/server/http_handler.go b/server/http_handler.go index 0642924359210..023bebfbf7226 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -59,6 +59,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/pdapi" @@ -703,6 +704,20 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { return } } + if deadlockHistoryCapacity := req.Form.Get("tidb_deadlock_history_capacity"); deadlockHistoryCapacity != "" { + capacity, err := strconv.Atoi(deadlockHistoryCapacity) + if err != nil { + writeError(w, errors.New("illegal argument")) + return + } else if capacity < 0 || capacity > 10000 { + writeError(w, errors.New("tidb_deadlock_history_capacity out of range, should be in 0 to 10000")) + return + } + cfg := config.GetGlobalConfig() + cfg.PessimisticTxn.DeadlockHistoryCapacity = uint(capacity) + config.StoreGlobalConfig(cfg) + deadlockhistory.GlobalDeadlockHistory.Resize(uint(capacity)) + } } else { writeData(w, config.GetGlobalConfig()) } diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 5d2b654262a54..1dc3d896bf14a 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -55,6 +55,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/rowcodec" "github.com/pingcap/tidb/util/versioninfo" "go.uber.org/zap" @@ -1139,6 +1140,10 @@ func (ts *HTTPHandlerTestSuite) TestAllHistory(c *C) { c.Assert(jobs, DeepEquals, data) } +func dummyRecord() *deadlockhistory.DeadlockRecord { + return &deadlockhistory.DeadlockRecord{} +} + func (ts *HTTPHandlerTestSuite) TestPostSettings(c *C) { ts.startServer(c) ts.prepareData(c) @@ -1224,6 +1229,29 @@ func (ts *HTTPHandlerTestSuite) TestPostSettings(c *C) { c.Assert(resp.StatusCode, Equals, http.StatusOK) c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, false) dbt.mustExec("insert t2 values (unhex('f09f8c80'));") + + // test tidb_deadlock_history_capacity + deadlockhistory.GlobalDeadlockHistory.Resize(10) + for i := 0; i < 10; i++ { + deadlockhistory.GlobalDeadlockHistory.Push(dummyRecord()) + } + form = make(url.Values) + form.Set("tidb_deadlock_history_capacity", "5") + resp, err = ts.formStatus("/settings", form) + c.Assert(len(deadlockhistory.GlobalDeadlockHistory.GetAll()), Equals, 5) + c.Assert(deadlockhistory.GlobalDeadlockHistory.GetAll()[0].ID, Equals, uint64(6)) + c.Assert(deadlockhistory.GlobalDeadlockHistory.GetAll()[4].ID, Equals, uint64(10)) + deadlockhistory.GlobalDeadlockHistory.Push(dummyRecord()) + c.Assert(len(deadlockhistory.GlobalDeadlockHistory.GetAll()), Equals, 5) + c.Assert(deadlockhistory.GlobalDeadlockHistory.GetAll()[0].ID, Equals, uint64(7)) + c.Assert(deadlockhistory.GlobalDeadlockHistory.GetAll()[4].ID, Equals, uint64(11)) + form = make(url.Values) + form.Set("tidb_deadlock_history_capacity", "6") + resp, err = ts.formStatus("/settings", form) + deadlockhistory.GlobalDeadlockHistory.Push(dummyRecord()) + c.Assert(len(deadlockhistory.GlobalDeadlockHistory.GetAll()), Equals, 6) + c.Assert(deadlockhistory.GlobalDeadlockHistory.GetAll()[0].ID, Equals, uint64(7)) + c.Assert(deadlockhistory.GlobalDeadlockHistory.GetAll()[5].ID, Equals, uint64(12)) } func (ts *HTTPHandlerTestSuite) TestPprof(c *C) { diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index b70a142cde171..f4162c7d1f4d3 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -175,6 +175,7 @@ func (s *testPessimisticSuite) TestTxnMode(c *C) { func (s *testPessimisticSuite) TestDeadlock(c *C) { deadlockhistory.GlobalDeadlockHistory.Clear() + deadlockhistory.GlobalDeadlockHistory.Resize(10) tk1 := testkit.NewTestKitWithInit(c, s.store) tk1.MustExec("drop table if exists deadlock") diff --git a/tidb-server/main.go b/tidb-server/main.go index f99722e8bfd13..344df22865577 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -51,6 +51,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/disk" "github.com/pingcap/tidb/util/domainutil" "github.com/pingcap/tidb/util/kvcache" @@ -581,6 +582,7 @@ func setGlobalVars() { } tikv.SetStoreLivenessTimeout(t) parsertypes.TiDBStrictIntegerDisplayWidth = cfg.DeprecateIntegerDisplayWidth + deadlockhistory.GlobalDeadlockHistory.Resize(cfg.PessimisticTxn.DeadlockHistoryCapacity) } func setupLog() { diff --git a/util/deadlockhistory/deadlock_history.go b/util/deadlockhistory/deadlock_history.go index ef80f0f5d7ead..68bf99beaaadf 100644 --- a/util/deadlockhistory/deadlock_history.go +++ b/util/deadlockhistory/deadlock_history.go @@ -46,7 +46,7 @@ type DeadlockRecord struct { WaitChain []WaitChainItem } -// DeadlockHistory is a collection for maintaining recent several deadlock events. +// DeadlockHistory is a collection for maintaining recent several deadlock events. All its public APIs are thread safe. type DeadlockHistory struct { sync.RWMutex @@ -64,7 +64,7 @@ type DeadlockHistory struct { } // NewDeadlockHistory creates an instance of DeadlockHistory -func NewDeadlockHistory(capacity int) *DeadlockHistory { +func NewDeadlockHistory(capacity uint) *DeadlockHistory { return &DeadlockHistory{ deadlocks: make([]*DeadlockRecord, capacity), currentID: 1, @@ -73,8 +73,29 @@ func NewDeadlockHistory(capacity int) *DeadlockHistory { // GlobalDeadlockHistory is the global instance of DeadlockHistory, which is used to maintain recent several recent // deadlock events globally. -// TODO: Make the capacity configurable -var GlobalDeadlockHistory = NewDeadlockHistory(10) +// The real size of the deadlock history table should be initialized with `Resize` +// in `setGlobalVars` in tidb-server/main.go +var GlobalDeadlockHistory = NewDeadlockHistory(0) + +// Resize update the DeadlockHistory's table max capacity to newCapacity +func (d *DeadlockHistory) Resize(newCapacity uint) { + d.Lock() + defer d.Unlock() + if newCapacity != uint(len(d.deadlocks)) { + current := d.getAll() + d.head = 0 + if uint(len(current)) < newCapacity { + // extend deadlocks + d.deadlocks = make([]*DeadlockRecord, newCapacity) + copy(d.deadlocks, current) + } else { + // shrink deadlocks, keep the last len(current)-newCapacity items + // use append here to force golang to realloc the underlying array to save memory + d.deadlocks = append([]*DeadlockRecord{}, current[uint(len(current))-newCapacity:]...) + d.size = int(newCapacity) + } + } +} // Push pushes an element into the queue. It will set the `ID` field of the record, and add the pointer directly to // the collection. Be aware that do not modify the record's content after pushing. @@ -106,7 +127,11 @@ func (d *DeadlockHistory) Push(record *DeadlockRecord) { func (d *DeadlockHistory) GetAll() []*DeadlockRecord { d.RLock() defer d.RUnlock() + return d.getAll() +} +// getAll is a thread unsafe version of GetAll() for internal use +func (d *DeadlockHistory) getAll() []*DeadlockRecord { res := make([]*DeadlockRecord, 0, d.size) capacity := len(d.deadlocks) if d.head+d.size <= capacity { diff --git a/util/deadlockhistory/deadlock_history_test.go b/util/deadlockhistory/deadlock_history_test.go index b3307c91cc3c5..f5f6f7285e1d6 100644 --- a/util/deadlockhistory/deadlock_history_test.go +++ b/util/deadlockhistory/deadlock_history_test.go @@ -277,3 +277,49 @@ func (s *testDeadlockHistorySuite) TestErrDeadlockToDeadlockRecord(c *C) { expectedRecord.OccurTime = record.OccurTime c.Assert(record, DeepEquals, expectedRecord) } + +func dummyRecord() *DeadlockRecord { + return &DeadlockRecord{} +} + +func (s *testDeadlockHistorySuite) TestResize(c *C) { + h := NewDeadlockHistory(2) + h.Push(dummyRecord()) // id=1 inserted + h.Push(dummyRecord()) // id=2 inserted, + h.Push(dummyRecord()) // id=3 inserted, id=1 is removed + c.Assert(h.head, Equals, 1) + c.Assert(h.size, Equals, 2) + c.Assert(len(h.GetAll()), Equals, 2) + c.Assert(h.GetAll()[0].ID, Equals, uint64(2)) + c.Assert(h.GetAll()[1].ID, Equals, uint64(3)) + + h.Resize(3) + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 2) + h.Push(dummyRecord()) // id=4 inserted + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 3) + c.Assert(len(h.GetAll()), Equals, 3) + c.Assert(h.GetAll()[0].ID, Equals, uint64(2)) + c.Assert(h.GetAll()[1].ID, Equals, uint64(3)) + c.Assert(h.GetAll()[2].ID, Equals, uint64(4)) + + h.Resize(2) // id=2 removed + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 2) + c.Assert(len(h.GetAll()), Equals, 2) + c.Assert(h.GetAll()[0].ID, Equals, uint64(3)) + c.Assert(h.GetAll()[1].ID, Equals, uint64(4)) + + h.Resize(0) // all removed + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 0) + c.Assert(len(h.GetAll()), Equals, 0) + + h.Resize(2) + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 0) + h.Push(dummyRecord()) // id=5 inserted + c.Assert(h.head, Equals, 0) + c.Assert(h.size, Equals, 1) +} From dffba2195600883b62231f51676b7bbd7d3b5a17 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Wed, 2 Jun 2021 23:22:26 -0600 Subject: [PATCH 249/343] *: move executor/set.go special handling to setter functions (#25008) --- domain/sysvar_cache.go | 4 ++ executor/set.go | 109 +++++++++++++--------------------- sessionctx/variable/sysvar.go | 45 +++++++++++--- util/stmtsummary/variables.go | 4 +- 4 files changed, 84 insertions(+), 78 deletions(-) diff --git a/domain/sysvar_cache.go b/domain/sysvar_cache.go index 8df217dd3ad42..0cb8b8977f329 100644 --- a/domain/sysvar_cache.go +++ b/domain/sysvar_cache.go @@ -145,6 +145,10 @@ func (svc *SysVarCache) RebuildSysVarCache(ctx sessionctx.Context) error { } // checkEnableServerGlobalVar processes variables that acts in server and global level. +// This is required because the SetGlobal function on the sysvar struct only executes on +// the initiating tidb-server. There is no current method to say "run this function on all +// tidb servers when the value of this variable changes". If you do not require changes to +// be applied on all servers, use a getter/setter instead! You don't need to add to this list. func checkEnableServerGlobalVar(name, sVal string) { var err error switch name { diff --git a/executor/set.go b/executor/set.go index 4ab6878a21d08..825f79e985fc9 100644 --- a/executor/set.go +++ b/executor/set.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/logutil" - "github.com/pingcap/tidb/util/stmtsummary" "go.uber.org/zap" ) @@ -107,10 +106,8 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e if sysVar == nil { return variable.ErrUnknownSystemVar.GenWithStackByArgs(name) } - var valStr string - var err error if v.IsGlobal { - valStr, err = e.getVarValue(v, sysVar) + valStr, err := e.getVarValue(v, sysVar) if err != nil { return err } @@ -125,80 +122,54 @@ func (e *SetExecutor) setSysVariable(name string, v *expression.VarAssignment) e } return nil }) - if err != nil { - return err - } logutil.BgLogger().Info("set global var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) - } else { - valStr, err = e.getVarValue(v, nil) - if err != nil { - return err - } - getSnapshotTSByName := func() uint64 { - if name == variable.TiDBSnapshot { - return sessionVars.SnapshotTS - } else if name == variable.TiDBTxnReadTS { - return sessionVars.TxnReadTS.PeakTxnReadTS() - } - return 0 - } - oldSnapshotTS := getSnapshotTSByName() - fallbackOldSnapshotTS := func() { - if name == variable.TiDBSnapshot { - sessionVars.SnapshotTS = oldSnapshotTS - } else if name == variable.TiDBTxnReadTS { - sessionVars.TxnReadTS.SetTxnReadTS(oldSnapshotTS) - } - } - if name == variable.TxnIsolationOneShot && sessionVars.InTxn() { - return errors.Trace(ErrCantChangeTxCharacteristics) - } - err = variable.SetSessionSystemVar(sessionVars, name, valStr) - if err != nil { - return err + return err + } + // Set session variable + valStr, err := e.getVarValue(v, nil) + if err != nil { + return err + } + getSnapshotTSByName := func() uint64 { + if name == variable.TiDBSnapshot { + return sessionVars.SnapshotTS + } else if name == variable.TiDBTxnReadTS { + return sessionVars.TxnReadTS.PeakTxnReadTS() } - newSnapshotTS := getSnapshotTSByName() - newSnapshotIsSet := newSnapshotTS > 0 && newSnapshotTS != oldSnapshotTS - if newSnapshotIsSet { - err = gcutil.ValidateSnapshot(e.ctx, newSnapshotTS) - if err != nil { - fallbackOldSnapshotTS() - return err - } + return 0 + } + oldSnapshotTS := getSnapshotTSByName() + fallbackOldSnapshotTS := func() { + if name == variable.TiDBSnapshot { + sessionVars.SnapshotTS = oldSnapshotTS + } else if name == variable.TiDBTxnReadTS { + sessionVars.TxnReadTS.SetTxnReadTS(oldSnapshotTS) } - err = e.loadSnapshotInfoSchemaIfNeeded(newSnapshotTS) + } + if name == variable.TxnIsolationOneShot && sessionVars.InTxn() { + return errors.Trace(ErrCantChangeTxCharacteristics) + } + err = variable.SetSessionSystemVar(sessionVars, name, valStr) + if err != nil { + return err + } + newSnapshotTS := getSnapshotTSByName() + newSnapshotIsSet := newSnapshotTS > 0 && newSnapshotTS != oldSnapshotTS + if newSnapshotIsSet { + err = gcutil.ValidateSnapshot(e.ctx, newSnapshotTS) if err != nil { fallbackOldSnapshotTS() return err } - // Clients are often noisy in setting session variables such as - // autocommit, timezone, query cache - logutil.BgLogger().Debug("set session var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) } - - // These are server instance scoped variables, and have special semantics. - // i.e. after SET SESSION, other users sessions will reflect the new value. - // TODO: in future these could be better managed as a post-set hook. - - valStrToBoolStr := variable.BoolToOnOff(variable.TiDBOptOn(valStr)) - - switch name { - case variable.TiDBEnableStmtSummary: - return stmtsummary.StmtSummaryByDigestMap.SetEnabled(valStr, !v.IsGlobal) - case variable.TiDBStmtSummaryInternalQuery: - return stmtsummary.StmtSummaryByDigestMap.SetEnabledInternalQuery(valStr, !v.IsGlobal) - case variable.TiDBStmtSummaryRefreshInterval: - return stmtsummary.StmtSummaryByDigestMap.SetRefreshInterval(valStr, !v.IsGlobal) - case variable.TiDBStmtSummaryHistorySize: - return stmtsummary.StmtSummaryByDigestMap.SetHistorySize(valStr, !v.IsGlobal) - case variable.TiDBStmtSummaryMaxStmtCount: - return stmtsummary.StmtSummaryByDigestMap.SetMaxStmtCount(valStr, !v.IsGlobal) - case variable.TiDBStmtSummaryMaxSQLLength: - return stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength(valStr, !v.IsGlobal) - case variable.TiDBCapturePlanBaseline: - variable.CapturePlanBaseline.Set(valStrToBoolStr, !v.IsGlobal) + err = e.loadSnapshotInfoSchemaIfNeeded(newSnapshotTS) + if err != nil { + fallbackOldSnapshotTS() + return err } - + // Clients are often noisy in setting session variables such as + // autocommit, timezone, query cache + logutil.BgLogger().Debug("set session var", zap.Uint64("conn", sessionVars.ConnectionID), zap.String("name", name), zap.String("val", valStr)) return nil } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index d49e4b1b19bda..b399be0415832 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tidb/util/versioninfo" atomic2 "go.uber.org/atomic" ) @@ -1399,14 +1400,44 @@ var defaultSysVars = []*SysVar{ s.AllowRemoveAutoInc = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableStmtSummary, Value: BoolToOnOff(config.GetGlobalConfig().StmtSummary.Enable), Type: TypeBool, AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryInternalQuery, Value: BoolToOnOff(config.GetGlobalConfig().StmtSummary.EnableInternalQuery), Type: TypeBool, AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryRefreshInterval, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.RefreshInterval), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryHistorySize, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.HistorySize), Type: TypeInt, MinValue: 0, MaxValue: math.MaxUint8, AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxStmtCount, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxStmtCount), 10), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt16, AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxSQLLength, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxSQLLength), 10), Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, AllowEmpty: true}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBCapturePlanBaseline, Value: Off, Type: TypeBool, AllowEmptyAll: true, GetSession: func(s *SessionVars) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableStmtSummary, Value: BoolToOnOff(config.GetGlobalConfig().StmtSummary.Enable), skipInit: true, Type: TypeBool, AllowEmpty: true, SetSession: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetEnabled(val, true) + }, SetGlobal: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetEnabled(val, false) + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryInternalQuery, Value: BoolToOnOff(config.GetGlobalConfig().StmtSummary.EnableInternalQuery), skipInit: true, Type: TypeBool, AllowEmpty: true, SetSession: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetEnabledInternalQuery(val, true) + }, SetGlobal: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetEnabledInternalQuery(val, false) + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryRefreshInterval, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.RefreshInterval), skipInit: true, Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt32, AllowEmpty: true, SetSession: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetRefreshInterval(val, true) + }, SetGlobal: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetRefreshInterval(val, false) + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryHistorySize, Value: strconv.Itoa(config.GetGlobalConfig().StmtSummary.HistorySize), skipInit: true, Type: TypeInt, MinValue: 0, MaxValue: math.MaxUint8, AllowEmpty: true, SetSession: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetHistorySize(val, true) + }, SetGlobal: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetHistorySize(val, false) + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxStmtCount, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxStmtCount), 10), skipInit: true, Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt16, AllowEmpty: true, SetSession: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetMaxStmtCount(val, true) + }, SetGlobal: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetMaxStmtCount(val, false) + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBStmtSummaryMaxSQLLength, Value: strconv.FormatUint(uint64(config.GetGlobalConfig().StmtSummary.MaxSQLLength), 10), skipInit: true, Type: TypeInt, MinValue: 0, MaxValue: math.MaxInt32, AllowEmpty: true, SetSession: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength(val, true) + }, SetGlobal: func(s *SessionVars, val string) error { + return stmtsummary.StmtSummaryByDigestMap.SetMaxSQLLength(val, false) + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBCapturePlanBaseline, Value: Off, Type: TypeBool, AllowEmptyAll: true, skipInit: true, GetSession: func(s *SessionVars) (string, error) { return CapturePlanBaseline.GetVal(), nil + }, SetSession: func(s *SessionVars, val string) error { + CapturePlanBaseline.Set(val, true) + return nil + }, SetGlobal: func(s *SessionVars, val string) error { + CapturePlanBaseline.Set(val, false) + return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBUsePlanBaselines, Value: BoolToOnOff(DefTiDBUsePlanBaselines), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.UsePlanBaselines = TiDBOptOn(val) diff --git a/util/stmtsummary/variables.go b/util/stmtsummary/variables.go index b179929002fd5..96f6b0fac7076 100644 --- a/util/stmtsummary/variables.go +++ b/util/stmtsummary/variables.go @@ -16,12 +16,12 @@ package stmtsummary import ( "fmt" "strconv" + "strings" "sync" "sync/atomic" "github.com/pingcap/errors" "github.com/pingcap/tidb/config" - svariable "github.com/pingcap/tidb/sessionctx/variable" ) const ( @@ -104,7 +104,7 @@ func getBoolFinalVariable(varType int, sessionValue, globalValue string) int64 { // normalizeEnableValue converts 'ON' or '1' to 1 and 'OFF' or '0' to 0. func normalizeEnableValue(value string) int64 { - if svariable.TiDBOptOn(value) { + if strings.EqualFold(value, "ON") || value == "1" { return 1 } return 0 From b21aed8cc46e229b74fcd4e96e9f4170ceecde98 Mon Sep 17 00:00:00 2001 From: kaixu120811 <78292293+kaixu120811@users.noreply.github.com> Date: Thu, 3 Jun 2021 14:08:26 +0800 Subject: [PATCH 250/343] Telemetry: Add slow query statistic bucket into telemetry data (#25006) --- telemetry/data.go | 6 + telemetry/data_feature_usage.go | 1 + telemetry/data_slow_query.go | 187 ++++++++++++++++++++++++++++++++ telemetry/telemetry.go | 2 +- 4 files changed, 195 insertions(+), 1 deletion(-) create mode 100644 telemetry/data_slow_query.go diff --git a/telemetry/data.go b/telemetry/data.go index 1cf0e3d66aa25..2600738f7f516 100644 --- a/telemetry/data.go +++ b/telemetry/data.go @@ -27,6 +27,7 @@ type telemetryData struct { TrackingID string `json:"trackingId"` FeatureUsage *featureUsage `json:"featureUsage"` WindowedStats []*windowData `json:"windowedStats"` + SlowQueryStats *slowQueryStats `json:"slowQueryStats"` } func generateTelemetryData(ctx sessionctx.Context, trackingID string) telemetryData { @@ -43,6 +44,10 @@ func generateTelemetryData(ctx sessionctx.Context, trackingID string) telemetryD if f, err := getFeatureUsage(ctx); err == nil { r.FeatureUsage = f } + if s, err := getSlowQueryStats(ctx); err == nil { + r.SlowQueryStats = s + } + r.WindowedStats = getWindowData() r.TelemetryHostExtra = getTelemetryHostExtraInfo() return r @@ -50,4 +55,5 @@ func generateTelemetryData(ctx sessionctx.Context, trackingID string) telemetryD func postReportTelemetryData() { postReportTxnUsage() + postReportSlowQueryStats() } diff --git a/telemetry/data_feature_usage.go b/telemetry/data_feature_usage.go index b1bb9f7e996e3..6767192d14e76 100644 --- a/telemetry/data_feature_usage.go +++ b/telemetry/data_feature_usage.go @@ -96,6 +96,7 @@ func GetClusterIndexUsageInfo(ctx sessionctx.Context) (cu *ClusterIndexUsage, er } } }() + err = ctx.RefreshTxnCtx(context.TODO()) if err != nil { return nil, err diff --git a/telemetry/data_slow_query.go b/telemetry/data_slow_query.go new file mode 100644 index 0000000000000..1b66e20ca2670 --- /dev/null +++ b/telemetry/data_slow_query.go @@ -0,0 +1,187 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package telemetry + +import ( + "context" + "errors" + "fmt" + "strconv" + "sync" + "time" + + pingcapErrors "github.com/pingcap/errors" + "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/logutil" + "github.com/prometheus/client_golang/api" + promv1 "github.com/prometheus/client_golang/api/prometheus/v1" + pmodel "github.com/prometheus/common/model" + "go.uber.org/zap" +) + +type slowQueryStats struct { + // Slow Query statistic buckets + SQBInfo *SlowQueryBucket `json:"slowQueryBucket"` +} + +// SlowQueryBucket records the statistic information of slow query buckets +// Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days // defined in metrics/server.go +type SlowQueryBucket map[string]int + +func (bucketMap SlowQueryBucket) String() string { + if bucketMap == nil { + return "nil" + } + var retStr string = "{" + for k, v := range bucketMap { + retStr += k + ":" + strconv.Itoa(v) + "," + } + retStr = retStr[:len(retStr)-1] + return retStr +} + +const slowQueryBucketNum = 29 //prometheus.ExponentialBuckets(0.001, 2, 28), and 1 more +Inf + +var ( + // LastSQBInfo records last statistic information of slow query buckets + LastSQBInfo SlowQueryBucket + // CurrentSQBInfo records current statitic information of slow query buckets + CurrentSQBInfo SlowQueryBucket + slowQueryLock = sync.RWMutex{} +) + +func getSlowQueryStats(ctx sessionctx.Context) (*slowQueryStats, error) { + slowQueryBucket, err := getSlowQueryBucket(ctx) + if err != nil { + logutil.BgLogger().Info(err.Error()) + return nil, err + } + + return &slowQueryStats{slowQueryBucket}, nil +} + +// getSlowQueryBucket genenrates the delta SlowQueryBucket to report +func getSlowQueryBucket(ctx sessionctx.Context) (*SlowQueryBucket, error) { + // update CurrentSQBInfo first, then gen delta + if err := updateCurrentSQB(ctx); err != nil { + return nil, err + } + delta := calculateDeltaSQB() + return delta, nil +} + +// updateCurrentSQB records current slow query buckets +func updateCurrentSQB(ctx sessionctx.Context) (err error) { + defer func() { + if r := recover(); r != nil { + err = pingcapErrors.Errorf(fmt.Sprintln(r)) + } + }() + + value, err := querySlowQueryMetric(ctx) //TODO: judge error here + if err != nil && err != infosync.ErrPrometheusAddrIsNotSet { + logutil.BgLogger().Info("querySlowQueryMetric got error") + return err + } + + if value.Type() != pmodel.ValVector { + return errors.New("Prom vector expected, got " + value.Type().String()) + } + promVec := value.(pmodel.Vector) + slowQueryLock.Lock() + for _, sample := range promVec { + metric := sample.Metric + bucketName := metric["le"] //hardcode bucket upper bound + CurrentSQBInfo[string(bucketName)] = int(sample.Value) + } + slowQueryLock.Unlock() + return nil +} + +func querySlowQueryMetric(sctx sessionctx.Context) (result pmodel.Value, err error) { + // Add retry to avoid network error. + var prometheusAddr string + for i := 0; i < 5; i++ { + //TODO: the prometheus will be Integrated into the PD, then we need to query the prometheus in PD directly, which need change the query API + prometheusAddr, err = infosync.GetPrometheusAddr() + if err == nil || err == infosync.ErrPrometheusAddrIsNotSet { + break + } + time.Sleep(100 * time.Millisecond) + } + if err != nil { + return nil, err + } + promClient, err := api.NewClient(api.Config{ + Address: prometheusAddr, + }) + if err != nil { + return nil, err + } + promQLAPI := promv1.NewAPI(promClient) + promQL := "tidb_server_slow_query_process_duration_seconds_bucket{sql_type=\"general\"}" + + ts := time.Now().Add(-time.Minute) + // Add retry to avoid network error. + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + for i := 0; i < 5; i++ { + result, _, err = promQLAPI.Query(ctx, promQL, ts) + if err == nil { + break + } + time.Sleep(100 * time.Millisecond) + } + return result, err +} + +// calculateDeltaSQB calculate the delta between current slow query bucket and last slow query bucket +func calculateDeltaSQB() *SlowQueryBucket { + deltaMap := make(SlowQueryBucket) + slowQueryLock.Lock() + for key, value := range CurrentSQBInfo { + deltaMap[key] = value - (LastSQBInfo)[key] + } + slowQueryLock.Unlock() + return &deltaMap +} + +// initSlowQueryStats Init LastSQBInfo, follow the definition of metrics/server.go +// Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days +func initSlowQueryStats() { + LastSQBInfo := make(SlowQueryBucket) + CurrentSQBInfo := make(SlowQueryBucket) + + bucketBase := 0.001 // From 0.001 to 134217.728, total 28 float number; the 29th is +Inf + for i := 0; i < slowQueryBucketNum-1; i++ { + LastSQBInfo[strconv.FormatFloat(bucketBase, 'f', 3, 32)] = 0 + CurrentSQBInfo[strconv.FormatFloat(bucketBase, 'f', 3, 32)] = 0 + bucketBase += bucketBase + } + LastSQBInfo["+Inf"] = 0 + CurrentSQBInfo["+Inf"] = 0 + + logutil.BgLogger().Info("Telemetry slow query stats initialized", zap.String("CurrentSQBInfo", CurrentSQBInfo.String()), zap.String("LastSQBInfo", LastSQBInfo.String())) +} + +// postReportSlowQueryStats copy CurrentSQBInfo to LastSQBInfo to be ready for next report +// this function is designed for being compatible with preview telemetry +func postReportSlowQueryStats() { + slowQueryLock.Lock() + LastSQBInfo = CurrentSQBInfo + CurrentSQBInfo = make(SlowQueryBucket) + slowQueryLock.Unlock() + logutil.BgLogger().Info("Telemetry slow query stats, postReportSlowQueryStats finished") +} diff --git a/telemetry/telemetry.go b/telemetry/telemetry.go index 55d6dfb215d34..23fe2462891d4 100644 --- a/telemetry/telemetry.go +++ b/telemetry/telemetry.go @@ -171,7 +171,7 @@ func InitialRun(ctx sessionctx.Context, etcdClient *clientv3.Client) error { return err } + initSlowQueryStats() logutil.BgLogger().Info("Telemetry configuration", zap.String("endpoint", apiEndpoint), zap.Duration("report_interval", ReportInterval), zap.Bool("enabled", enabled)) - return ReportUsageData(ctx, etcdClient) } From 3e3f9778ad9349f9fce322af98097ae578930130 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 3 Jun 2021 14:28:27 +0800 Subject: [PATCH 251/343] statistics: fix the top-n size to not hold the small things (#24906) --- statistics/builder.go | 55 ++++++++++++++++++++++++++++++++ statistics/handle/handle_test.go | 18 +++++++---- statistics/statistics_test.go | 17 ++++++---- 3 files changed, 76 insertions(+), 14 deletions(-) diff --git a/statistics/builder.go b/statistics/builder.go index 6dad07b844e35..2c36b0c27917a 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -15,6 +15,7 @@ package statistics import ( "bytes" + "math" "github.com/pingcap/errors" "github.com/pingcap/tidb/sessionctx" @@ -329,6 +330,8 @@ func BuildHistAndTopN( } } + topNList = pruneTopNItem(topNList, ndv, nullCount, sampleNum, count) + // Step2: exclude topn from samples for i := int64(0); i < int64(len(samples)); i++ { sampleBytes, err := getComparedBytes(samples[i].Value) @@ -366,3 +369,55 @@ func BuildHistAndTopN( return hg, topn, nil } + +// 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 { + return topns + } + // Sum the occurrence except the least common one from the top-n list. To check whether the lest common one is worth + // storing later. + sumCount := uint64(0) + for i := 0; i < len(topns)-1; i++ { + sumCount += topns[i].Count + } + topNNum := len(topns) + for topNNum > 0 { + // Selectivity for the ones not in the top-n list. + // (1 - things in top-n list - null) / remained ndv. + selectivity := 1.0 - float64(sumCount)/float64(sampleRows) - float64(nullCount)/float64(totalRows) + if selectivity < 0.0 { + selectivity = 0 + } + if selectivity > 1 { + selectivity = 1 + } + otherNDV := float64(ndv) - float64(topNNum) + if otherNDV > 1 { + selectivity /= otherNDV + } + N := float64(totalRows) + n := float64(sampleRows) + K := N * float64(topns[topNNum-1].Count) / n + // Since we are sampling without replacement. The distribution would be a hypergeometric distribution. + // 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. + // 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. + break + } + // Current one is not worth storing, remove it and subtract it from sumCount, go to next one. + topNNum-- + if topNNum == 0 { + break + } + sumCount -= topns[topNNum-1].Count + } + return topns[:topNNum] +} diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 1518503fa73ba..58245841785fb 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -848,9 +848,13 @@ func (s *testStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.TestKit buf1.WriteString(fmt.Sprintf(", (%v)", i)) buf2.WriteString(fmt.Sprintf(", (%v)", 100000+i)) } + for i := 0; i < 1000; i++ { + buf1.WriteString(fmt.Sprintf(", (%v)", 0)) + buf2.WriteString(fmt.Sprintf(", (%v)", 100000)) + } tk.MustExec(buf1.String()) tk.MustExec(buf2.String()) - tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("set @@tidb_analyze_version=3") tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) } @@ -907,23 +911,23 @@ func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts2(c *C) { tk := testkit.NewTestKit(c, s.store) s.prepareForGlobalStatsWithOpts(c, tk) - tk.MustExec("analyze table t with 20 topn, 50 buckets") - s.checkForGlobalStatsWithOpts(c, tk, "global", 20, 50) - s.checkForGlobalStatsWithOpts(c, tk, "p0", 20, 50) - s.checkForGlobalStatsWithOpts(c, tk, "p1", 20, 50) + tk.MustExec("analyze table t with 20 topn, 50 buckets, 1000 samples") + s.checkForGlobalStatsWithOpts(c, tk, "global", 2, 50) + s.checkForGlobalStatsWithOpts(c, tk, "p0", 1, 50) + s.checkForGlobalStatsWithOpts(c, tk, "p1", 1, 50) // analyze a partition to let its options be different with others' tk.MustExec("analyze table t partition p0 with 10 topn, 20 buckets") s.checkForGlobalStatsWithOpts(c, tk, "global", 10, 20) // use new options s.checkForGlobalStatsWithOpts(c, tk, "p0", 10, 20) - s.checkForGlobalStatsWithOpts(c, tk, "p1", 20, 50) + s.checkForGlobalStatsWithOpts(c, tk, "p1", 1, 50) tk.MustExec("analyze table t partition p1 with 100 topn, 200 buckets") s.checkForGlobalStatsWithOpts(c, tk, "global", 100, 200) s.checkForGlobalStatsWithOpts(c, tk, "p0", 10, 20) s.checkForGlobalStatsWithOpts(c, tk, "p1", 100, 200) - tk.MustExec("analyze table t partition p0") // default options + tk.MustExec("analyze table t partition p0 with 20 topn") // change back to 20 topn s.checkForGlobalStatsWithOpts(c, tk, "global", 20, 256) s.checkForGlobalStatsWithOpts(c, tk, "p0", 20, 256) s.checkForGlobalStatsWithOpts(c, tk, "p1", 100, 200) diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index c352c3576c89b..2b9f9771ed12a 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -281,23 +281,26 @@ func (s *testStatisticsSuite) TestBuild(c *C) { colv2, topnv2, err := BuildHistAndTopN(ctx, int(bucketCount), topNCount, 2, collector, types.NewFieldType(mysql.TypeLonglong), true) c.Check(err, IsNil) c.Check(topnv2.TopN, NotNil) - expectedTopNCount := []uint64{9990, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30} + // The most common one's occurrence is 9990, the second most common one's occurrence is 30. + // The ndv of the histogram is 73344, the total count of it is 90010. 90010/73344 vs 30, it's not a bad estimate. + expectedTopNCount := []uint64{9990} + c.Assert(len(topnv2.TopN), Equals, len(expectedTopNCount)) for i, meta := range topnv2.TopN { c.Check(meta.Count, Equals, expectedTopNCount[i]) } - c.Check(colv2.Len(), Equals, 256) + c.Check(colv2.Len(), Equals, 251) count = colv2.lessRowCount(types.NewIntDatum(1000)) - c.Check(int(count), Equals, 325) + c.Check(int(count), Equals, 328) count = colv2.lessRowCount(types.NewIntDatum(2000)) - c.Check(int(count), Equals, 9430) + c.Check(int(count), Equals, 10007) count = colv2.greaterRowCount(types.NewIntDatum(2000)) - c.Check(int(count), Equals, 80008) + c.Check(int(count), Equals, 80001) count = colv2.lessRowCount(types.NewIntDatum(200000000)) - c.Check(int(count), Equals, 89440) + c.Check(int(count), Equals, 90010) count = colv2.greaterRowCount(types.NewIntDatum(200000000)) c.Check(count, Equals, 0.0) count = colv2.BetweenRowCount(types.NewIntDatum(3000), types.NewIntDatum(3500)) - c.Check(int(count), Equals, 4995) + c.Check(int(count), Equals, 5001) count = colv2.lessRowCount(types.NewIntDatum(1)) c.Check(int(count), Equals, 0) From 0f151af7c5f86e205ff726ee707b7a2e9bb8844f Mon Sep 17 00:00:00 2001 From: wjHuang Date: Thu, 3 Jun 2021 14:46:27 +0800 Subject: [PATCH 252/343] *: add telemetry support for CTE (#25022) --- executor/adapter.go | 11 +++- executor/benchmark_test.go | 10 ++-- executor/builder.go | 10 +++- executor/compiler.go | 1 + executor/coprocessor.go | 2 +- executor/executor_required_rows_test.go | 2 +- executor/prepared.go | 1 + metrics/session.go | 1 + metrics/telemetry.go | 68 +++++++++++++++++++++++++ session/session.go | 30 +++++++++-- telemetry/data.go | 1 + telemetry/data_feature_usage.go | 19 ++++++- telemetry/telemetry_test.go | 36 +++++++++++++ 13 files changed, 177 insertions(+), 15 deletions(-) create mode 100644 metrics/telemetry.go diff --git a/executor/adapter.go b/executor/adapter.go index f1d6a8b8e6040..308676dc704a7 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -176,6 +176,12 @@ func (a *recordSet) OnFetchReturned() { a.stmt.LogSlowQuery(a.txnStartTS, a.lastErr == nil, true) } +// TelemetryInfo records some telemetry information during execution. +type TelemetryInfo struct { + UseNonRecursive bool + UseRecursive bool +} + // ExecStmt implements the sqlexec.Statement interface, it builds a planner.Plan to an sqlexec.Statement. type ExecStmt struct { // GoCtx stores parent go context.Context for a stmt. @@ -204,6 +210,7 @@ type ExecStmt struct { // OutputNames will be set if using cached plan OutputNames []*types.FieldName PsStmt *plannercore.CachedPrepareStmt + Ti *TelemetryInfo } // PointGet short path for point exec directly from plan, keep only necessary steps @@ -236,7 +243,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec } } if a.PsStmt.Executor == nil { - b := newExecutorBuilder(a.Ctx, is) + b := newExecutorBuilder(a.Ctx, is, a.Ti) newExecutor := b.build(a.Plan) if b.err != nil { return nil, b.err @@ -782,7 +789,7 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityLow } - b := newExecutorBuilder(ctx, a.InfoSchema) + b := newExecutorBuilder(ctx, a.InfoSchema, a.Ti) b.snapshotTS = a.SnapshotTS e := b.build(a.Plan) if b.err != nil { diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index adf6cc6c854ed..df593a6e3d7cc 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -290,7 +290,7 @@ func buildHashAggExecutor(ctx sessionctx.Context, src Executor, schema *expressi plan.SetSchema(schema) plan.Init(ctx, nil, 0) plan.SetChildren(nil) - b := newExecutorBuilder(ctx, nil) + b := newExecutorBuilder(ctx, nil, nil) exec := b.build(plan) hashAgg := exec.(*HashAggExec) hashAgg.children[0] = src @@ -342,7 +342,7 @@ func buildStreamAggExecutor(ctx sessionctx.Context, srcExec Executor, schema *ex plan = sg } - b := newExecutorBuilder(ctx, nil) + b := newExecutorBuilder(ctx, nil, nil) return b.build(plan) } @@ -575,7 +575,7 @@ func buildWindowExecutor(ctx sessionctx.Context, windowFunc string, funcs int, f plan = win } - b := newExecutorBuilder(ctx, nil) + b := newExecutorBuilder(ctx, nil, nil) exec := b.build(plan) return exec } @@ -1290,7 +1290,7 @@ func prepare4IndexInnerHashJoin(tc *indexJoinTestCase, outerDS *mockDataSource, hashCols: tc.outerHashKeyIdx, }, innerCtx: innerCtx{ - readerBuilder: &dataReaderBuilder{Plan: &mockPhysicalIndexReader{e: innerDS}, executorBuilder: newExecutorBuilder(tc.ctx, nil)}, + readerBuilder: &dataReaderBuilder{Plan: &mockPhysicalIndexReader{e: innerDS}, executorBuilder: newExecutorBuilder(tc.ctx, nil, nil)}, rowTypes: rightTypes, colLens: colLens, keyCols: tc.innerJoinKeyIdx, @@ -1356,7 +1356,7 @@ func prepare4IndexMergeJoin(tc *indexJoinTestCase, outerDS *mockDataSource, inne compareFuncs: outerCompareFuncs, }, innerMergeCtx: innerMergeCtx{ - readerBuilder: &dataReaderBuilder{Plan: &mockPhysicalIndexReader{e: innerDS}, executorBuilder: newExecutorBuilder(tc.ctx, nil)}, + readerBuilder: &dataReaderBuilder{Plan: &mockPhysicalIndexReader{e: innerDS}, executorBuilder: newExecutorBuilder(tc.ctx, nil, nil)}, rowTypes: rightTypes, joinKeys: innerJoinKeys, colLens: colLens, diff --git a/executor/builder.go b/executor/builder.go index 66f20c000a92b..0b9711ba95a34 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -82,6 +82,7 @@ type executorBuilder struct { snapshotTSCached bool err error // err is set when there is error happened during Executor building process. hasLock bool + Ti *TelemetryInfo } // CTEStorages stores resTbl and iterInTbl for CTEExec. @@ -92,10 +93,11 @@ type CTEStorages struct { IterInTbl cteutil.Storage } -func newExecutorBuilder(ctx sessionctx.Context, is infoschema.InfoSchema) *executorBuilder { +func newExecutorBuilder(ctx sessionctx.Context, is infoschema.InfoSchema, ti *TelemetryInfo) *executorBuilder { return &executorBuilder{ ctx: ctx, is: is, + Ti: ti, } } @@ -4086,6 +4088,9 @@ func (b *executorBuilder) buildTableSample(v *plannercore.PhysicalTableSample) * func (b *executorBuilder) buildCTE(v *plannercore.PhysicalCTE) Executor { // 1. Build seedPlan. + if b.Ti != nil { + b.Ti.UseNonRecursive = true + } seedExec := b.build(v.SeedPlan) if b.err != nil { return nil @@ -4127,6 +4132,9 @@ func (b *executorBuilder) buildCTE(v *plannercore.PhysicalCTE) Executor { } // 3. Build recursive part. + if v.RecurPlan != nil && b.Ti != nil { + b.Ti.UseRecursive = true + } recursiveExec := b.build(v.RecurPlan) if b.err != nil { return nil diff --git a/executor/compiler.go b/executor/compiler.go index 673f17f24f40e..5f0454c66390a 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -78,6 +78,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm StmtNode: stmtNode, Ctx: c.Ctx, OutputNames: names, + Ti: &TelemetryInfo{}, }, nil } diff --git a/executor/coprocessor.go b/executor/coprocessor.go index f46ba8774c2be..04b90af2ad63b 100644 --- a/executor/coprocessor.go +++ b/executor/coprocessor.go @@ -168,7 +168,7 @@ func (h *CoprocessorDAGHandler) buildDAGExecutor(req *coprocessor.Request) (Exec } plan = core.InjectExtraProjection(plan) // Build executor. - b := newExecutorBuilder(h.sctx, is) + b := newExecutorBuilder(h.sctx, is, nil) return b.build(plan), nil } diff --git a/executor/executor_required_rows_test.go b/executor/executor_required_rows_test.go index 720b5300677c9..a0eb15ca77860 100644 --- a/executor/executor_required_rows_test.go +++ b/executor/executor_required_rows_test.go @@ -843,7 +843,7 @@ func buildMergeJoinExec(ctx sessionctx.Context, joinType plannercore.JoinType, i j.CompareFuncs = append(j.CompareFuncs, expression.GetCmpFunction(nil, j.LeftJoinKeys[i], j.RightJoinKeys[i])) } - b := newExecutorBuilder(ctx, nil) + b := newExecutorBuilder(ctx, nil, nil) return b.build(j) } diff --git a/executor/prepared.go b/executor/prepared.go index 2a11977ef607e..a5fd131d1a87b 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -337,6 +337,7 @@ func CompileExecutePreparedStmt(ctx context.Context, sctx sessionctx.Context, StmtNode: execStmt, Ctx: sctx, OutputNames: names, + Ti: &TelemetryInfo{}, } if preparedPointer, ok := sctx.GetSessionVars().PreparedStmts[ID]; ok { preparedObj, ok := preparedPointer.(*plannercore.CachedPrepareStmt) diff --git a/metrics/session.go b/metrics/session.go index f2d682a6dd22b..775ffc0f5c630 100644 --- a/metrics/session.go +++ b/metrics/session.go @@ -146,4 +146,5 @@ const ( LblInTxn = "in_txn" LblVersion = "version" LblHash = "hash" + LblCTEType = "cte_type" ) diff --git a/metrics/telemetry.go b/metrics/telemetry.go new file mode 100644 index 0000000000000..054001f1a7c48 --- /dev/null +++ b/metrics/telemetry.go @@ -0,0 +1,68 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package metrics + +import ( + "github.com/prometheus/client_golang/prometheus" + dto "github.com/prometheus/client_model/go" +) + +// Metrics +var ( + TelemetrySQLCTECnt = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "telemetry", + Name: "non_recursive_cte_usage", + Help: "Counter of usage of CTE", + }, []string{LblCTEType}) +) + +// readCounter reads the value of a prometheus.Counter. +// Returns -1 when failing to read the value. +func readCounter(m prometheus.Counter) int64 { + // Actually, it's not recommended to read the value of prometheus metric types directly: + // https://github.com/prometheus/client_golang/issues/486#issuecomment-433345239 + pb := &dto.Metric{} + // It's impossible to return an error though. + if err := m.Write(pb); err != nil { + return -1 + } + return int64(pb.GetCounter().GetValue()) +} + +// CTEUsageCounter records the usages of CTE. +type CTEUsageCounter struct { + NonRecursiveCTEUsed int64 `json:"nonRecursiveCTEUsed"` + RecursiveUsed int64 `json:"recursiveUsed"` + NonCTEUsed int64 `json:"nonCTEUsed"` +} + +// Sub returns the difference of two counters. +func (c CTEUsageCounter) Sub(rhs CTEUsageCounter) CTEUsageCounter { + new := CTEUsageCounter{} + new.NonRecursiveCTEUsed = c.NonRecursiveCTEUsed - rhs.NonRecursiveCTEUsed + new.RecursiveUsed = c.RecursiveUsed - rhs.RecursiveUsed + new.NonCTEUsed = c.NonCTEUsed - rhs.NonCTEUsed + return new +} + +// GetCTECounter gets the TxnCommitCounter. +func GetCTECounter() CTEUsageCounter { + return CTEUsageCounter{ + NonRecursiveCTEUsed: readCounter(TelemetrySQLCTECnt.With(prometheus.Labels{LblCTEType: "nonRecurCTE"})), + RecursiveUsed: readCounter(TelemetrySQLCTECnt.With(prometheus.Labels{LblCTEType: "recurCTE"})), + NonCTEUsed: readCounter(TelemetrySQLCTECnt.With(prometheus.Labels{LblCTEType: "notCTE"})), + } +} diff --git a/session/session.go b/session/session.go index 0c10d2edbcc2d..7435ef5882aaa 100644 --- a/session/session.go +++ b/session/session.go @@ -101,6 +101,8 @@ var ( sessionExecuteParseDurationInternal = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblInternal) sessionExecuteParseDurationGeneral = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblGeneral) + telemetryCTEUsage = metrics.TelemetrySQLCTECnt + tiKVGCAutoConcurrency = "tikv_gc_auto_concurrency" ) @@ -609,7 +611,7 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { // Finally t1 will have more data than t2, with no errors return to user! if s.isTxnRetryableError(err) && !s.sessionVars.BatchInsert && commitRetryLimit > 0 && !isPessimistic { logutil.Logger(ctx).Warn("sql", - zap.String("label", s.getSQLLabel()), + zap.String("label", s.GetSQLLabel()), zap.Error(err), zap.String("txn", s.txn.GoString())) // Transactions will retry 2 ~ commitRetryLimit times. @@ -619,7 +621,7 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { err = s.retry(ctx, uint(maxRetryCount)) } else if !errIsNoisy(err) { logutil.Logger(ctx).Warn("can not retry txn", - zap.String("label", s.getSQLLabel()), + zap.String("label", s.GetSQLLabel()), zap.Error(err), zap.Bool("IsBatchInsert", s.sessionVars.BatchInsert), zap.Bool("IsPessimistic", isPessimistic), @@ -734,7 +736,7 @@ const sqlLogMaxLen = 1024 // SchemaChangedWithoutRetry is used for testing. var SchemaChangedWithoutRetry uint32 -func (s *session) getSQLLabel() string { +func (s *session) GetSQLLabel() string { if s.sessionVars.InRestrictedSQL { return metrics.LblInternal } @@ -794,7 +796,7 @@ func (s *session) retry(ctx context.Context, maxCnt uint) (err error) { var schemaVersion int64 sessVars := s.GetSessionVars() orgStartTS := sessVars.TxnCtx.StartTS - label := s.getSQLLabel() + label := s.GetSQLLabel() for { s.PrepareTxnCtx(ctx) s.sessionVars.RetryInfo.ResetOffset() @@ -1649,6 +1651,7 @@ func runStmt(ctx context.Context, se *session, s sqlexec.Statement) (rs sqlexec. return nil, err } rs, err = s.Exec(ctx) + se.updateTelemetryMetric(s.(*executor.ExecStmt)) sessVars.TxnCtx.StatementCount++ if rs != nil { return &execStmtResult{ @@ -1816,6 +1819,7 @@ func (s *session) cachedPlanExec(ctx context.Context, Ctx: s, OutputNames: execPlan.OutputNames(), PsStmt: prepareStmt, + Ti: &executor.TelemetryInfo{}, } compileDuration := time.Since(s.sessionVars.StartTime) sessionExecuteCompileDurationGeneral.Observe(compileDuration.Seconds()) @@ -2926,3 +2930,21 @@ func (s *session) GetInfoSchema() sessionctx.InfoschemaMetaVersion { } return domain.GetDomain(s).InfoSchema() } + +func (s *session) updateTelemetryMetric(es *executor.ExecStmt) { + if es.Ti == nil { + return + } + if s.isInternal() { + return + } + + ti := es.Ti + if ti.UseRecursive { + telemetryCTEUsage.WithLabelValues("recurCTE").Inc() + } else if ti.UseNonRecursive { + telemetryCTEUsage.WithLabelValues("nonRecurCTE").Inc() + } else { + telemetryCTEUsage.WithLabelValues("notCTE").Inc() + } +} diff --git a/telemetry/data.go b/telemetry/data.go index 2600738f7f516..fa1de90efe092 100644 --- a/telemetry/data.go +++ b/telemetry/data.go @@ -55,5 +55,6 @@ func generateTelemetryData(ctx sessionctx.Context, trackingID string) telemetryD func postReportTelemetryData() { postReportTxnUsage() + postReportCTEUsage() postReportSlowQueryStats() } diff --git a/telemetry/data_feature_usage.go b/telemetry/data_feature_usage.go index 6767192d14e76..508bd70e8c23c 100644 --- a/telemetry/data_feature_usage.go +++ b/telemetry/data_feature_usage.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/tidb/infoschema" + m "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/tikv/metrics" @@ -33,6 +34,7 @@ type featureUsage struct { // key is the first 6 characters of sha2(TABLE_NAME, 256) ClusterIndex *ClusterIndexUsage `json:"clusterIndex"` TemporaryTable bool `json:"temporaryTable"` + CTE *m.CTEUsageCounter `json:"cte"` } func getFeatureUsage(ctx sessionctx.Context) (*featureUsage, error) { @@ -49,7 +51,9 @@ func getFeatureUsage(ctx sessionctx.Context) (*featureUsage, error) { // Avoid the circle dependency. temporaryTable := ctx.(TemporaryTableFeatureChecker).TemporaryTableExists() - return &featureUsage{txnUsage, clusterIdxUsage, temporaryTable}, nil + cteUsage := GetCTEUsageInfo(ctx) + + return &featureUsage{txnUsage, clusterIdxUsage, temporaryTable, cteUsage}, nil } // ClusterIndexUsage records the usage info of all the tables, no more than 10k tables @@ -149,6 +153,7 @@ type TxnUsage struct { } var initialTxnCommitCounter metrics.TxnCommitCounter +var initialCTECounter m.CTEUsageCounter // GetTxnUsageInfo gets the usage info of transaction related features. It's exported for tests. func GetTxnUsageInfo(ctx sessionctx.Context) *TxnUsage { @@ -168,3 +173,15 @@ func GetTxnUsageInfo(ctx sessionctx.Context) *TxnUsage { func postReportTxnUsage() { initialTxnCommitCounter = metrics.GetTxnCommitCounter() } + +// ResetCTEUsage resets CTE usages. +func postReportCTEUsage() { + initialCTECounter = m.GetCTECounter() +} + +// GetCTEUsageInfo gets the CTE usages. +func GetCTEUsageInfo(ctx sessionctx.Context) *m.CTEUsageCounter { + curr := m.GetCTECounter() + diff := curr.Sub(initialCTECounter) + return &diff +} diff --git a/telemetry/telemetry_test.go b/telemetry/telemetry_test.go index fab7f82ade9fe..ecb6aa00768b6 100644 --- a/telemetry/telemetry_test.go +++ b/telemetry/telemetry_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/telemetry" + "github.com/pingcap/tidb/util/testkit" "go.etcd.io/etcd/integration" ) @@ -145,3 +146,38 @@ func (s *testSuite) Test03Report(c *C) { c.Assert(jsonParsed.Path("error_msg").Data().(string), Equals, "telemetry is disabled") c.Assert(jsonParsed.Path("is_request_sent").Data().(bool), Equals, false) } + +func (s *testSuite) TestCTEPreviewAndReport(c *C) { + config.GetGlobalConfig().EnableTelemetry = true + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("with cte as (select 1) select * from cte") + tk.MustExec("with recursive cte as (select 1) select * from cte") + tk.MustExec("with recursive cte(n) as (select 1 union select * from cte where n < 5) select * from cte") + tk.MustExec("select 1") + + res, err := telemetry.PreviewUsageData(s.se, s.etcdCluster.RandClient()) + c.Assert(err, IsNil) + + jsonParsed, err := gabs.ParseJSON([]byte(res)) + c.Assert(err, IsNil) + + c.Assert(int(jsonParsed.Path("featureUsage.cte.nonRecursiveCTEUsed").Data().(float64)), Equals, 2) + c.Assert(int(jsonParsed.Path("featureUsage.cte.recursiveUsed").Data().(float64)), Equals, 1) + // TODO: Fix this case. If run this test singly, the result is 2. But if run the whole test, the result is 4. + //c.Assert(int(jsonParsed.Path("featureUsage.cte.nonCTEUsed").Data().(float64)), Equals, 2) + + err = telemetry.ReportUsageData(s.se, s.etcdCluster.RandClient()) + c.Assert(err, IsNil) + + res, err = telemetry.PreviewUsageData(s.se, s.etcdCluster.RandClient()) + c.Assert(err, IsNil) + + jsonParsed, err = gabs.ParseJSON([]byte(res)) + c.Assert(err, IsNil) + + c.Assert(int(jsonParsed.Path("featureUsage.cte.nonRecursiveCTEUsed").Data().(float64)), Equals, 0) + c.Assert(int(jsonParsed.Path("featureUsage.cte.recursiveUsed").Data().(float64)), Equals, 0) + c.Assert(int(jsonParsed.Path("featureUsage.cte.nonCTEUsed").Data().(float64)), Equals, 0) +} From 6a1ebeb020d2c6399a44ce991c3fd586e2e3a089 Mon Sep 17 00:00:00 2001 From: Zhi Qi <30543181+LittleFall@users.noreply.github.com> Date: Thu, 3 Jun 2021 15:12:26 +0800 Subject: [PATCH 253/343] expression: Support push fucntion unix_timestamp, concat down to TiFlash. (#25083) --- expression/expr_to_pb_test.go | 27 +++++++++++++++++++++++++++ expression/expression.go | 6 ++++++ 2 files changed, 33 insertions(+) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index d997e9cf1691f..1a60a1266b8df 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -729,14 +729,41 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) + // ScalarFuncSig_RoundReal function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeDouble), realColumn) c.Assert(err, IsNil) exprs = append(exprs, function) + // ScalarFuncSig_RoundInt function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeLonglong), intColumn) c.Assert(err, IsNil) exprs = append(exprs, function) + // concat + function, err = NewFunction(mock.NewContext(), ast.Concat, types.NewFieldType(mysql.TypeString), stringColumn, intColumn, realColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // UnixTimestampCurrent + function, err = NewFunction(mock.NewContext(), ast.UnixTimestamp, types.NewFieldType(mysql.TypeLonglong)) + c.Assert(err, IsNil) + _, ok := function.(*Constant) + c.Assert(ok, IsTrue) + + // UnixTimestampInt + datetimeColumn.RetType.Decimal = 0 + function, err = NewFunction(mock.NewContext(), ast.UnixTimestamp, types.NewFieldType(mysql.TypeLonglong), datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_UnixTimestampInt) + exprs = append(exprs, function) + + // UnixTimestampDecimal + datetimeColumn.RetType.Decimal = types.UnspecifiedLength + function, err = NewFunction(mock.NewContext(), ast.UnixTimestamp, types.NewFieldType(mysql.TypeNewDecimal), datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_UnixTimestampDec) + exprs = append(exprs, function) + canPush := CanExprsPushDown(sc, exprs, client, kv.TiFlash) c.Assert(canPush, Equals, true) diff --git a/expression/expression.go b/expression/expression.go index 20178919a89b4..9a14537cac248 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1001,6 +1001,7 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { ast.GE, ast.LE, ast.EQ, ast.NE, ast.LT, ast.GT, ast.In, ast.IsNull, ast.Like, ast.Plus, ast.Minus, ast.Div, ast.Mul, /*ast.Mod,*/ ast.If, ast.Ifnull, ast.Case, + ast.Concat, ast.Month, ast.TimestampDiff, ast.DateFormat, ast.FromUnixTime, ast.JSONLength: @@ -1026,6 +1027,11 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { case tipb.ScalarFuncSig_AddDateDatetimeInt, tipb.ScalarFuncSig_AddDateStringInt: return true } + case ast.UnixTimestamp: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_UnixTimestampInt, tipb.ScalarFuncSig_UnixTimestampDec: + return true + } case ast.Round: switch function.Function.PbCode() { case tipb.ScalarFuncSig_RoundInt, tipb.ScalarFuncSig_RoundReal: From a37a30b883114f73a82b87cf4712666cd2180fa5 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Thu, 3 Jun 2021 15:27:29 +0800 Subject: [PATCH 254/343] planner: try to fix some unstable test cases about partition table statistics (#25026) --- executor/analyze_test.go | 84 ++++++++++++++++++-------------- executor/simple_test.go | 28 ++++++----- statistics/handle/handle_test.go | 34 ++++++++----- 3 files changed, 84 insertions(+), 62 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index f5b822b51f45f..45be004a2578a 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -46,6 +46,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" ) @@ -237,7 +238,9 @@ func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) { } func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { - c.Skip("unstable") + if israce.RaceEnabled { + c.Skip("unstable, skip race test") + } store, err := mockstore.NewMockStore() c.Assert(err, IsNil) defer func() { @@ -252,7 +255,8 @@ func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { defer dom.Close() tk := testkit.NewTestKit(c, store) - tk.MustExec("use test") + tk.MustExec("create database test_index_extract_topn") + tk.MustExec("use test_index_extract_topn") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, index idx(a, b))") tk.MustExec("insert into t values(1, 1), (1, 1), (1, 2), (1, 2)") @@ -260,7 +264,7 @@ func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { tk.MustExec("analyze table t with 10 cmsketch width") is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(model.NewCIStr("test_index_extract_topn"), model.NewCIStr("t")) c.Assert(err, IsNil) tableInfo := table.Meta() tbl := dom.StatsHandle().GetTableStats(tableInfo) @@ -511,18 +515,21 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { } func (s *testSerialSuite2) TestFastAnalyze4GlobalStats(c *C) { - c.Skip("unstable") + if israce.RaceEnabled { + c.Skip("unstable, skip race test") + } tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") + tk.MustExec(`create database if not exists test_fast_gstats`) + tk.MustExec("use test_fast_gstats") tk.MustExec("set @@session.tidb_enable_fast_analyze=1") tk.MustExec("set @@session.tidb_build_stats_concurrency=1") // test fast analyze in dynamic mode tk.MustExec("set @@session.tidb_analyze_version = 2;") tk.MustExec("set @@session.tidb_partition_prune_mode = 'dynamic';") - tk.MustExec("drop table if exists t4;") - tk.MustExec("create table t4(a int, b int) PARTITION BY HASH(a) PARTITIONS 2;") - tk.MustExec("insert into t4 values(1,1),(3,3),(4,4),(2,2),(5,5);") - err := tk.ExecToErr("analyze table t4;") + tk.MustExec("drop table if exists test_fast_gstats;") + tk.MustExec("create table test_fast_gstats(a int, b int) PARTITION BY HASH(a) PARTITIONS 2;") + tk.MustExec("insert into test_fast_gstats values(1,1),(3,3),(4,4),(2,2),(5,5);") + err := tk.ExecToErr("analyze table test_fast_gstats;") c.Assert(err.Error(), Equals, "Fast analyze hasn't reached General Availability and only support analyze version 1 currently.") } @@ -747,21 +754,24 @@ func (s *testSuite10) TestFailedAnalyzeRequest(c *C) { } func (s *testSuite1) TestExtractTopN(c *C) { - c.Skip("unstable") + if israce.RaceEnabled { + c.Skip("unstable, skip race test") + } tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int primary key, b int, index index_b(b))") + tk.MustExec("create database if not exists test_extract_topn") + tk.MustExec("use test_extract_topn") + tk.MustExec("drop table if exists test_extract_topn") + tk.MustExec("create table test_extract_topn(a int primary key, b int, index index_b(b))") tk.MustExec("set @@session.tidb_analyze_version=2") for i := 0; i < 10; i++ { - tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) + tk.MustExec(fmt.Sprintf("insert into test_extract_topn values (%d, %d)", i, i)) } for i := 0; i < 10; i++ { - tk.MustExec(fmt.Sprintf("insert into t values (%d, 0)", i+10)) + tk.MustExec(fmt.Sprintf("insert into test_extract_topn values (%d, 0)", i+10)) } - tk.MustExec("analyze table t") + tk.MustExec("analyze table test_extract_topn") is := s.dom.InfoSchema() - table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + table, err := is.TableByName(model.NewCIStr("test_extract_topn"), model.NewCIStr("test_extract_topn")) c.Assert(err, IsNil) tblInfo := table.Meta() tblStats := s.dom.StatsHandle().GetTableStats(tblInfo) @@ -774,26 +784,26 @@ func (s *testSuite1) TestExtractTopN(c *C) { idxItem := idxStats.TopN.TopN[0] c.Assert(idxItem.Count, Equals, uint64(11)) // The columns are: DBName, table name, column name, is index, value, count. - tk.MustQuery("show stats_topn").Sort().Check(testkit.Rows("test t b 0 0 11", - "test t b 0 1 1", - "test t b 0 2 1", - "test t b 0 3 1", - "test t b 0 4 1", - "test t b 0 5 1", - "test t b 0 6 1", - "test t b 0 7 1", - "test t b 0 8 1", - "test t b 0 9 1", - "test t index_b 1 0 11", - "test t index_b 1 1 1", - "test t index_b 1 2 1", - "test t index_b 1 3 1", - "test t index_b 1 4 1", - "test t index_b 1 5 1", - "test t index_b 1 6 1", - "test t index_b 1 7 1", - "test t index_b 1 8 1", - "test t index_b 1 9 1", + tk.MustQuery("show stats_topn").Sort().Check(testkit.Rows("test_extract_topn test_extract_topn b 0 0 11", + "test_extract_topn test_extract_topn b 0 1 1", + "test_extract_topn test_extract_topn b 0 2 1", + "test_extract_topn test_extract_topn b 0 3 1", + "test_extract_topn test_extract_topn b 0 4 1", + "test_extract_topn test_extract_topn b 0 5 1", + "test_extract_topn test_extract_topn b 0 6 1", + "test_extract_topn test_extract_topn b 0 7 1", + "test_extract_topn test_extract_topn b 0 8 1", + "test_extract_topn test_extract_topn b 0 9 1", + "test_extract_topn test_extract_topn index_b 1 0 11", + "test_extract_topn test_extract_topn index_b 1 1 1", + "test_extract_topn test_extract_topn index_b 1 2 1", + "test_extract_topn test_extract_topn index_b 1 3 1", + "test_extract_topn test_extract_topn index_b 1 4 1", + "test_extract_topn test_extract_topn index_b 1 5 1", + "test_extract_topn test_extract_topn index_b 1 6 1", + "test_extract_topn test_extract_topn index_b 1 7 1", + "test_extract_topn test_extract_topn index_b 1 8 1", + "test_extract_topn test_extract_topn index_b 1 9 1", )) } diff --git a/executor/simple_test.go b/executor/simple_test.go index 8c09fc336f5d2..dc464c38b83c0 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testutil" ) @@ -604,12 +605,15 @@ func (s *testFlushSuite) TestFlushPrivilegesPanic(c *C) { } func (s *testSerialSuite) TestDropPartitionStats(c *C) { - c.Skip("unstable") + if israce.RaceEnabled { + c.Skip("unstable, skip race test") + } // Use the testSerialSuite to fix the unstable test tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t;") - tk.MustExec(`create table t ( + tk.MustExec(`create database if not exists test_drop_gstats`) + tk.MustExec("use test_drop_gstats") + tk.MustExec("drop table if exists test_drop_gstats;") + tk.MustExec(`create table test_drop_gstats ( a int, key(a) ) @@ -620,7 +624,7 @@ partition by range (a) ( )`) tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") - tk.MustExec("insert into t values (1), (5), (11), (15), (21), (25)") + tk.MustExec("insert into test_drop_gstats values (1), (5), (11), (15), (21), (25)") c.Assert(s.domain.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) checkPartitionStats := func(names ...string) { @@ -631,26 +635,26 @@ partition by range (a) ( } } - tk.MustExec("analyze table t") + tk.MustExec("analyze table test_drop_gstats") checkPartitionStats("global", "p0", "p1", "global") - tk.MustExec("drop stats t partition p0") + tk.MustExec("drop stats test_drop_gstats partition p0") checkPartitionStats("global", "p1", "global") - err := tk.ExecToErr("drop stats t partition abcde") + err := tk.ExecToErr("drop stats test_drop_gstats partition abcde") c.Assert(err, NotNil) c.Assert(err.Error(), Equals, "can not found the specified partition name abcde in the table definition") - tk.MustExec("drop stats t partition global") + tk.MustExec("drop stats test_drop_gstats partition global") checkPartitionStats("global", "p1") - tk.MustExec("drop stats t global") + tk.MustExec("drop stats test_drop_gstats global") checkPartitionStats("p1") - tk.MustExec("analyze table t") + tk.MustExec("analyze table test_drop_gstats") checkPartitionStats("global", "p0", "p1", "global") - tk.MustExec("drop stats t partition p0, p1, global") + tk.MustExec("drop stats test_drop_gstats partition p0, p1, global") checkPartitionStats("global") } diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 58245841785fb..234e5f3cae0cc 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" ) @@ -837,13 +838,14 @@ func (s *testStatsSuite) TestBuildGlobalLevelStats(c *C) { c.Assert(len(result.Rows()), Equals, 20) } -func (s *testStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.TestKit) { - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec(` create table t (a int, key(a)) partition by range (a) ` + +func (s *testStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, tblName, dbName string) { + tk.MustExec("create database if not exists " + dbName) + tk.MustExec("use " + dbName) + tk.MustExec("drop table if exists " + tblName) + tk.MustExec(` create table ` + tblName + ` (a int, key(a)) partition by range (a) ` + `(partition p0 values less than (100000), partition p1 values less than (200000))`) - buf1 := bytes.NewBufferString("insert into t values (0)") - buf2 := bytes.NewBufferString("insert into t values (100000)") + buf1 := bytes.NewBufferString("insert into " + tblName + " values (0)") + buf2 := bytes.NewBufferString("insert into " + tblName + " values (100000)") for i := 0; i < 5000; i += 3 { buf1.WriteString(fmt.Sprintf(", (%v)", i)) buf2.WriteString(fmt.Sprintf(", (%v)", 100000+i)) @@ -872,9 +874,12 @@ func (s *testStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, } func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts(c *C) { + if israce.RaceEnabled { + c.Skip("unstable, skip race test") + } defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) - s.prepareForGlobalStatsWithOpts(c, tk) + s.prepareForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt") type opt struct { topn int @@ -893,7 +898,7 @@ func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts(c *C) { {77, 47000, true}, } for _, ca := range cases { - sql := fmt.Sprintf("analyze table t with %v topn, %v buckets", ca.topn, ca.buckets) + sql := fmt.Sprintf("analyze table test_gstats_opt with %v topn, %v buckets", ca.topn, ca.buckets) if !ca.err { tk.MustExec(sql) s.checkForGlobalStatsWithOpts(c, tk, "global", ca.topn, ca.buckets) @@ -907,27 +912,30 @@ func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts(c *C) { } func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts2(c *C) { + if israce.RaceEnabled { + c.Skip("unstable, skip race test") + } defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) - s.prepareForGlobalStatsWithOpts(c, tk) + s.prepareForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2") - tk.MustExec("analyze table t with 20 topn, 50 buckets, 1000 samples") + tk.MustExec("analyze table test_gstats_opt2 with 20 topn, 50 buckets, 1000 samples") s.checkForGlobalStatsWithOpts(c, tk, "global", 2, 50) s.checkForGlobalStatsWithOpts(c, tk, "p0", 1, 50) s.checkForGlobalStatsWithOpts(c, tk, "p1", 1, 50) // analyze a partition to let its options be different with others' - tk.MustExec("analyze table t partition p0 with 10 topn, 20 buckets") + tk.MustExec("analyze table test_gstats_opt2 partition p0 with 10 topn, 20 buckets") s.checkForGlobalStatsWithOpts(c, tk, "global", 10, 20) // use new options s.checkForGlobalStatsWithOpts(c, tk, "p0", 10, 20) s.checkForGlobalStatsWithOpts(c, tk, "p1", 1, 50) - tk.MustExec("analyze table t partition p1 with 100 topn, 200 buckets") + tk.MustExec("analyze table test_gstats_opt2 partition p1 with 100 topn, 200 buckets") s.checkForGlobalStatsWithOpts(c, tk, "global", 100, 200) s.checkForGlobalStatsWithOpts(c, tk, "p0", 10, 20) s.checkForGlobalStatsWithOpts(c, tk, "p1", 100, 200) - tk.MustExec("analyze table t partition p0 with 20 topn") // change back to 20 topn + tk.MustExec("analyze table test_gstats_opt2 partition p0 with 20 topn") // change back to 20 topn s.checkForGlobalStatsWithOpts(c, tk, "global", 20, 256) s.checkForGlobalStatsWithOpts(c, tk, "p0", 20, 256) s.checkForGlobalStatsWithOpts(c, tk, "p1", 100, 200) From dbb753fd028bfe30738c1104a7d0a78ee5f31d2d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=99=8E?= Date: Thu, 3 Jun 2021 15:38:27 +0800 Subject: [PATCH 255/343] executor: create PipelinedWindowExec (#23022) --- cmd/explaintest/r/select.result | 2 +- executor/aggfuncs/aggfuncs.go | 2 +- executor/aggfuncs/func_avg.go | 16 +- executor/aggfuncs/func_bitfuncs.go | 8 +- executor/aggfuncs/func_count.go | 56 ++-- executor/aggfuncs/func_max_min.go | 48 ++- executor/aggfuncs/func_sum.go | 16 +- executor/aggfuncs/row_number.go | 6 + executor/benchmark_test.go | 79 +++-- executor/builder.go | 36 +- executor/pipelined_window.go | 457 ++++++++++++++++++++++++++ executor/window.go | 16 +- executor/window_test.go | 40 +++ expression/aggregation/window_func.go | 16 + planner/core/logical_plan_builder.go | 64 +++- sessionctx/variable/session.go | 3 + sessionctx/variable/sysvar.go | 4 + sessionctx/variable/tidb_vars.go | 4 + 18 files changed, 784 insertions(+), 89 deletions(-) create mode 100644 executor/pipelined_window.go diff --git a/cmd/explaintest/r/select.result b/cmd/explaintest/r/select.result index 0836e2461a2a3..41369bffcfbbf 100644 --- a/cmd/explaintest/r/select.result +++ b/cmd/explaintest/r/select.result @@ -468,7 +468,7 @@ PRIMARY KEY (`id`) explain format = 'brief' select row_number() over( partition by i ) - x as rnk from t; id estRows task access object operator info Projection 10000.00 root minus(Column#5, test.t.x)->Column#7 -└─Window 10000.00 root row_number()->Column#5 over(partition by test.t.i) +└─Window 10000.00 root row_number()->Column#5 over(partition by test.t.i rows between current row and current row) └─Sort 10000.00 root test.t.i └─TableReader 10000.00 root data:TableRangeScan └─TableRangeScan 10000.00 cop[tikv] table:t range:[0,+inf], keep order:false, stats:pseudo diff --git a/executor/aggfuncs/aggfuncs.go b/executor/aggfuncs/aggfuncs.go index cacb418203cc2..6dd807c2ea5e1 100644 --- a/executor/aggfuncs/aggfuncs.go +++ b/executor/aggfuncs/aggfuncs.go @@ -201,7 +201,7 @@ type SlidingWindowAggFunc interface { // PartialResult stores the intermediate result which will be used in the next // sliding window, ensure call ResetPartialResult after a frame are evaluated // completely. - Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error + Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error } // MaxMinSlidingWindowAggFunc is the interface to evaluate the max/min agg function using sliding window diff --git a/executor/aggfuncs/func_avg.go b/executor/aggfuncs/func_avg.go index 216778d643c53..a62565ab43fa6 100644 --- a/executor/aggfuncs/func_avg.go +++ b/executor/aggfuncs/func_avg.go @@ -105,10 +105,12 @@ func (e *avgOriginal4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsI return 0, nil } -func (e *avgOriginal4Decimal) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &avgOriginal4Decimal{} + +func (e *avgOriginal4Decimal) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4AvgDecimal)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalDecimal(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalDecimal(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -124,7 +126,7 @@ func (e *avgOriginal4Decimal) Slide(sctx sessionctx.Context, rows []chunk.Row, l p.count++ } for i := uint64(0); i < shiftStart; i++ { - input, isNull, err := e.args[0].EvalDecimal(sctx, rows[lastStart+i]) + input, isNull, err := e.args[0].EvalDecimal(sctx, getRow(lastStart+i)) if err != nil { return err } @@ -325,10 +327,12 @@ type avgOriginal4Float64 struct { avgOriginal4Float64HighPrecision } -func (e *avgOriginal4Float64) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &avgOriginal4Float64{} + +func (e *avgOriginal4Float64) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4AvgFloat64)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalReal(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalReal(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -339,7 +343,7 @@ func (e *avgOriginal4Float64) Slide(sctx sessionctx.Context, rows []chunk.Row, l p.count++ } for i := uint64(0); i < shiftStart; i++ { - input, isNull, err := e.args[0].EvalReal(sctx, rows[lastStart+i]) + input, isNull, err := e.args[0].EvalReal(sctx, getRow(lastStart+i)) if err != nil { return err } diff --git a/executor/aggfuncs/func_bitfuncs.go b/executor/aggfuncs/func_bitfuncs.go index 214a1a9da0f1d..00aee7af2e364 100644 --- a/executor/aggfuncs/func_bitfuncs.go +++ b/executor/aggfuncs/func_bitfuncs.go @@ -91,10 +91,12 @@ func (e *bitXorUint64) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup return memDelta, nil } -func (e *bitXorUint64) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &bitXorUint64{} + +func (e *bitXorUint64) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4BitFunc)(pr) for i := uint64(0); i < shiftStart; i++ { - inputValue, isNull, err := e.args[0].EvalInt(sctx, rows[lastStart+i]) + inputValue, isNull, err := e.args[0].EvalInt(sctx, getRow(lastStart+i)) if err != nil { return err } @@ -104,7 +106,7 @@ func (e *bitXorUint64) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStar *p ^= uint64(inputValue) } for i := uint64(0); i < shiftEnd; i++ { - inputValue, isNull, err := e.args[0].EvalInt(sctx, rows[lastEnd+i]) + inputValue, isNull, err := e.args[0].EvalInt(sctx, getRow(lastEnd+i)) if err != nil { return err } diff --git a/executor/aggfuncs/func_count.go b/executor/aggfuncs/func_count.go index abc2bc17cde17..8b0c4f558744b 100644 --- a/executor/aggfuncs/func_count.go +++ b/executor/aggfuncs/func_count.go @@ -68,10 +68,12 @@ func (e *countOriginal4Int) UpdatePartialResult(sctx sessionctx.Context, rowsInG return 0, nil } -func (e *countOriginal4Int) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &countOriginal4Int{} + +func (e *countOriginal4Int) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4Count)(pr) for i := uint64(0); i < shiftStart; i++ { - _, isNull, err := e.args[0].EvalInt(sctx, rows[lastStart+i]) + _, isNull, err := e.args[0].EvalInt(sctx, getRow(lastStart+i)) if err != nil { return err } @@ -81,7 +83,7 @@ func (e *countOriginal4Int) Slide(sctx sessionctx.Context, rows []chunk.Row, las *p-- } for i := uint64(0); i < shiftEnd; i++ { - _, isNull, err := e.args[0].EvalInt(sctx, rows[lastEnd+i]) + _, isNull, err := e.args[0].EvalInt(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -115,10 +117,12 @@ func (e *countOriginal4Real) UpdatePartialResult(sctx sessionctx.Context, rowsIn return 0, nil } -func (e *countOriginal4Real) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &countOriginal4Real{} + +func (e *countOriginal4Real) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4Count)(pr) for i := uint64(0); i < shiftStart; i++ { - _, isNull, err := e.args[0].EvalReal(sctx, rows[lastStart+i]) + _, isNull, err := e.args[0].EvalReal(sctx, getRow(lastStart+i)) if err != nil { return err } @@ -128,7 +132,7 @@ func (e *countOriginal4Real) Slide(sctx sessionctx.Context, rows []chunk.Row, la *p-- } for i := uint64(0); i < shiftEnd; i++ { - _, isNull, err := e.args[0].EvalReal(sctx, rows[lastEnd+i]) + _, isNull, err := e.args[0].EvalReal(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -162,10 +166,12 @@ func (e *countOriginal4Decimal) UpdatePartialResult(sctx sessionctx.Context, row return 0, nil } -func (e *countOriginal4Decimal) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &countOriginal4Decimal{} + +func (e *countOriginal4Decimal) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4Count)(pr) for i := uint64(0); i < shiftStart; i++ { - _, isNull, err := e.args[0].EvalDecimal(sctx, rows[lastStart+i]) + _, isNull, err := e.args[0].EvalDecimal(sctx, getRow(lastStart+i)) if err != nil { return err } @@ -175,7 +181,7 @@ func (e *countOriginal4Decimal) Slide(sctx sessionctx.Context, rows []chunk.Row, *p-- } for i := uint64(0); i < shiftEnd; i++ { - _, isNull, err := e.args[0].EvalDecimal(sctx, rows[lastEnd+i]) + _, isNull, err := e.args[0].EvalDecimal(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -209,10 +215,12 @@ func (e *countOriginal4Time) UpdatePartialResult(sctx sessionctx.Context, rowsIn return 0, nil } -func (e *countOriginal4Time) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &countOriginal4Time{} + +func (e *countOriginal4Time) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4Count)(pr) for i := uint64(0); i < shiftStart; i++ { - _, isNull, err := e.args[0].EvalTime(sctx, rows[lastStart+i]) + _, isNull, err := e.args[0].EvalTime(sctx, getRow(lastStart+i)) if err != nil { return err } @@ -222,7 +230,7 @@ func (e *countOriginal4Time) Slide(sctx sessionctx.Context, rows []chunk.Row, la *p-- } for i := uint64(0); i < shiftEnd; i++ { - _, isNull, err := e.args[0].EvalTime(sctx, rows[lastEnd+i]) + _, isNull, err := e.args[0].EvalTime(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -256,10 +264,12 @@ func (e *countOriginal4Duration) UpdatePartialResult(sctx sessionctx.Context, ro return 0, nil } -func (e *countOriginal4Duration) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &countOriginal4Duration{} + +func (e *countOriginal4Duration) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4Count)(pr) for i := uint64(0); i < shiftStart; i++ { - _, isNull, err := e.args[0].EvalDuration(sctx, rows[lastStart+i]) + _, isNull, err := e.args[0].EvalDuration(sctx, getRow(lastStart+i)) if err != nil { return err } @@ -269,7 +279,7 @@ func (e *countOriginal4Duration) Slide(sctx sessionctx.Context, rows []chunk.Row *p-- } for i := uint64(0); i < shiftEnd; i++ { - _, isNull, err := e.args[0].EvalDuration(sctx, rows[lastEnd+i]) + _, isNull, err := e.args[0].EvalDuration(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -303,10 +313,12 @@ func (e *countOriginal4JSON) UpdatePartialResult(sctx sessionctx.Context, rowsIn return 0, nil } -func (e *countOriginal4JSON) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &countOriginal4JSON{} + +func (e *countOriginal4JSON) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4Count)(pr) for i := uint64(0); i < shiftStart; i++ { - _, isNull, err := e.args[0].EvalJSON(sctx, rows[lastStart+i]) + _, isNull, err := e.args[0].EvalJSON(sctx, getRow(lastStart+i)) if err != nil { return err } @@ -316,7 +328,7 @@ func (e *countOriginal4JSON) Slide(sctx sessionctx.Context, rows []chunk.Row, la *p-- } for i := uint64(0); i < shiftEnd; i++ { - _, isNull, err := e.args[0].EvalJSON(sctx, rows[lastEnd+i]) + _, isNull, err := e.args[0].EvalJSON(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -350,10 +362,12 @@ func (e *countOriginal4String) UpdatePartialResult(sctx sessionctx.Context, rows return 0, nil } -func (e *countOriginal4String) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &countOriginal4String{} + +func (e *countOriginal4String) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4Count)(pr) for i := uint64(0); i < shiftStart; i++ { - _, isNull, err := e.args[0].EvalString(sctx, rows[lastStart+i]) + _, isNull, err := e.args[0].EvalString(sctx, getRow(lastStart+i)) if err != nil { return err } @@ -363,7 +377,7 @@ func (e *countOriginal4String) Slide(sctx sessionctx.Context, rows []chunk.Row, *p-- } for i := uint64(0); i < shiftEnd; i++ { - _, isNull, err := e.args[0].EvalString(sctx, rows[lastEnd+i]) + _, isNull, err := e.args[0].EvalString(sctx, getRow(lastEnd+i)) if err != nil { return err } diff --git a/executor/aggfuncs/func_max_min.go b/executor/aggfuncs/func_max_min.go index 63a00ddd5bede..2c10b9251a463 100644 --- a/executor/aggfuncs/func_max_min.go +++ b/executor/aggfuncs/func_max_min.go @@ -343,10 +343,12 @@ func (e *maxMin4IntSliding) UpdatePartialResult(sctx sessionctx.Context, rowsInG return 0, nil } -func (e *maxMin4IntSliding) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &maxMin4IntSliding{} + +func (e *maxMin4IntSliding) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4MaxMinInt)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalInt(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalInt(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -480,10 +482,12 @@ func (e *maxMin4UintSliding) UpdatePartialResult(sctx sessionctx.Context, rowsIn return 0, nil } -func (e *maxMin4UintSliding) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &maxMin4UintSliding{} + +func (e *maxMin4UintSliding) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4MaxMinUint)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalInt(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalInt(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -617,10 +621,12 @@ func (e *maxMin4Float32Sliding) UpdatePartialResult(sctx sessionctx.Context, row return 0, nil } -func (e *maxMin4Float32Sliding) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &maxMin4Float32Sliding{} + +func (e *maxMin4Float32Sliding) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4MaxMinFloat32)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalReal(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalReal(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -752,10 +758,12 @@ func (e *maxMin4Float64Sliding) UpdatePartialResult(sctx sessionctx.Context, row return 0, nil } -func (e *maxMin4Float64Sliding) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &maxMin4Float64Sliding{} + +func (e *maxMin4Float64Sliding) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4MaxMinFloat64)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalReal(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalReal(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -904,10 +912,12 @@ func (e *maxMin4DecimalSliding) UpdatePartialResult(sctx sessionctx.Context, row return 0, nil } -func (e *maxMin4DecimalSliding) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &maxMin4DecimalSliding{} + +func (e *maxMin4DecimalSliding) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4MaxMinDecimal)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalDecimal(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalDecimal(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -1052,10 +1062,12 @@ func (e *maxMin4StringSliding) UpdatePartialResult(sctx sessionctx.Context, rows return 0, nil } -func (e *maxMin4StringSliding) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &maxMin4StringSliding{} + +func (e *maxMin4StringSliding) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4MaxMinString)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalString(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalString(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -1190,10 +1202,12 @@ func (e *maxMin4TimeSliding) UpdatePartialResult(sctx sessionctx.Context, rowsIn return 0, nil } -func (e *maxMin4TimeSliding) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &maxMin4DurationSliding{} + +func (e *maxMin4TimeSliding) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4MaxMinTime)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalTime(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalTime(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -1328,10 +1342,12 @@ func (e *maxMin4DurationSliding) UpdatePartialResult(sctx sessionctx.Context, ro return 0, nil } -func (e *maxMin4DurationSliding) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &maxMin4DurationSliding{} + +func (e *maxMin4DurationSliding) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4MaxMinDuration)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalDuration(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalDuration(sctx, getRow(lastEnd+i)) if err != nil { return err } diff --git a/executor/aggfuncs/func_sum.go b/executor/aggfuncs/func_sum.go index 127e743b000a9..77cc6745ddf65 100644 --- a/executor/aggfuncs/func_sum.go +++ b/executor/aggfuncs/func_sum.go @@ -115,10 +115,12 @@ type sum4Float64 struct { baseSum4Float64 } -func (e *sum4Float64) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &sum4Float64{} + +func (e *sum4Float64) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4SumFloat64)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalReal(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalReal(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -129,7 +131,7 @@ func (e *sum4Float64) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart p.notNullRowCount++ } for i := uint64(0); i < shiftStart; i++ { - input, isNull, err := e.args[0].EvalReal(sctx, rows[lastStart+i]) + input, isNull, err := e.args[0].EvalReal(sctx, getRow(lastStart+i)) if err != nil { return err } @@ -201,10 +203,12 @@ func (e *sum4Decimal) UpdatePartialResult(sctx sessionctx.Context, rowsInGroup [ return 0, nil } -func (e *sum4Decimal) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { +var _ SlidingWindowAggFunc = &sum4Decimal{} + +func (e *sum4Decimal) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { p := (*partialResult4SumDecimal)(pr) for i := uint64(0); i < shiftEnd; i++ { - input, isNull, err := e.args[0].EvalDecimal(sctx, rows[lastEnd+i]) + input, isNull, err := e.args[0].EvalDecimal(sctx, getRow(lastEnd+i)) if err != nil { return err } @@ -225,7 +229,7 @@ func (e *sum4Decimal) Slide(sctx sessionctx.Context, rows []chunk.Row, lastStart p.notNullRowCount++ } for i := uint64(0); i < shiftStart; i++ { - input, isNull, err := e.args[0].EvalDecimal(sctx, rows[lastStart+i]) + input, isNull, err := e.args[0].EvalDecimal(sctx, getRow(lastStart+i)) if err != nil { return err } diff --git a/executor/aggfuncs/row_number.go b/executor/aggfuncs/row_number.go index f1257d15f0411..b1f95a008ec9b 100644 --- a/executor/aggfuncs/row_number.go +++ b/executor/aggfuncs/row_number.go @@ -52,3 +52,9 @@ func (rn *rowNumber) AppendFinalResult2Chunk(sctx sessionctx.Context, pr Partial chk.AppendInt64(rn.ordinal, p.curIdx) return nil } + +var _ SlidingWindowAggFunc = &rowNumber{} + +func (rn *rowNumber) Slide(sctx sessionctx.Context, getRow func(uint64) chunk.Row, lastStart, lastEnd uint64, shiftStart, shiftEnd uint64, pr PartialResult) error { + return nil +} diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index df593a6e3d7cc..8e19d39ddb23d 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -588,6 +588,7 @@ type windowTestCase struct { ndv int // the number of distinct group-by keys rows int concurrency int + pipelined int dataSourceSorted bool ctx sessionctx.Context rawDataSmall string @@ -595,15 +596,15 @@ type windowTestCase struct { } func (a windowTestCase) String() string { - return fmt.Sprintf("(func:%v, aggColType:%s, numFunc:%v, ndv:%v, rows:%v, sorted:%v, concurrency:%v)", - a.windowFunc, a.columns[0].RetType, a.numFunc, a.ndv, a.rows, a.dataSourceSorted, a.concurrency) + return fmt.Sprintf("(func:%v, aggColType:%s, numFunc:%v, ndv:%v, rows:%v, sorted:%v, concurrency:%v, pipelined:%v)", + a.windowFunc, a.columns[0].RetType, a.numFunc, a.ndv, a.rows, a.dataSourceSorted, a.concurrency, a.pipelined) } func defaultWindowTestCase() *windowTestCase { ctx := mock.NewContext() ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize - return &windowTestCase{ast.WindowFuncRowNumber, 1, nil, 1000, 10000000, 1, true, ctx, strings.Repeat("x", 16), + return &windowTestCase{ast.WindowFuncRowNumber, 1, nil, 1000, 10000000, 1, 0, true, ctx, strings.Repeat("x", 16), []*expression.Column{ {Index: 0, RetType: types.NewFieldType(mysql.TypeDouble)}, {Index: 1, RetType: types.NewFieldType(mysql.TypeLonglong)}, @@ -617,6 +618,9 @@ func benchmarkWindowExecWithCase(b *testing.B, casTest *windowTestCase) { if err := ctx.GetSessionVars().SetSystemVar(variable.TiDBWindowConcurrency, fmt.Sprintf("%v", casTest.concurrency)); err != nil { b.Fatal(err) } + if err := ctx.GetSessionVars().SetSystemVar(variable.TiDBEnablePipelinedWindowFunction, fmt.Sprintf("%v", casTest.pipelined)); err != nil { + b.Fatal(err) + } cols := casTest.columns dataSource := buildMockDataSource(mockDataSourceParameters{ @@ -657,10 +661,10 @@ func benchmarkWindowExecWithCase(b *testing.B, casTest *windowTestCase) { } } -func BenchmarkWindowRows(b *testing.B) { +func baseBenchmarkWindowRows(b *testing.B, pipelined int) { b.ReportAllocs() rows := []int{1000, 100000} - ndvs := []int{10, 1000} + ndvs := []int{1, 10, 1000} concs := []int{1, 2, 4} for _, row := range rows { for _, ndv := range ndvs { @@ -671,6 +675,7 @@ func BenchmarkWindowRows(b *testing.B) { cas.concurrency = con cas.dataSourceSorted = false cas.windowFunc = ast.WindowFuncRowNumber // cheapest + cas.pipelined = pipelined b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { benchmarkWindowExecWithCase(b, cas) }) @@ -679,20 +684,25 @@ func BenchmarkWindowRows(b *testing.B) { } } -func BenchmarkWindowFunctions(b *testing.B) { +func BenchmarkWindowRows(b *testing.B) { + baseBenchmarkWindowRows(b, 0) + baseBenchmarkWindowRows(b, 1) +} + +func baseBenchmarkWindowFunctions(b *testing.B, pipelined int) { b.ReportAllocs() windowFuncs := []string{ - ast.WindowFuncRowNumber, - ast.WindowFuncRank, - ast.WindowFuncDenseRank, - ast.WindowFuncCumeDist, - ast.WindowFuncPercentRank, - ast.WindowFuncNtile, - ast.WindowFuncLead, + // ast.WindowFuncRowNumber, + // ast.WindowFuncRank, + // ast.WindowFuncDenseRank, + // ast.WindowFuncCumeDist, + // ast.WindowFuncPercentRank, + // ast.WindowFuncNtile, + // ast.WindowFuncLead, ast.WindowFuncLag, - ast.WindowFuncFirstValue, - ast.WindowFuncLastValue, - ast.WindowFuncNthValue, + // ast.WindowFuncFirstValue, + // ast.WindowFuncLastValue, + // ast.WindowFuncNthValue, } concs := []int{1, 4} for _, windowFunc := range windowFuncs { @@ -703,6 +713,7 @@ func BenchmarkWindowFunctions(b *testing.B) { cas.concurrency = con cas.dataSourceSorted = false cas.windowFunc = windowFunc + cas.pipelined = pipelined b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { benchmarkWindowExecWithCase(b, cas) }) @@ -710,7 +721,12 @@ func BenchmarkWindowFunctions(b *testing.B) { } } -func BenchmarkWindowFunctionsWithFrame(b *testing.B) { +func BenchmarkWindowFunctions(b *testing.B) { + baseBenchmarkWindowFunctions(b, 0) + baseBenchmarkWindowFunctions(b, 1) +} + +func baseBenchmarkWindowFunctionsWithFrame(b *testing.B, pipelined int) { b.ReportAllocs() windowFuncs := []string{ ast.WindowFuncRowNumber, @@ -736,6 +752,7 @@ func BenchmarkWindowFunctionsWithFrame(b *testing.B) { if i < len(frames) { cas.frame = frames[i] } + cas.pipelined = pipelined b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { benchmarkWindowExecWithCase(b, cas) }) @@ -743,9 +760,15 @@ func BenchmarkWindowFunctionsWithFrame(b *testing.B) { } } } + } -func BenchmarkWindowFunctionsAggWindowProcessorAboutFrame(b *testing.B) { +func BenchmarkWindowFunctionsWithFrame(b *testing.B) { + baseBenchmarkWindowFunctionsWithFrame(b, 0) + baseBenchmarkWindowFunctionsWithFrame(b, 1) +} + +func baseBenchmarkWindowFunctionsAggWindowProcessorAboutFrame(b *testing.B, pipelined int) { b.ReportAllocs() windowFunc := ast.AggFuncMax frame := &core.WindowFrame{Type: ast.Rows, Start: &core.FrameBound{UnBounded: true}, End: &core.FrameBound{UnBounded: true}} @@ -757,12 +780,18 @@ func BenchmarkWindowFunctionsAggWindowProcessorAboutFrame(b *testing.B) { cas.windowFunc = windowFunc cas.numFunc = 1 cas.frame = frame + cas.pipelined = pipelined b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { benchmarkWindowExecWithCase(b, cas) }) } -func baseBenchmarkWindowFunctionsWithSlidingWindow(b *testing.B, frameType ast.FrameType) { +func BenchmarkWindowFunctionsAggWindowProcessorAboutFrame(b *testing.B) { + baseBenchmarkWindowFunctionsAggWindowProcessorAboutFrame(b, 0) + baseBenchmarkWindowFunctionsAggWindowProcessorAboutFrame(b, 1) +} + +func baseBenchmarkWindowFunctionsWithSlidingWindow(b *testing.B, frameType ast.FrameType, pipelined int) { b.ReportAllocs() windowFuncs := []struct { aggFunc string @@ -794,6 +823,7 @@ func baseBenchmarkWindowFunctionsWithSlidingWindow(b *testing.B, frameType ast.F cas.windowFunc = windowFunc.aggFunc cas.frame = frame cas.columns[0].RetType.Tp = windowFunc.aggColTypes + cas.pipelined = pipelined b.Run(fmt.Sprintf("%v", cas), func(b *testing.B) { benchmarkWindowExecWithCase(b, cas) }) @@ -801,8 +831,10 @@ func baseBenchmarkWindowFunctionsWithSlidingWindow(b *testing.B, frameType ast.F } func BenchmarkWindowFunctionsWithSlidingWindow(b *testing.B) { - baseBenchmarkWindowFunctionsWithSlidingWindow(b, ast.Rows) - baseBenchmarkWindowFunctionsWithSlidingWindow(b, ast.Ranges) + baseBenchmarkWindowFunctionsWithSlidingWindow(b, ast.Rows, 0) + baseBenchmarkWindowFunctionsWithSlidingWindow(b, ast.Ranges, 0) + baseBenchmarkWindowFunctionsWithSlidingWindow(b, ast.Rows, 1) + baseBenchmarkWindowFunctionsWithSlidingWindow(b, ast.Ranges, 1) } type hashJoinTestCase struct { @@ -2061,3 +2093,8 @@ func BenchmarkAggPartialResultMapperMemoryUsage(b *testing.B) { }) } } + +func BenchmarkPipelinedRowNumberWindowFunctionExecution(b *testing.B) { + b.ReportAllocs() + +} diff --git a/executor/builder.go b/executor/builder.go index 0b9711ba95a34..6827e18da55ba 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3741,7 +3741,7 @@ func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, l return distsql.IndexRangesToKVRanges(ctx.GetSessionVars().StmtCtx, tableID, indexID, tmpDatumRanges, nil) } -func (b *executorBuilder) buildWindow(v *plannercore.PhysicalWindow) *WindowExec { +func (b *executorBuilder) buildWindow(v *plannercore.PhysicalWindow) Executor { childExec := b.build(v.Children()[0]) if b.err != nil { return nil @@ -3770,6 +3770,40 @@ func (b *executorBuilder) buildWindow(v *plannercore.PhysicalWindow) *WindowExec partialResults = append(partialResults, partialResult) resultColIdx++ } + + if b.ctx.GetSessionVars().EnablePipelinedWindowExec { + exec := &PipelinedWindowExec{ + baseExecutor: base, + groupChecker: newVecGroupChecker(b.ctx, groupByItems), + numWindowFuncs: len(v.WindowFuncDescs), + } + + exec.windowFuncs = windowFuncs + exec.partialResults = partialResults + if v.Frame == nil { + exec.start = &plannercore.FrameBound{ + Type: ast.Preceding, + UnBounded: true, + } + exec.end = &plannercore.FrameBound{ + Type: ast.Following, + UnBounded: true, + } + } else { + exec.start = v.Frame.Start + exec.end = v.Frame.End + if v.Frame.Type == ast.Ranges { + cmpResult := int64(-1) + if len(v.OrderBy) > 0 && v.OrderBy[0].Desc { + cmpResult = 1 + } + exec.orderByCols = orderByCols + exec.expectedCmpResult = cmpResult + exec.isRangeFrame = true + } + } + return exec + } var processor windowProcessor if v.Frame == nil { processor = &aggWindowProcessor{ diff --git a/executor/pipelined_window.go b/executor/pipelined_window.go new file mode 100644 index 0000000000000..3f016ff32860c --- /dev/null +++ b/executor/pipelined_window.go @@ -0,0 +1,457 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor + +import ( + "context" + + "github.com/cznic/mathutil" + "github.com/pingcap/errors" + "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/executor/aggfuncs" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/chunk" +) + +type dataInfo struct { + chk *chunk.Chunk + remaining uint64 + accumulated uint64 +} + +// PipelinedWindowExec is the executor for window functions. +type PipelinedWindowExec struct { + baseExecutor + numWindowFuncs int + windowFuncs []aggfuncs.AggFunc + slidingWindowFuncs []aggfuncs.SlidingWindowAggFunc + partialResults []aggfuncs.PartialResult + start *core.FrameBound + end *core.FrameBound + groupChecker *vecGroupChecker + + // childResult stores the child chunk. Note that even if remaining is 0, e.rows might still references rows in data[0].chk after returned it to upper executor, since there is no guarantee what the upper executor will do to the returned chunk, it might destroy the data (as in the benchmark test, it reused the chunk to pull data, and it will be chk.Reset(), causing panicking). So dataIdx, accumulated and dropped are added to ensure that chunk will only be returned if there is no row reference. + childResult *chunk.Chunk + data []dataInfo + dataIdx int + + // done indicates the child executor is drained or something unexpected happened. + done bool + accumulated uint64 + dropped uint64 + rowToConsume uint64 + newPartition bool + + curRowIdx uint64 + // curStartRow and curEndRow defines the current frame range + lastStartRow uint64 + lastEndRow uint64 + stagedStartRow uint64 + stagedEndRow uint64 + rowStart uint64 + orderByCols []*expression.Column + // expectedCmpResult is used to decide if one value is included in the frame. + expectedCmpResult int64 + + // rows keeps rows starting from curStartRow + rows []chunk.Row + rowCnt uint64 + whole bool + isRangeFrame bool + emptyFrame bool + initializedSlidingWindow bool +} + +// Close implements the Executor Close interface. +func (e *PipelinedWindowExec) Close() error { + return errors.Trace(e.baseExecutor.Close()) +} + +// Open implements the Executor Open interface +func (e *PipelinedWindowExec) Open(ctx context.Context) (err error) { + e.rowToConsume = 0 + e.done = false + e.accumulated = 0 + e.dropped = 0 + e.data = make([]dataInfo, 0) + e.dataIdx = 0 + e.slidingWindowFuncs = make([]aggfuncs.SlidingWindowAggFunc, len(e.windowFuncs)) + for i, windowFunc := range e.windowFuncs { + if slidingWindowAggFunc, ok := windowFunc.(aggfuncs.SlidingWindowAggFunc); ok { + e.slidingWindowFuncs[i] = slidingWindowAggFunc + } + } + e.rows = make([]chunk.Row, 0) + return e.baseExecutor.Open(ctx) +} + +func (e *PipelinedWindowExec) firstResultChunkNotReady() bool { + // chunk can't be ready unless, 1. all of the rows in the chunk is filled, 2. e.rows doesn't contain rows in the chunk + return len(e.data) > 0 && (e.data[0].remaining != 0 || e.data[0].accumulated > e.dropped) +} + +// Next implements the Executor Next interface. +func (e *PipelinedWindowExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { + chk.Reset() + + for !e.done || e.firstResultChunkNotReady() { + // we firstly gathering enough rows and consume them, until we are able to produce. + // for unbounded frame, it needs consume the whole partition before being able to produce, in this case + // e.p.enoughToProduce will be false until so. + var enough bool + enough, err = e.enoughToProduce(e.ctx) + if err != nil { + return + } + if !enough { + if !e.done && e.rowToConsume == 0 { + err = e.getRowsInPartition(ctx) + if err != nil { + return err + } + } + if e.done || e.newPartition { + e.finish() + // if we continued, the rows will not be consumed, so next time we should consume it instead of calling e.getRowsInPartition + enough, err = e.enoughToProduce(e.ctx) + if err != nil { + return + } + if enough { + continue + } + e.newPartition = false + e.reset() + if e.rowToConsume == 0 { + // no more data + break + } + } + e.rowCnt += e.rowToConsume + e.rowToConsume = 0 + } + + // e.p is ready to produce data + if len(e.data) > e.dataIdx && e.data[e.dataIdx].remaining != 0 { + produced, err := e.produce(e.ctx, e.data[e.dataIdx].chk, e.data[e.dataIdx].remaining) + if err != nil { + return err + } + e.data[e.dataIdx].remaining -= produced + if e.data[e.dataIdx].remaining == 0 { + e.dataIdx++ + } + } + } + if len(e.data) > 0 { + chk.SwapColumns(e.data[0].chk) + e.data = e.data[1:] + e.dataIdx-- + } + return nil +} + +func (e *PipelinedWindowExec) getRowsInPartition(ctx context.Context) (err error) { + e.newPartition = true + if len(e.rows) == 0 { + // if getRowsInPartition is called for the first time, we ignore it as a new partition + e.newPartition = false + } + + if e.groupChecker.isExhausted() { + var drained, samePartition bool + drained, err = e.fetchChild(ctx) + if err != nil { + err = errors.Trace(err) + } + // we return immediately to use a combination of true newPartition but 0 in e.rowToConsume to indicate the data source is drained, + if drained { + e.done = true + return nil + } + samePartition, err = e.groupChecker.splitIntoGroups(e.childResult) + if samePartition { + // the only case that when getRowsInPartition gets called, it is not a new partition. + e.newPartition = false + } + if err != nil { + return errors.Trace(err) + } + } + begin, end := e.groupChecker.getNextGroup() + e.rowToConsume += uint64(end - begin) + for i := begin; i < end; i++ { + e.rows = append(e.rows, e.childResult.GetRow(i)) + } + return +} + +func (e *PipelinedWindowExec) fetchChild(ctx context.Context) (EOF bool, err error) { + // TODO: reuse chunks + childResult := newFirstChunk(e.children[0]) + err = Next(ctx, e.children[0], childResult) + if err != nil { + return false, errors.Trace(err) + } + // No more data. + numRows := childResult.NumRows() + if numRows == 0 { + return true, nil + } + + // TODO: reuse chunks + resultChk := chunk.New(e.retFieldTypes, 0, numRows) + err = e.copyChk(childResult, resultChk) + if err != nil { + return false, err + } + e.accumulated += uint64(numRows) + e.data = append(e.data, dataInfo{chk: resultChk, remaining: uint64(numRows), accumulated: e.accumulated}) + + e.childResult = childResult + return false, nil +} + +func (e *PipelinedWindowExec) copyChk(src, dst *chunk.Chunk) error { + columns := e.Schema().Columns[:len(e.Schema().Columns)-e.numWindowFuncs] + for i, col := range columns { + if err := dst.MakeRefTo(i, src, col.Index); err != nil { + return err + } + } + return nil +} + +func (e *PipelinedWindowExec) getRow(i uint64) chunk.Row { + return e.rows[i-e.rowStart] +} + +func (e *PipelinedWindowExec) getRows(start, end uint64) []chunk.Row { + return e.rows[start-e.rowStart : end-e.rowStart] +} + +// finish is called upon a whole partition is consumed +func (e *PipelinedWindowExec) finish() { + e.whole = true +} + +func (e *PipelinedWindowExec) getStart(ctx sessionctx.Context) (uint64, error) { + if e.start.UnBounded { + return 0, nil + } + if e.isRangeFrame { + var start uint64 + for start = mathutil.MaxUint64(e.lastStartRow, e.stagedStartRow); start < e.rowCnt; start++ { + var res int64 + var err error + for i := range e.orderByCols { + res, _, err = e.start.CmpFuncs[i](ctx, e.orderByCols[i], e.start.CalcFuncs[i], e.getRow(start), e.getRow(e.curRowIdx)) + if err != nil { + return 0, err + } + if res != 0 { + break + } + } + // For asc, break when the calculated result is greater than the current value. + // For desc, break when the calculated result is less than the current value. + if res != e.expectedCmpResult { + break + } + } + e.stagedStartRow = start + return start, nil + } + switch e.start.Type { + case ast.Preceding: + if e.curRowIdx > e.start.Num { + return e.curRowIdx - e.start.Num, nil + } + return 0, nil + case ast.Following: + return e.curRowIdx + e.start.Num, nil + default: // ast.CurrentRow + return e.curRowIdx, nil + } +} + +func (e *PipelinedWindowExec) getEnd(ctx sessionctx.Context) (uint64, error) { + if e.end.UnBounded { + return e.rowCnt, nil + } + if e.isRangeFrame { + var end uint64 + for end = mathutil.MaxUint64(e.lastEndRow, e.stagedEndRow); end < e.rowCnt; end++ { + var res int64 + var err error + for i := range e.orderByCols { + res, _, err = e.end.CmpFuncs[i](ctx, e.end.CalcFuncs[i], e.orderByCols[i], e.getRow(e.curRowIdx), e.getRow(end)) + if err != nil { + return 0, err + } + if res != 0 { + break + } + } + // For asc, break when the calculated result is greater than the current value. + // For desc, break when the calculated result is less than the current value. + if res == e.expectedCmpResult { + break + } + } + e.stagedEndRow = end + return end, nil + } + switch e.end.Type { + case ast.Preceding: + if e.curRowIdx >= e.end.Num { + return e.curRowIdx - e.end.Num + 1, nil + } + return 0, nil + case ast.Following: + return e.curRowIdx + e.end.Num + 1, nil + default: // ast.CurrentRow: + return e.curRowIdx + 1, nil + } +} + +// produce produces rows and append it to chk, return produced means number of rows appended into chunk, available means +// number of rows processed but not fetched +func (e *PipelinedWindowExec) produce(ctx sessionctx.Context, chk *chunk.Chunk, remained uint64) (produced uint64, err error) { + var ( + start uint64 + end uint64 + enough bool + ) + for remained > 0 { + enough, err = e.enoughToProduce(ctx) + if err != nil { + return + } + if !enough { + break + } + start, err = e.getStart(ctx) + if err != nil { + return + } + end, err = e.getEnd(ctx) + if err != nil { + return + } + if end > e.rowCnt { + end = e.rowCnt + } + if start >= e.rowCnt { + start = e.rowCnt + } + // if start >= end, we should return a default value, and we reset the frame to empty. + if start >= end { + for i, wf := range e.windowFuncs { + if !e.emptyFrame { + wf.ResetPartialResult(e.partialResults[i]) + } + err = wf.AppendFinalResult2Chunk(ctx, e.partialResults[i], chk) + if err != nil { + return + } + } + if !e.emptyFrame { + e.emptyFrame = true + e.initializedSlidingWindow = false + } + } else { + e.emptyFrame = false + for i, wf := range e.windowFuncs { + slidingWindowAggFunc := e.slidingWindowFuncs[i] + if e.lastStartRow != start || e.lastEndRow != end { + if slidingWindowAggFunc != nil && e.initializedSlidingWindow { + err = slidingWindowAggFunc.Slide(ctx, e.getRow, e.lastStartRow, e.lastEndRow, start-e.lastStartRow, end-e.lastEndRow, e.partialResults[i]) + } else { + // For MinMaxSlidingWindowAggFuncs, it needs the absolute value of each start of window, to compare + // whether elements inside deque are out of current window. + if minMaxSlidingWindowAggFunc, ok := wf.(aggfuncs.MaxMinSlidingWindowAggFunc); ok { + // Store start inside MaxMinSlidingWindowAggFunc.windowInfo + minMaxSlidingWindowAggFunc.SetWindowStart(start) + } + // TODO(zhifeng): track memory usage here + wf.ResetPartialResult(e.partialResults[i]) + _, err = wf.UpdatePartialResult(ctx, e.getRows(start, end), e.partialResults[i]) + } + } + if err != nil { + return + } + err = wf.AppendFinalResult2Chunk(ctx, e.partialResults[i], chk) + if err != nil { + return + } + } + e.initializedSlidingWindow = true + } + e.curRowIdx++ + e.lastStartRow, e.lastEndRow = start, end + + produced++ + remained-- + } + extend := mathutil.MinUint64Val(e.curRowIdx, e.lastEndRow, e.lastStartRow) + if extend > e.rowStart { + numDrop := extend - e.rowStart + e.dropped += numDrop + e.rows = e.rows[numDrop:] + e.rowStart = extend + } + return +} + +func (e *PipelinedWindowExec) enoughToProduce(ctx sessionctx.Context) (enough bool, err error) { + if e.curRowIdx >= e.rowCnt { + return false, nil + } + if e.whole { + return true, nil + } + start, err := e.getStart(ctx) + if err != nil { + return + } + end, err := e.getEnd(ctx) + if err != nil { + return + } + return end < e.rowCnt && start < e.rowCnt, nil +} + +// reset resets the processor +func (e *PipelinedWindowExec) reset() { + e.lastStartRow = 0 + e.lastEndRow = 0 + e.stagedStartRow = 0 + e.stagedEndRow = 0 + e.emptyFrame = false + e.curRowIdx = 0 + e.whole = false + numDrop := e.rowCnt - e.rowStart + e.dropped += numDrop + e.rows = e.rows[numDrop:] + e.rowStart = 0 + e.rowCnt = 0 + e.initializedSlidingWindow = false + for i, windowFunc := range e.windowFuncs { + windowFunc.ResetPartialResult(e.partialResults[i]) + } +} diff --git a/executor/window.go b/executor/window.go index 4675902318b7a..19e92687fffda 100644 --- a/executor/window.go +++ b/executor/window.go @@ -319,7 +319,9 @@ func (p *rowFrameWindowProcessor) appendResult2Chunk(ctx sessionctx.Context, row for i, windowFunc := range p.windowFuncs { slidingWindowAggFunc := slidingWindowAggFuncs[i] if slidingWindowAggFunc != nil && initializedSlidingWindow { - err = slidingWindowAggFunc.Slide(ctx, rows, lastStart, lastEnd, shiftStart, shiftEnd, p.partialResults[i]) + err = slidingWindowAggFunc.Slide(ctx, func(u uint64) chunk.Row { + return rows[u] + }, lastStart, lastEnd, shiftStart, shiftEnd, p.partialResults[i]) if err != nil { return nil, err } @@ -335,7 +337,9 @@ func (p *rowFrameWindowProcessor) appendResult2Chunk(ctx sessionctx.Context, row for i, windowFunc := range p.windowFuncs { slidingWindowAggFunc := slidingWindowAggFuncs[i] if slidingWindowAggFunc != nil && initializedSlidingWindow { - err = slidingWindowAggFunc.Slide(ctx, rows, lastStart, lastEnd, shiftStart, shiftEnd, p.partialResults[i]) + err = slidingWindowAggFunc.Slide(ctx, func(u uint64) chunk.Row { + return rows[u] + }, lastStart, lastEnd, shiftStart, shiftEnd, p.partialResults[i]) } else { // For MinMaxSlidingWindowAggFuncs, it needs the absolute value of each start of window, to compare // whether elements inside deque are out of current window. @@ -469,7 +473,9 @@ func (p *rangeFrameWindowProcessor) appendResult2Chunk(ctx sessionctx.Context, r for i, windowFunc := range p.windowFuncs { slidingWindowAggFunc := slidingWindowAggFuncs[i] if slidingWindowAggFunc != nil && initializedSlidingWindow { - err = slidingWindowAggFunc.Slide(ctx, rows, lastStart, lastEnd, shiftStart, shiftEnd, p.partialResults[i]) + err = slidingWindowAggFunc.Slide(ctx, func(u uint64) chunk.Row { + return rows[u] + }, lastStart, lastEnd, shiftStart, shiftEnd, p.partialResults[i]) if err != nil { return nil, err } @@ -485,7 +491,9 @@ func (p *rangeFrameWindowProcessor) appendResult2Chunk(ctx sessionctx.Context, r for i, windowFunc := range p.windowFuncs { slidingWindowAggFunc := slidingWindowAggFuncs[i] if slidingWindowAggFunc != nil && initializedSlidingWindow { - err = slidingWindowAggFunc.Slide(ctx, rows, lastStart, lastEnd, shiftStart, shiftEnd, p.partialResults[i]) + err = slidingWindowAggFunc.Slide(ctx, func(u uint64) chunk.Row { + return rows[u] + }, lastStart, lastEnd, shiftStart, shiftEnd, p.partialResults[i]) } else { if minMaxSlidingWindowAggFunc, ok := windowFunc.(aggfuncs.MaxMinSlidingWindowAggFunc); ok { minMaxSlidingWindowAggFunc.SetWindowStart(start) diff --git a/executor/window_test.go b/executor/window_test.go index e530aa03f0549..970d7f7354100 100644 --- a/executor/window_test.go +++ b/executor/window_test.go @@ -23,10 +23,30 @@ import ( func (s *testSuite7) TestWindowFunctions(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@tidb_window_concurrency = 1") + tk.MustExec("set @@tidb_enable_pipelined_window_function = 0") + defer func() { + tk.MustExec("set @@tidb_enable_pipelined_window_function=1;") + }() doTestWindowFunctions(tk) } func (s *testSuite7) TestWindowParallelFunctions(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@tidb_window_concurrency = 4") + tk.MustExec("set @@tidb_enable_pipelined_window_function = 0") + defer func() { + tk.MustExec("set @@tidb_enable_pipelined_window_function=1;") + }() + doTestWindowFunctions(tk) +} + +func (s *testSuite7) TestPipelinedWindowFunctions(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set @@tidb_window_concurrency = 1") + doTestWindowFunctions(tk) +} + +func (s *testSuite7) TestPipelinedWindowParallelFunctions(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@tidb_window_concurrency = 4") doTestWindowFunctions(tk) @@ -222,6 +242,25 @@ func (s *testSuite7) TestWindowFunctionsDataReference(c *C) { } func (s *testSuite7) TestSlidingWindowFunctions(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("set @@tidb_enable_pipelined_window_function=0;") + defer func() { + tk.MustExec("set @@tidb_enable_pipelined_window_function=1;") + }() + idTypes := []string{"FLOAT", "DOUBLE"} + useHighPrecisions := []string{"ON", "OFF"} + for _, idType := range idTypes { + for _, useHighPrecision := range useHighPrecisions { + tk.MustExec("drop table if exists t;") + tk.MustExec(fmt.Sprintf("CREATE TABLE t (id %s, sex CHAR(1));", idType)) + tk.MustExec(fmt.Sprintf("SET SESSION windowing_use_high_precision = %s;", useHighPrecision)) + baseTestSlidingWindowFunctions(tk) + } + } +} + +func (s *testSuite7) TestPipelinedSlidingWindowFunctions(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") idTypes := []string{"FLOAT", "DOUBLE"} @@ -245,6 +284,7 @@ func baseTestSlidingWindowFunctions(tk *testkit.TestKit) { tk.MustExec("insert into t values (5,'M')") tk.MustExec("insert into t values (10,null)") tk.MustExec("insert into t values (11,null)") + tk.MustExec("PREPARE p FROM 'SELECT sex, COUNT(id) OVER (ORDER BY id ROWS BETWEEN ? PRECEDING and ? PRECEDING) FROM t';") tk.MustExec("SET @p1= 1;") tk.MustExec("SET @p2= 2;") diff --git a/expression/aggregation/window_func.go b/expression/aggregation/window_func.go index 8f963480dde16..64e412ad2149a 100644 --- a/expression/aggregation/window_func.go +++ b/expression/aggregation/window_func.go @@ -70,6 +70,22 @@ var noFrameWindowFuncs = map[string]struct{}{ ast.WindowFuncRowNumber: {}, } +var useDefaultFrameWindowFuncs = map[string]ast.FrameClause{ + ast.WindowFuncRowNumber: { + Type: ast.Rows, + Extent: ast.FrameExtent{ + Start: ast.FrameBound{Type: ast.CurrentRow}, + End: ast.FrameBound{Type: ast.CurrentRow}, + }, + }, +} + +// UseDefaultFrame indicates if the window function has a provided frame that will override user's designation +func UseDefaultFrame(name string) (bool, ast.FrameClause) { + frame, ok := useDefaultFrameWindowFuncs[strings.ToLower(name)] + return ok, frame +} + // NeedFrame checks if the function need frame specification. func NeedFrame(name string) bool { _, ok := noFrameWindowFuncs[strings.ToLower(name)] diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index ccb93029edd25..c92c11ac4ec2d 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -5469,13 +5469,27 @@ func (b *PlanBuilder) handleDefaultFrame(spec *ast.WindowSpec, windowFuncName st newSpec.Frame = nil return &newSpec, true } - // For functions that operate on the entire partition, the frame clause will be ignored. - if !needFrame && spec.Frame != nil { - specName := spec.Name.O - b.ctx.GetSessionVars().StmtCtx.AppendNote(ErrWindowFunctionIgnoresFrame.GenWithStackByArgs(windowFuncName, getWindowName(specName))) + if !needFrame { + var updated bool newSpec := *spec - newSpec.Frame = nil - return &newSpec, true + + // For functions that operate on the entire partition, the frame clause will be ignored. + if spec.Frame != nil { + specName := spec.Name.O + b.ctx.GetSessionVars().StmtCtx.AppendNote(ErrWindowFunctionIgnoresFrame.GenWithStackByArgs(windowFuncName, getWindowName(specName))) + newSpec.Frame = nil + updated = true + } + if b.ctx.GetSessionVars().EnablePipelinedWindowExec { + useDefaultFrame, defaultFrame := aggregation.UseDefaultFrame(windowFuncName) + if useDefaultFrame { + newSpec.Frame = &defaultFrame + updated = true + } + } + if updated { + return &newSpec, true + } } return spec, false } @@ -5490,11 +5504,27 @@ func appendIfAbsentWindowSpec(specs []*ast.WindowSpec, ns *ast.WindowSpec) []*as return append(specs, ns) } +func specEqual(s1, s2 *ast.WindowSpec) (equal bool, err error) { + if (s1 == nil && s2 != nil) || (s1 != nil && s2 == nil) { + return false, nil + } + var sb1, sb2 strings.Builder + ctx1 := format.NewRestoreCtx(0, &sb1) + ctx2 := format.NewRestoreCtx(0, &sb2) + if err = s1.Restore(ctx1); err != nil { + return + } + if err = s2.Restore(ctx2); err != nil { + return + } + return sb1.String() == sb2.String(), nil +} + // groupWindowFuncs groups the window functions according to the window specification name. // TODO: We can group the window function by the definition of window specification. func (b *PlanBuilder) groupWindowFuncs(windowFuncs []*ast.WindowFuncExpr) (map[*ast.WindowSpec][]*ast.WindowFuncExpr, []*ast.WindowSpec, error) { // updatedSpecMap is used to handle the specifications that have frame clause changed. - updatedSpecMap := make(map[string]*ast.WindowSpec) + updatedSpecMap := make(map[string][]*ast.WindowSpec) groupedWindow := make(map[*ast.WindowSpec][]*ast.WindowFuncExpr) orderedSpec := make([]*ast.WindowSpec, 0, len(windowFuncs)) for _, windowFunc := range windowFuncs { @@ -5526,10 +5556,26 @@ func (b *PlanBuilder) groupWindowFuncs(windowFuncs []*ast.WindowFuncExpr) (map[* groupedWindow[spec] = append(groupedWindow[spec], windowFunc) orderedSpec = appendIfAbsentWindowSpec(orderedSpec, spec) } else { + var updatedSpec *ast.WindowSpec if _, ok := updatedSpecMap[name]; !ok { - updatedSpecMap[name] = newSpec + updatedSpecMap[name] = []*ast.WindowSpec{newSpec} + updatedSpec = newSpec + } else { + for _, spec := range updatedSpecMap[name] { + eq, err := specEqual(spec, newSpec) + if err != nil { + return nil, nil, err + } + if eq { + updatedSpec = spec + break + } + } + if updatedSpec == nil { + updatedSpec = newSpec + updatedSpecMap[name] = append(updatedSpecMap[name], newSpec) + } } - updatedSpec := updatedSpecMap[name] groupedWindow[updatedSpec] = append(groupedWindow[updatedSpec], windowFunc) orderedSpec = appendIfAbsentWindowSpec(orderedSpec, updatedSpec) } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 6e6d766bd8bf5..90dd2b90b91f6 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -596,6 +596,9 @@ type SessionVars struct { // EnableWindowFunction enables the window function. EnableWindowFunction bool + // EnablePipelinedWindowExec enables executing window functions in a pipelined manner. + EnablePipelinedWindowExec bool + // EnableStrictDoubleTypeCheck enables table field double type check. EnableStrictDoubleTypeCheck bool diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index b399be0415832..447e09e141e84 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1202,6 +1202,10 @@ var defaultSysVars = []*SysVar{ s.EnableWindowFunction = TiDBOptOn(val) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePipelinedWindowFunction, Value: BoolToOnOff(DefEnablePipelinedWindowFunction), Hidden: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.EnablePipelinedWindowExec = TiDBOptOn(val) + return nil + }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableStrictDoubleTypeCheck, Value: BoolToOnOff(DefEnableStrictDoubleTypeCheck), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableStrictDoubleTypeCheck = TiDBOptOn(val) return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index b262bfd69ff19..64901a02bd8a9 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -403,6 +403,9 @@ const ( // tidb_enable_window_function is used to control whether to enable the window function. TiDBEnableWindowFunction = "tidb_enable_window_function" + // tidb_enable_pipelined_window_function is used to control whether to use pipelined window function, it only works when tidb_enable_window_function = true. + TiDBEnablePipelinedWindowFunction = "tidb_enable_pipelined_window_function" + // tidb_enable_strict_double_type_check is used to control table field double type syntax check. TiDBEnableStrictDoubleTypeCheck = "tidb_enable_strict_double_type_check" @@ -644,6 +647,7 @@ const ( DefTiDBStreamAggConcurrency = 1 DefTiDBForcePriority = mysql.NoPriority DefEnableWindowFunction = true + DefEnablePipelinedWindowFunction = true DefEnableStrictDoubleTypeCheck = true DefEnableVectorizedExpression = true DefTiDBOptJoinReorderThreshold = 0 From 30fcecad7fa9873a618c8612dfb9761759dbf669 Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Thu, 3 Jun 2021 15:50:27 +0800 Subject: [PATCH 256/343] planner: add dml support in CTE (#25024) --- expression/integration_test.go | 26 +++++++++++++++ planner/core/logical_plan_builder.go | 47 ++++++++++++++++++++++++++-- planner/core/preprocess.go | 3 ++ 3 files changed, 73 insertions(+), 3 deletions(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 061ef6ed81de2..a26600ec98e54 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -7362,6 +7362,32 @@ func (s *testIntegrationSuite) TestIssue15992(c *C) { tk.MustExec("drop table t0;") } +func (s *testIntegrationSuite) TestCTEWithDML(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(a int);") + tk.MustExec("insert into t1 values(2),(3);") + tk.MustQuery("with t1 as (select 36 as col from t1 where a=3) select * from t1;").Check(testkit.Rows("36")) + tk.MustExec("insert into t1 with t1 as (select 36 as col from t1) select * from t1;") + tk.MustQuery("select * from t1").Check(testkit.Rows("2", "3", "36", "36")) + tk.MustExec("with cte1(a) as (select 36) update t1 set a = 1 where a in (select a from cte1);") + tk.MustQuery("select * from t1").Check(testkit.Rows("2", "3", "1", "1")) + tk.MustExec("with recursive cte(a) as (select 1 union select a + 1 from cte where a < 10) update cte, t1 set t1.a=1") + tk.MustQuery("select * from t1").Check(testkit.Rows("1", "1", "1", "1")) + + tk.MustGetErrCode("with recursive cte(a) as (select 1 union select a + 1 from cte where a < 10) update cte set a=1", mysql.ErrNonUpdatableTable) + tk.MustGetErrCode("with recursive cte(a) as (select 1 union select a + 1 from cte where a < 10) delete from cte", mysql.ErrNonUpdatableTable) + tk.MustGetErrCode("with cte(a) as (select a from t1) delete from cte", mysql.ErrNonUpdatableTable) + tk.MustGetErrCode("with cte(a) as (select a from t1) update cte set a=1", mysql.ErrNonUpdatableTable) + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(a int, b int, primary key(a));") + tk.MustExec("insert into t1 values (1, 1),(2,1),(3,1);") + tk.MustExec("replace into t1 with recursive cte(a,b) as (select 1, 1 union select a + 1,b+1 from cte where a < 5) select * from cte;") + tk.MustQuery("select * from t1").Check(testkit.Rows("1 1", "2 2", "3 3", "4 4", "5 5")) +} + func (s *testIntegrationSuite) TestIssue16419(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("use test;") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index c92c11ac4ec2d..9a9a9351a92c1 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3260,6 +3260,9 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev } func (b *PlanBuilder) popVisitInfo() { + if len(b.visitInfo) == 0 { + return + } b.visitInfo = b.visitInfo[:len(b.visitInfo)-1] } @@ -4382,6 +4385,17 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( b.inUpdateStmt = true b.isForUpdateRead = true + if update.With != nil { + l := len(b.outerCTEs) + defer func() { + b.outerCTEs = b.outerCTEs[:l] + }() + err := b.buildWith(ctx, update.With) + if err != nil { + return nil, err + } + } + p, err := b.buildResultSetNode(ctx, update.TableRefs.TableRefs) if err != nil { return nil, err @@ -4520,6 +4534,12 @@ func CheckUpdateList(assignFlags []int, updt *Update) error { return nil } +// If tl is CTE, its TableInfo will be nil. +// Only used in build plan from AST after preprocess. +func isCTE(tl *ast.TableName) bool { + return tl.TableInfo == nil +} + func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.TableName, list []*ast.Assignment, p LogicalPlan, notUpdatableTbl []string) (newList []*expression.Assignment, po LogicalPlan, allAssignmentsAreConstant bool, e error) { b.curClause = fieldList @@ -4546,7 +4566,7 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab name := p.OutputNames()[idx] for _, tl := range tableList { if (tl.Schema.L == "" || tl.Schema.L == name.DBName.L) && (tl.Name.L == name.TblName.L) { - if tl.TableInfo.IsView() || tl.TableInfo.IsSequence() { + if isCTE(tl) || tl.TableInfo.IsView() || tl.TableInfo.IsSequence() { return nil, nil, false, ErrNonUpdatableTable.GenWithStackByArgs(name.TblName.O, "UPDATE") } // may be a subquery @@ -4581,7 +4601,7 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab break } } - if !updatable || tn.TableInfo.IsView() || tn.TableInfo.IsSequence() { + if !updatable || isCTE(tn) || tn.TableInfo.IsView() || tn.TableInfo.IsSequence() { continue } @@ -4615,6 +4635,9 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab newList = make([]*expression.Assignment, 0, p.Schema().Len()) tblDbMap := make(map[string]string, len(tableList)) for _, tbl := range tableList { + if isCTE(tbl) { + continue + } tblDbMap[tbl.Name.L] = tbl.DBInfo.Name.L } @@ -4721,6 +4744,17 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( b.inDeleteStmt = true b.isForUpdateRead = true + if delete.With != nil { + l := len(b.outerCTEs) + defer func() { + b.outerCTEs = b.outerCTEs[:l] + }() + err := b.buildWith(ctx, delete.With) + if err != nil { + return nil, err + } + } + p, err := b.buildResultSetNode(ctx, delete.TableRefs.TableRefs) if err != nil { return nil, err @@ -4847,6 +4881,9 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( var tableList []*ast.TableName tableList = extractTableList(delete.TableRefs.TableRefs, tableList, false) for _, v := range tableList { + if isCTE(v) { + return nil, ErrNonUpdatableTable.GenWithStackByArgs(v.Name.O, "DELETE") + } if v.TableInfo.IsView() { return nil, errors.Errorf("delete view %s is not supported now.", v.Name.O) } @@ -5702,10 +5739,14 @@ func collectTableName(node ast.ResultSetNode, updatableName *map[string]bool, in if s, canUpdate = x.Source.(*ast.TableName); canUpdate { if name == "" { name = s.Schema.L + "." + s.Name.L + // it may be a CTE + if s.Schema.L == "" { + name = s.Name.L + } } (*info)[name] = s } - (*updatableName)[name] = canUpdate + (*updatableName)[name] = canUpdate && s.Schema.L != "" } } diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 07d7a485ab04e..5e6f41b7b4fb8 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -165,6 +165,9 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { p.stmtTp = TypeUpdate case *ast.InsertStmt: p.stmtTp = TypeInsert + // handle the insert table name imminently + // insert into t with t ..., the insert can not see t here. We should hand it before the CTE statement + p.handleTableName(node.Table.TableRefs.Left.(*ast.TableSource).Source.(*ast.TableName)) case *ast.CreateTableStmt: p.stmtTp = TypeCreate p.flag |= inCreateOrDropTable From 83a9a12a1436fd1250271bc98fc7eeb86b17d49e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Thu, 3 Jun 2021 16:10:45 +0800 Subject: [PATCH 257/343] *: add sys var tidb_enable_global_temporary_table to open global temp table (#25075) --- ddl/db_integration_test.go | 1 + ddl/db_partition_test.go | 1 + ddl/db_test.go | 2 ++ ddl/ddl_api.go | 3 +++ ddl/serial_test.go | 2 ++ executor/admin_test.go | 1 + executor/batch_point_get_test.go | 1 + executor/executor_test.go | 4 +++ executor/insert_test.go | 3 +++ executor/show_test.go | 1 + executor/write_test.go | 1 + session/session_test.go | 32 ++++++++++++++++++++++++ sessionctx/binloginfo/binloginfo_test.go | 1 + sessionctx/variable/session.go | 4 +++ sessionctx/variable/sysvar.go | 4 +++ sessionctx/variable/tidb_vars.go | 4 +++ telemetry/data_feature_usage_test.go | 1 + 17 files changed, 66 insertions(+) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 9fe96a50afa02..c65fb27de399e 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2776,6 +2776,7 @@ func (s *testIntegrationSuite3) TestCreateTemporaryTable(c *C) { tk.MustExec("drop table if exists t;") // Grammar error. + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustGetErrCode("create global temporary table t(a double(0, 0))", errno.ErrParse) tk.MustGetErrCode("create temporary table t(id int) on commit delete rows", errno.ErrParse) tk.MustGetErrCode("create temporary table t(id int) on commit preserve rows", errno.ErrParse) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index b261ca3f98519..04e69ee89dd82 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -3393,6 +3393,7 @@ func (s *testSerialDBSuite1) TestPartitionListWithNewCollation(c *C) { func (s *testSerialDBSuite1) TestAddTableWithPartition(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("use test;") tk.MustExec("drop table if exists global_partition_table;") tk.MustGetErrCode("create global temporary table global_partition_table (a int, b int) partition by hash(a) partitions 3 ON COMMIT DELETE ROWS;", errno.ErrPartitionNoTemporary) diff --git a/ddl/db_test.go b/ddl/db_test.go index 0bfc654941727..1c0e1f4cb48de 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -3037,6 +3037,7 @@ func (s *testDBSuite2) TestTemporaryTableForeignKey(c *C) { tk.MustExec("drop table if exists t1;") tk.MustExec("create table t1 (a int, b int);") tk.MustExec("drop table if exists t1_tmp;") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table t1_tmp (a int, b int) on commit delete rows;") // test add foreign key. tk.MustExec("drop table if exists t2;") @@ -5383,6 +5384,7 @@ func (s *testSerialDBSuite) TestShardRowIDBitsOnTemporaryTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists shard_row_id_temporary") + tk.MustExec("set tidb_enable_global_temporary_table=true") _, err := tk.Exec("create global temporary table shard_row_id_temporary (a int) shard_row_id_bits = 5 on commit delete rows;") c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("shard_row_id_bits").Error()) tk.MustExec("create global temporary table shard_row_id_temporary (a int) on commit delete rows;") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 640823b23305a..8583fa6414386 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1736,6 +1736,9 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh switch s.TemporaryKeyword { case ast.TemporaryGlobal: tbInfo.TempTableType = model.TempTableGlobal + if !ctx.GetSessionVars().EnableGlobalTemporaryTable { + return nil, errors.New("global temporary table is experimental and it is switched off by tidb_enable_global_temporary_table") + } // "create global temporary table ... on commit preserve rows" if !s.OnCommitDelete { return nil, errors.Trace(errUnsupportedOnCommitPreserve) diff --git a/ddl/serial_test.go b/ddl/serial_test.go index 30c81bf2f2e7b..e6e3164381566 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -527,6 +527,7 @@ func (s *testSerialSuite) TestCreateTableWithLike(c *C) { tk.MustExec("drop database ctwl_db1") // Test create table like at temporary mode. + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("use test") tk.MustExec("drop table if exists temporary_table;") tk.MustExec("create global temporary table temporary_table (a int, b int,index(a)) on commit delete rows") @@ -939,6 +940,7 @@ func (s *testSerialDBSuite) TestAutoRandomOnTemporaryTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists auto_random_temporary") + tk.MustExec("set tidb_enable_global_temporary_table=true") _, err := tk.Exec("create global temporary table auto_random_temporary (a bigint primary key auto_random(3), b varchar(255)) on commit delete rows;") c.Assert(err.Error(), Equals, core.ErrOptOnTemporaryTable.GenWithStackByArgs("auto_random").Error()) tk.MustExec("set @@tidb_enable_noop_functions = 1") diff --git a/executor/admin_test.go b/executor/admin_test.go index 2e1d24bd32bc6..f23680e789e56 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -81,6 +81,7 @@ func (s *testSuite5) TestAdminCheckIndexInTemporaryMode(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists temporary_admin_test;") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table temporary_admin_test (c1 int, c2 int, c3 int default 1, primary key (c1), index (c1), unique key(c2)) ON COMMIT DELETE ROWS;") tk.MustExec("insert temporary_admin_test (c1, c2) values (1, 1), (2, 2), (3, 3);") _, err := tk.Exec("admin check table temporary_admin_test;") diff --git a/executor/batch_point_get_test.go b/executor/batch_point_get_test.go index 69a60535ce1ab..5db23ed39b0b8 100644 --- a/executor/batch_point_get_test.go +++ b/executor/batch_point_get_test.go @@ -326,6 +326,7 @@ func (s *testBatchPointGetSuite) TestPointGetForTemporaryTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table t1 (id int primary key, val int) on commit delete rows") tk.MustExec("begin") tk.MustExec("insert into t1 values (1,1)") diff --git a/executor/executor_test.go b/executor/executor_test.go index 1f5cc0b6b7b3d..450e83f152e39 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -5203,6 +5203,7 @@ func (s *testSplitTable) TestShowTableRegion(c *C) { // Test show table regions and split table on temporary table. tk.MustExec("drop table if exists t_regions_temporary_table") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table t_regions_temporary_table (a int key, b int, c int, index idx(b), index idx2(c)) ON COMMIT DELETE ROWS;") // Test show table regions. _, err = tk.Exec("show table t_regions_temporary_table regions") @@ -5969,6 +5970,7 @@ func (s *testRecoverTable) TestRecoverTempTable(c *C) { tk.MustExec("create database if not exists test_recover") tk.MustExec("use test_recover") tk.MustExec("drop table if exists t_recover") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table t_recover (a int) on commit delete rows;") timeBeforeDrop, _, safePointSQL, resetGC := s.mockGC(tk) @@ -8245,6 +8247,7 @@ func (s *testSerialSuite) TestDeadlockTable(c *C) { func (s *testSuite1) TestTemporaryTableNoPessimisticLock(c *C) { tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("use test") tk.MustExec("drop table if exists t;") tk.MustExec("create global temporary table t (a int primary key, b int) on commit delete rows") @@ -8365,6 +8368,7 @@ func (s testSerialSuite) TestTemporaryTableNoNetwork(c *C) { tk.MustExec("use test") tk1.MustExec("use test") tk.MustExec("create table normal (id int, a int, index(a))") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table tmp_t (id int, a int, index(a)) on commit delete rows") tk.MustExec("begin") diff --git a/executor/insert_test.go b/executor/insert_test.go index 351b337bd84ae..1e7a5bb36ef87 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -1601,6 +1601,7 @@ func (s *testSuite13) TestGlobalTempTableAutoInc(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) tk.MustExec("drop table if exists temp_test") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table temp_test(id int primary key auto_increment) on commit delete rows") defer tk.MustExec("drop table if exists temp_test") @@ -1646,6 +1647,7 @@ func (s *testSuite13) TestGlobalTempTableRowID(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) tk.MustExec("drop table if exists temp_test") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table temp_test(id int) on commit delete rows") defer tk.MustExec("drop table if exists temp_test") @@ -1681,6 +1683,7 @@ func (s *testSuite13) TestGlobalTempTableParallel(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) tk.MustExec("drop table if exists temp_test") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table temp_test(id int primary key auto_increment) on commit delete rows") defer tk.MustExec("drop table if exists temp_test") diff --git a/executor/show_test.go b/executor/show_test.go index 38f513642df99..778f9452b5347 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -1319,6 +1319,7 @@ func (s *testSuite5) TestShowPerformanceSchema(c *C) { func (s *testSuite5) TestShowTemporaryTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table t1 (id int) on commit delete rows") tk.MustExec("create global temporary table t3 (i int primary key, j int) on commit delete rows") // For issue https://github.com/pingcap/tidb/issues/24752 diff --git a/executor/write_test.go b/executor/write_test.go index e9111154cfc87..7d39df1b71939 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -2862,6 +2862,7 @@ func (s *testSuite7) TestDeferConstraintCheckForInsert(c *C) { c.Assert(err, NotNil) // Cover the temporary table. + tk.MustExec("set tidb_enable_global_temporary_table=true") for val := range []int{0, 1} { tk.MustExec("set tidb_constraint_check_in_place = ?", val) diff --git a/session/session_test.go b/session/session_test.go index f68f75f133da9..0e9364018c574 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -789,6 +789,7 @@ func (s *testSessionSuite) TestRetryUnion(c *C) { func (s *testSessionSuite) TestRetryGlobalTempTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("drop table if exists normal_table") tk.MustExec("create table normal_table(a int primary key, b int)") defer tk.MustExec("drop table if exists normal_table") @@ -2139,6 +2140,7 @@ func (s *testSchemaSerialSuite) TestSchemaCheckerTempTable(c *C) { tk.MustExec(`drop table if exists normal_table`) tk.MustExec(`create table normal_table (id int, c int);`) defer tk.MustExec(`drop table if exists normal_table`) + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec(`drop table if exists temp_table`) tk.MustExec(`create global temporary table temp_table (id int, c int) on commit delete rows;`) defer tk.MustExec(`drop table if exists temp_table`) @@ -4158,6 +4160,7 @@ func (s *testSessionSerialSuite) TestParseWithParams(c *C) { func (s *testSessionSuite3) TestGlobalTemporaryTable(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("create global temporary table g_tmp (a int primary key, b int, c int, index i_b(b)) on commit delete rows") tk.MustExec("begin") tk.MustExec("insert into g_tmp values (3, 3, 3)") @@ -4528,3 +4531,32 @@ func (s *testSessionSuite) TestInTxnPSProtoPointGet(c *C) { c.Assert(txn.Valid(), IsTrue) tk.MustExec("commit") } + +func (s *testSessionSuite) TestTiDBEnableGlobalTemporaryTable(c *C) { + // Test the @@tidb_enable_global_temporary_table system variable. + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + // variable 'tidb_enable_global_temporary_table' should not be seen when show variables + tk.MustQuery("show variables like 'tidb_enable_global_temporary_table'").Check(testkit.Rows()) + tk.MustQuery("show global variables like 'tidb_enable_global_temporary_table'").Check(testkit.Rows()) + + // variable 'tidb_enable_global_temporary_table' is turned off by default + tk.MustQuery("select @@global.tidb_enable_global_temporary_table").Check(testkit.Rows("0")) + tk.MustQuery("select @@tidb_enable_global_temporary_table").Check(testkit.Rows("0")) + c.Assert(tk.Se.GetSessionVars().EnableGlobalTemporaryTable, IsFalse) + + // cannot create global temporary table when 'tidb_enable_global_temporary_table' is off + tk.MustGetErrMsg( + "create global temporary table temp_test(id int primary key auto_increment) on commit delete rows", + "global temporary table is experimental and it is switched off by tidb_enable_global_temporary_table", + ) + tk.MustQuery("show tables like 'temp_test'").Check(testkit.Rows()) + + // you can create global temporary table when 'tidb_enable_global_temporary_table' is on + tk.MustExec("set tidb_enable_global_temporary_table=on") + tk.MustQuery("select @@tidb_enable_global_temporary_table").Check(testkit.Rows("1")) + c.Assert(tk.Se.GetSessionVars().EnableGlobalTemporaryTable, IsTrue) + tk.MustExec("create global temporary table temp_test(id int primary key auto_increment) on commit delete rows") + tk.MustQuery("show tables like 'temp_test'").Check(testkit.Rows("temp_test")) +} diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index b0d9ec91888e4..6a586615bd8a0 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -703,6 +703,7 @@ func (s *testBinlogSuite) TestTempTableBinlog(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.Se.GetSessionVars().BinlogClient = s.client tk.MustExec("begin") + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("drop table if exists temp_table") ddlQuery := "create global temporary table temp_table(id int) on commit delete rows" tk.MustExec(ddlQuery) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 90dd2b90b91f6..247f2b12c2884 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -829,6 +829,9 @@ type SessionVars struct { // CTEMaxRecursionDepth indicates The common table expression (CTE) maximum recursion depth. // see https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_cte_max_recursion_depth CTEMaxRecursionDepth int + + // EnableGlobalTemporaryTable indicates whether to enable global temporary table + EnableGlobalTemporaryTable bool } // AllocMPPTaskID allocates task id for mpp tasks. It will reset the task id if the query's @@ -1031,6 +1034,7 @@ func NewSessionVars() *SessionVars { EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, AllowFallbackToTiKV: make(map[kv.StoreType]struct{}), CTEMaxRecursionDepth: DefCTEMaxRecursionDepth, + EnableGlobalTemporaryTable: DefTiDBEnableGlobalTemporaryTable, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 447e09e141e84..cab7e1b2e7ef2 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1722,6 +1722,10 @@ var defaultSysVars = []*SysVar{ TopSQLVariable.MaxStatementCount.Store(val) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableGlobalTemporaryTable, Value: BoolToOnOff(DefTiDBEnableGlobalTemporaryTable), Hidden: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.EnableGlobalTemporaryTable = TiDBOptOn(val) + return nil + }}, } // FeedbackProbability points to the FeedbackProbability in statistics package. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 64901a02bd8a9..1c1145c2ad36a 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -546,6 +546,9 @@ const ( // TiDBTopSQLMaxStatementCount indicates the max number of statements been collected. TiDBTopSQLMaxStatementCount = "tidb_top_sql_max_statement_count" + + // TiDBEnableGlobalTemporaryTable indicates whether to enable global temporary table + TiDBEnableGlobalTemporaryTable = "tidb_enable_global_temporary_table" ) // TiDB vars that have only global scope @@ -693,6 +696,7 @@ const ( DefTiDBTopSQLAgentAddress = "" DefTiDBTopSQLPrecisionSeconds = 1 DefTiDBTopSQLMaxStatementCount = 200 + DefTiDBEnableGlobalTemporaryTable = false ) // Process global variables. diff --git a/telemetry/data_feature_usage_test.go b/telemetry/data_feature_usage_test.go index 790a24779f234..2d69d093e5360 100644 --- a/telemetry/data_feature_usage_test.go +++ b/telemetry/data_feature_usage_test.go @@ -93,6 +93,7 @@ func (s *testFeatureInfoSuite) TestTxnUsageInfo(c *C) { func (s *testFeatureInfoSuite) TestTemporaryTable(c *C) { tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set tidb_enable_global_temporary_table=true") tk.MustExec("use test") usage, err := telemetry.GetFeatureUsage(tk.Se) c.Assert(err, IsNil) From 50b6da0f894dd982ddddff5494c1c216a2eeafdb Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 3 Jun 2021 16:26:27 +0800 Subject: [PATCH 258/343] executor: add limit implementation for CTEExec (#24870) --- cmd/explaintest/r/explain_cte.result | 87 +++++++++++++++-- cmd/explaintest/t/explain_cte.test | 13 +++ executor/builder.go | 3 + executor/cte.go | 76 ++++++++++++++- executor/cte_test.go | 137 +++++++++++++++++++++++++++ planner/core/logical_plan_builder.go | 35 ++++++- planner/core/logical_plans.go | 5 +- planner/core/physical_plans.go | 10 +- planner/core/planbuilder.go | 1 + util/cteutil/storage.go | 8 ++ 10 files changed, 362 insertions(+), 13 deletions(-) diff --git a/cmd/explaintest/r/explain_cte.result b/cmd/explaintest/r/explain_cte.result index 4092fe73fbd07..36eda33222fd6 100644 --- a/cmd/explaintest/r/explain_cte.result +++ b/cmd/explaintest/r/explain_cte.result @@ -7,19 +7,19 @@ insert into t2 values(1, 0), (2, 1); explain with cte(a) as (select 1) select * from cte; id estRows task access object operator info CTEFullScan_8 1.00 root CTE:cte data:CTE_0 -CTE_0 1.00 root None Recursive CTE +CTE_0 1.00 root Non-Recursive CTE └─Projection_6(Seed Part) 1.00 root 1->Column#1 └─TableDual_7 1.00 root rows:1 explain with cte(a) as (select c1 from t1) select * from cte; id estRows task access object operator info CTEFullScan_11 1.00 root CTE:cte data:CTE_0 -CTE_0 1.00 root None Recursive CTE +CTE_0 1.00 root Non-Recursive CTE └─TableReader_8(Seed Part) 10000.00 root data:TableFullScan_7 └─TableFullScan_7 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain with cte(a,b,c,d) as (select * from t1, t2) select * from cte; id estRows task access object operator info CTEFullScan_18 1.00 root CTE:cte data:CTE_0 -CTE_0 1.00 root None Recursive CTE +CTE_0 1.00 root Non-Recursive CTE └─HashJoin_10(Seed Part) 100000000.00 root CARTESIAN inner join ├─TableReader_17(Build) 10000.00 root data:TableFullScan_16 │ └─TableFullScan_16 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo @@ -46,7 +46,7 @@ CTE_0 1.00 root Recursive CTE explain with cte(a) as (with recursive cte1(a) as (select 1 union select a + 1 from cte1 where a < 10) select * from cte1) select * from cte; id estRows task access object operator info CTEFullScan_21 1.00 root CTE:cte data:CTE_0 -CTE_0 1.00 root None Recursive CTE +CTE_0 1.00 root Non-Recursive CTE └─CTEFullScan_20(Seed Part) 1.00 root CTE:cte1 data:CTE_1 CTE_1 1.00 root Recursive CTE ├─Projection_15(Seed Part) 1.00 root 1->Column#2 @@ -70,7 +70,7 @@ id estRows task access object operator info HashJoin_17 1.00 root CARTESIAN inner join ├─CTEFullScan_27(Build) 1.00 root CTE:t2 data:CTE_0 └─CTEFullScan_26(Probe) 1.00 root CTE:t1 data:CTE_0 -CTE_0 1.00 root None Recursive CTE +CTE_0 1.00 root Non-Recursive CTE └─CTEFullScan_25(Seed Part) 1.00 root CTE:cte1 data:CTE_1 CTE_1 1.00 root Recursive CTE ├─Projection_20(Seed Part) 1.00 root 1->Column#2 @@ -102,7 +102,7 @@ HashJoin_12 0.64 root CARTESIAN inner join │ └─CTEFullScan_22 1.00 root CTE:q1 data:CTE_0 └─Selection_14(Probe) 0.80 root eq(test.t1.c1, 1) └─CTEFullScan_20 1.00 root CTE:q data:CTE_0 -CTE_0 1.00 root None Recursive CTE +CTE_0 1.00 root Non-Recursive CTE └─TableReader_17(Seed Part) 10000.00 root data:TableFullScan_16 └─TableFullScan_16 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo explain with recursive cte(a,b) as (select 1, concat('a', 1) union select a+1, concat(b, 1) from cte where a < 5) select * from cte; @@ -127,3 +127,78 @@ CTE_0 1.00 root Recursive CTE └─Projection_27(Recursive Part) 0.80 root plus(Column#5, 1)->Column#7 └─Selection_28 0.80 root eq(Column#5, 0) └─CTETable_29 1.00 root Scan on CTE_0 +explain with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 1) select * from cte1; +id estRows task access object operator info +CTEFullScan_19 1.00 root CTE:cte1 data:CTE_0 +CTE_0 1.00 root Recursive CTE, limit(offset:0, count:1) +├─TableReader_14(Seed Part) 10000.00 root data:TableFullScan_13 +│ └─TableFullScan_13 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Projection_17(Recursive Part) 1.00 root cast(plus(test.t1.c1, 1), int(11))->test.t1.c1 + └─CTETable_18 1.00 root Scan on CTE_0 +explain with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 100 offset 100) select * from cte1; +id estRows task access object operator info +CTEFullScan_19 1.00 root CTE:cte1 data:CTE_0 +CTE_0 1.00 root Recursive CTE, limit(offset:100, count:100) +├─TableReader_14(Seed Part) 10000.00 root data:TableFullScan_13 +│ └─TableFullScan_13 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Projection_17(Recursive Part) 1.00 root cast(plus(test.t1.c1, 1), int(11))->test.t1.c1 + └─CTETable_18 1.00 root Scan on CTE_0 +explain with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 0 offset 0) select * from cte1; +id estRows task access object operator info +CTEFullScan_19 1.00 root CTE:cte1 data:CTE_0 +CTE_0 1.00 root Recursive CTE, limit(offset:0, count:0) +├─TableReader_14(Seed Part) 10000.00 root data:TableFullScan_13 +│ └─TableFullScan_13 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Projection_17(Recursive Part) 1.00 root cast(plus(test.t1.c1, 1), int(11))->test.t1.c1 + └─CTETable_18 1.00 root Scan on CTE_0 +explain with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 1) select * from cte1 dt1 join cte1 dt2 on dt1.c1 = dt2.c1; +id estRows task access object operator info +HashJoin_18 0.64 root inner join, equal:[eq(test.t1.c1, test.t1.c1)] +├─Selection_29(Build) 0.80 root not(isnull(test.t1.c1)) +│ └─CTEFullScan_30 1.00 root CTE:dt2 data:CTE_0 +└─Selection_20(Probe) 0.80 root not(isnull(test.t1.c1)) + └─CTEFullScan_28 1.00 root CTE:dt1 data:CTE_0 +CTE_0 1.00 root Recursive CTE, limit(offset:0, count:1) +├─TableReader_23(Seed Part) 10000.00 root data:TableFullScan_22 +│ └─TableFullScan_22 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Projection_26(Recursive Part) 1.00 root cast(plus(test.t1.c1, 1), int(11))->test.t1.c1 + └─CTETable_27 1.00 root Scan on CTE_0 +explain with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 0 offset 0) select * from cte1 dt1 join cte1 dt2 on dt1.c1 = dt2.c1; +id estRows task access object operator info +HashJoin_18 0.64 root inner join, equal:[eq(test.t1.c1, test.t1.c1)] +├─Selection_29(Build) 0.80 root not(isnull(test.t1.c1)) +│ └─CTEFullScan_30 1.00 root CTE:dt2 data:CTE_0 +└─Selection_20(Probe) 0.80 root not(isnull(test.t1.c1)) + └─CTEFullScan_28 1.00 root CTE:dt1 data:CTE_0 +CTE_0 1.00 root Recursive CTE, limit(offset:0, count:0) +├─TableReader_23(Seed Part) 10000.00 root data:TableFullScan_22 +│ └─TableFullScan_22 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +└─Projection_26(Recursive Part) 1.00 root cast(plus(test.t1.c1, 1), int(11))->test.t1.c1 + └─CTETable_27 1.00 root Scan on CTE_0 +explain with recursive cte1(c1) as (select c1 from t1 union select c1 from t2 limit 1) select * from cte1; +id estRows task access object operator info +CTEFullScan_34 1.00 root CTE:cte1 data:CTE_0 +CTE_0 1.00 root Non-Recursive CTE +└─Limit_21(Seed Part) 1.00 root offset:0, count:1 + └─HashAgg_22 1.00 root group by:Column#11, funcs:firstrow(Column#11)->Column#11 + └─Union_23 20000.00 root + ├─TableReader_26 10000.00 root data:TableFullScan_25 + │ └─TableFullScan_25 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + └─IndexReader_33 10000.00 root index:IndexFullScan_32 + └─IndexFullScan_32 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo +explain with recursive cte1(c1) as (select c1 from t1 union select c1 from t2 limit 100 offset 100) select * from cte1; +id estRows task access object operator info +CTEFullScan_34 1.00 root CTE:cte1 data:CTE_0 +CTE_0 1.00 root Non-Recursive CTE +└─Limit_21(Seed Part) 100.00 root offset:100, count:100 + └─HashAgg_22 200.00 root group by:Column#11, funcs:firstrow(Column#11)->Column#11 + └─Union_23 20000.00 root + ├─TableReader_26 10000.00 root data:TableFullScan_25 + │ └─TableFullScan_25 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo + └─IndexReader_33 10000.00 root index:IndexFullScan_32 + └─IndexFullScan_32 10000.00 cop[tikv] table:t2, index:c1(c1) keep order:false, stats:pseudo +explain with recursive cte1(c1) as (select c1 from t1 union select c1 from t2 limit 0 offset 0) select * from cte1; +id estRows task access object operator info +CTEFullScan_18 1.00 root CTE:cte1 data:CTE_0 +CTE_0 1.00 root Non-Recursive CTE +└─TableDual_17(Seed Part) 0.00 root rows:0 diff --git a/cmd/explaintest/t/explain_cte.test b/cmd/explaintest/t/explain_cte.test index 50032776f85dd..c657ad5c68898 100644 --- a/cmd/explaintest/t/explain_cte.test +++ b/cmd/explaintest/t/explain_cte.test @@ -29,3 +29,16 @@ explain with q(a,b) as (select * from t1) select /*+ merge(q) no_merge(q1) */ * # explain with cte(a,b) as (select * from t1) select (select 1 from cte limit 1) from cte; explain with recursive cte(a,b) as (select 1, concat('a', 1) union select a+1, concat(b, 1) from cte where a < 5) select * from cte; explain select * from t1 dt where exists(with recursive qn as (select c1*0+1 as b union all select b+1 from qn where b=0) select * from qn where b=1); + +# recursive limit +explain with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 1) select * from cte1; +explain with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 100 offset 100) select * from cte1; +explain with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 0 offset 0) select * from cte1; + +explain with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 1) select * from cte1 dt1 join cte1 dt2 on dt1.c1 = dt2.c1; +explain with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 0 offset 0) select * from cte1 dt1 join cte1 dt2 on dt1.c1 = dt2.c1; + +# non-recursive limit +explain with recursive cte1(c1) as (select c1 from t1 union select c1 from t2 limit 1) select * from cte1; +explain with recursive cte1(c1) as (select c1 from t1 union select c1 from t2 limit 100 offset 100) select * from cte1; +explain with recursive cte1(c1) as (select c1 from t1 union select c1 from t2 limit 0 offset 0) select * from cte1; diff --git a/executor/builder.go b/executor/builder.go index 6827e18da55ba..5e24ae8b880c0 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -4192,6 +4192,9 @@ func (b *executorBuilder) buildCTE(v *plannercore.PhysicalCTE) Executor { chkIdx: 0, isDistinct: v.CTE.IsDistinct, sel: sel, + hasLimit: v.CTE.HasLimit, + limitBeg: v.CTE.LimitBeg, + limitEnd: v.CTE.LimitEnd, } } diff --git a/executor/cte.go b/executor/cte.go index 055163dc17e4f..fb0aaccb3d512 100644 --- a/executor/cte.go +++ b/executor/cte.go @@ -79,6 +79,13 @@ type CTEExec struct { hCtx *hashContext sel []int + // Limit related info. + hasLimit bool + limitBeg uint64 + limitEnd uint64 + cursor uint64 + meetFirstBatch bool + memTracker *memory.Tracker diskTracker *disk.Tracker } @@ -131,8 +138,8 @@ func (e *CTEExec) Open(ctx context.Context) (err error) { func (e *CTEExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { req.Reset() e.resTbl.Lock() + defer e.resTbl.Unlock() if !e.resTbl.Done() { - defer e.resTbl.Unlock() resAction := setupCTEStorageTracker(e.resTbl, e.ctx, e.memTracker, e.diskTracker) iterInAction := setupCTEStorageTracker(e.iterInTbl, e.ctx, e.memTracker, e.diskTracker) iterOutAction := setupCTEStorageTracker(e.iterOutTbl, e.ctx, e.memTracker, e.diskTracker) @@ -160,10 +167,11 @@ func (e *CTEExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { return err } e.resTbl.SetDone() - } else { - e.resTbl.Unlock() } + if e.hasLimit { + return e.nextChunkLimit(req) + } if e.chkIdx < e.resTbl.NumChunks() { res, err := e.resTbl.GetChunk(e.chkIdx) if err != nil { @@ -205,6 +213,9 @@ func (e *CTEExec) computeSeedPart(ctx context.Context) (err error) { defer close(e.iterInTbl.GetBegCh()) chks := make([]*chunk.Chunk, 0, 10) for { + if e.limitDone(e.iterInTbl) { + break + } chk := newFirstChunk(e.seedExec) if err = Next(ctx, e.seedExec, chk); err != nil { return err @@ -239,6 +250,10 @@ func (e *CTEExec) computeRecursivePart(ctx context.Context) (err error) { return ErrCTEMaxRecursionDepth.GenWithStackByArgs(e.curIter) } + if e.limitDone(e.resTbl) { + return nil + } + for { chk := newFirstChunk(e.recursiveExec) if err = Next(ctx, e.recursiveExec, chk); err != nil { @@ -248,6 +263,9 @@ func (e *CTEExec) computeRecursivePart(ctx context.Context) (err error) { if err = e.setupTblsForNewIteration(); err != nil { return err } + if e.limitDone(e.resTbl) { + break + } if e.iterInTbl.NumChunks() == 0 { break } @@ -274,6 +292,51 @@ func (e *CTEExec) computeRecursivePart(ctx context.Context) (err error) { return nil } +// Get next chunk from resTbl for limit. +func (e *CTEExec) nextChunkLimit(req *chunk.Chunk) error { + if !e.meetFirstBatch { + for e.chkIdx < e.resTbl.NumChunks() { + res, err := e.resTbl.GetChunk(e.chkIdx) + if err != nil { + return err + } + e.chkIdx++ + numRows := uint64(res.NumRows()) + if newCursor := e.cursor + numRows; newCursor >= e.limitBeg { + e.meetFirstBatch = true + begInChk, endInChk := e.limitBeg-e.cursor, numRows + if newCursor > e.limitEnd { + endInChk = e.limitEnd - e.cursor + } + e.cursor += endInChk + if begInChk == endInChk { + break + } + tmpChk := res.CopyConstructSel() + req.Append(tmpChk, int(begInChk), int(endInChk)) + return nil + } + e.cursor += numRows + } + } + if e.chkIdx < e.resTbl.NumChunks() && e.cursor < e.limitEnd { + res, err := e.resTbl.GetChunk(e.chkIdx) + if err != nil { + return err + } + e.chkIdx++ + numRows := uint64(res.NumRows()) + if e.cursor+numRows > e.limitEnd { + numRows = e.limitEnd - e.cursor + req.Append(res.CopyConstructSel(), 0, int(numRows)+1) + } else { + req.SwapColumns(res.CopyConstructSel()) + } + e.cursor += numRows + } + return nil +} + func (e *CTEExec) setupTblsForNewIteration() (err error) { num := e.iterOutTbl.NumChunks() chks := make([]*chunk.Chunk, 0, num) @@ -322,6 +385,8 @@ func (e *CTEExec) reset() { e.curIter = 0 e.chkIdx = 0 e.hashTbl = nil + e.cursor = 0 + e.meetFirstBatch = false } func (e *CTEExec) reopenTbls() (err error) { @@ -332,6 +397,11 @@ func (e *CTEExec) reopenTbls() (err error) { return e.iterInTbl.Reopen() } +// Check if tbl meets the requirement of limit. +func (e *CTEExec) limitDone(tbl cteutil.Storage) bool { + return e.hasLimit && uint64(tbl.NumRows()) >= e.limitEnd +} + func setupCTEStorageTracker(tbl cteutil.Storage, ctx sessionctx.Context, parentMemTracker *memory.Tracker, parentDiskTracker *disk.Tracker) (actionSpill *chunk.SpillDiskAction) { memTracker := tbl.GetMemTracker() diff --git a/executor/cte_test.go b/executor/cte_test.go index d6e212484ad29..aa7c2804425c8 100644 --- a/executor/cte_test.go +++ b/executor/cte_test.go @@ -237,3 +237,140 @@ func (test *CTETestSuite) TestCTEMaxRecursionDepth(c *check.C) { rows = tk.MustQuery("with cte1(c1) as (select 1 union select 2) select * from cte1 order by c1;") rows.Check(testkit.Rows("1", "2")) } + +func (test *CTETestSuite) TestCTEWithLimit(c *check.C) { + tk := testkit.NewTestKit(c, test.store) + tk.MustExec("use test;") + + // Basic recursive tests. + rows := tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 5 offset 0) select * from cte1") + rows.Check(testkit.Rows("1", "2", "3", "4", "5")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 5 offset 1) select * from cte1") + rows.Check(testkit.Rows("2", "3", "4", "5", "6")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 5 offset 10) select * from cte1") + rows.Check(testkit.Rows("11", "12", "13", "14", "15")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 5 offset 995) select * from cte1") + rows.Check(testkit.Rows("996", "997", "998", "999", "1000")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 5 offset 6) select * from cte1;") + rows.Check(testkit.Rows("7", "8", "9", "10", "11")) + + // Test with cte_max_recursion_depth + tk.MustExec("set cte_max_recursion_depth=2;") + rows = tk.MustQuery("with recursive cte1(c1) as (select 0 union select c1 + 1 from cte1 limit 1 offset 2) select * from cte1;") + rows.Check(testkit.Rows("2")) + + err := tk.QueryToErr("with recursive cte1(c1) as (select 0 union select c1 + 1 from cte1 limit 1 offset 3) select * from cte1;") + c.Assert(err, check.NotNil) + c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 3 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + + tk.MustExec("set cte_max_recursion_depth=1000;") + rows = tk.MustQuery("with recursive cte1(c1) as (select 0 union select c1 + 1 from cte1 limit 5 offset 996) select * from cte1;") + rows.Check(testkit.Rows("996", "997", "998", "999", "1000")) + + err = tk.QueryToErr("with recursive cte1(c1) as (select 0 union select c1 + 1 from cte1 limit 5 offset 997) select * from cte1;") + c.Assert(err, check.NotNil) + c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1001 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 0 offset 1) select * from cte1") + rows.Check(testkit.Rows()) + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 0 offset 10) select * from cte1") + rows.Check(testkit.Rows()) + + // Test join. + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 2 offset 1) select * from cte1 dt1 join cte1 dt2 order by dt1.c1, dt2.c1;") + rows.Check(testkit.Rows("2 2", "2 3", "3 2", "3 3")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 2 offset 1) select * from cte1 dt1 join cte1 dt2 on dt1.c1 = dt2.c1 order by dt1.c1, dt1.c1;") + rows.Check(testkit.Rows("2 2", "3 3")) + + // Test subquery. + // Different with mysql, maybe it's mysql bug?(https://bugs.mysql.com/bug.php?id=103890&thanks=4) + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 2 offset 1) select c1 from cte1 where c1 in (select 2);") + rows.Check(testkit.Rows("2")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 2 offset 1) select c1 from cte1 dt where c1 in (select c1 from cte1 where 1 = dt.c1 - 1);") + rows.Check(testkit.Rows("2")) + + // Test Apply. + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 2 offset 1) select c1 from cte1 where cte1.c1 = (select dt1.c1 from cte1 dt1 where dt1.c1 = cte1.c1);") + rows.Check(testkit.Rows("2", "3")) + + // Recursive tests with table. + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int);") + tk.MustExec("insert into t1 values(1), (2), (3);") + + // Error: ERROR 1221 (HY000): Incorrect usage of UNION and LIMIT. + // Limit can only be at the end of SQL stmt. + err = tk.ExecToErr("with recursive cte1(c1) as (select c1 from t1 limit 1 offset 1 union select c1 + 1 from cte1 limit 0 offset 1) select * from cte1") + c.Assert(err.Error(), check.Equals, "[planner:1221]Incorrect usage of UNION and LIMIT") + + // Basic non-recusive tests. + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select 2 order by 1 limit 1 offset 1) select * from cte1") + rows.Check(testkit.Rows("2")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select 2 order by 1 limit 0 offset 1) select * from cte1") + rows.Check(testkit.Rows()) + + rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select 2 order by 1 limit 2 offset 0) select * from cte1") + rows.Check(testkit.Rows("1", "2")) + + // Test with table. + tk.MustExec("drop table if exists t1;") + insertStr := "insert into t1 values(0)" + for i := 1; i < 5000; i++ { + insertStr += fmt.Sprintf(", (%d)", i) + } + + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int);") + tk.MustExec(insertStr) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 1) select * from cte1") + rows.Check(testkit.Rows("0")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 1 offset 100) select * from cte1") + rows.Check(testkit.Rows("100")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 c1 from cte1 limit 5 offset 100) select * from cte1") + rows.Check(testkit.Rows("100", "101", "102", "103", "104")) + + // Basic non-recursive tests. + rows = tk.MustQuery("with cte1 as (select c1 from t1 limit 2 offset 1) select * from cte1") + rows.Check(testkit.Rows("1", "2")) + + rows = tk.MustQuery("with cte1 as (select c1 from t1 limit 2 offset 1) select * from cte1 dt1 join cte1 dt2 on dt1.c1 = dt2.c1") + rows.Check(testkit.Rows("1 1", "2 2")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select 2 limit 0 offset 1) select * from cte1") + rows.Check(testkit.Rows()) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select 2 limit 0 offset 1) select * from cte1 dt1 join cte1 dt2 on dt1.c1 = dt2.c1") + rows.Check(testkit.Rows()) + + // rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select 2 limit 5 offset 100) select * from cte1") + // rows.Check(testkit.Rows("100", "101", "102", "103", "104")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 limit 3 offset 100) select * from cte1") + rows.Check(testkit.Rows("100", "101", "102")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 limit 3 offset 100) select * from cte1 dt1 join cte1 dt2 on dt1.c1 = dt2.c1") + rows.Check(testkit.Rows("100 100", "101 101", "102 102")) + + // Test limit 0. + tk.MustExec("set cte_max_recursion_depth = 0;") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int);") + tk.MustExec("insert into t1 values(0);") + rows = tk.MustQuery("with recursive cte1 as (select 1/c1 c1 from t1 union select c1 + 1 c1 from cte1 where c1 < 2 limit 0) select * from cte1;") + rows.Check(testkit.Rows()) + // MySQL err: ERROR 1365 (22012): Division by 0. Because it gives error when computing 1/c1. + err = tk.QueryToErr("with recursive cte1 as (select 1/c1 c1 from t1 union select c1 + 1 c1 from cte1 where c1 < 2 limit 1) select * from cte1;") + c.Assert(err, check.NotNil) + c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 9a9a9351a92c1..e575bb79e7135 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3674,8 +3674,28 @@ func (b *PlanBuilder) tryBuildCTE(ctx context.Context, tn *ast.TableName, asName } b.handleHelper.pushMap(nil) + + hasLimit := false + limitBeg := uint64(0) + limitEnd := uint64(0) + if cte.limitLP != nil { + hasLimit = true + switch x := cte.limitLP.(type) { + case *LogicalLimit: + limitBeg = x.Offset + limitEnd = x.Offset + x.Count + case *LogicalTableDual: + // Beg and End will both be 0. + default: + return nil, errors.Errorf("invalid type for limit plan: %v", cte.limitLP) + } + } + var p LogicalPlan - lp := LogicalCTE{cteAsName: tn.Name, cte: &CTEClass{IsDistinct: cte.isDistinct, seedPartLogicalPlan: cte.seedLP, recursivePartLogicalPlan: cte.recurLP, IDForStorage: cte.storageID, optFlag: cte.optFlag}}.Init(b.ctx, b.getSelectOffset()) + lp := LogicalCTE{cteAsName: tn.Name, cte: &CTEClass{IsDistinct: cte.isDistinct, seedPartLogicalPlan: cte.seedLP, + recursivePartLogicalPlan: cte.recurLP, IDForStorage: cte.storageID, + optFlag: cte.optFlag, HasLimit: hasLimit, LimitBeg: limitBeg, + LimitEnd: limitEnd}}.Init(b.ctx, b.getSelectOffset()) lp.SetSchema(getResultCTESchema(cte.seedLP.Schema(), b.ctx.GetSessionVars())) p = lp p.SetOutputNames(cte.seedLP.OutputNames()) @@ -5907,6 +5927,9 @@ func (b *PlanBuilder) buildRecursiveCTE(ctx context.Context, cte ast.ResultSetNo if x.OrderBy != nil { return ErrNotSupportedYet.GenWithStackByArgs("ORDER BY over UNION in recursive Common Table Expression") } + // Limit clause is for the whole CTE instead of only for the seed part. + oriLimit := x.Limit + x.Limit = nil // Check union type. if afterOpr != nil { @@ -5936,6 +5959,7 @@ func (b *PlanBuilder) buildRecursiveCTE(ctx context.Context, cte ast.ResultSetNo // Rebuild the plan. i-- b.buildingRecursivePartForCTE = true + x.Limit = oriLimit continue } if err != nil { @@ -5984,6 +6008,15 @@ func (b *PlanBuilder) buildRecursiveCTE(ctx context.Context, cte ast.ResultSetNo } // 4. Finally, we get the seed part plan and recursive part plan. cInfo.recurLP = recurPart + // Only need to handle limit if x is SetOprStmt. + if x.Limit != nil { + limit, err := b.buildLimit(cInfo.seedLP, x.Limit) + if err != nil { + return err + } + limit.SetChildren(limit.Children()[:0]...) + cInfo.limitLP = limit + } return nil default: p, err := b.buildResultSetNode(ctx, x) diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 16489edc02cba..7186813a0fe04 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -1183,7 +1183,10 @@ type CTEClass struct { // storageID for this CTE. IDForStorage int // optFlag is the optFlag for the whole CTE. - optFlag uint64 + optFlag uint64 + HasLimit bool + LimitBeg uint64 + LimitEnd uint64 } // LogicalCTE is for CTE. diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 74c70e1fce3c9..a80dd96a28259 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1463,10 +1463,16 @@ type CTEDefinition PhysicalCTE // ExplainInfo overrides the ExplainInfo func (p *CTEDefinition) ExplainInfo() string { + var res string if p.RecurPlan != nil { - return "Recursive CTE" + res = "Recursive CTE" + } else { + res = "Non-Recursive CTE" + } + if p.CTE.HasLimit { + res += fmt.Sprintf(", limit(offset:%v, count:%v)", p.CTE.LimitBeg, p.CTE.LimitEnd-p.CTE.LimitBeg) } - return "None Recursive CTE" + return res } // ExplainID overrides the ExplainID. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index b743204a1db91..b287c42c8b4f7 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -432,6 +432,7 @@ type cteInfo struct { // enterSubquery and recursiveRef are used to check "recursive table must be referenced only once, and not in any subquery". enterSubquery bool recursiveRef bool + limitLP LogicalPlan } // PlanBuilder builds Plan from an ast.Node. diff --git a/util/cteutil/storage.go b/util/cteutil/storage.go index 9d42b1a11c015..d2607892db62c 100644 --- a/util/cteutil/storage.go +++ b/util/cteutil/storage.go @@ -63,6 +63,9 @@ type Storage interface { // NumChunks return chunk number of the underlying storage. NumChunks() int + // NumRows return row number of the underlying storage. + NumRows() int + // Storage is not thread-safe. // By using Lock(), users can achieve the purpose of ensuring thread safety. Lock() @@ -200,6 +203,11 @@ func (s *StorageRC) NumChunks() int { return s.rc.NumChunks() } +// NumRows impls Storage NumRows interface. +func (s *StorageRC) NumRows() int { + return s.rc.NumRow() +} + // Lock impls Storage Lock interface. func (s *StorageRC) Lock() { s.mu.Lock() From 49cd57174b58351c5195cb1ed01b6275b5cd4fce Mon Sep 17 00:00:00 2001 From: mmyj Date: Thu, 3 Jun 2021 16:50:27 +0800 Subject: [PATCH 259/343] session, planner: Add a factor getter for networkFactor/scanFactor/descScanFactor/seekFactor (#25046) --- planner/core/exhaust_physical_plans.go | 4 +- planner/core/find_best_task.go | 26 +++++----- planner/core/point_get_plan.go | 8 ++-- planner/core/task.go | 18 ++++--- planner/implementation/datasource.go | 16 ++++--- sessionctx/variable/session.go | 66 +++++++++++++++++++++----- sessionctx/variable/sysvar.go | 8 ++-- sessionctx/variable/varsutil_test.go | 16 +++---- 8 files changed, 107 insertions(+), 55 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b2ba91ff99dcd..f61f138861e2e 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -930,7 +930,7 @@ func (p *LogicalJoin) constructInnerTableScanTask( copTask := &copTask{ tablePlan: ts, indexPlanFinished: true, - cst: sessVars.ScanFactor * rowSize * ts.stats.RowCount, + cst: sessVars.GetScanFactor(ts.Table) * rowSize * ts.stats.RowCount, tblColHists: ds.TblColHists, keepOrder: ts.KeepOrder, } @@ -1088,7 +1088,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( is.stats = ds.tableStats.ScaleByExpectCnt(tmpPath.CountAfterAccess) rowSize := is.indexScanRowSize(path.Index, ds, true) sessVars := ds.ctx.GetSessionVars() - cop.cst = tmpPath.CountAfterAccess * rowSize * sessVars.ScanFactor + cop.cst = tmpPath.CountAfterAccess * rowSize * sessVars.GetScanFactor(ds.tableInfo) finalStats := ds.tableStats.ScaleByExpectCnt(rowCount) is.addPushedDownSelection(cop, ds, tmpPath, finalStats) t := cop.convertToRootTask(ds.ctx) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 5207390aa8276..a9d82d308acc5 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -880,10 +880,10 @@ func (ds *DataSource) convertToPartialIndexScan(prop *property.PhysicalProperty, } indexPlan := PhysicalSelection{Conditions: indexConds}.Init(is.ctx, stats, ds.blockOffset) indexPlan.SetChildren(is) - partialCost += rowCount * rowSize * sessVars.NetworkFactor + partialCost += rowCount * rowSize * sessVars.GetNetworkFactor(ds.tableInfo) return indexPlan, partialCost } - partialCost += rowCount * rowSize * sessVars.NetworkFactor + partialCost += rowCount * rowSize * sessVars.GetNetworkFactor(ds.tableInfo) indexPlan = is return indexPlan, partialCost } @@ -903,10 +903,10 @@ func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, tablePlan = PhysicalSelection{Conditions: ts.filterCondition}.Init(ts.ctx, ts.stats.ScaleByExpectCnt(selectivity*rowCount), ds.blockOffset) tablePlan.SetChildren(ts) partialCost += rowCount * sessVars.CopCPUFactor - partialCost += selectivity * rowCount * rowSize * sessVars.NetworkFactor + partialCost += selectivity * rowCount * rowSize * sessVars.GetNetworkFactor(ds.tableInfo) return tablePlan, partialCost } - partialCost += rowCount * rowSize * sessVars.NetworkFactor + partialCost += rowCount * rowSize * sessVars.GetNetworkFactor(ds.tableInfo) tablePlan = ts return tablePlan, partialCost } @@ -971,7 +971,7 @@ func (ds *DataSource) buildIndexMergeTableScan(prop *property.PhysicalProperty, } } rowSize := ds.TblColHists.GetTableAvgRowSize(ds.ctx, ds.TblCols, ts.StoreType, true) - partialCost += totalRowCount * rowSize * sessVars.ScanFactor + partialCost += totalRowCount * rowSize * sessVars.GetScanFactor(ds.tableInfo) ts.stats = ds.tableStats.ScaleByExpectCnt(totalRowCount) if ds.statisticTable.Pseudo { ts.stats.StatsVersion = statistics.PseudoVersion @@ -1873,22 +1873,22 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper rowSize = ds.TblColHists.GetTableAvgRowSize(ds.ctx, ts.Schema().Columns, ts.StoreType, ds.handleCols != nil) } sessVars := ds.ctx.GetSessionVars() - cost := rowCount * rowSize * sessVars.ScanFactor + cost := rowCount * rowSize * sessVars.GetScanFactor(ds.tableInfo) if ts.IsGlobalRead { - cost += rowCount * sessVars.NetworkFactor * rowSize + cost += rowCount * sessVars.GetNetworkFactor(ds.tableInfo) * rowSize } if isMatchProp { ts.Desc = prop.SortItems[0].Desc if prop.SortItems[0].Desc && prop.ExpectedCnt >= smallScanThreshold { - cost = rowCount * rowSize * sessVars.DescScanFactor + cost = rowCount * rowSize * sessVars.GetDescScanFactor(ds.tableInfo) } ts.KeepOrder = true } switch ts.StoreType { case kv.TiKV: - cost += float64(len(ts.Ranges)) * sessVars.SeekFactor + cost += float64(len(ts.Ranges)) * sessVars.GetSeekFactor(ds.tableInfo) case kv.TiFlash: - cost += float64(len(ts.Ranges)) * float64(len(ts.Columns)) * sessVars.SeekFactor + cost += float64(len(ts.Ranges)) * float64(len(ts.Columns)) * sessVars.GetSeekFactor(ds.tableInfo) } return ts, cost, rowCount } @@ -1928,15 +1928,15 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper is.stats = ds.tableStats.ScaleByExpectCnt(rowCount) rowSize := is.indexScanRowSize(idx, ds, true) sessVars := ds.ctx.GetSessionVars() - cost := rowCount * rowSize * sessVars.ScanFactor + cost := rowCount * rowSize * sessVars.GetScanFactor(ds.tableInfo) if isMatchProp { is.Desc = prop.SortItems[0].Desc if prop.SortItems[0].Desc && prop.ExpectedCnt >= smallScanThreshold { - cost = rowCount * rowSize * sessVars.DescScanFactor + cost = rowCount * rowSize * sessVars.GetDescScanFactor(ds.tableInfo) } is.KeepOrder = true } - cost += float64(len(is.Ranges)) * sessVars.SeekFactor + cost += float64(len(is.Ranges)) * sessVars.GetSeekFactor(ds.tableInfo) is.cost = cost return is, cost, rowCount } diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 64642d4bef335..37747f33b0167 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -247,8 +247,8 @@ func (p *PointGetPlan) GetCost(cols []*expression.Column) float64 { } else { rowSize = p.stats.HistColl.GetIndexAvgRowSize(p.ctx, cols, p.IndexInfo.Unique) } - cost += rowSize * sessVars.NetworkFactor - cost += sessVars.SeekFactor + cost += rowSize * sessVars.GetNetworkFactor(p.TblInfo) + cost += sessVars.GetSeekFactor(p.TblInfo) cost /= float64(sessVars.DistSQLScanConcurrency()) return cost } @@ -428,8 +428,8 @@ func (p *BatchPointGetPlan) GetCost(cols []*expression.Column) float64 { rowCount = float64(len(p.IndexValues)) rowSize = p.stats.HistColl.GetIndexAvgRowSize(p.ctx, cols, p.IndexInfo.Unique) } - cost += rowCount * rowSize * sessVars.NetworkFactor - cost += rowCount * sessVars.SeekFactor + cost += rowCount * rowSize * sessVars.GetNetworkFactor(p.TblInfo) + cost += rowCount * sessVars.GetSeekFactor(p.TblInfo) cost /= float64(sessVars.DistSQLScanConcurrency()) return cost } diff --git a/planner/core/task.go b/planner/core/task.go index f4c1d1ca72c65..d3b8b84d78ead 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/charset" + "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/expression" @@ -150,19 +151,24 @@ func (t *copTask) finishIndexPlan() { cnt := t.count() t.indexPlanFinished = true sessVars := t.indexPlan.SCtx().GetSessionVars() + var tableInfo *model.TableInfo + if t.tablePlan != nil { + ts := t.tablePlan.(*PhysicalTableScan) + ts.stats = t.indexPlan.statsInfo() + tableInfo = ts.Table + } // Network cost of transferring rows of index scan to TiDB. - t.cst += cnt * sessVars.NetworkFactor * t.tblColHists.GetAvgRowSize(t.indexPlan.SCtx(), t.indexPlan.Schema().Columns, true, false) - + t.cst += cnt * sessVars.GetNetworkFactor(tableInfo) * t.tblColHists.GetAvgRowSize(t.indexPlan.SCtx(), t.indexPlan.Schema().Columns, true, false) if t.tablePlan == nil { return } + // Calculate the IO cost of table scan here because we cannot know its stats until we finish index plan. - t.tablePlan.(*PhysicalTableScan).stats = t.indexPlan.statsInfo() var p PhysicalPlan for p = t.indexPlan; len(p.Children()) > 0; p = p.Children()[0] { } rowSize := t.tblColHists.GetIndexAvgRowSize(t.indexPlan.SCtx(), t.tblCols, p.(*PhysicalIndexScan).Index.Unique) - t.cst += cnt * rowSize * sessVars.ScanFactor + t.cst += cnt * rowSize * sessVars.GetScanFactor(tableInfo) } func (t *copTask) getStoreType() kv.StoreType { @@ -935,7 +941,7 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { var prevSchema *expression.Schema // Network cost of transferring rows of table scan to TiDB. if t.tablePlan != nil { - t.cst += t.count() * sessVars.NetworkFactor * t.tblColHists.GetAvgRowSize(ctx, t.tablePlan.Schema().Columns, false, false) + t.cst += t.count() * sessVars.GetNetworkFactor(nil) * t.tblColHists.GetAvgRowSize(ctx, t.tablePlan.Schema().Columns, false, false) tp := t.tablePlan for len(tp.Children()) > 0 { @@ -2111,7 +2117,7 @@ func (t *mppTask) enforceExchangerImpl(prop *property.PhysicalProperty) *mppTask sender.SetChildren(t.p) receiver := PhysicalExchangeReceiver{}.Init(ctx, t.p.statsInfo()) receiver.SetChildren(sender) - cst := t.cst + t.count()*ctx.GetSessionVars().NetworkFactor + cst := t.cst + t.count()*ctx.GetSessionVars().GetNetworkFactor(nil) sender.cost = cst receiver.cost = cst return &mppTask{ diff --git a/planner/implementation/datasource.go b/planner/implementation/datasource.go index 1135ddbc1569d..c18d9ae64f586 100644 --- a/planner/implementation/datasource.go +++ b/planner/implementation/datasource.go @@ -74,7 +74,9 @@ func (impl *TableReaderImpl) CalcCost(outCount float64, children ...memo.Impleme reader := impl.plan.(*plannercore.PhysicalTableReader) width := impl.tblColHists.GetAvgRowSize(impl.plan.SCtx(), reader.Schema().Columns, false, false) sessVars := reader.SCtx().GetSessionVars() - networkCost := outCount * sessVars.NetworkFactor * width + // TableReaderImpl don't have tableInfo property, so using nil to replace it. + // Todo add the tableInfo property for the TableReaderImpl. + networkCost := outCount * sessVars.GetNetworkFactor(nil) * width // copTasks are run in parallel, to make the estimated cost closer to execution time, we amortize // the cost to cop iterator workers. According to `CopClient::Send`, the concurrency // is Min(DistSQLScanConcurrency, numRegionsInvolvedInScan), since we cannot infer @@ -118,9 +120,9 @@ func (impl *TableScanImpl) CalcCost(outCount float64, children ...memo.Implement ts := impl.plan.(*plannercore.PhysicalTableScan) width := impl.tblColHists.GetTableAvgRowSize(impl.plan.SCtx(), impl.tblCols, kv.TiKV, true) sessVars := ts.SCtx().GetSessionVars() - impl.cost = outCount * sessVars.ScanFactor * width + impl.cost = outCount * sessVars.GetScanFactor(ts.Table) * width if ts.Desc { - impl.cost = outCount * sessVars.DescScanFactor * width + impl.cost = outCount * sessVars.GetDescScanFactor(ts.Table) * width } return impl.cost } @@ -146,7 +148,7 @@ func (impl *IndexReaderImpl) GetCostLimit(costLimit float64, children ...memo.Im func (impl *IndexReaderImpl) CalcCost(outCount float64, children ...memo.Implementation) float64 { reader := impl.plan.(*plannercore.PhysicalIndexReader) sessVars := reader.SCtx().GetSessionVars() - networkCost := outCount * sessVars.NetworkFactor * impl.tblColHists.GetAvgRowSize(reader.SCtx(), children[0].GetPlan().Schema().Columns, true, false) + networkCost := outCount * sessVars.GetNetworkFactor(nil) * impl.tblColHists.GetAvgRowSize(reader.SCtx(), children[0].GetPlan().Schema().Columns, true, false) copIterWorkers := float64(sessVars.DistSQLScanConcurrency()) impl.cost = (networkCost + children[0].GetCost()) / copIterWorkers return impl.cost @@ -171,11 +173,11 @@ func (impl *IndexScanImpl) CalcCost(outCount float64, children ...memo.Implement is := impl.plan.(*plannercore.PhysicalIndexScan) sessVars := is.SCtx().GetSessionVars() rowSize := impl.tblColHists.GetIndexAvgRowSize(is.SCtx(), is.Schema().Columns, is.Index.Unique) - cost := outCount * rowSize * sessVars.ScanFactor + cost := outCount * rowSize * sessVars.GetScanFactor(is.Table) if is.Desc { - cost = outCount * rowSize * sessVars.DescScanFactor + cost = outCount * rowSize * sessVars.GetDescScanFactor(is.Table) } - cost += float64(len(is.Ranges)) * sessVars.SeekFactor + cost += float64(len(is.Ranges)) * sessVars.GetSeekFactor(is.Table) impl.cost = cost return impl.cost } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 247f2b12c2884..ea8ad75390a52 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -526,14 +526,14 @@ type SessionVars struct { CopCPUFactor float64 // CopTiFlashConcurrencyFactor is the concurrency number of computation in tiflash coprocessor. CopTiFlashConcurrencyFactor float64 - // NetworkFactor is the network cost of transferring 1 byte data. - NetworkFactor float64 + // networkFactor is the network cost of transferring 1 byte data. + networkFactor float64 // ScanFactor is the IO cost of scanning 1 byte data on TiKV and TiFlash. - ScanFactor float64 - // DescScanFactor is the IO cost of scanning 1 byte data on TiKV and TiFlash in desc order. - DescScanFactor float64 - // SeekFactor is the IO cost of seeking the start value of a range in TiKV or TiFlash. - SeekFactor float64 + scanFactor float64 + // descScanFactor is the IO cost of scanning 1 byte data on TiKV and TiFlash in desc order. + descScanFactor float64 + // seekFactor is the IO cost of seeking the start value of a range in TiKV or TiFlash. + seekFactor float64 // MemoryFactor is the memory cost of storing one tuple. MemoryFactor float64 // DiskFactor is the IO cost of reading/writing one byte to temporary disk. @@ -984,10 +984,10 @@ func NewSessionVars() *SessionVars { CPUFactor: DefOptCPUFactor, CopCPUFactor: DefOptCopCPUFactor, CopTiFlashConcurrencyFactor: DefOptTiFlashConcurrencyFactor, - NetworkFactor: DefOptNetworkFactor, - ScanFactor: DefOptScanFactor, - DescScanFactor: DefOptDescScanFactor, - SeekFactor: DefOptSeekFactor, + networkFactor: DefOptNetworkFactor, + scanFactor: DefOptScanFactor, + descScanFactor: DefOptDescScanFactor, + seekFactor: DefOptSeekFactor, MemoryFactor: DefOptMemoryFactor, DiskFactor: DefOptDiskFactor, ConcurrencyFactor: DefOptConcurrencyFactor, @@ -2113,3 +2113,47 @@ func (s *SessionVars) CleanupTxnReadTSIfUsed() { s.SnapshotInfoschema = nil } } + +// GetNetworkFactor returns the session variable networkFactor +// returns 0 when tbl is a temporary table. +func (s *SessionVars) GetNetworkFactor(tbl *model.TableInfo) float64 { + if tbl != nil { + if tbl.TempTableType != model.TempTableNone { + return 0 + } + } + return s.networkFactor +} + +// GetScanFactor returns the session variable scanFactor +// returns 0 when tbl is a temporary table. +func (s *SessionVars) GetScanFactor(tbl *model.TableInfo) float64 { + if tbl != nil { + if tbl.TempTableType != model.TempTableNone { + return 0 + } + } + return s.scanFactor +} + +// GetDescScanFactor returns the session variable descScanFactor +// returns 0 when tbl is a temporary table. +func (s *SessionVars) GetDescScanFactor(tbl *model.TableInfo) float64 { + if tbl != nil { + if tbl.TempTableType != model.TempTableNone { + return 0 + } + } + return s.descScanFactor +} + +// GetSeekFactor returns the session variable seekFactor +// returns 0 when tbl is a temporary table. +func (s *SessionVars) GetSeekFactor(tbl *model.TableInfo) float64 { + if tbl != nil { + if tbl.TempTableType != model.TempTableNone { + return 0 + } + } + return s.seekFactor +} diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index cab7e1b2e7ef2..3763229fb47ca 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -918,19 +918,19 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptNetworkFactor, Value: strconv.FormatFloat(DefOptNetworkFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.NetworkFactor = tidbOptFloat64(val, DefOptNetworkFactor) + s.networkFactor = tidbOptFloat64(val, DefOptNetworkFactor) return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptScanFactor, Value: strconv.FormatFloat(DefOptScanFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.ScanFactor = tidbOptFloat64(val, DefOptScanFactor) + s.scanFactor = tidbOptFloat64(val, DefOptScanFactor) return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptDescScanFactor, Value: strconv.FormatFloat(DefOptDescScanFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.DescScanFactor = tidbOptFloat64(val, DefOptDescScanFactor) + s.descScanFactor = tidbOptFloat64(val, DefOptDescScanFactor) return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptSeekFactor, Value: strconv.FormatFloat(DefOptSeekFactor, 'f', -1, 64), skipInit: true, Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { - s.SeekFactor = tidbOptFloat64(val, DefOptSeekFactor) + s.seekFactor = tidbOptFloat64(val, DefOptSeekFactor) return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptMemoryFactor, Value: strconv.FormatFloat(DefOptMemoryFactor, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64, SetSession: func(s *SessionVars, val string) error { diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 5843e4b91c77d..fa3fe12144efa 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -369,37 +369,37 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(val, Equals, "5.0") c.Assert(v.CopCPUFactor, Equals, 5.0) - c.Assert(v.NetworkFactor, Equals, 1.0) + c.Assert(v.GetNetworkFactor(nil), Equals, 1.0) err = SetSessionSystemVar(v, TiDBOptNetworkFactor, "3.0") c.Assert(err, IsNil) val, err = GetSessionOrGlobalSystemVar(v, TiDBOptNetworkFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "3.0") - c.Assert(v.NetworkFactor, Equals, 3.0) + c.Assert(v.GetNetworkFactor(nil), Equals, 3.0) - c.Assert(v.ScanFactor, Equals, 1.5) + c.Assert(v.GetScanFactor(nil), Equals, 1.5) err = SetSessionSystemVar(v, TiDBOptScanFactor, "3.0") c.Assert(err, IsNil) val, err = GetSessionOrGlobalSystemVar(v, TiDBOptScanFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "3.0") - c.Assert(v.ScanFactor, Equals, 3.0) + c.Assert(v.GetScanFactor(nil), Equals, 3.0) - c.Assert(v.DescScanFactor, Equals, 3.0) + c.Assert(v.GetDescScanFactor(nil), Equals, 3.0) err = SetSessionSystemVar(v, TiDBOptDescScanFactor, "5.0") c.Assert(err, IsNil) val, err = GetSessionOrGlobalSystemVar(v, TiDBOptDescScanFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "5.0") - c.Assert(v.DescScanFactor, Equals, 5.0) + c.Assert(v.GetDescScanFactor(nil), Equals, 5.0) - c.Assert(v.SeekFactor, Equals, 20.0) + c.Assert(v.GetSeekFactor(nil), Equals, 20.0) err = SetSessionSystemVar(v, TiDBOptSeekFactor, "50.0") c.Assert(err, IsNil) val, err = GetSessionOrGlobalSystemVar(v, TiDBOptSeekFactor) c.Assert(err, IsNil) c.Assert(val, Equals, "50.0") - c.Assert(v.SeekFactor, Equals, 50.0) + c.Assert(v.GetSeekFactor(nil), Equals, 50.0) c.Assert(v.MemoryFactor, Equals, 0.001) err = SetSessionSystemVar(v, TiDBOptMemoryFactor, "1.0") From 982dcece3b8b8eaf2b69cbc544cf9265a95cbe38 Mon Sep 17 00:00:00 2001 From: sylzd Date: Thu, 3 Jun 2021 17:24:26 +0800 Subject: [PATCH 260/343] executor: fix incorrect result of enum type merge join (#24775) --- executor/index_lookup_join_test.go | 3 ++- executor/index_lookup_merge_join_test.go | 11 ++++++++++ planner/core/exhaust_physical_plans.go | 26 ++++++++++++++++++++++++ 3 files changed, 39 insertions(+), 1 deletion(-) diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index f524acc6acca5..4a3923298ae43 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -248,7 +248,8 @@ func (s *testSuite5) TestIndexJoinEnumSetIssue19233(c *C) { tk.MustExec(`insert into p1 values('HOST_PORT');`) tk.MustExec(`insert into p2 values('HOST_PORT');`) for _, table := range []string{"p1", "p2"} { - for _, hint := range []string{"INL_HASH_JOIN", "INL_MERGE_JOIN", "INL_JOIN"} { + // INL_MERGE_JOIN do not support enum type. ref: https://github.com/pingcap/tidb/issues/24473 + for _, hint := range []string{"INL_HASH_JOIN", "INL_JOIN"} { sql := fmt.Sprintf(`select /*+ %s(%s) */ * from i, %s where i.objectType = %s.type;`, hint, table, table, table) rows := tk.MustQuery(sql).Rows() c.Assert(len(rows), Equals, 64) diff --git a/executor/index_lookup_merge_join_test.go b/executor/index_lookup_merge_join_test.go index 37943c8ec5f11..2158026f1966f 100644 --- a/executor/index_lookup_merge_join_test.go +++ b/executor/index_lookup_merge_join_test.go @@ -158,3 +158,14 @@ func (s *testSuite9) TestIssue20549(c *C) { tk.MustQuery("SELECT /*+ HASH_JOIN(t1,t2) */ 1 from t1 left outer join t2 on t1.t2id=t2.id;\n").Check( testkit.Rows("1")) } + +func (s *testSuite9) TestIssue24473(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists x, t2, t3") + tk.MustExec("CREATE TABLE `x` ( `a` enum('y','b','1','x','0','null') DEFAULT NULL, KEY `a` (`a`));") + tk.MustExec("insert into x values(\"x\"),(\"x\"),(\"b\"),(\"y\");") + tk.MustQuery("SELECT /*+ merge_join (t2,t3) */ t2.a,t3.a FROM x t2 inner join x t3 on t2.a = t3.a;").Check( + testkit.Rows("y y", "b b", "x x", "x x", "x x", "x x")) + tk.MustQuery("SELECT /*+ inl_merge_join (t2,t3) */ t2.a,t3.a FROM x t2 inner join x t3 on t2.a = t3.a;").Check( + testkit.Rows("y y", "b b", "x x", "x x", "x x", "x x")) +} diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index f61f138861e2e..1937ad6b09a26 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -147,6 +147,19 @@ func (p *LogicalJoin) GetMergeJoin(prop *property.PhysicalProperty, schema *expr joins := make([]PhysicalPlan, 0, len(p.leftProperties)+1) // The leftProperties caches all the possible properties that are provided by its children. leftJoinKeys, rightJoinKeys, isNullEQ, hasNullEQ := p.GetJoinKeys() + + // EnumType Unsupported: merge join conflicts with index order. ref: https://github.com/pingcap/tidb/issues/24473 + for _, leftKey := range leftJoinKeys { + if leftKey.RetType.Tp == mysql.TypeEnum { + return nil + } + } + for _, rightKey := range rightJoinKeys { + if rightKey.RetType.Tp == mysql.TypeEnum { + return nil + } + } + // TODO: support null equal join keys for merge join if hasNullEQ { return nil @@ -513,6 +526,19 @@ func (p *LogicalJoin) constructIndexMergeJoin( if len(join.InnerHashKeys) > len(join.InnerJoinKeys) { return nil } + + // EnumType Unsupported: merge join conflicts with index order. ref: https://github.com/pingcap/tidb/issues/24473 + for _, innerKey := range join.InnerJoinKeys { + if innerKey.RetType.Tp == mysql.TypeEnum { + return nil + } + } + for _, outerKey := range join.OuterJoinKeys { + if outerKey.RetType.Tp == mysql.TypeEnum { + return nil + } + } + hasPrefixCol := false for _, l := range join.IdxColLens { if l != types.UnspecifiedLength { From d3de5479a06a45336d2eab7bd88c50e89f4b8373 Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 3 Jun 2021 17:38:27 +0800 Subject: [PATCH 261/343] planner: support push down broadcast cartesian join to TiFlash (#25049) --- go.mod | 2 +- go.sum | 4 +- planner/core/exhaust_physical_plans.go | 15 ++++- planner/core/plan_to_pb.go | 43 ++++++++---- .../testdata/integration_serial_suite_in.json | 6 +- .../integration_serial_suite_out.json | 65 ++++++++++++++++++- sessionctx/variable/session.go | 7 ++ sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 6 ++ 9 files changed, 129 insertions(+), 20 deletions(-) diff --git a/go.mod b/go.mod index ac4ef88fd215d..ad115a92b4be3 100644 --- a/go.mod +++ b/go.mod @@ -48,7 +48,7 @@ require ( github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307 github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible - github.com/pingcap/tipb v0.0.0-20210525032549-b80be13ddf6c + github.com/pingcap/tipb v0.0.0-20210601083426-79a378b6d1c4 github.com/prometheus/client_golang v1.5.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.9.1 diff --git a/go.sum b/go.sum index 6336c1749649d..db9370b27cdac 100644 --- a/go.sum +++ b/go.sum @@ -448,8 +448,8 @@ github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041 github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:OzFN8H0EDMMqeulPhPMw2i2JaiZWOKFQ7zdRPhENNgo= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible h1:ceznmu/lLseGHP/jKyOa/3u/5H3wtLLLqkH2V3ssSjg= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20210525032549-b80be13ddf6c h1:El3pMBpJHuSkItkHsnBqsaaHzJwFBNDt3Aul98AhREY= -github.com/pingcap/tipb v0.0.0-20210525032549-b80be13ddf6c/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= +github.com/pingcap/tipb v0.0.0-20210601083426-79a378b6d1c4 h1:n47+OwdI/uxKenfBT8Y2/be11MwbeLKNLdzOWnxNQKg= +github.com/pingcap/tipb v0.0.0-20210601083426-79a378b6d1c4/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 1937ad6b09a26..6ad716c93cac2 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1661,6 +1661,9 @@ func (p *LogicalJoin) shouldUseMPPBCJ() bool { if p.ctx.GetSessionVars().BroadcastJoinThresholdSize == 0 || p.ctx.GetSessionVars().BroadcastJoinThresholdCount == 0 { return p.ctx.GetSessionVars().AllowBCJ } + if len(p.EqualConditions) == 0 && p.ctx.GetSessionVars().AllowCartesianBCJ == 2 { + return true + } if p.JoinType == LeftOuterJoin || p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin { return checkChildFitBC(p.children[1]) } else if p.JoinType == RightOuterJoin { @@ -1769,9 +1772,19 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC return nil } - if (p.JoinType != InnerJoin && p.JoinType != LeftOuterJoin && p.JoinType != RightOuterJoin && p.JoinType != SemiJoin && p.JoinType != AntiSemiJoin) || len(p.EqualConditions) == 0 { + if p.JoinType != InnerJoin && p.JoinType != LeftOuterJoin && p.JoinType != RightOuterJoin && p.JoinType != SemiJoin && p.JoinType != AntiSemiJoin { return nil } + + if len(p.EqualConditions) == 0 { + if p.ctx.GetSessionVars().AllowCartesianBCJ == 0 || !useBCJ { + return nil + } + } + if (len(p.LeftConditions) != 0 && p.JoinType != LeftOuterJoin) || (len(p.RightConditions) != 0 && p.JoinType != RightOuterJoin) { + return nil + } + if prop.PartitionTp == property.BroadcastType { return nil } diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index e095f78dff960..1b6ef79bbc6bc 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -366,7 +366,25 @@ func (p *PhysicalHashJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreType) if err != nil { return nil, err } - otherConditions, err := expression.ExpressionsToPBList(sc, p.OtherConditions, client) + + var otherConditionsInJoin expression.CNFExprs + var otherEqConditionsFromIn expression.CNFExprs + if p.JoinType == AntiSemiJoin { + for _, condition := range p.OtherConditions { + if expression.IsEQCondFromIn(condition) { + otherEqConditionsFromIn = append(otherEqConditionsFromIn, condition) + } else { + otherConditionsInJoin = append(otherConditionsInJoin, condition) + } + } + } else { + otherConditionsInJoin = p.OtherConditions + } + otherConditions, err := expression.ExpressionsToPBList(sc, otherConditionsInJoin, client) + if err != nil { + return nil, err + } + otherEqConditions, err := expression.ExpressionsToPBList(sc, otherEqConditionsFromIn, client) if err != nil { return nil, err } @@ -397,17 +415,18 @@ func (p *PhysicalHashJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreType) buildFiledTypes = append(buildFiledTypes, expression.ToPBFieldType(retType)) } join := &tipb.Join{ - JoinType: pbJoinType, - JoinExecType: tipb.JoinExecType_TypeHashJoin, - InnerIdx: int64(p.InnerChildIdx), - LeftJoinKeys: left, - RightJoinKeys: right, - ProbeTypes: probeFiledTypes, - BuildTypes: buildFiledTypes, - LeftConditions: leftConditions, - RightConditions: rightConditions, - OtherConditions: otherConditions, - Children: []*tipb.Executor{lChildren, rChildren}, + JoinType: pbJoinType, + JoinExecType: tipb.JoinExecType_TypeHashJoin, + InnerIdx: int64(p.InnerChildIdx), + LeftJoinKeys: left, + RightJoinKeys: right, + ProbeTypes: probeFiledTypes, + BuildTypes: buildFiledTypes, + LeftConditions: leftConditions, + RightConditions: rightConditions, + OtherConditions: otherConditions, + OtherEqConditionsFromIn: otherEqConditions, + Children: []*tipb.Executor{lChildren, rChildren}, } executorID := p.ExplainID().String() diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 34e50df03661b..8d6f3dee0bc53 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -52,7 +52,11 @@ "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)" + "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", + "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)" ] }, { diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 026ef6ea1bce7..46f04a2af0340 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -460,6 +460,65 @@ " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 batchCop[tiflash] ", + " │ └─ExchangeSender 2.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", + " └─HashJoin 16.00 batchCop[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 8.00 batchCop[tiflash] ", + " │ └─ExchangeSender 8.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 batchCop[tiflash] table:d1_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 cop[tiflash] CARTESIAN anti semi join, other cond:eq(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─TableFullScan 2.00 cop[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 cop[tiflash] table:fact_t keep order:false" + ] } ] }, @@ -2618,11 +2677,11 @@ "HashAgg 7992.00 root group by:test.ts.col_char_64, funcs:firstrow(test.ts.col_char_64)->test.ts.col_char_64", "└─HashJoin 9990.00 root CARTESIAN inner join, other cond:or(ge(test.ts.col_char_64_not_null, Column#25), if(ne(Column#26, 0), NULL, 0))", " ├─Selection(Build) 0.80 root ne(Column#27, 0)", - " │ └─HashAgg 1.00 root funcs:min(Column#33)->Column#25, funcs:sum(Column#34)->Column#26, funcs:count(Column#35)->Column#27", + " │ └─HashAgg 1.00 root funcs:min(Column#36)->Column#25, funcs:sum(Column#37)->Column#26, funcs:count(Column#38)->Column#27", " │ └─TableReader 1.00 root data:ExchangeSender", " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#39)->Column#33, funcs:sum(Column#40)->Column#34, funcs:count(1)->Column#35", - " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(22,0) BINARY)->Column#40", + " │ └─HashAgg 1.00 batchCop[tiflash] funcs:min(Column#42)->Column#36, funcs:sum(Column#43)->Column#37, funcs:count(1)->Column#38", + " │ └─Projection 10000.00 batchCop[tiflash] test.ts.col_varchar_64, cast(isnull(test.ts.col_varchar_64), decimal(22,0) BINARY)->Column#43", " │ └─TableFullScan 10000.00 batchCop[tiflash] table:SUBQUERY4_t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 12487.50 root data:ExchangeSender", " └─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index ea8ad75390a52..16172ca1ed213 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -486,6 +486,12 @@ type SessionVars struct { // AllowBCJ means allow broadcast join. AllowBCJ bool + + // AllowCartesianBCJ means allow broadcast CARTESIAN join, 0 means not allow, 1 means allow broadcast CARTESIAN join + // but the table size should under the broadcast threshold, 2 means allow broadcast CARTESIAN join even if the table + // size exceeds the broadcast threshold + AllowCartesianBCJ int + // AllowDistinctAggPushDown can be set true to allow agg with distinct push down to tikv/tiflash. AllowDistinctAggPushDown bool @@ -971,6 +977,7 @@ func NewSessionVars() *SessionVars { StmtCtx: new(stmtctx.StatementContext), AllowAggPushDown: false, AllowBCJ: false, + AllowCartesianBCJ: DefOptCartesianBCJ, BroadcastJoinThresholdSize: DefBroadcastJoinThresholdSize, BroadcastJoinThresholdCount: DefBroadcastJoinThresholdSize, OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 3763229fb47ca..98e8ca0c94ed3 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -877,6 +877,7 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildStatsConcurrency, skipInit: true, Value: strconv.Itoa(DefBuildStatsConcurrency)}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCartesianBCJ, Value: strconv.Itoa(DefOptCartesianBCJ), Type: TypeInt, MinValue: 0, MaxValue: 2}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeRatio, Value: strconv.FormatFloat(DefAutoAnalyzeRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeStartTime, Value: DefAutoAnalyzeStartTime, Type: TypeTime}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeEndTime, Value: DefAutoAnalyzeEndTime, Type: TypeTime}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 1c1145c2ad36a..29f10bda8cdb8 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -46,7 +46,12 @@ const ( // tidb_opt_agg_push_down is used to enable/disable the optimizer rule of aggregation push down. TiDBOptAggPushDown = "tidb_opt_agg_push_down" + // TiDBOptBCJ is used to enable/disable broadcast join in MPP mode TiDBOptBCJ = "tidb_opt_broadcast_join" + + // TiDBOptCartesianBCJ is used to disable/enable broadcast cartesian join in MPP mode + TiDBOptCartesianBCJ = "tidb_opt_broadcast_cartesian_join" + // tidb_opt_distinct_agg_push_down is used to decide whether agg with distinct should be pushed to tikv/tiflash. TiDBOptDistinctAggPushDown = "tidb_opt_distinct_agg_push_down" @@ -588,6 +593,7 @@ const ( DefSkipASCIICheck = false DefOptAggPushDown = false DefOptBCJ = false + DefOptCartesianBCJ = 1 DefOptWriteRowID = false DefOptCorrelationThreshold = 0.9 DefOptCorrelationExpFactor = 1 From 69274d886463724a96f1c9ca12de4a105613cd6d Mon Sep 17 00:00:00 2001 From: JmPotato Date: Thu, 3 Jun 2021 17:52:26 +0800 Subject: [PATCH 262/343] store: refine the error handling and retry mechanism for stale read (#24956) --- executor/batch_point_get.go | 1 + executor/point_get.go | 1 + go.mod | 2 +- go.sum | 4 +- kv/kv.go | 2 + store/copr/coprocessor.go | 3 +- store/mockstore/unistore/tikv/server.go | 5 ++ store/tikv/region_cache.go | 27 ++++++-- store/tikv/region_cache_test.go | 9 +-- store/tikv/region_request.go | 45 +++++++++++- store/tikv/region_request_test.go | 91 +++++++++++++++++++++++++ store/tikv/snapshot.go | 24 ++++--- store/tikv/tikv_test.go | 10 +++ store/tikv/tikvrpc/tikvrpc.go | 9 +++ 14 files changed, 208 insertions(+), 25 deletions(-) diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 48b33e7c1bef3..35ccecec65670 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -124,6 +124,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } snapshot.SetOption(kv.TaskID, stmtCtx.TaskID) + snapshot.SetOption(kv.TxnScope, e.ctx.GetSessionVars().TxnCtx.TxnScope) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness snapshot.SetOption(kv.IsStalenessReadOnly, isStaleness) if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != kv.GlobalTxnScope { diff --git a/executor/point_get.go b/executor/point_get.go index 78f5553069063..9d09a6acf6fb0 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -149,6 +149,7 @@ func (e *PointGetExecutor) Open(context.Context) error { e.snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } e.snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) + e.snapshot.SetOption(kv.TxnScope, e.ctx.GetSessionVars().TxnCtx.TxnScope) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness e.snapshot.SetOption(kv.IsStalenessReadOnly, isStaleness) if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != kv.GlobalTxnScope { diff --git a/go.mod b/go.mod index ad115a92b4be3..74ef9b0bb07d6 100644 --- a/go.mod +++ b/go.mod @@ -43,7 +43,7 @@ require ( github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c + github.com/pingcap/kvproto v0.0.0-20210531063847-f42e582bf0bb github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307 github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 diff --git a/go.sum b/go.sum index db9370b27cdac..b2583c595388f 100644 --- a/go.sum +++ b/go.sum @@ -433,8 +433,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17Xtb github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c h1:cy87vgUJT0U4JuxC7R14PuwBrabI9fDawYhyKTbjOBQ= -github.com/pingcap/kvproto v0.0.0-20210507054410-a8152f8a876c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210531063847-f42e582bf0bb h1:6isHwZRl1fc9i1Mggiza2iQcfvVvYAAerFIs5t9msXg= +github.com/pingcap/kvproto v0.0.0-20210531063847-f42e582bf0bb/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= diff --git a/kv/kv.go b/kv/kv.go index d65258131d184..283bc40078247 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -283,6 +283,8 @@ type Request struct { TaskID uint64 // TiDBServerID is the specified TiDB serverID to execute request. `0` means all TiDB instances. TiDBServerID uint64 + // TxnScope is the scope of the current txn. + TxnScope string // IsStaleness indicates whether the request read staleness data IsStaleness bool // MatchStoreLabels indicates the labels the store should be matched diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index 50d7d6c0a1546..d8ebf02d48136 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -713,10 +713,11 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch if worker.kvclient.Stats == nil { worker.kvclient.Stats = make(map[tikvrpc.CmdType]*tikv.RPCRuntimeStats) } + req.TxnScope = worker.req.TxnScope if worker.req.IsStaleness { req.EnableStaleRead() } - var ops []tikv.StoreSelectorOption + ops := make([]tikv.StoreSelectorOption, 0, 2) if len(worker.req.MatchStoreLabels) > 0 { ops = append(ops, tikv.WithMatchLabels(worker.req.MatchStoreLabels)) } diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index 036d824a39ff9..1b8bfa163ec85 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -618,6 +618,11 @@ func (svr *Server) BatchCoprocessor(req *coprocessor.BatchRequest, batchCopServe return nil } +// RawCoprocessor implements implements the tikvpb.TikvServer interface. +func (svr *Server) RawCoprocessor(context.Context, *kvrpcpb.RawCoprocessorRequest) (*kvrpcpb.RawCoprocessorResponse, error) { + panic("unimplemented") +} + func (mrm *MockRegionManager) removeMPPTaskHandler(taskID int64, storeID uint64) error { set := mrm.getMPPTaskSet(storeID) if set == nil { diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 50eb437545024..76da1a2ef8927 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -172,14 +172,19 @@ func (r *RegionStore) follower(seed uint32, op *storeSelectorOp) AccessIndex { // return next leader or follower store's index func (r *RegionStore) kvPeer(seed uint32, op *storeSelectorOp) AccessIndex { + if op.leaderOnly { + return r.workTiKVIdx + } candidates := make([]AccessIndex, 0, r.accessStoreNum(TiKVOnly)) for i := 0; i < r.accessStoreNum(TiKVOnly); i++ { - storeIdx, s := r.accessStore(TiKVOnly, AccessIndex(i)) - if r.storeEpochs[storeIdx] != atomic.LoadUint32(&s.epoch) || !r.filterStoreCandidate(AccessIndex(i), op) { + accessIdx := AccessIndex(i) + storeIdx, s := r.accessStore(TiKVOnly, accessIdx) + if r.storeEpochs[storeIdx] != atomic.LoadUint32(&s.epoch) || !r.filterStoreCandidate(accessIdx, op) { continue } - candidates = append(candidates, AccessIndex(i)) + candidates = append(candidates, accessIdx) } + // If there is no candidates, send to current workTiKVIdx which generally is the leader. if len(candidates) == 0 { return r.workTiKVIdx } @@ -422,6 +427,8 @@ type RPCContext struct { ProxyAccessIdx AccessIndex // valid when ProxyStore is not nil ProxyAddr string // valid when ProxyStore is not nil TiKVNum int // Number of TiKV nodes among the region's peers. Assuming non-TiKV peers are all TiFlash peers. + + tryTimes int } func (c *RPCContext) String() string { @@ -438,16 +445,24 @@ func (c *RPCContext) String() string { } type storeSelectorOp struct { - labels []*metapb.StoreLabel + leaderOnly bool + labels []*metapb.StoreLabel } // StoreSelectorOption configures storeSelectorOp. type StoreSelectorOption func(*storeSelectorOp) -// WithMatchLabels indicates selecting stores with matched labels +// WithMatchLabels indicates selecting stores with matched labels. func WithMatchLabels(labels []*metapb.StoreLabel) StoreSelectorOption { return func(op *storeSelectorOp) { - op.labels = labels + op.labels = append(op.labels, labels...) + } +} + +// WithLeaderOnly indicates selecting stores with leader only. +func WithLeaderOnly() StoreSelectorOption { + return func(op *storeSelectorOp) { + op.leaderOnly = true } } diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index ed19243a40a6a..c7fe03638ee5a 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" "github.com/pingcap/tidb/store/tikv/retry" + "github.com/pingcap/tidb/store/tikv/tikvrpc" pd "github.com/tikv/pd/client" ) @@ -1043,7 +1044,7 @@ func (s *testRegionCacheSuite) TestRegionEpochOnTiFlash(c *C) { r := ctxTiFlash.Meta reqSend := NewRegionRequestSender(s.cache, nil) regionErr := &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{CurrentRegions: []*metapb.Region{r}}} - reqSend.onRegionError(s.bo, ctxTiFlash, nil, regionErr) + reqSend.onRegionError(s.bo, ctxTiFlash, nil, regionErr, nil) // check leader read should not go to tiflash lctx, err = s.cache.GetTiKVRPCContext(s.bo, loc1.Region, kv.ReplicaReadLeader, 0) @@ -1406,11 +1407,11 @@ func (s *testRegionCacheSuite) TestFollowerMeetEpochNotMatch(c *C) { c.Assert(ctxFollower1.Store.storeID, Equals, s.store2) regionErr := &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}} - reqSend.onRegionError(s.bo, ctxFollower1, &followReqSeed, regionErr) + reqSend.onRegionError(s.bo, ctxFollower1, &tikvrpc.Request{ReplicaReadSeed: &followReqSeed}, regionErr, nil) c.Assert(followReqSeed, Equals, uint32(1)) regionErr = &errorpb.Error{RegionNotFound: &errorpb.RegionNotFound{}} - reqSend.onRegionError(s.bo, ctxFollower1, &followReqSeed, regionErr) + reqSend.onRegionError(s.bo, ctxFollower1, &tikvrpc.Request{ReplicaReadSeed: &followReqSeed}, regionErr, nil) c.Assert(followReqSeed, Equals, uint32(2)) } @@ -1437,7 +1438,7 @@ func (s *testRegionCacheSuite) TestMixedMeetEpochNotMatch(c *C) { c.Assert(ctxFollower1.Store.storeID, Equals, s.store1) regionErr := &errorpb.Error{EpochNotMatch: &errorpb.EpochNotMatch{}} - reqSend.onRegionError(s.bo, ctxFollower1, &followReqSeed, regionErr) + reqSend.onRegionError(s.bo, ctxFollower1, &tikvrpc.Request{ReplicaReadSeed: &followReqSeed}, regionErr, nil) c.Assert(followReqSeed, Equals, uint32(1)) } diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index a90d638233578..9f180c26269f2 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/util" @@ -286,6 +287,9 @@ func (s *RegionRequestSender) SendReqCtx( if err != nil { return nil, nil, err } + if rpcCtx != nil { + rpcCtx.tryTimes = tryTimes + } failpoint.Inject("invalidCacheAndRetry", func() { // cooperate with github.com/pingcap/tidb/store/gcworker/setGcResolveMaxBackoff @@ -334,8 +338,14 @@ func (s *RegionRequestSender) SendReqCtx( if err != nil { return nil, nil, errors.Trace(err) } + failpoint.Inject("mockDataIsNotReadyError", func(val failpoint.Value) { + regionErr = &errorpb.Error{} + if tryTimesLimit, ok := val.(int); ok && tryTimes <= tryTimesLimit { + regionErr.DataIsNotReady = &errorpb.DataIsNotReady{} + } + }) if regionErr != nil { - retry, err = s.onRegionError(bo, rpcCtx, req.ReplicaReadSeed, regionErr) + retry, err = s.onRegionError(bo, rpcCtx, req, regionErr, &opts) if err != nil { return nil, nil, errors.Trace(err) } @@ -644,7 +654,7 @@ func regionErrorToLabel(e *errorpb.Error) string { return "unknown" } -func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed *uint32, regionErr *errorpb.Error) (shouldRetry bool, err error) { +func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req *tikvrpc.Request, regionErr *errorpb.Error, opts *[]StoreSelectorOption) (shouldRetry bool, err error) { if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("tikv.onRegionError", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -652,6 +662,12 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed // bo = bo.Clone() bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } + // Stale Read request will retry the leader or next peer on error, + // if txnScope is global, we will only retry the leader by using the WithLeaderOnly option, + // if txnScope is local, we will retry both other peers and the leader by the incresing seed. + if ctx.tryTimes < 1 && req != nil && req.TxnScope == oracle.GlobalTxnScope && req.GetStaleRead() { + *opts = append(*opts, WithLeaderOnly()) + } metrics.TiKVRegionErrorCounter.WithLabelValues(regionErrorToLabel(regionErr)).Inc() if notLeader := regionErr.GetNotLeader(); notLeader != nil { @@ -687,6 +703,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed return true, nil } + seed := req.GetReplicaReadSeed() if epochNotMatch := regionErr.GetEpochNotMatch(); epochNotMatch != nil { logutil.BgLogger().Debug("tikv reports `EpochNotMatch` retry later", zap.Stringer("EpochNotMatch", epochNotMatch), @@ -723,6 +740,30 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, seed logutil.BgLogger().Warn("tikv reports `RaftEntryTooLarge`", zap.Stringer("ctx", ctx)) return false, errors.New(regionErr.String()) } + // A stale read request may be sent to a peer which the data is not ready yet, we should retry in this case. + if regionErr.GetDataIsNotReady() != nil { + logutil.BgLogger().Warn("tikv reports `DataIsNotReady` retry later", + zap.Uint64("store-id", ctx.Store.storeID), + zap.Uint64("peer-id", regionErr.GetDataIsNotReady().GetPeerId()), + zap.Uint64("region-id", regionErr.GetDataIsNotReady().GetRegionId()), + zap.Uint64("safe-ts", regionErr.GetDataIsNotReady().GetSafeTs()), + zap.Stringer("ctx", ctx)) + if seed != nil { + *seed = *seed + 1 + } + return true, nil + } + // A read request may be sent to a peer which has not been initialized yet, we should retry in this case. + if regionErr.GetRegionNotInitialized() != nil { + logutil.BgLogger().Warn("tikv reports `RegionNotInitialized` retry later", + zap.Uint64("store-id", ctx.Store.storeID), + zap.Uint64("region-id", regionErr.GetRegionNotInitialized().GetRegionId()), + zap.Stringer("ctx", ctx)) + if seed != nil { + *seed = *seed + 1 + } + return true, nil + } if regionErr.GetRegionNotFound() != nil && seed != nil { logutil.BgLogger().Debug("tikv reports `RegionNotFound` in follow-reader", zap.Stringer("ctx", ctx), zap.Uint32("seed", *seed)) diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index d323f9f2f7caa..35bb0c185cdb5 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -24,6 +24,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/coprocessor_v2" "github.com/pingcap/kvproto/pkg/errorpb" @@ -33,6 +34,7 @@ import ( "github.com/pingcap/kvproto/pkg/tikvpb" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/config" @@ -185,6 +187,92 @@ func (s *testRegionRequestToThreeStoresSuite) TestStoreTokenLimit(c *C) { kv.StoreLimit.Store(oldStoreLimit) } +// Test whether the Stale Read request will retry the leader or other peers on error. +func (s *testRegionRequestToThreeStoresSuite) TestStaleReadRetry(c *C) { + var seed uint32 = 0 + req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{}, kv.ReplicaReadMixed, &seed) + req.EnableStaleRead() + + // Test whether a global Stale Read request will only retry on the leader. + req.TxnScope = oracle.GlobalTxnScope + region, err := s.cache.LocateRegionByID(s.bo, s.regionID) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + // Retry 1 time. + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockDataIsNotReadyError", `return(1)`), IsNil) + resp, ctx, err := s.regionRequestSender.SendReqCtx(s.bo, req, region.Region, time.Second, tikvrpc.TiKV) + c.Assert(err, IsNil) + c.Assert(resp.Resp, NotNil) + c.Assert(ctx, NotNil) + c.Assert(ctx.Peer.GetId(), Equals, s.leaderPeer) + + seed = 0 + region, err = s.cache.LocateRegionByID(s.bo, s.regionID) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + // Retry 2 times. + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockDataIsNotReadyError", `return(2)`), IsNil) + resp, ctx, err = s.regionRequestSender.SendReqCtx(s.bo, req, region.Region, time.Second, tikvrpc.TiKV) + c.Assert(err, IsNil) + c.Assert(resp.Resp, NotNil) + c.Assert(ctx, NotNil) + c.Assert(ctx.Peer.GetId(), Equals, s.leaderPeer) + + seed = 0 + region, err = s.cache.LocateRegionByID(s.bo, s.regionID) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + // Retry 3 times. + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockDataIsNotReadyError", `return(3)`), IsNil) + resp, ctx, err = s.regionRequestSender.SendReqCtx(s.bo, req, region.Region, time.Second, tikvrpc.TiKV) + c.Assert(err, IsNil) + c.Assert(resp.Resp, NotNil) + c.Assert(ctx, NotNil) + c.Assert(ctx.Peer.GetId(), Equals, s.leaderPeer) + + // Test whether a local Stale Read request will retry on the leader and other peers. + req.TxnScope = "local" + seed = 0 + region, err = s.cache.LocateRegionByID(s.bo, s.regionID) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + // Retry 1 time. + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockDataIsNotReadyError", `return(1)`), IsNil) + resp, ctx, err = s.regionRequestSender.SendReqCtx(s.bo, req, region.Region, time.Second, tikvrpc.TiKV) + c.Assert(err, IsNil) + c.Assert(resp.Resp, NotNil) + c.Assert(ctx, NotNil) + peerID1 := ctx.Peer.GetId() + + seed = 0 + region, err = s.cache.LocateRegionByID(s.bo, s.regionID) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + // Retry 2 times. + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockDataIsNotReadyError", `return(2)`), IsNil) + resp, ctx, err = s.regionRequestSender.SendReqCtx(s.bo, req, region.Region, time.Second, tikvrpc.TiKV) + c.Assert(err, IsNil) + c.Assert(resp.Resp, NotNil) + c.Assert(ctx, NotNil) + peerID2 := ctx.Peer.GetId() + c.Assert(peerID2, Not(Equals), peerID1) + + seed = 0 + region, err = s.cache.LocateRegionByID(s.bo, s.regionID) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + // Retry 3 times. + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockDataIsNotReadyError", `return(3)`), IsNil) + resp, ctx, err = s.regionRequestSender.SendReqCtx(s.bo, req, region.Region, time.Second, tikvrpc.TiKV) + c.Assert(err, IsNil) + c.Assert(resp.Resp, NotNil) + c.Assert(ctx, NotNil) + peerID3 := ctx.Peer.GetId() + c.Assert(peerID3, Not(Equals), peerID1) + c.Assert(peerID3, Not(Equals), peerID2) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockDataIsNotReadyError"), IsNil) +} + func (s *testRegionRequestToSingleStoreSuite) TestOnSendFailedWithStoreRestart(c *C) { req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ Key: []byte("key"), @@ -435,6 +523,9 @@ func (s *mockTikvGrpcServer) Coprocessor(context.Context, *coprocessor.Request) func (s *mockTikvGrpcServer) BatchCoprocessor(*coprocessor.BatchRequest, tikvpb.Tikv_BatchCoprocessorServer) error { return errors.New("unreachable") } +func (s *mockTikvGrpcServer) RawCoprocessor(context.Context, *kvrpcpb.RawCoprocessorRequest) (*kvrpcpb.RawCoprocessorResponse, error) { + return nil, errors.New("unreachable") +} func (s *mockTikvGrpcServer) DispatchMPPTask(context.Context, *mpp.DispatchTaskRequest) (*mpp.DispatchTaskResponse, error) { return nil, errors.New("unreachable") } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 743111827e85d..e9398e336ce86 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -105,6 +105,7 @@ type KVSnapshot struct { replicaRead kv.ReplicaReadType taskID uint64 isStaleness bool + txnScope string // MatchStoreLabels indicates the labels the store should be matched matchStoreLabels []*metapb.StoreLabel } @@ -306,8 +307,6 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec pending := batch.keys for { - isStaleness := false - var matchStoreLabels []*metapb.StoreLabel s.mu.RLock() req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdBatchGet, &pb.BatchGetRequest{ Keys: pending, @@ -318,13 +317,15 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec TaskId: s.mu.taskID, ResourceGroupTag: s.resourceGroupTag, }) - isStaleness = s.mu.isStaleness - matchStoreLabels = s.mu.matchStoreLabels + txnScope := s.mu.txnScope + isStaleness := s.mu.isStaleness + matchStoreLabels := s.mu.matchStoreLabels s.mu.RUnlock() - var ops []StoreSelectorOption + req.TxnScope = txnScope if isStaleness { req.EnableStaleRead() } + ops := make([]StoreSelectorOption, 0, 2) if len(matchStoreLabels) > 0 { ops = append(ops, WithMatchLabels(matchStoreLabels)) } @@ -452,8 +453,6 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, cli := NewClientHelper(s.store, s.resolvedLocks) - isStaleness := false - var matchStoreLabels []*metapb.StoreLabel s.mu.RLock() if s.mu.stats != nil { cli.Stats = make(map[tikvrpc.CmdType]*RPCRuntimeStats) @@ -471,8 +470,8 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, TaskId: s.mu.taskID, ResourceGroupTag: s.resourceGroupTag, }) - isStaleness = s.mu.isStaleness - matchStoreLabels = s.mu.matchStoreLabels + isStaleness := s.mu.isStaleness + matchStoreLabels := s.mu.matchStoreLabels s.mu.RUnlock() var ops []StoreSelectorOption if isStaleness { @@ -620,6 +619,13 @@ func (s *KVSnapshot) SetRuntimeStats(stats *SnapshotRuntimeStats) { s.mu.stats = stats } +// SetTxnScope sets up the txn scope. +func (s *KVSnapshot) SetTxnScope(txnScope string) { + s.mu.Lock() + defer s.mu.Unlock() + s.mu.txnScope = txnScope +} + // SetIsStatenessReadOnly indicates whether the transaction is staleness read only transaction func (s *KVSnapshot) SetIsStatenessReadOnly(b bool) { s.mu.Lock() diff --git a/store/tikv/tikv_test.go b/store/tikv/tikv_test.go index a5b703ee3ff7a..391af8e2663f9 100644 --- a/store/tikv/tikv_test.go +++ b/store/tikv/tikv_test.go @@ -15,9 +15,12 @@ package tikv import ( "flag" + "os" "sync" + "testing" . "github.com/pingcap/check" + "github.com/pingcap/tidb/util/logutil" ) var ( @@ -48,6 +51,13 @@ type testTiKVSuite struct { OneByOneSuite } +func TestT(t *testing.T) { + CustomVerboseFlag = true + logLevel := os.Getenv("log_level") + logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false)) + TestingT(t) +} + var _ = Suite(&testTiKVSuite{}) func (s *testTiKVSuite) TestBasicFunc(c *C) { diff --git a/store/tikv/tikvrpc/tikvrpc.go b/store/tikv/tikvrpc/tikvrpc.go index 9f1b4fc0806aa..ae450b6019799 100644 --- a/store/tikv/tikvrpc/tikvrpc.go +++ b/store/tikv/tikvrpc/tikvrpc.go @@ -180,6 +180,7 @@ type Request struct { Type CmdType Req interface{} kvrpcpb.Context + TxnScope string ReplicaReadType kv.ReplicaReadType // different from `kvrpcpb.Context.ReplicaRead` ReplicaReadSeed *uint32 // pointer to follower read seed in snapshot/coprocessor StoreTp EndpointType @@ -214,6 +215,14 @@ func NewReplicaReadRequest(typ CmdType, pointer interface{}, replicaReadType kv. return req } +// GetReplicaReadSeed returns ReplicaReadSeed pointer. +func (req *Request) GetReplicaReadSeed() *uint32 { + if req != nil { + return req.ReplicaReadSeed + } + return nil +} + // EnableStaleRead enables stale read func (req *Request) EnableStaleRead() { req.StaleRead = true From 4d9c667aab039dd02d36d9f65bc99409e059ac40 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Thu, 3 Jun 2021 18:14:26 +0800 Subject: [PATCH 263/343] infoschema, executor, txn: encode `key` field in DATA_LOCK_WAITS table (#25060) --- executor/infoschema_reader.go | 2 +- infoschema/tables_test.go | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index b2d733851c998..b01972726991a 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1038,7 +1038,7 @@ func (e *memtableRetriever) setDataForTableDataLockWaits(ctx sessionctx.Context) digestStr = hex.EncodeToString(digest) } e.rows = append(e.rows, types.MakeDatums( - wait.Key, + hex.EncodeToString(wait.Key), wait.Txn, wait.WaitForTxn, digestStr, diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 3de42c28e1235..dbba2dbcdba38 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -15,6 +15,7 @@ package infoschema_test import ( "crypto/tls" + "encoding/hex" "fmt" "math" "net" @@ -1625,8 +1626,11 @@ func (s *testDataLockWaitSuite) SetUpSuite(c *C) { func (s *testDataLockWaitSuite) TestDataLockWait(c *C) { _, digest1 := parser.NormalizeDigest("select * from t1 for update;") _, digest2 := parser.NormalizeDigest("update t1 set f1=1 where id=2;") + keyHex1 := hex.EncodeToString([]byte("a")) + keyHex2 := hex.EncodeToString([]byte("b")) tk := s.newTestKitWithRoot(c) - tk.MustQuery("select * from information_schema.DATA_LOCK_WAITS;").Check(testkit.Rows("a 1 2 "+digest1.String(), "b 4 5 "+digest2.String())) + tk.MustQuery("select * from information_schema.DATA_LOCK_WAITS;"). + Check(testkit.Rows(keyHex1+" 1 2 "+digest1.String(), keyHex2+" 4 5 "+digest2.String())) } func (s *testDataLockWaitSuite) TestDataLockPrivilege(c *C) { From 8111d14c9022a7f47891ab5f45c86cc9f991e63e Mon Sep 17 00:00:00 2001 From: tangenta Date: Thu, 3 Jun 2021 18:15:35 +0800 Subject: [PATCH 264/343] tests: make TestIssue20658 stable (#25092) * tests: make TestIssue20658 stable --- executor/aggregate_test.go | 81 +++++++++++++++++++++----------------- 1 file changed, 44 insertions(+), 37 deletions(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index b93d8dd619d88..32041a29df8fd 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -1310,50 +1310,57 @@ func (s *testSuiteAgg) TestIssue20658(c *C) { tk.MustExec("set tidb_max_chunk_size=32;") randSeed := time.Now().UnixNano() r := rand.New(rand.NewSource(randSeed)) - var insertSQL string + var insertSQL strings.Builder for i := 0; i < 1000; i++ { - if i == 0 { - insertSQL += fmt.Sprintf("(%d, %d)", r.Intn(100), r.Intn(100)) - } else { - insertSQL += fmt.Sprintf(",(%d, %d)", r.Intn(100), r.Intn(100)) + insertSQL.WriteString("(") + insertSQL.WriteString(strconv.Itoa(r.Intn(10))) + insertSQL.WriteString(",") + insertSQL.WriteString(strconv.Itoa(r.Intn(10))) + insertSQL.WriteString(")") + if i < 1000-1 { + insertSQL.WriteString(",") } } - tk.MustExec(fmt.Sprintf("insert into t values %s;", insertSQL)) - - concurrencies := []int{1, 2, 4, 8} + tk.MustExec(fmt.Sprintf("insert into t values %s;", insertSQL.String())) + + mustParseAndSort := func(rows [][]interface{}, cmt CommentInterface) []float64 { + ret := make([]float64, len(rows)) + for i := 0; i < len(rows); i++ { + rowStr := rows[i][0].(string) + if rowStr == "" { + ret[i] = 0 + continue + } + v, err := strconv.ParseFloat(rowStr, 64) + c.Assert(err, IsNil, cmt) + ret[i] = v + } + sort.Float64s(ret) + return ret + } for _, sql := range sqls { - var expected [][]interface{} - for _, con := range concurrencies { - comment := Commentf("sql: %s; concurrency: %d, seed: ", sql, con, randSeed) + tk.MustExec("set @@tidb_streamagg_concurrency = 1;") + exp := tk.MustQuery(sql).Rows() + expected := mustParseAndSort(exp, Commentf("sql: %s; seed: %d", sql, randSeed)) + for _, con := range []int{2, 4, 8} { + comment := Commentf("sql: %s; concurrency: %d, seed: %d", sql, con, randSeed) tk.MustExec(fmt.Sprintf("set @@tidb_streamagg_concurrency=%d;", con)) - if con == 1 { - expected = tk.MustQuery(sql).Sort().Rows() - } else { - er := tk.MustQuery("explain format = 'brief' " + sql).Rows() - ok := false - for _, l := range er { - str := fmt.Sprintf("%v", l) - if strings.Contains(str, "Shuffle") { - ok = true - break - } - } - c.Assert(ok, Equals, true, comment) - rows := tk.MustQuery(sql).Sort().Rows() - - c.Assert(len(rows), Equals, len(expected), comment) - for i := range rows { - rowStr, expStr := rows[i][0].(string), expected[i][0].(string) - if rowStr == "" && expStr == "" { - continue - } - v1, err := strconv.ParseFloat(rowStr, 64) - c.Assert(err, IsNil, comment) - v2, err := strconv.ParseFloat(expStr, 64) - c.Assert(err, IsNil, comment) - c.Assert(math.Abs(v1-v2), Less, 1e-3, comment) + er := tk.MustQuery("explain format = 'brief' " + sql).Rows() + ok := false + for _, l := range er { + str := fmt.Sprintf("%v", l) + if strings.Contains(str, "Shuffle") { + ok = true + break } } + c.Assert(ok, Equals, true, comment) + rows := mustParseAndSort(tk.MustQuery(sql).Rows(), comment) + + c.Assert(len(rows), Equals, len(expected), comment) + for i := range rows { + c.Assert(math.Abs(rows[i]-expected[i]), Less, 1e-3, comment) + } } } } From 090892822b17d94345492e95e3bb1793532431de Mon Sep 17 00:00:00 2001 From: JaySon Date: Thu, 3 Jun 2021 18:28:26 +0800 Subject: [PATCH 265/343] expression: support `str_to_date` push to TiFlash (#25095) --- expression/expr_to_pb_test.go | 6 ++++++ expression/expression.go | 9 +++++++++ 2 files changed, 15 insertions(+) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index 1a60a1266b8df..66f07e4d7846e 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -764,6 +764,12 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_UnixTimestampDec) exprs = append(exprs, function) + // StrToDateDateTime + function, err = NewFunction(mock.NewContext(), ast.StrToDate, types.NewFieldType(mysql.TypeDatetime), stringColumn, stringColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_StrToDateDatetime) + exprs = append(exprs, function) + canPush := CanExprsPushDown(sc, exprs, client, kv.TiFlash) c.Assert(canPush, Equals, true) diff --git a/expression/expression.go b/expression/expression.go index 9a14537cac248..e4732440ef1b6 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1042,6 +1042,15 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { case tipb.ScalarFuncSig_ExtractDatetime: return true } + case ast.StrToDate: + switch function.Function.PbCode() { + case + tipb.ScalarFuncSig_StrToDateDate, + tipb.ScalarFuncSig_StrToDateDatetime: + return true + default: + return false + } } return false } From 3846378f7125d586994082ac4502ce7590696136 Mon Sep 17 00:00:00 2001 From: djshow832 Date: Thu, 3 Jun 2021 18:54:26 +0800 Subject: [PATCH 266/343] ddl: Forbid set tiflash replicas on temporary tables (#25036) --- ddl/db_test.go | 28 ++++++++++++++++++++++++++++ ddl/ddl_api.go | 15 ++++++++++----- 2 files changed, 38 insertions(+), 5 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 1c0e1f4cb48de..fb4e38f37f109 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -5339,6 +5339,34 @@ func (s *testSerialDBSuite) TestSetTableFlashReplicaForSystemTable(c *C) { } } +func (s *testSerialDBSuite) TestSetTiFlashReplicaForTemporaryTable(c *C) { + // test for tiflash replica + c.Assert(failpoint.Enable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount", `return(true)`), IsNil) + defer func() { + err := failpoint.Disable("github.com/pingcap/tidb/infoschema/mockTiFlashStoreCount") + c.Assert(err, IsNil) + }() + + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("set tidb_enable_global_temporary_table=true") + tk.MustExec("drop table if exists temp") + tk.MustExec("create global temporary table temp(id int) on commit delete rows") + tk.MustGetErrCode("alter table temp set tiflash replica 1", errno.ErrOptOnTemporaryTable) + tk.MustExec("drop table temp") + + tk.MustExec("drop table if exists normal") + tk.MustExec("create table normal(id int)") + defer tk.MustExec("drop table normal") + tk.MustExec("alter table normal set tiflash replica 1") + tk.MustQuery("select REPLICA_COUNT from information_schema.tiflash_replica where table_schema='test' and table_name='normal'").Check(testkit.Rows("1")) + tk.MustExec("create global temporary table temp like normal on commit delete rows") + tk.MustQuery("select REPLICA_COUNT from information_schema.tiflash_replica where table_schema='test' and table_name='temp'").Check(testkit.Rows()) + tk.MustExec("drop table temp") + tk.MustExec("set tidb_enable_noop_functions = 1") + tk.MustExec("create temporary table temp like normal") + tk.MustQuery("select REPLICA_COUNT from information_schema.tiflash_replica where table_schema='test' and table_name='temp'").Check(testkit.Rows()) +} + func (s *testSerialDBSuite) TestAlterShardRowIDBits(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) defer func() { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 8583fa6414386..30ad5bb6f2878 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1643,7 +1643,7 @@ func checkTableInfoValid(tblInfo *model.TableInfo) error { return checkInvisibleIndexOnPK(tblInfo) } -func buildTableInfoWithLike(ident ast.Ident, referTblInfo *model.TableInfo) (*model.TableInfo, error) { +func buildTableInfoWithLike(ident ast.Ident, referTblInfo *model.TableInfo, s *ast.CreateTableStmt) (*model.TableInfo, error) { // Check the referred table is a real table object. if referTblInfo.IsSequence() || referTblInfo.IsView() { return nil, ErrWrongObject.GenWithStackByArgs(ident.Schema, referTblInfo.Name, "BASE TABLE") @@ -1662,7 +1662,10 @@ func buildTableInfoWithLike(ident ast.Ident, referTblInfo *model.TableInfo) (*mo tblInfo.Name = ident.Name tblInfo.AutoIncID = 0 tblInfo.ForeignKeys = nil - if tblInfo.TiFlashReplica != nil { + // Ignore TiFlash replicas for temporary tables. + if s.TemporaryKeyword != ast.TemporaryNone { + tblInfo.TiFlashReplica = nil + } else if tblInfo.TiFlashReplica != nil { replica := *tblInfo.TiFlashReplica // Keep the tiflash replica setting, remove the replica available status. replica.AvailablePartitionIDs = nil @@ -1810,7 +1813,7 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e // build tableInfo var tbInfo *model.TableInfo if s.ReferTable != nil { - tbInfo, err = buildTableInfoWithLike(ident, referTbl.Meta()) + tbInfo, err = buildTableInfoWithLike(ident, referTbl.Meta(), s) } else { tbInfo, err = buildTableInfoWithStmt(ctx, s, schema.Charset, schema.Collate) } @@ -4387,14 +4390,16 @@ func (d *ddl) AlterTableSetTiFlashReplica(ctx sessionctx.Context, ident ast.Iden // Ban setting replica count for tables in system database. if util.IsMemOrSysDB(schema.Name.L) { return errors.Trace(errUnsupportedAlterReplicaForSysTable) + } else if tb.Meta().TempTableType != model.TempTableNone { + return ErrOptOnTemporaryTable.GenWithStackByArgs("set tiflash replica") } tbReplicaInfo := tb.Meta().TiFlashReplica if tbReplicaInfo != nil && tbReplicaInfo.Count == replicaInfo.Count && len(tbReplicaInfo.LocationLabels) == len(replicaInfo.Labels) { changed := false - for i, lable := range tbReplicaInfo.LocationLabels { - if replicaInfo.Labels[i] != lable { + for i, label := range tbReplicaInfo.LocationLabels { + if replicaInfo.Labels[i] != label { changed = true break } From 6900a46221aa986732103328cf7635f8ca7d68ed Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BE=99=E6=96=B9=E6=B7=9E?= Date: Thu, 3 Jun 2021 19:06:26 +0800 Subject: [PATCH 267/343] infoschema, executor, txn: change some field name in TIDB_TRX table (#25061) --- docs/design/2021-04-26-lock-view.md | 3 ++- infoschema/tables.go | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/design/2021-04-26-lock-view.md b/docs/design/2021-04-26-lock-view.md index 814294c8e929f..79afbce94abef 100644 --- a/docs/design/2021-04-26-lock-view.md +++ b/docs/design/2021-04-26-lock-view.md @@ -49,7 +49,8 @@ Several tables will be provided in `information_schema`. Some tables has both lo | `DB` | `varchar` | | | `SET_COUNT` | `int` | Modified keys of the current transaction | | `LOCKED_COUNT` | `int` | Locked keys of the current transaction | -| `MEM_BUFFER_SIZE` | `int` | Size occupied by the transaction's membuffer | +| `MEM_BUFFER_KEYS` | `int` | Entries in transaction's membuffer | +| `MEM_BUFFER_BYTES` | `int` | Size occupied by the transaction's membuffer | * Life span of rows: * Create on first writing or locking operation in a transaction diff --git a/infoschema/tables.go b/infoschema/tables.go index d320c633d2398..df6c926b6354d 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1356,8 +1356,8 @@ var tableTiDBTrxCols = []columnInfo{ {name: "CURRENT_SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the sql the transaction are currently running"}, {name: "STATE", tp: mysql.TypeEnum, enumElems: txninfo.TxnRunningStateStrs, comment: "Current running state of the transaction"}, {name: "WAITING_START_TIME", tp: mysql.TypeTimestamp, decimal: 6, size: 26, comment: "Current lock waiting's start time"}, - {name: "LEN", tp: mysql.TypeLonglong, size: 64, comment: "How many entries are in MemDB"}, - {name: "SIZE", tp: mysql.TypeLonglong, size: 64, comment: "MemDB used memory"}, + {name: "MEM_BUFFER_KEYS", tp: mysql.TypeLonglong, size: 64, comment: "How many entries are in MemDB"}, + {name: "MEM_BUFFER_BYTES", tp: mysql.TypeLonglong, size: 64, comment: "MemDB used memory"}, {name: "SESSION_ID", tp: mysql.TypeLonglong, size: 21, flag: mysql.UnsignedFlag, comment: "Which session this transaction belongs to"}, {name: "USER", tp: mysql.TypeVarchar, size: 16, comment: "The user who open this session"}, {name: "DB", tp: mysql.TypeVarchar, size: 64, comment: "The schema this transaction works on"}, From e2340c1d86191445e01f0e4297e8ac3a36310acc Mon Sep 17 00:00:00 2001 From: Yu Lei Date: Thu, 3 Jun 2021 20:30:26 +0800 Subject: [PATCH 268/343] expression: Support cast string as real push down (#25096) --- expression/expr_to_pb_test.go | 5 +++++ expression/expression.go | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index 66f07e4d7846e..c7b97e0227626 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -719,6 +719,11 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) + // CastStringAsReal + function, err = NewFunction(mock.NewContext(), ast.Cast, types.NewFieldType(mysql.TypeDouble), stringColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + // Substring2ArgsUTF8 function, err = NewFunction(mock.NewContext(), ast.Substr, types.NewFieldType(mysql.TypeString), stringColumn, intColumn) c.Assert(err, IsNil) diff --git a/expression/expression.go b/expression/expression.go index e4732440ef1b6..cbbc200811e23 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1017,7 +1017,7 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { switch function.Function.PbCode() { case tipb.ScalarFuncSig_CastIntAsInt, tipb.ScalarFuncSig_CastIntAsDecimal, tipb.ScalarFuncSig_CastIntAsString, tipb.ScalarFuncSig_CastIntAsTime, tipb.ScalarFuncSig_CastRealAsInt, tipb.ScalarFuncSig_CastRealAsDecimal, tipb.ScalarFuncSig_CastRealAsString, tipb.ScalarFuncSig_CastRealAsTime, - tipb.ScalarFuncSig_CastStringAsInt, tipb.ScalarFuncSig_CastStringAsDecimal, tipb.ScalarFuncSig_CastStringAsString, tipb.ScalarFuncSig_CastStringAsTime, + tipb.ScalarFuncSig_CastStringAsInt, tipb.ScalarFuncSig_CastStringAsDecimal, tipb.ScalarFuncSig_CastStringAsString, tipb.ScalarFuncSig_CastStringAsTime, tipb.ScalarFuncSig_CastStringAsReal, tipb.ScalarFuncSig_CastDecimalAsInt, tipb.ScalarFuncSig_CastDecimalAsDecimal, tipb.ScalarFuncSig_CastDecimalAsString, tipb.ScalarFuncSig_CastDecimalAsTime, tipb.ScalarFuncSig_CastTimeAsInt, tipb.ScalarFuncSig_CastTimeAsDecimal, tipb.ScalarFuncSig_CastTimeAsTime, tipb.ScalarFuncSig_CastIntAsReal, tipb.ScalarFuncSig_CastRealAsReal: return true From 06b80b74c5ce7f8cd739df04229b9748882f127b Mon Sep 17 00:00:00 2001 From: Lei Zhao Date: Thu, 3 Jun 2021 21:04:27 +0800 Subject: [PATCH 269/343] store/tikv: new retry logic for RegionRequestSender (#25040) --- session/pessimistic_test.go | 2 + store/gcworker/gc_worker_test.go | 2 +- store/mockstore/unistore/rpc.go | 2 + store/tikv/2pc.go | 61 ++- store/tikv/backoff.go | 3 + store/tikv/client/client_batch.go | 1 + store/tikv/commit.go | 178 +++++---- store/tikv/prewrite.go | 54 ++- store/tikv/region_cache.go | 111 ++++-- store/tikv/region_cache_test.go | 24 +- store/tikv/region_request.go | 355 +++++++++++++++-- store/tikv/region_request_test.go | 425 ++++++++++++++++++++- store/tikv/retry/backoff.go | 37 +- store/tikv/tests/2pc_fail_test.go | 16 + store/tikv/tests/2pc_test.go | 2 + store/tikv/tests/async_commit_fail_test.go | 22 ++ 16 files changed, 1090 insertions(+), 205 deletions(-) diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index f4162c7d1f4d3..6add080091fbf 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -66,11 +66,13 @@ func (s *testPessimisticSuite) SetUpSuite(c *C) { // Set it to 300ms for testing lock resolve. atomic.StoreUint64(&tikv.ManagedLockTTL, 300) tikv.PrewriteMaxBackoff = 500 + tikv.VeryLongMaxBackoff = 500 } func (s *testPessimisticSuite) TearDownSuite(c *C) { s.testSessionSuiteBase.TearDownSuite(c) tikv.PrewriteMaxBackoff = 20000 + tikv.VeryLongMaxBackoff = 600000 } func (s *testPessimisticSuite) TestPessimisticTxn(c *C) { diff --git a/store/gcworker/gc_worker_test.go b/store/gcworker/gc_worker_test.go index 2beef12da62c0..c3ec1c9cbb601 100644 --- a/store/gcworker/gc_worker_test.go +++ b/store/gcworker/gc_worker_test.go @@ -943,7 +943,7 @@ func (s *testGCWorkerSuite) TestResolveLockRangeMeetRegionEnlargeCausedByRegionM mCluster := s.cluster.(*mocktikv.Cluster) mCluster.Merge(s.initRegion.regionID, region2) regionMeta, _ := mCluster.GetRegion(s.initRegion.regionID) - err := s.tikvStore.GetRegionCache().OnRegionEpochNotMatch( + _, err := s.tikvStore.GetRegionCache().OnRegionEpochNotMatch( retry.NewNoopBackoff(context.Background()), &tikv.RPCContext{Region: regionID, Store: &tikv.Store{}}, []*metapb.Region{regionMeta}) diff --git a/store/mockstore/unistore/rpc.go b/store/mockstore/unistore/rpc.go index 1cde24dd5ed94..51c26529d4593 100644 --- a/store/mockstore/unistore/rpc.go +++ b/store/mockstore/unistore/rpc.go @@ -125,6 +125,8 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R failpoint.Inject("rpcPrewriteResult", func(val failpoint.Value) { if val != nil { switch val.(string) { + case "timeout": + failpoint.Return(nil, errors.New("timeout")) case "notLeader": failpoint.Return(&tikvrpc.Response{ Resp: &kvrpcpb.PrewriteResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}}, diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 64d3701eb7efd..66957629e2473 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -44,6 +44,9 @@ import ( zap "go.uber.org/zap" ) +// If the duration of a single request exceeds the slowRequestThreshold, a warning log will be logged. +const slowRequestThreshold = time.Minute + type twoPhaseCommitAction interface { handleSingleBatch(*twoPhaseCommitter, *Backoffer, batchMutations) error tiKVTxnRegionsNumHistogram() prometheus.Observer @@ -520,8 +523,7 @@ func (c *twoPhaseCommitter) groupMutations(bo *Backoffer, mutations CommitterMut logutil.BgLogger().Info("2PC detect large amount of mutations on a single region", zap.Uint64("region", group.region.GetID()), zap.Int("mutations count", group.mutations.Len())) - // Use context.Background, this time should not add up to Backoffer. - if c.store.preSplitRegion(context.Background(), group) { + if c.store.preSplitRegion(bo.GetCtx(), group) { didPreSplit = true } } @@ -547,7 +549,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh switch act := action.(type) { case actionPrewrite: // Do not update regionTxnSize on retries. They are not used when building a PrewriteRequest. - if bo.ErrorsNum() == 0 { + if !act.retry { for _, group := range groups { c.regionTxnSize[group.region.id] = group.mutations.Len() } @@ -612,7 +614,7 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh } // Already spawned a goroutine for async commit transaction. if actionIsCommit && !actionCommit.retry && !c.isAsyncCommit() { - secondaryBo := retry.NewBackofferWithVars(context.Background(), int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) + secondaryBo := retry.NewBackofferWithVars(context.Background(), CommitSecondaryMaxBackoff, c.txn.vars) go func() { if c.sessionID > 0 { failpoint.Inject("beforeCommitSecondaries", func(v failpoint.Value) { @@ -897,6 +899,9 @@ const ( tsoMaxBackoff = 15000 ) +// VeryLongMaxBackoff is the max sleep time of transaction commit. +var VeryLongMaxBackoff = uint64(600000) // 10mins + func (c *twoPhaseCommitter) cleanup(ctx context.Context) { c.cleanWg.Add(1) go func() { @@ -993,13 +998,22 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { c.setOnePC(true) c.hasTriedOnePC = true } + + // TODO(youjiali1995): It's better to use different maxSleep for different operations + // and distinguish permanent errors from temporary errors, for example: + // - If all PDs are down, all requests to PD will fail due to network error. + // The maxSleep should't be very long in this case. + // - If the region isn't found in PD, it's possible the reason is write-stall. + // The maxSleep can be long in this case. + bo := retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&VeryLongMaxBackoff)), c.txn.vars) + // If we want to use async commit or 1PC and also want linearizability across // all nodes, we have to make sure the commit TS of this transaction is greater // than the snapshot TS of all existent readers. So we get a new timestamp // from PD and plus one as our MinCommitTS. if commitTSMayBeCalculated && c.needLinearizability() { failpoint.Inject("getMinCommitTSFromTSO", nil) - latestTS, err := c.store.getTimestampWithRetry(retry.NewBackofferWithVars(ctx, tsoMaxBackoff, c.txn.vars), c.txn.GetScope()) + latestTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) // If we fail to get a timestamp from PD, we just propagate the failure // instead of falling back to the normal 2PC because a normal 2PC will // also be likely to fail due to the same timestamp issue. @@ -1025,9 +1039,9 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { if c.shouldWriteBinlog() { binlogChan = c.binlog.Prewrite(ctx, c.primary()) } - prewriteBo := retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, c.txn.vars) + start := time.Now() - err = c.prewriteMutations(prewriteBo, c.mutations) + err = c.prewriteMutations(bo, c.mutations) if err != nil { // TODO: Now we return an undetermined error as long as one of the prewrite @@ -1044,12 +1058,14 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { commitDetail := c.getDetail() commitDetail.PrewriteTime = time.Since(start) - if prewriteBo.GetTotalSleep() > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(prewriteBo.GetTotalSleep())*int64(time.Millisecond)) + // TODO(youjiali1995): Record the backoff time of the last finished batch. It doesn't make sense to aggregate all batches'. + if bo.GetTotalSleep() > 0 { + atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(bo.GetTotalSleep())*int64(time.Millisecond)) commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, prewriteBo.GetTypes()...) + commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, bo.GetTypes()...) commitDetail.Mu.Unlock() } + if binlogChan != nil { startWaitBinlog := time.Now() binlogWriteResult := <-binlogChan @@ -1184,7 +1200,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { failpoint.Inject("asyncCommitDoNothing", func() { failpoint.Return() }) - commitBo := retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) + commitBo := retry.NewBackofferWithVars(ctx, CommitSecondaryMaxBackoff, c.txn.vars) err := c.commitMutations(commitBo, c.mutations) if err != nil { logutil.Logger(ctx).Warn("2PC async commit failed", zap.Uint64("sessionID", c.sessionID), @@ -1200,7 +1216,8 @@ func (c *twoPhaseCommitter) commitTxn(ctx context.Context, commitDetail *util.Co c.txn.GetMemBuffer().DiscardValues() start := time.Now() - commitBo := retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&CommitMaxBackoff)), c.txn.vars) + // Use the VeryLongMaxBackoff to commit the primary key. + commitBo := retry.NewBackofferWithVars(ctx, int(atomic.LoadUint64(&VeryLongMaxBackoff)), c.txn.vars) err := c.commitMutations(commitBo, c.mutations) commitDetail.CommitTime = time.Since(start) if commitBo.GetTotalSleep() > 0 { @@ -1466,6 +1483,20 @@ type batchMutations struct { mutations CommitterMutations isPrimary bool } + +func (b *batchMutations) relocate(bo *Backoffer, c *RegionCache) (bool, error) { + begin, end := b.mutations.GetKey(0), b.mutations.GetKey(b.mutations.Len()-1) + loc, err := c.LocateKey(bo, begin) + if err != nil { + return false, errors.Trace(err) + } + if !loc.Contains(end) { + return false, nil + } + b.region = loc.Region + return true, nil +} + type batched struct { batches []batchMutations primaryIdx int @@ -1550,7 +1581,7 @@ type batchExecutor struct { func newBatchExecutor(rateLimit int, committer *twoPhaseCommitter, action twoPhaseCommitAction, backoffer *Backoffer) *batchExecutor { return &batchExecutor{rateLimit, nil, committer, - action, backoffer, 1 * time.Millisecond} + action, backoffer, 0} } // initUtils do initialize batchExecutor related policies like rateLimit util @@ -1647,7 +1678,9 @@ func (batchExe *batchExecutor) process(batches []batchMutations) error { } } close(exitCh) - metrics.TiKVTokenWaitDuration.Observe(float64(batchExe.tokenWaitDuration.Nanoseconds())) + if batchExe.tokenWaitDuration > 0 { + metrics.TiKVTokenWaitDuration.Observe(float64(batchExe.tokenWaitDuration.Nanoseconds())) + } return err } diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index 918acc9addcb8..9dd4c3d8b0e04 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -29,6 +29,9 @@ type BackoffConfig = retry.Config // Maximum total sleep time(in ms) for kv/cop commands. const ( gcResolveLockMaxBackoff = 100000 + pdRPCMaxBackoff = 20000 + // CommitSecondaryMaxBackoff is max sleep time of the 'commit' command + CommitSecondaryMaxBackoff = 41000 ) var ( diff --git a/store/tikv/client/client_batch.go b/store/tikv/client/client_batch.go index 10ebced212d6a..5cac70f38b260 100644 --- a/store/tikv/client/client_batch.go +++ b/store/tikv/client/client_batch.go @@ -769,6 +769,7 @@ func sendBatchRequest( zap.String("to", addr), zap.String("cause", ctx.Err().Error())) return nil, errors.Trace(ctx.Err()) case <-timer.C: + atomic.StoreInt32(&entry.canceled, 1) return nil, errors.SuspendStack(errors.Annotate(context.DeadlineExceeded, "wait recvLoop")) } } diff --git a/store/tikv/commit.go b/store/tikv/commit.go index d5f88f4311e1c..73ed6bc0b422a 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -15,6 +15,7 @@ package tikv import ( "encoding/hex" + "time" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" @@ -41,7 +42,7 @@ func (actionCommit) tiKVTxnRegionsNumHistogram() prometheus.Observer { return metrics.TxnRegionsNumHistogramCommit } -func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { +func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) (err error) { keys := batch.mutations.GetKeys() req := tikvrpc.NewRequest(tikvrpc.CmdCommit, &pb.CommitRequest{ StartVersion: c.startTS, @@ -49,97 +50,122 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch CommitVersion: c.commitTS, }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) + tBegin := time.Now() + attempts := 0 + sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) - resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort) - - // If we fail to receive response for the request that commits primary key, it will be undetermined whether this - // transaction has been successfully committed. - // Under this circumstance, we can not declare the commit is complete (may lead to data lost), nor can we throw - // an error (may lead to the duplicated key error when upper level restarts the transaction). Currently the best - // solution is to populate this error and let upper layer drop the connection to the corresponding mysql client. - if batch.isPrimary && sender.rpcError != nil { - c.setUndeterminedErr(errors.Trace(sender.rpcError)) - } + defer func() { + if err != nil { + // If we fail to receive response for the request that commits primary key, it will be undetermined whether this + // transaction has been successfully committed. + // Under this circumstance, we can not declare the commit is complete (may lead to data lost), nor can we throw + // an error (may lead to the duplicated key error when upper level restarts the transaction). Currently the best + // solution is to populate this error and let upper layer drop the connection to the corresponding mysql client. + if batch.isPrimary && sender.rpcError != nil && !c.isAsyncCommit() { + c.setUndeterminedErr(errors.Trace(sender.rpcError)) + } + } + }() + for { + attempts++ + if time.Since(tBegin) > slowRequestThreshold { + logutil.BgLogger().Warn("slow commit request", zap.Uint64("startTS", c.startTS), zap.Stringer("region", &batch.region), zap.Int("attempts", attempts)) + tBegin = time.Now() + } - if err != nil { - return errors.Trace(err) - } - regionErr, err := resp.GetRegionError() - if err != nil { - return errors.Trace(err) - } - if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort) + // Unexpected error occurs, return it. if err != nil { return errors.Trace(err) } - // re-split keys and commit again. - err = c.doActionOnMutations(bo, actionCommit{retry: true}, batch.mutations) - return errors.Trace(err) - } - if resp.Resp == nil { - return errors.Trace(tikverr.ErrBodyMissing) - } - commitResp := resp.Resp.(*pb.CommitResponse) - // Here we can make sure tikv has processed the commit primary key request. So - // we can clean undetermined error. - if batch.isPrimary { - c.setUndeterminedErr(nil) - } - if keyErr := commitResp.GetError(); keyErr != nil { - if rejected := keyErr.GetCommitTsExpired(); rejected != nil { - logutil.Logger(bo.GetCtx()).Info("2PC commitTS rejected by TiKV, retry with a newer commitTS", - zap.Uint64("txnStartTS", c.startTS), - zap.Stringer("info", logutil.Hex(rejected))) - - // Do not retry for a txn which has a too large MinCommitTs - // 3600000 << 18 = 943718400000 - if rejected.MinCommitTs-rejected.AttemptedCommitTs > 943718400000 { - err := errors.Errorf("2PC MinCommitTS is too large, we got MinCommitTS: %d, and AttemptedCommitTS: %d", - rejected.MinCommitTs, rejected.AttemptedCommitTs) - return errors.Trace(err) - } - // Update commit ts and retry. - commitTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) + regionErr, err := resp.GetRegionError() + if err != nil { + return errors.Trace(err) + } + if regionErr != nil { + // For other region error and the fake region error, backoff because + // there's something wrong. + // For the real EpochNotMatch error, don't backoff. + if regionErr.GetEpochNotMatch() == nil || isFakeRegionError(regionErr) { + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + if err != nil { + return errors.Trace(err) + } + } + same, err := batch.relocate(bo, c.store.regionCache) if err != nil { - logutil.Logger(bo.GetCtx()).Warn("2PC get commitTS failed", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) return errors.Trace(err) } + if same { + continue + } + err = c.doActionOnMutations(bo, actionCommit{true}, batch.mutations) + return errors.Trace(err) + } - c.mu.Lock() - c.commitTS = commitTS - c.mu.Unlock() - return c.commitMutations(bo, batch.mutations) + if resp.Resp == nil { + return errors.Trace(tikverr.ErrBodyMissing) } + commitResp := resp.Resp.(*pb.CommitResponse) + if keyErr := commitResp.GetError(); keyErr != nil { + if rejected := keyErr.GetCommitTsExpired(); rejected != nil { + logutil.Logger(bo.GetCtx()).Info("2PC commitTS rejected by TiKV, retry with a newer commitTS", + zap.Uint64("txnStartTS", c.startTS), + zap.Stringer("info", logutil.Hex(rejected))) + + // Do not retry for a txn which has a too large MinCommitTs + // 3600000 << 18 = 943718400000 + if rejected.MinCommitTs-rejected.AttemptedCommitTs > 943718400000 { + err := errors.Errorf("2PC MinCommitTS is too large, we got MinCommitTS: %d, and AttemptedCommitTS: %d", + rejected.MinCommitTs, rejected.AttemptedCommitTs) + return errors.Trace(err) + } - c.mu.RLock() - defer c.mu.RUnlock() - err = extractKeyErr(keyErr) - if c.mu.committed { - // No secondary key could be rolled back after it's primary key is committed. - // There must be a serious bug somewhere. - hexBatchKeys := func(keys [][]byte) []string { - var res []string - for _, k := range keys { - res = append(res, hex.EncodeToString(k)) + // Update commit ts and retry. + commitTS, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) + if err != nil { + logutil.Logger(bo.GetCtx()).Warn("2PC get commitTS failed", + zap.Error(err), + zap.Uint64("txnStartTS", c.startTS)) + return errors.Trace(err) } - return res + + c.mu.Lock() + c.commitTS = commitTS + c.mu.Unlock() + // Update the commitTS of the request and retry. + req.Commit().CommitVersion = commitTS + continue } - logutil.Logger(bo.GetCtx()).Error("2PC failed commit key after primary key committed", + + c.mu.RLock() + defer c.mu.RUnlock() + err = extractKeyErr(keyErr) + if c.mu.committed { + // No secondary key could be rolled back after it's primary key is committed. + // There must be a serious bug somewhere. + hexBatchKeys := func(keys [][]byte) []string { + var res []string + for _, k := range keys { + res = append(res, hex.EncodeToString(k)) + } + return res + } + logutil.Logger(bo.GetCtx()).Error("2PC failed commit key after primary key committed", + zap.Error(err), + zap.Uint64("txnStartTS", c.startTS), + zap.Uint64("commitTS", c.commitTS), + zap.Strings("keys", hexBatchKeys(keys))) + return errors.Trace(err) + } + // The transaction maybe rolled back by concurrent transactions. + logutil.Logger(bo.GetCtx()).Debug("2PC failed commit primary key", zap.Error(err), - zap.Uint64("txnStartTS", c.startTS), - zap.Uint64("commitTS", c.commitTS), - zap.Strings("keys", hexBatchKeys(keys))) - return errors.Trace(err) + zap.Uint64("txnStartTS", c.startTS)) + return err } - // The transaction maybe rolled back by concurrent transactions. - logutil.Logger(bo.GetCtx()).Debug("2PC failed commit primary key", - zap.Error(err), - zap.Uint64("txnStartTS", c.startTS)) - return err + break } c.mu.Lock() diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index 3dc0fd13a1e74..e5a3c0c8419da 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -34,7 +34,7 @@ import ( "go.uber.org/zap" ) -type actionPrewrite struct{} +type actionPrewrite struct{ retry bool } var _ twoPhaseCommitAction = actionPrewrite{} @@ -120,7 +120,7 @@ func (c *twoPhaseCommitter) buildPrewriteRequest(batch batchMutations, txnSize u return tikvrpc.NewRequest(tikvrpc.CmdPrewrite, req, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) } -func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { +func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) (err error) { // WARNING: This function only tries to send a single request to a single region, so it don't // need to unset the `useOnePC` flag when it fails. A special case is that when TiKV returns // regionErr, it's uncertain if the request will be splitted into multiple and sent to multiple @@ -152,38 +152,64 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff txnSize := uint64(c.regionTxnSize[batch.region.id]) // When we retry because of a region miss, we don't know the transaction size. We set the transaction size here // to MaxUint64 to avoid unexpected "resolve lock lite". - if bo.ErrorsNum() > 0 { + if action.retry { txnSize = math.MaxUint64 } + tBegin := time.Now() + attempts := 0 + req := c.buildPrewriteRequest(batch, txnSize) + sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) + defer func() { + if err != nil { + // If we fail to receive response for async commit prewrite, it will be undetermined whether this + // transaction has been successfully committed. + // If prewrite has been cancelled, all ongoing prewrite RPCs will become errors, we needn't set undetermined + // errors. + if (c.isAsyncCommit() || c.isOnePC()) && sender.rpcError != nil && atomic.LoadUint32(&c.prewriteCancelled) == 0 { + c.setUndeterminedErr(errors.Trace(sender.rpcError)) + } + } + }() for { - sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) - resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort) - - // If we fail to receive response for async commit prewrite, it will be undetermined whether this - // transaction has been successfully committed. - // If prewrite has been cancelled, all ongoing prewrite RPCs will become errors, we needn't set undetermined - // errors. - if (c.isAsyncCommit() || c.isOnePC()) && sender.rpcError != nil && atomic.LoadUint32(&c.prewriteCancelled) == 0 { - c.setUndeterminedErr(errors.Trace(sender.rpcError)) + attempts++ + if time.Since(tBegin) > slowRequestThreshold { + logutil.BgLogger().Warn("slow prewrite request", zap.Uint64("startTS", c.startTS), zap.Stringer("region", &batch.region), zap.Int("attempts", attempts)) + tBegin = time.Now() } + resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort) + // Unexpected error occurs, return it if err != nil { return errors.Trace(err) } + regionErr, err := resp.GetRegionError() if err != nil { return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + // For other region error and the fake region error, backoff because + // there's something wrong. + // For the real EpochNotMatch error, don't backoff. + if regionErr.GetEpochNotMatch() == nil || isFakeRegionError(regionErr) { + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + if err != nil { + return errors.Trace(err) + } + } + same, err := batch.relocate(bo, c.store.regionCache) if err != nil { return errors.Trace(err) } - err = c.prewriteMutations(bo, batch.mutations) + if same { + continue + } + err = c.doActionOnMutations(bo, actionPrewrite{true}, batch.mutations) return errors.Trace(err) } + if resp.Resp == nil { return errors.Trace(tikverr.ErrBodyMissing) } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 76da1a2ef8927..6e6303b3ace99 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -303,6 +303,10 @@ func (r *Region) checkNeedReload() bool { return v != updated } +func (r *Region) isValid() bool { + return r != nil && !r.checkNeedReload() && r.checkRegionCacheTTL(time.Now().Unix()) +} + // RegionCache caches Regions loaded from PD. type RegionCache struct { pdClient pd.Client @@ -477,8 +481,6 @@ func (c *RegionCache) GetTiKVRPCContext(bo *Backoffer, id RegionVerID, replicaRe } if cachedRegion.checkNeedReload() { - // TODO: This may cause a fake EpochNotMatch error, and reload the region after a backoff. It's better to reload - // the region directly here. return nil, nil } @@ -517,7 +519,7 @@ func (c *RegionCache) GetTiKVRPCContext(bo *Backoffer, id RegionVerID, replicaRe isLeaderReq = true store, peer, accessIdx, storeIdx = cachedRegion.WorkStorePeer(regionStore) } - addr, err := c.getStoreAddr(bo, cachedRegion, store, storeIdx) + addr, err := c.getStoreAddr(bo, cachedRegion, store) if err != nil { return nil, err } @@ -546,15 +548,14 @@ func (c *RegionCache) GetTiKVRPCContext(bo *Backoffer, id RegionVerID, replicaRe proxyStore *Store proxyAddr string proxyAccessIdx AccessIndex - proxyStoreIdx int ) if c.enableForwarding && isLeaderReq { if atomic.LoadInt32(&store.needForwarding) == 0 { regionStore.unsetProxyStoreIfNeeded(cachedRegion) } else { - proxyStore, proxyAccessIdx, proxyStoreIdx = c.getProxyStore(cachedRegion, store, regionStore, accessIdx) + proxyStore, proxyAccessIdx, _ = c.getProxyStore(cachedRegion, store, regionStore, accessIdx) if proxyStore != nil { - proxyAddr, err = c.getStoreAddr(bo, cachedRegion, proxyStore, proxyStoreIdx) + proxyAddr, err = c.getStoreAddr(bo, cachedRegion, proxyStore) if err != nil { return nil, err } @@ -637,7 +638,7 @@ func (c *RegionCache) GetTiFlashRPCContext(bo *Backoffer, id RegionVerID, loadBa for i := 0; i < regionStore.accessStoreNum(TiFlashOnly); i++ { accessIdx := AccessIndex((sIdx + i) % regionStore.accessStoreNum(TiFlashOnly)) storeIdx, store := regionStore.accessStore(TiFlashOnly, accessIdx) - addr, err := c.getStoreAddr(bo, cachedRegion, store, storeIdx) + addr, err := c.getStoreAddr(bo, cachedRegion, store) if err != nil { return nil, err } @@ -1085,16 +1086,15 @@ func (c *RegionCache) InvalidateCachedRegionWithReason(id RegionVerID, reason In } // UpdateLeader update some region cache with newer leader info. -func (c *RegionCache) UpdateLeader(regionID RegionVerID, leaderStoreID uint64, currentPeerIdx AccessIndex) { +func (c *RegionCache) UpdateLeader(regionID RegionVerID, leader *metapb.Peer, currentPeerIdx AccessIndex) { r := c.GetCachedRegionWithRLock(regionID) if r == nil { logutil.BgLogger().Debug("regionCache: cannot find region when updating leader", - zap.Uint64("regionID", regionID.GetID()), - zap.Uint64("leaderStoreID", leaderStoreID)) + zap.Uint64("regionID", regionID.GetID())) return } - if leaderStoreID == 0 { + if leader == nil { rs := r.getStore() rs.switchNextTiKVPeer(r, currentPeerIdx) logutil.BgLogger().Info("switch region peer to next due to NotLeader with NULL leader", @@ -1103,17 +1103,17 @@ func (c *RegionCache) UpdateLeader(regionID RegionVerID, leaderStoreID uint64, c return } - if !c.switchWorkLeaderToPeer(r, leaderStoreID) { + if !c.switchWorkLeaderToPeer(r, leader) { logutil.BgLogger().Info("invalidate region cache due to cannot find peer when updating leader", zap.Uint64("regionID", regionID.GetID()), zap.Int("currIdx", int(currentPeerIdx)), - zap.Uint64("leaderStoreID", leaderStoreID)) + zap.Uint64("leaderStoreID", leader.GetStoreId())) r.invalidate(StoreNotFound) } else { logutil.BgLogger().Info("switch region leader to specific leader due to kv return NotLeader", zap.Uint64("regionID", regionID.GetID()), zap.Int("currIdx", int(currentPeerIdx)), - zap.Uint64("leaderStoreID", leaderStoreID)) + zap.Uint64("leaderStoreID", leader.GetStoreId())) } } @@ -1134,6 +1134,8 @@ func (c *RegionCache) insertRegionToCache(cachedRegion *Region) { store := cachedRegion.getStore() oldRegion := old.(*btreeItem).cachedRegion oldRegionStore := oldRegion.getStore() + // TODO(youjiali1995): remove this because the new retry logic can handle this issue. + // // Joint consensus is enabled in v5.0, which is possible to make a leader step down as a learner during a conf change. // And if hibernate region is enabled, after the leader step down, there can be a long time that there is no leader // in the region and the leader info in PD is stale until requests are sent to followers or hibernate timeout. @@ -1142,6 +1144,8 @@ func (c *RegionCache) insertRegionToCache(cachedRegion *Region) { if InvalidReason(atomic.LoadInt32((*int32)(&oldRegion.invalidReason))) == NoLeader { store.workTiKVIdx = (oldRegionStore.workTiKVIdx + 1) % AccessIndex(store.accessStoreNum(TiKVOnly)) } + // Invalidate the old region in case it's not invalidated and some requests try with the stale region information. + oldRegion.invalidate(Other) // Don't refresh TiFlash work idx for region. Otherwise, it will always goto a invalid store which // is under transferring regions. store.workTiFlashIdx = atomic.LoadInt32(&oldRegionStore.workTiFlashIdx) @@ -1312,7 +1316,7 @@ func (c *RegionCache) loadRegion(bo *Backoffer, key []byte, isEndKey bool) (*Reg return nil, err } if reg.Leader != nil { - c.switchWorkLeaderToPeer(region, reg.Leader.StoreId) + c.switchWorkLeaderToPeer(region, reg.Leader) } return region, nil } @@ -1357,7 +1361,7 @@ func (c *RegionCache) loadRegionByID(bo *Backoffer, regionID uint64) (*Region, e return nil, err } if reg.Leader != nil { - c.switchWorkLeaderToPeer(region, reg.Leader.GetStoreId()) + c.switchWorkLeaderToPeer(region, reg.Leader) } return region, nil } @@ -1409,8 +1413,8 @@ func (c *RegionCache) scanRegions(bo *Backoffer, startKey, endKey []byte, limit } leader := r.Leader // Leader id = 0 indicates no leader. - if leader.GetId() != 0 { - c.switchWorkLeaderToPeer(region, leader.GetStoreId()) + if leader != nil && leader.GetId() != 0 { + c.switchWorkLeaderToPeer(region, leader) regions = append(regions, region) } } @@ -1433,7 +1437,7 @@ func (c *RegionCache) GetCachedRegionWithRLock(regionID RegionVerID) (r *Region) return } -func (c *RegionCache) getStoreAddr(bo *Backoffer, region *Region, store *Store, storeIdx int) (addr string, err error) { +func (c *RegionCache) getStoreAddr(bo *Backoffer, region *Region, store *Store) (addr string, err error) { state := store.getResolveState() switch state { case resolved, needCheck: @@ -1443,7 +1447,7 @@ func (c *RegionCache) getStoreAddr(bo *Backoffer, region *Region, store *Store, addr, err = store.initResolve(bo, c) return case deleted: - addr = c.changeToActiveStore(region, store, storeIdx) + addr = c.changeToActiveStore(region, store) return case tombstone: return "", nil @@ -1496,7 +1500,7 @@ func (c *RegionCache) getProxyStore(region *Region, store *Store, rs *RegionStor // changeToActiveStore replace the deleted store in the region by an up-to-date store in the stores map. // The order is guaranteed by reResolve() which adds the new store before marking old store deleted. -func (c *RegionCache) changeToActiveStore(region *Region, store *Store, storeIdx int) (addr string) { +func (c *RegionCache) changeToActiveStore(region *Region, store *Store) (addr string) { c.storeMu.RLock() store = c.storeMu.stores[store.storeID] c.storeMu.RUnlock() @@ -1504,8 +1508,8 @@ func (c *RegionCache) changeToActiveStore(region *Region, store *Store, storeIdx oldRegionStore := region.getStore() newRegionStore := oldRegionStore.clone() newRegionStore.stores = make([]*Store, 0, len(oldRegionStore.stores)) - for i, s := range oldRegionStore.stores { - if i == storeIdx { + for _, s := range oldRegionStore.stores { + if s.storeID == store.storeID { newRegionStore.stores = append(newRegionStore.stores, store) } else { newRegionStore.stores = append(newRegionStore.stores, s) @@ -1546,7 +1550,12 @@ func (c *RegionCache) getStoresByLabels(labels []*metapb.StoreLabel) []*Store { } // OnRegionEpochNotMatch removes the old region and inserts new regions into the cache. -func (c *RegionCache) OnRegionEpochNotMatch(bo *Backoffer, ctx *RPCContext, currentRegions []*metapb.Region) error { +func (c *RegionCache) OnRegionEpochNotMatch(bo *Backoffer, ctx *RPCContext, currentRegions []*metapb.Region) (bool, error) { + if len(currentRegions) == 0 { + c.InvalidateCachedRegionWithReason(ctx.Region, EpochNotMatch) + return false, nil + } + // Find whether the region epoch in `ctx` is ahead of TiKV's. If so, backoff. for _, meta := range currentRegions { if meta.GetId() == ctx.Region.id && @@ -1554,45 +1563,49 @@ func (c *RegionCache) OnRegionEpochNotMatch(bo *Backoffer, ctx *RPCContext, curr meta.GetRegionEpoch().GetVersion() < ctx.Region.ver) { err := errors.Errorf("region epoch is ahead of tikv. rpc ctx: %+v, currentRegions: %+v", ctx, currentRegions) logutil.BgLogger().Info("region epoch is ahead of tikv", zap.Error(err)) - return bo.Backoff(retry.BoRegionMiss, err) + return true, bo.Backoff(retry.BoRegionMiss, err) } } - c.mu.Lock() - defer c.mu.Unlock() needInvalidateOld := true + newRegions := make([]*Region, 0, len(currentRegions)) // If the region epoch is not ahead of TiKV's, replace region meta in region cache. for _, meta := range currentRegions { if _, ok := c.pdClient.(*CodecPDClient); ok { var err error if meta, err = decodeRegionMetaKeyWithShallowCopy(meta); err != nil { - return errors.Errorf("newRegion's range key is not encoded: %v, %v", meta, err) + return false, errors.Errorf("newRegion's range key is not encoded: %v, %v", meta, err) } } region := &Region{meta: meta} err := region.init(bo, c) if err != nil { - return err + return false, err } - var initLeader uint64 + var initLeaderStoreID uint64 if ctx.Store.storeType == tikvrpc.TiFlash { - initLeader = region.findElectableStoreID() + initLeaderStoreID = region.findElectableStoreID() } else { - initLeader = ctx.Store.storeID + initLeaderStoreID = ctx.Store.storeID } - c.switchWorkLeaderToPeer(region, initLeader) - c.insertRegionToCache(region) + c.switchWorkLeaderToPeer(region, region.getPeerOnStore(initLeaderStoreID)) + newRegions = append(newRegions, region) if ctx.Region == region.VerID() { needInvalidateOld = false } } + c.mu.Lock() + for _, region := range newRegions { + c.insertRegionToCache(region) + } if needInvalidateOld { cachedRegion, ok := c.mu.regions[ctx.Region] if ok { cachedRegion.invalidate(EpochNotMatch) } } - return nil + c.mu.Unlock() + return false, nil } // PDClient returns the pd.Client in RegionCache. @@ -1745,9 +1758,12 @@ func (r *Region) EndKey() []byte { } // switchWorkLeaderToPeer switches current store to the one on specific store. It returns -// false if no peer matches the storeID. -func (c *RegionCache) switchWorkLeaderToPeer(r *Region, targetStoreID uint64) (found bool) { - globalStoreIdx, found := c.getPeerStoreIndex(r, targetStoreID) +// false if no peer matches the peer. +func (c *RegionCache) switchWorkLeaderToPeer(r *Region, peer *metapb.Peer) (found bool) { + globalStoreIdx, found := c.getPeerStoreIndex(r, peer) + if !found { + return + } retry: // switch to new leader. oldRegionStore := r.getStore() @@ -1850,12 +1866,21 @@ func (r *Region) findElectableStoreID() uint64 { return 0 } -func (c *RegionCache) getPeerStoreIndex(r *Region, id uint64) (idx int, found bool) { - if len(r.meta.Peers) == 0 { +func (r *Region) getPeerOnStore(storeID uint64) *metapb.Peer { + for _, p := range r.meta.Peers { + if p.StoreId == storeID { + return p + } + } + return nil +} + +func (c *RegionCache) getPeerStoreIndex(r *Region, peer *metapb.Peer) (idx int, found bool) { + if len(r.meta.Peers) == 0 || peer == nil { return } for i, p := range r.meta.Peers { - if p.GetStoreId() == id { + if isSamePeer(p, peer) { idx = i found = true return @@ -2295,3 +2320,7 @@ func createKVHealthClient(ctx context.Context, addr string) (*grpc.ClientConn, h cli := healthpb.NewHealthClient(conn) return conn, cli, nil } + +func isSamePeer(lhs *metapb.Peer, rhs *metapb.Peer) bool { + return lhs == rhs || (lhs.GetId() == rhs.GetId() && lhs.GetStoreId() == rhs.GetStoreId()) +} diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index c7fe03638ee5a..5ad0721c75c00 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -359,7 +359,7 @@ func (s *testRegionCacheSuite) TestUpdateLeader(c *C) { loc, err := s.cache.LocateKey(s.bo, []byte("a")) c.Assert(err, IsNil) // tikv-server reports `NotLeader` - s.cache.UpdateLeader(loc.Region, s.store2, 0) + s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer2, StoreId: s.store2}, 0) r := s.getRegion(c, []byte("a")) c.Assert(r, NotNil) @@ -384,7 +384,7 @@ func (s *testRegionCacheSuite) TestUpdateLeader2(c *C) { s.cluster.AddStore(store3, s.storeAddr(store3)) s.cluster.AddPeer(s.region1, store3, peer3) // tikv-server reports `NotLeader` - s.cache.UpdateLeader(loc.Region, store3, 0) + s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: peer3, StoreId: store3}, 0) // Store3 does not exist in cache, causes a reload from PD. r := s.getRegion(c, []byte("a")) @@ -408,7 +408,7 @@ func (s *testRegionCacheSuite) TestUpdateLeader2(c *C) { // tikv-server notifies new leader to pd-server. s.cluster.ChangeLeader(s.region1, peer3) // tikv-server reports `NotLeader` again. - s.cache.UpdateLeader(r.VerID(), store3, 0) + s.cache.UpdateLeader(r.VerID(), &metapb.Peer{Id: peer3, StoreId: store3}, 0) r = s.getRegion(c, []byte("a")) c.Assert(r, NotNil) c.Assert(r.GetID(), Equals, s.region1) @@ -443,7 +443,7 @@ func (s *testRegionCacheSuite) TestUpdateLeader3(c *C) { // tikv-server notifies new leader to pd-server. s.cluster.ChangeLeader(s.region1, peer3) // tikv-server reports `NotLeader`(store2 is the leader) - s.cache.UpdateLeader(loc.Region, s.store2, 0) + s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer2, StoreId: s.store2}, 0) // Store2 does not exist any more, causes a reload from PD. r := s.getRegion(c, []byte("a")) @@ -458,7 +458,7 @@ func (s *testRegionCacheSuite) TestUpdateLeader3(c *C) { c.Assert(ctx.Addr, Equals, "store2") s.cache.OnSendFail(retry.NewNoopBackoff(context.Background()), ctx, false, errors.New("send fail")) s.cache.checkAndResolve(nil, func(*Store) bool { return true }) - s.cache.UpdateLeader(loc.Region, s.store2, 0) + s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer2, StoreId: s.store2}, 0) addr := s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0) c.Assert(addr, Equals, "") addr = s.getAddr(c, []byte("a"), kv.ReplicaReadLeader, 0) @@ -527,7 +527,7 @@ func (s *testRegionCacheSuite) TestSendFailedButLeaderNotChange(c *C) { c.Assert(ctxFollower1.Peer.Id, Not(Equals), ctxFollower2.Peer.Id) // access 1 it will return NotLeader, leader back to 2 again - s.cache.UpdateLeader(loc.Region, s.store2, ctx.AccessIdx) + s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer2, StoreId: s.store2}, ctx.AccessIdx) ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) c.Assert(err, IsNil) c.Assert(ctx.Peer.Id, Equals, s.peer2) @@ -608,7 +608,7 @@ func (s *testRegionCacheSuite) TestSendFailedInHibernateRegion(c *C) { c.Assert(ctxFollower1.Peer.Id, Not(Equals), ctxFollower2.Peer.Id) // access 2, it's in hibernate and return 0 leader, so switch to 3 - s.cache.UpdateLeader(loc.Region, 0, ctx.AccessIdx) + s.cache.UpdateLeader(loc.Region, nil, ctx.AccessIdx) ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) c.Assert(err, IsNil) c.Assert(ctx.Peer.Id, Equals, peer3) @@ -633,7 +633,7 @@ func (s *testRegionCacheSuite) TestSendFailedInHibernateRegion(c *C) { // again peer back to 1 ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) c.Assert(err, IsNil) - s.cache.UpdateLeader(loc.Region, 0, ctx.AccessIdx) + s.cache.UpdateLeader(loc.Region, nil, ctx.AccessIdx) ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) c.Assert(err, IsNil) c.Assert(ctx.Peer.Id, Equals, s.peer1) @@ -817,7 +817,7 @@ func (s *testRegionCacheSuite) TestSendFailedInMultipleNode(c *C) { c.Assert(ctxFollower1.Peer.Id, Equals, ctxFollower2.Peer.Id) // 3 can be access, so switch to 1 - s.cache.UpdateLeader(loc.Region, s.store1, ctx.AccessIdx) + s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer1, StoreId: s.store1}, ctx.AccessIdx) ctx, err = s.cache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadLeader, 0) c.Assert(err, IsNil) c.Assert(ctx.Peer.Id, Equals, s.peer1) @@ -1012,9 +1012,9 @@ func (s *testRegionCacheSuite) TestRegionEpochAheadOfTiKV(c *C) { bo := NewBackofferWithVars(context.Background(), 2000000, nil) - err := cache.OnRegionEpochNotMatch(bo, &RPCContext{Region: region.VerID()}, []*metapb.Region{&r1}) + _, err := cache.OnRegionEpochNotMatch(bo, &RPCContext{Region: region.VerID()}, []*metapb.Region{&r1}) c.Assert(err, IsNil) - err = cache.OnRegionEpochNotMatch(bo, &RPCContext{Region: region.VerID()}, []*metapb.Region{&r2}) + _, err = cache.OnRegionEpochNotMatch(bo, &RPCContext{Region: region.VerID()}, []*metapb.Region{&r2}) c.Assert(err, IsNil) c.Assert(bo.ErrorsNum(), Equals, 2) } @@ -1446,7 +1446,7 @@ func (s *testRegionCacheSuite) TestPeersLenChange(c *C) { // 2 peers [peer1, peer2] and let peer2 become leader loc, err := s.cache.LocateKey(s.bo, []byte("a")) c.Assert(err, IsNil) - s.cache.UpdateLeader(loc.Region, s.store2, 0) + s.cache.UpdateLeader(loc.Region, &metapb.Peer{Id: s.peer2, StoreId: s.store2}, 0) // current leader is peer2 in [peer1, peer2] loc, err = s.cache.LocateKey(s.bo, []byte("a")) diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 9f180c26269f2..3efb3f4a5ec06 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/metapb" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" @@ -70,16 +71,20 @@ func LoadShuttingDown() uint32 { // merge, or region balance, tikv server may not able to process request and // send back a RegionError. // RegionRequestSender takes care of errors that does not relevant to region -// range, such as 'I/O timeout', 'NotLeader', and 'ServerIsBusy'. For other -// errors, since region range have changed, the request may need to split, so we -// simply return the error to caller. +// range, such as 'I/O timeout', 'NotLeader', and 'ServerIsBusy'. If fails to +// send the request to all replicas, a fake rregion error may be returned. +// Caller which receives the error should retry the request. +// +// For other region errors, since region range have changed, the request may need to +// split, so we simply return the error to caller. type RegionRequestSender struct { - regionCache *RegionCache - client Client - storeAddr string - rpcError error - failStoreIDs map[uint64]struct{} - failProxyStoreIDs map[uint64]struct{} + regionCache *RegionCache + client Client + storeAddr string + rpcError error + leaderReplicaSelector *replicaSelector + failStoreIDs map[uint64]struct{} + failProxyStoreIDs map[uint64]struct{} RegionRequestRuntimeStats } @@ -194,12 +199,188 @@ func (s *RegionRequestSender) SetRPCError(err error) { s.rpcError = err } -// SendReq sends a request to tikv server. +// SendReq sends a request to tikv server. If fails to send the request to all replicas, +// a fake region error may be returned. Caller which receives the error should retry the request. func (s *RegionRequestSender) SendReq(bo *Backoffer, req *tikvrpc.Request, regionID RegionVerID, timeout time.Duration) (*tikvrpc.Response, error) { resp, _, err := s.SendReqCtx(bo, req, regionID, timeout, tikvrpc.TiKV) return resp, err } +type replica struct { + store *Store + peer *metapb.Peer + epoch uint32 + attempts int +} + +type replicaSelector struct { + regionCache *RegionCache + region *Region + // replicas contains all TiKV replicas for now and the leader is at the + // head of the slice. + replicas []*replica + // nextReplicaIdx points to the candidate for the next attempt. + nextReplicaIdx int +} + +func newReplicaSelector(regionCache *RegionCache, regionID RegionVerID) (*replicaSelector, error) { + cachedRegion := regionCache.GetCachedRegionWithRLock(regionID) + if cachedRegion == nil || !cachedRegion.isValid() { + return nil, nil + } + regionStore := cachedRegion.getStore() + replicas := make([]*replica, 0, regionStore.accessStoreNum(TiKVOnly)) + for _, storeIdx := range regionStore.accessIndex[TiKVOnly] { + replicas = append(replicas, &replica{ + store: regionStore.stores[storeIdx], + peer: cachedRegion.meta.Peers[storeIdx], + epoch: regionStore.storeEpochs[storeIdx], + attempts: 0, + }) + } + // Move the leader to the first slot. + replicas[regionStore.workTiKVIdx], replicas[0] = replicas[0], replicas[regionStore.workTiKVIdx] + return &replicaSelector{ + regionCache, + cachedRegion, + replicas, + 0, + }, nil +} + +// isExhausted returns true if runs out of all replicas. +func (s *replicaSelector) isExhausted() bool { + return s.nextReplicaIdx >= len(s.replicas) +} + +func (s *replicaSelector) nextReplica() *replica { + if s.isExhausted() { + return nil + } + return s.replicas[s.nextReplicaIdx] +} + +const maxReplicaAttempt = 10 + +// next creates the RPCContext of the current candidate replica. +// It returns a SendError if runs out of all replicas of the cached region is invalidated. +func (s *replicaSelector) next(bo *Backoffer) (*RPCContext, error) { + for { + if !s.region.isValid() { + return nil, nil + } + if s.isExhausted() { + s.invalidateRegion() + return nil, nil + } + replica := s.replicas[s.nextReplicaIdx] + s.nextReplicaIdx++ + + // Limit the max attempts of each replica to prevent endless retry. + if replica.attempts >= maxReplicaAttempt { + continue + } + replica.attempts++ + + storeFailEpoch := atomic.LoadUint32(&replica.store.epoch) + if storeFailEpoch != replica.epoch { + // TODO(youjiali1995): Is it necessary to invalidate the region? + s.invalidateRegion() + return nil, nil + } + addr, err := s.regionCache.getStoreAddr(bo, s.region, replica.store) + if err == nil && len(addr) != 0 { + return &RPCContext{ + Region: s.region.VerID(), + Meta: s.region.meta, + Peer: replica.peer, + Store: replica.store, + Addr: addr, + AccessMode: TiKVOnly, + TiKVNum: len(s.replicas), + }, nil + } + } +} + +func (s *replicaSelector) onSendFailure(bo *Backoffer, err error) { + metrics.RegionCacheCounterWithSendFail.Inc() + replica := s.replicas[s.nextReplicaIdx-1] + if replica.store.requestLiveness(bo, s.regionCache) == reachable { + s.rewind() + return + } + + store := replica.store + // invalidate regions in store. + if atomic.CompareAndSwapUint32(&store.epoch, replica.epoch, replica.epoch+1) { + logutil.BgLogger().Info("mark store's regions need be refill", zap.Uint64("id", store.storeID), zap.String("addr", store.addr), zap.Error(err)) + metrics.RegionCacheCounterWithInvalidateStoreRegionsOK.Inc() + // schedule a store addr resolve. + store.markNeedCheck(s.regionCache.notifyCheckCh) + } + // TODO(youjiali1995): It's not necessary, but some tests depend on it and it's not easy to fix. + if s.isExhausted() { + s.region.scheduleReload() + } +} + +// OnSendSuccess updates the leader of the cached region since the replicaSelector +// is only used for leader request. It's called when the request is sent to the +// replica successfully. +func (s *replicaSelector) OnSendSuccess() { + // The successful replica is not at the head of replicas which means it's not the + // leader in the cached region, so update leader. + if s.nextReplicaIdx-1 != 0 { + leader := s.replicas[s.nextReplicaIdx-1].peer + if !s.regionCache.switchWorkLeaderToPeer(s.region, leader) { + panic("the store must exist") + } + } +} + +func (s *replicaSelector) rewind() { + s.nextReplicaIdx-- +} + +// updateLeader updates the leader of the cached region. +// If the leader peer isn't found in the region, the region will be invalidated. +func (s *replicaSelector) updateLeader(leader *metapb.Peer) { + if leader == nil { + return + } + for i, replica := range s.replicas { + if isSamePeer(replica.peer, leader) { + if i < s.nextReplicaIdx { + s.nextReplicaIdx-- + } + // Move the leader replica to the front of candidates. + s.replicas[i], s.replicas[s.nextReplicaIdx] = s.replicas[s.nextReplicaIdx], s.replicas[i] + if s.replicas[s.nextReplicaIdx].attempts == maxReplicaAttempt { + // Give the replica one more chance and because the current replica is skipped, it + // won't result in infinite retry. + s.replicas[s.nextReplicaIdx].attempts = maxReplicaAttempt - 1 + } + // Update the workTiKVIdx so that following requests can be sent to the leader immediately. + if !s.regionCache.switchWorkLeaderToPeer(s.region, leader) { + panic("the store must exist") + } + logutil.BgLogger().Debug("switch region leader to specific leader due to kv return NotLeader", + zap.Uint64("regionID", s.region.GetID()), + zap.Uint64("leaderStoreID", leader.GetStoreId())) + return + } + } + // Invalidate the region since the new leader is not in the cached version. + s.region.invalidate(StoreNotFound) +} + +func (s *replicaSelector) invalidateRegion() { + if s.region != nil { + s.region.invalidate(Other) + } +} + func (s *RegionRequestSender) getRPCContext( bo *Backoffer, req *tikvrpc.Request, @@ -209,6 +390,20 @@ func (s *RegionRequestSender) getRPCContext( ) (*RPCContext, error) { switch et { case tikvrpc.TiKV: + // Now only requests sent to the replica leader will use the replica selector to get + // the RPC context. + // TODO(youjiali1995): make all requests use the replica selector. + if !s.regionCache.enableForwarding && req.ReplicaReadType == kv.ReplicaReadLeader { + if s.leaderReplicaSelector == nil { + selector, err := newReplicaSelector(s.regionCache, regionID) + if selector == nil || err != nil { + return nil, err + } + s.leaderReplicaSelector = selector + } + return s.leaderReplicaSelector.next(bo) + } + var seed uint32 if req.ReplicaReadSeed != nil { seed = *req.ReplicaReadSeed @@ -223,6 +418,16 @@ func (s *RegionRequestSender) getRPCContext( } } +func (s *RegionRequestSender) reset() { + s.leaderReplicaSelector = nil + s.failStoreIDs = nil + s.failProxyStoreIDs = nil +} + +func isFakeRegionError(err *errorpb.Error) bool { + return err != nil && err.GetEpochNotMatch() != nil && len(err.GetEpochNotMatch().CurrentRegions) == 0 +} + // SendReqCtx sends a request to tikv server and return response and RPCCtx of this RPC. func (s *RegionRequestSender) SendReqCtx( bo *Backoffer, @@ -239,8 +444,6 @@ func (s *RegionRequestSender) SendReqCtx( if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("regionRequest.SendReqCtx", opentracing.ChildOf(span.Context())) defer span1.Finish() - // TODO(MyonKeminta): Make sure trace works without cloning the backoffer. - // bo = bo.Clone() bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } @@ -277,10 +480,11 @@ func (s *RegionRequestSender) SendReqCtx( req.Context.MaxExecutionDurationMs = uint64(timeout.Milliseconds()) } + s.reset() tryTimes := 0 for { - if (tryTimes > 0) && (tryTimes%1000 == 0) { - logutil.Logger(bo.GetCtx()).Warn("retry get ", zap.Uint64("region = ", regionID.GetID()), zap.Int("times = ", tryTimes)) + if (tryTimes > 0) && (tryTimes%100 == 0) { + logutil.Logger(bo.GetCtx()).Warn("retry", zap.Uint64("region", regionID.GetID()), zap.Int("times", tryTimes)) } rpcCtx, err = s.getRPCContext(bo, req, regionID, et, opts...) @@ -299,6 +503,7 @@ func (s *RegionRequestSender) SendReqCtx( } }) if rpcCtx == nil { + // TODO(youjiali1995): remove it when using the replica selector for all requests. // If the region is not found in cache, it must be out // of date and already be cleaned up. We can skip the // RPC by returning RegionError directly. @@ -353,6 +558,12 @@ func (s *RegionRequestSender) SendReqCtx( tryTimes++ continue } + } else { + // Clear the RPC Error since the request is evaluated successfully on a store. + s.rpcError = nil + if s.leaderReplicaSelector != nil { + s.leaderReplicaSelector.OnSendSuccess() + } } return resp, rpcCtx, nil } @@ -569,8 +780,6 @@ func (s *RegionRequestSender) onSendFail(bo *Backoffer, ctx *RPCContext, err err if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("regionRequest.onSendFail", opentracing.ChildOf(span.Context())) defer span1.Finish() - // TODO(MyonKeminta): Make sure trace works without cloning the backoffer. - // bo = bo.Clone() bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } // If it failed because the context is cancelled by ourself, don't retry. @@ -592,7 +801,11 @@ func (s *RegionRequestSender) onSendFail(bo *Backoffer, ctx *RPCContext, err err } if ctx.Meta != nil { - s.regionCache.OnSendFail(bo, ctx, s.NeedReloadRegion(ctx), err) + if s.leaderReplicaSelector != nil { + s.leaderReplicaSelector.onSendFailure(bo, err) + } else { + s.regionCache.OnSendFail(bo, ctx, s.NeedReloadRegion(ctx), err) + } } // Retry on send request failure when it's not canceled. @@ -658,8 +871,6 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("tikv.onRegionError", opentracing.ChildOf(span.Context())) defer span1.Finish() - // TODO(MyonKeminta): Make sure trace works without cloning the backoffer. - // bo = bo.Clone() bo.SetCtx(opentracing.ContextWithSpan(bo.GetCtx(), span1)) } // Stale Read request will retry the leader or next peer on error, @@ -676,7 +887,18 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req zap.String("notLeader", notLeader.String()), zap.String("ctx", ctx.String())) - if notLeader.GetLeader() == nil { + if s.leaderReplicaSelector != nil { + leader := notLeader.GetLeader() + if leader == nil { + // The region may be during transferring leader. + if err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("no leader, ctx: %v", ctx)); err != nil { + return false, errors.Trace(err) + } + } else { + s.leaderReplicaSelector.updateLeader(notLeader.GetLeader()) + } + return true, nil + } else if notLeader.GetLeader() == nil { // The peer doesn't know who is the current leader. Generally it's because // the Raft group is in an election, but it's possible that the peer is // isolated and removed from the Raft group. So it's necessary to reload @@ -685,12 +907,12 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req if err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("not leader: %v, ctx: %v", notLeader, ctx)); err != nil { return false, errors.Trace(err) } + return false, nil } else { // don't backoff if a new leader is returned. - s.regionCache.UpdateLeader(ctx.Region, notLeader.GetLeader().GetStoreId(), ctx.AccessIdx) + s.regionCache.UpdateLeader(ctx.Region, notLeader.GetLeader(), ctx.AccessIdx) + return true, nil } - - return true, nil } if storeNotMatch := regionErr.GetStoreNotMatch(); storeNotMatch != nil { @@ -700,7 +922,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req zap.Stringer("ctx", ctx)) ctx.Store.markNeedCheck(s.regionCache.notifyCheckCh) s.regionCache.InvalidateCachedRegion(ctx.Region) - return true, nil + return false, nil } seed := req.GetReplicaReadSeed() @@ -711,8 +933,11 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req if seed != nil { *seed = *seed + 1 } - err = s.regionCache.OnRegionEpochNotMatch(bo, ctx, epochNotMatch.CurrentRegions) - return false, errors.Trace(err) + retry, err := s.regionCache.OnRegionEpochNotMatch(bo, ctx, epochNotMatch.CurrentRegions) + if !retry && s.leaderReplicaSelector != nil { + s.leaderReplicaSelector.invalidateRegion() + } + return retry, errors.Trace(err) } if regionErr.GetServerIsBusy() != nil { logutil.BgLogger().Warn("tikv reports `ServerIsBusy` retry later", @@ -726,13 +951,21 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req if err != nil { return false, errors.Trace(err) } + if s.leaderReplicaSelector != nil { + s.leaderReplicaSelector.rewind() + } return true, nil } if regionErr.GetStaleCommand() != nil { logutil.BgLogger().Debug("tikv reports `StaleCommand`", zap.Stringer("ctx", ctx)) - err = bo.Backoff(retry.BoStaleCmd, errors.Errorf("stale command, ctx: %v", ctx)) - if err != nil { - return false, errors.Trace(err) + if s.leaderReplicaSelector != nil { + // Needn't backoff because stale command indicates the command is sent to the old leader. + // The new leader should be elected soon and the leaderReplicaSelector will try the next peer. + } else { + err = bo.Backoff(retry.BoStaleCmd, errors.Errorf("stale command, ctx: %v", ctx)) + if err != nil { + return false, errors.Trace(err) + } } return true, nil } @@ -748,8 +981,18 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req zap.Uint64("region-id", regionErr.GetDataIsNotReady().GetRegionId()), zap.Uint64("safe-ts", regionErr.GetDataIsNotReady().GetSafeTs()), zap.Stringer("ctx", ctx)) + // Don't backoff if it's a replica-read. if seed != nil { *seed = *seed + 1 + } else { + // The region is merging or splitting. + err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("data is not ready, ctx: %v", ctx)) + if err != nil { + return false, errors.Trace(err) + } + if s.leaderReplicaSelector != nil { + s.leaderReplicaSelector.rewind() + } } return true, nil } @@ -764,28 +1007,66 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req } return true, nil } - if regionErr.GetRegionNotFound() != nil && seed != nil { - logutil.BgLogger().Debug("tikv reports `RegionNotFound` in follow-reader", - zap.Stringer("ctx", ctx), zap.Uint32("seed", *seed)) - *seed = *seed + 1 - } if regionErr.GetMaxTimestampNotSynced() != nil { - logutil.BgLogger().Warn("tikv reports `MaxTimestampNotSynced`", zap.Stringer("ctx", ctx)) + logutil.BgLogger().Debug("tikv reports `MaxTimestampNotSynced`", zap.Stringer("ctx", ctx)) err = bo.Backoff(retry.BoMaxTsNotSynced, errors.Errorf("max timestamp not synced, ctx: %v", ctx)) if err != nil { return false, errors.Trace(err) } + if s.leaderReplicaSelector != nil { + s.leaderReplicaSelector.rewind() + } return true, nil } - // For other errors, we only drop cache here. - // Because caller may need to re-split the request. + if regionErr.GetReadIndexNotReady() != nil { + logutil.BgLogger().Debug("tikv reports `ReadIndexNotReady`", zap.Stringer("ctx", ctx)) + // The region is merging or splitting. + err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("read index not ready, ctx: %v", ctx)) + if err != nil { + return false, errors.Trace(err) + } + if s.leaderReplicaSelector != nil { + s.leaderReplicaSelector.rewind() + } + return true, nil + } + if regionErr.GetProposalInMergingMode() != nil { + logutil.BgLogger().Debug("tikv reports `ProposalInMergingMode`", zap.Stringer("ctx", ctx)) + // The region is merging. + err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("region is merging, ctx: %v", ctx)) + if err != nil { + return false, errors.Trace(err) + } + if s.leaderReplicaSelector != nil { + s.leaderReplicaSelector.rewind() + } + return true, nil + } + if regionErr.GetRegionNotFound() != nil { + if seed != nil { + logutil.BgLogger().Debug("tikv reports `RegionNotFound` in follow-reader", + zap.Stringer("ctx", ctx), zap.Uint32("seed", *seed)) + *seed = *seed + 1 + } + s.regionCache.InvalidateCachedRegion(ctx.Region) + return false, nil + } + logutil.BgLogger().Debug("tikv reports region failed", zap.Stringer("regionErr", regionErr), zap.Stringer("ctx", ctx)) + + if s.leaderReplicaSelector != nil { + // Try the next replica. + return true, nil + } + // When the request is sent to TiDB, there is no region in the request, so the region id will be 0. // So when region id is 0, there is no business with region cache. if ctx.Region.id != 0 { s.regionCache.InvalidateCachedRegion(ctx.Region) } + // For other errors, we only drop cache here. + // Because caller may need to re-split the request. return false, nil } diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 35bb0c185cdb5..7b8016e50abc9 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -117,7 +117,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestGetRPCContext(c *C) { _, err := s.cache.BatchLoadRegionsFromKey(s.bo, []byte{}, 1) c.Assert(err, IsNil) - var seed uint32 = 0 + var seed uint32 var regionID = RegionVerID{s.regionID, 0, 0} req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{}, kv.ReplicaReadLeader, &seed) @@ -164,8 +164,10 @@ func (s *testRegionRequestToSingleStoreSuite) TestOnRegionError(c *C) { }} bo := NewBackofferWithVars(context.Background(), 5, nil) resp, err := s.regionRequestSender.SendReq(bo, req, region.Region, time.Second) - c.Assert(err, NotNil) - c.Assert(resp, IsNil) + c.Assert(err, IsNil) + c.Assert(resp, NotNil) + regionErr, _ := resp.GetRegionError() + c.Assert(regionErr, NotNil) }() } @@ -284,11 +286,14 @@ func (s *testRegionRequestToSingleStoreSuite) TestOnSendFailedWithStoreRestart(c resp, err := s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) c.Assert(err, IsNil) c.Assert(resp.Resp, NotNil) + c.Assert(s.regionRequestSender.rpcError, IsNil) // stop store. s.cluster.StopStore(s.store) _, err = s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) c.Assert(err, NotNil) + // The RPC error shouldn't be nil since it failed to sent the request. + c.Assert(s.regionRequestSender.rpcError, NotNil) // start store. s.cluster.StartStore(s.store) @@ -298,9 +303,12 @@ func (s *testRegionRequestToSingleStoreSuite) TestOnSendFailedWithStoreRestart(c region, err = s.cache.LocateRegionByID(s.bo, s.region) c.Assert(err, IsNil) c.Assert(region, NotNil) + c.Assert(s.regionRequestSender.rpcError, NotNil) resp, err = s.regionRequestSender.SendReq(s.bo, req, region.Region, time.Second) c.Assert(err, IsNil) c.Assert(resp.Resp, NotNil) + // The RPC error should be nil since it's evaluated successfully. + c.Assert(s.regionRequestSender.rpcError, IsNil) } func (s *testRegionRequestToSingleStoreSuite) TestOnSendFailedWithCloseKnownStoreThenUseNewOne(c *C) { @@ -691,9 +699,9 @@ func (s *testRegionRequestToThreeStoresSuite) TestSwitchPeerWhenNoLeader(c *C) { func (s *testRegionRequestToThreeStoresSuite) loadAndGetLeaderStore(c *C) (*Store, string) { region, err := s.regionRequestSender.regionCache.findRegionByKey(s.bo, []byte("a"), false) c.Assert(err, IsNil) - leaderStore, leaderPeer, _, leaderStoreIdx := region.WorkStorePeer(region.getStore()) + leaderStore, leaderPeer, _, _ := region.WorkStorePeer(region.getStore()) c.Assert(leaderPeer.Id, Equals, s.leaderPeer) - leaderAddr, err := s.regionRequestSender.regionCache.getStoreAddr(s.bo, region, leaderStore, leaderStoreIdx) + leaderAddr, err := s.regionRequestSender.regionCache.getStoreAddr(s.bo, region, leaderStore) c.Assert(err, IsNil) return leaderStore, leaderAddr } @@ -859,3 +867,410 @@ func (s *testRegionRequestToSingleStoreSuite) TestGetRegionByIDFromCache(c *C) { c.Assert(region.Region.confVer, Equals, region.Region.confVer) c.Assert(region.Region.ver, Equals, v3) } + +func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector(c *C) { + regionLoc, err := s.cache.LocateRegionByID(s.bo, s.regionID) + c.Assert(err, IsNil) + c.Assert(regionLoc, NotNil) + region := s.cache.GetCachedRegionWithRLock(regionLoc.Region) + regionStore := region.getStore() + + // Create a fake region and change its leader to the last peer. + regionStore = regionStore.clone() + regionStore.workTiKVIdx = AccessIndex(len(regionStore.stores) - 1) + sidx, _ := regionStore.accessStore(TiKVOnly, regionStore.workTiKVIdx) + regionStore.stores[sidx].epoch++ + regionStore.storeEpochs[sidx]++ + // Add a TiFlash peer to the region. + peer := &metapb.Peer{Id: s.cluster.AllocID(), StoreId: s.cluster.AllocID()} + regionStore.accessIndex[TiFlashOnly] = append(regionStore.accessIndex[TiFlashOnly], len(regionStore.stores)) + regionStore.stores = append(regionStore.stores, &Store{storeID: peer.StoreId, storeType: tikvrpc.TiFlash}) + regionStore.storeEpochs = append(regionStore.storeEpochs, 0) + + region = &Region{ + meta: region.GetMeta(), + } + region.lastAccess = time.Now().Unix() + region.meta.Peers = append(region.meta.Peers, peer) + atomic.StorePointer(®ion.store, unsafe.Pointer(regionStore)) + + cache := NewRegionCache(s.cache.pdClient) + defer cache.Close() + cache.insertRegionToCache(region) + + // Verify creating the replicaSelector. + replicaSelector, err := newReplicaSelector(cache, regionLoc.Region) + c.Assert(replicaSelector, NotNil) + c.Assert(err, IsNil) + c.Assert(replicaSelector.region, Equals, region) + // Should only contains TiKV stores. + c.Assert(len(replicaSelector.replicas), Equals, regionStore.accessStoreNum(TiKVOnly)) + c.Assert(len(replicaSelector.replicas), Equals, len(regionStore.stores)-1) + c.Assert(replicaSelector.nextReplicaIdx == 0, IsTrue) + c.Assert(replicaSelector.isExhausted(), IsFalse) + + // Verify that the store matches the peer and epoch. + for _, replica := range replicaSelector.replicas { + c.Assert(replica.store.storeID, Equals, replica.peer.GetStoreId()) + c.Assert(replica.peer, Equals, region.getPeerOnStore(replica.store.storeID)) + c.Assert(replica.attempts == 0, IsTrue) + + for i, store := range regionStore.stores { + if replica.store == store { + c.Assert(replica.epoch, Equals, regionStore.storeEpochs[i]) + } + } + } + // Verify that the leader replica is at the head of replicas. + leaderStore, leaderPeer, _, _ := region.WorkStorePeer(regionStore) + leaderReplica := replicaSelector.replicas[0] + c.Assert(leaderReplica.store, Equals, leaderStore) + c.Assert(leaderReplica.peer, Equals, leaderPeer) + + assertRPCCtxEqual := func(rpcCtx *RPCContext, replica *replica) { + c.Assert(rpcCtx.Store, Equals, replicaSelector.replicas[replicaSelector.nextReplicaIdx-1].store) + c.Assert(rpcCtx.Peer, Equals, replicaSelector.replicas[replicaSelector.nextReplicaIdx-1].peer) + c.Assert(rpcCtx.Addr, Equals, replicaSelector.replicas[replicaSelector.nextReplicaIdx-1].store.addr) + c.Assert(rpcCtx.AccessMode, Equals, TiKVOnly) + } + + // Verify the correctness of next() + for i := 0; i < len(replicaSelector.replicas); i++ { + rpcCtx, err := replicaSelector.next(s.bo) + c.Assert(rpcCtx, NotNil) + c.Assert(err, IsNil) + c.Assert(rpcCtx.Region, Equals, regionLoc.Region) + c.Assert(rpcCtx.Meta, Equals, region.meta) + replica := replicaSelector.replicas[replicaSelector.nextReplicaIdx-1] + assertRPCCtxEqual(rpcCtx, replica) + c.Assert(replica.attempts, Equals, 1) + c.Assert(replicaSelector.nextReplicaIdx, Equals, i+1) + } + c.Assert(replicaSelector.isExhausted(), IsTrue) + rpcCtx, err := replicaSelector.next(s.bo) + c.Assert(rpcCtx, IsNil) + c.Assert(err, IsNil) + // The region should be invalidated if runs out of all replicas. + c.Assert(replicaSelector.region.isValid(), IsFalse) + + region.lastAccess = time.Now().Unix() + replicaSelector, err = newReplicaSelector(cache, regionLoc.Region) + c.Assert(replicaSelector, NotNil) + cache.testingKnobs.mockRequestLiveness = func(s *Store, bo *Backoffer) livenessState { + return reachable + } + for i := 0; i < maxReplicaAttempt; i++ { + rpcCtx, err := replicaSelector.next(s.bo) + c.Assert(rpcCtx, NotNil) + c.Assert(err, IsNil) + nextIdx := replicaSelector.nextReplicaIdx + // Verify that retry the same store if it's reachable. + replicaSelector.onSendFailure(s.bo, nil) + c.Assert(nextIdx, Equals, replicaSelector.nextReplicaIdx+1) + c.Assert(replicaSelector.nextReplica().attempts, Equals, i+1) + } + // Verify the maxReplicaAttempt limit for each replica. + rpcCtx, err = replicaSelector.next(s.bo) + c.Assert(err, IsNil) + assertRPCCtxEqual(rpcCtx, replicaSelector.replicas[1]) + c.Assert(replicaSelector.nextReplicaIdx, Equals, 2) + + // Verify updating leader. + replicaSelector, _ = newReplicaSelector(cache, regionLoc.Region) + replicaSelector.next(s.bo) + // The leader is the 3rd replica. After updating leader, it should be the next. + leader := replicaSelector.replicas[2] + replicaSelector.updateLeader(leader.peer) + c.Assert(replicaSelector.nextReplica(), Equals, leader) + c.Assert(replicaSelector.nextReplicaIdx, Equals, 1) + rpcCtx, _ = replicaSelector.next(s.bo) + assertRPCCtxEqual(rpcCtx, leader) + // Verify the regionStore is updated and the workTiKVIdx points to the leader. + regionStore = region.getStore() + leaderStore, leaderPeer, _, _ = region.WorkStorePeer(regionStore) + c.Assert(leaderStore, Equals, leader.store) + c.Assert(leaderPeer, Equals, leader.peer) + + replicaSelector, _ = newReplicaSelector(cache, regionLoc.Region) + replicaSelector.next(s.bo) + replicaSelector.next(s.bo) + replicaSelector.next(s.bo) + c.Assert(replicaSelector.isExhausted(), IsTrue) + // The leader is the 1st replica. After updating leader, it should be the next and + // the currnet replica is skipped. + leader = replicaSelector.replicas[0] + replicaSelector.updateLeader(leader.peer) + // The leader should be the next replica. + c.Assert(replicaSelector.nextReplica(), Equals, leader) + c.Assert(replicaSelector.nextReplicaIdx, Equals, 2) + rpcCtx, _ = replicaSelector.next(s.bo) + c.Assert(replicaSelector.isExhausted(), IsTrue) + assertRPCCtxEqual(rpcCtx, leader) + // Verify the regionStore is updated and the workTiKVIdx points to the leader. + regionStore = region.getStore() + leaderStore, leaderPeer, _, _ = region.WorkStorePeer(regionStore) + c.Assert(leaderStore, Equals, leader.store) + c.Assert(leaderPeer, Equals, leader.peer) + + // Give the leader one more chance even if it exceeds the maxReplicaAttempt. + replicaSelector, _ = newReplicaSelector(cache, regionLoc.Region) + leader = replicaSelector.replicas[0] + leader.attempts = maxReplicaAttempt + replicaSelector.updateLeader(leader.peer) + c.Assert(leader.attempts, Equals, maxReplicaAttempt-1) + rpcCtx, _ = replicaSelector.next(s.bo) + assertRPCCtxEqual(rpcCtx, leader) + c.Assert(leader.attempts, Equals, maxReplicaAttempt) + + // Invalidate the region if the leader is not in the region. + region.lastAccess = time.Now().Unix() + replicaSelector.updateLeader(&metapb.Peer{Id: s.cluster.AllocID(), StoreId: s.cluster.AllocID()}) + c.Assert(region.isValid(), IsFalse) + // Don't try next replica if the region is invalidated. + rpcCtx, err = replicaSelector.next(s.bo) + c.Assert(rpcCtx, IsNil) + c.Assert(err, IsNil) + + // Verify on send success. + region.lastAccess = time.Now().Unix() + replicaSelector, _ = newReplicaSelector(cache, regionLoc.Region) + replicaSelector.next(s.bo) + rpcCtx, err = replicaSelector.next(s.bo) + replicaSelector.OnSendSuccess() + // Verify the regionStore is updated and the workTiKVIdx points to the leader. + leaderStore, leaderPeer, _, _ = region.WorkStorePeer(region.getStore()) + c.Assert(leaderStore, Equals, rpcCtx.Store) + c.Assert(leaderPeer, Equals, rpcCtx.Peer) +} + +// TODO(youjiali1995): Remove duplicated tests. This test may be duplicated with other +// tests but it's a dedicated one to test sending requests with the replica selector. +func (s *testRegionRequestToThreeStoresSuite) TestSendReqWithReplicaSelector(c *C) { + req := tikvrpc.NewRequest(tikvrpc.CmdRawPut, &kvrpcpb.RawPutRequest{ + Key: []byte("key"), + Value: []byte("value"), + }) + region, err := s.cache.LocateRegionByID(s.bo, s.regionID) + c.Assert(err, IsNil) + c.Assert(region, NotNil) + + reloadRegion := func() { + s.regionRequestSender.leaderReplicaSelector.region.invalidate(Other) + region, _ = s.cache.LocateRegionByID(s.bo, s.regionID) + } + + hasFakeRegionError := func(resp *tikvrpc.Response) bool { + if resp == nil { + return false + } + regionErr, err := resp.GetRegionError() + if err != nil { + return false + } + return isFakeRegionError(regionErr) + } + + // Normal + bo := retry.NewBackoffer(context.Background(), -1) + sender := s.regionRequestSender + resp, err := sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(resp, NotNil) + c.Assert(bo.GetTotalBackoffTimes() == 0, IsTrue) + + // Switch to the next Peer due to store failure and the leader is on the next peer. + bo = retry.NewBackoffer(context.Background(), -1) + s.cluster.ChangeLeader(s.regionID, s.peerIDs[1]) + s.cluster.StopStore(s.storeIDs[0]) + resp, err = sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(resp, NotNil) + c.Assert(sender.leaderReplicaSelector.nextReplicaIdx, Equals, 2) + c.Assert(bo.GetTotalBackoffTimes() == 1, IsTrue) + s.cluster.StartStore(s.storeIDs[0]) + + // Leader is updated because of send success, so no backoff. + bo = retry.NewBackoffer(context.Background(), -1) + resp, err = sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(resp, NotNil) + c.Assert(sender.leaderReplicaSelector.nextReplicaIdx, Equals, 1) + c.Assert(bo.GetTotalBackoffTimes() == 0, IsTrue) + + // Switch to the next peer due to leader failure but the new leader is not elected. + // Region will be invalidated due to store epoch changed. + reloadRegion() + s.cluster.StopStore(s.storeIDs[1]) + bo = retry.NewBackoffer(context.Background(), -1) + resp, err = sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(hasFakeRegionError(resp), IsTrue) + c.Assert(sender.leaderReplicaSelector.isExhausted(), IsFalse) + c.Assert(bo.GetTotalBackoffTimes(), Equals, 1) + s.cluster.StartStore(s.storeIDs[1]) + + // Leader is changed. No backoff. + reloadRegion() + s.cluster.ChangeLeader(s.regionID, s.peerIDs[0]) + bo = retry.NewBackoffer(context.Background(), -1) + resp, err = sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(resp, NotNil) + c.Assert(bo.GetTotalBackoffTimes(), Equals, 0) + + // No leader. Backoff for each replica and runs out all replicas. + s.cluster.GiveUpLeader(s.regionID) + bo = retry.NewBackoffer(context.Background(), -1) + resp, err = sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(hasFakeRegionError(resp), IsTrue) + c.Assert(bo.GetTotalBackoffTimes(), Equals, 3) + c.Assert(sender.leaderReplicaSelector.isExhausted(), IsTrue) + c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) + s.cluster.ChangeLeader(s.regionID, s.peerIDs[0]) + + // The leader store is alive but can't provide service. + // Region will be invalidated due to running out of all replicas. + s.regionRequestSender.regionCache.testingKnobs.mockRequestLiveness = func(s *Store, bo *Backoffer) livenessState { + return reachable + } + reloadRegion() + s.cluster.StopStore(s.storeIDs[0]) + bo = retry.NewBackoffer(context.Background(), -1) + resp, err = sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(hasFakeRegionError(resp), IsTrue) + c.Assert(sender.leaderReplicaSelector.isExhausted(), IsTrue) + c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) + c.Assert(bo.GetTotalBackoffTimes(), Equals, maxReplicaAttempt+2) + s.cluster.StartStore(s.storeIDs[0]) + + // Verify that retry the same replica when meets ServerIsBusy/MaxTimestampNotSynced/ReadIndexNotReady/ProposalInMergingMode/DataIsNotReady. + for _, regionErr := range []*errorpb.Error{ + // ServerIsBusy takes too much time to test. + // {ServerIsBusy: &errorpb.ServerIsBusy{}}, + {MaxTimestampNotSynced: &errorpb.MaxTimestampNotSynced{}}, + {ReadIndexNotReady: &errorpb.ReadIndexNotReady{}}, + {ProposalInMergingMode: &errorpb.ProposalInMergingMode{}}, + {DataIsNotReady: &errorpb.DataIsNotReady{}}} { + func() { + oc := sender.client + defer func() { + sender.client = oc + }() + s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + // Return the specific region error when accesses the leader. + if addr == s.cluster.GetStore(s.storeIDs[0]).Address { + return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{RegionError: regionErr}}, nil + } + // Return the not leader error when accesses followers. + return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{RegionError: &errorpb.Error{ + NotLeader: &errorpb.NotLeader{ + RegionId: region.Region.id, Leader: &metapb.Peer{Id: s.peerIDs[0], StoreId: s.storeIDs[0]}, + }}}}, nil + + }} + reloadRegion() + bo = retry.NewBackoffer(context.Background(), -1) + resp, err := sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(hasFakeRegionError(resp), IsTrue) + c.Assert(sender.leaderReplicaSelector.isExhausted(), IsTrue) + c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) + c.Assert(bo.GetTotalBackoffTimes(), Equals, maxReplicaAttempt+2) + }() + } + + // Verify switch to the next peer immediately when meets StaleCommand. + reloadRegion() + func() { + oc := sender.client + defer func() { + sender.client = oc + }() + s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{RegionError: &errorpb.Error{StaleCommand: &errorpb.StaleCommand{}}}}, nil + }} + reloadRegion() + bo = retry.NewBackoffer(context.Background(), -1) + resp, err := sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(hasFakeRegionError(resp), IsTrue) + c.Assert(sender.leaderReplicaSelector.isExhausted(), IsTrue) + c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) + c.Assert(bo.GetTotalBackoffTimes(), Equals, 0) + }() + + // Verify don't invalidate region when meets unknown region errors. + reloadRegion() + func() { + oc := sender.client + defer func() { + sender.client = oc + }() + s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{RegionError: &errorpb.Error{Message: ""}}}, nil + }} + reloadRegion() + bo = retry.NewBackoffer(context.Background(), -1) + resp, err := sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(hasFakeRegionError(resp), IsTrue) + c.Assert(sender.leaderReplicaSelector.isExhausted(), IsTrue) + c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) + c.Assert(bo.GetTotalBackoffTimes(), Equals, 0) + }() + + // Verify invalidate region when meets StoreNotMatch/RegionNotFound/EpochNotMatch/NotLeader and can't find the leader in region. + for i, regionErr := range []*errorpb.Error{ + {StoreNotMatch: &errorpb.StoreNotMatch{}}, + {RegionNotFound: &errorpb.RegionNotFound{}}, + {EpochNotMatch: &errorpb.EpochNotMatch{}}, + {NotLeader: &errorpb.NotLeader{Leader: &metapb.Peer{}}}} { + func() { + oc := sender.client + defer func() { + sender.client = oc + }() + s.regionRequestSender.client = &fnClient{func(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (response *tikvrpc.Response, err error) { + return &tikvrpc.Response{Resp: &kvrpcpb.RawPutResponse{RegionError: regionErr}}, nil + + }} + reloadRegion() + bo = retry.NewBackoffer(context.Background(), -1) + resp, err := sender.SendReq(bo, req, region.Region, time.Second) + + // Return a sendError when meets NotLeader and can't find the leader in the region. + if i == 3 { + c.Assert(err, IsNil) + c.Assert(hasFakeRegionError(resp), IsTrue) + } else { + c.Assert(err, IsNil) + c.Assert(resp, NotNil) + regionErr, _ := resp.GetRegionError() + c.Assert(regionErr, NotNil) + } + c.Assert(sender.leaderReplicaSelector.isExhausted(), IsFalse) + c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) + c.Assert(bo.GetTotalBackoffTimes(), Equals, 0) + }() + } + + // Runs out of all replicas and then returns a send error. + s.regionRequestSender.regionCache.testingKnobs.mockRequestLiveness = func(s *Store, bo *Backoffer) livenessState { + return unreachable + } + reloadRegion() + for _, store := range s.storeIDs { + s.cluster.StopStore(store) + } + bo = retry.NewBackoffer(context.Background(), -1) + resp, err = sender.SendReq(bo, req, region.Region, time.Second) + c.Assert(err, IsNil) + c.Assert(hasFakeRegionError(resp), IsTrue) + c.Assert(bo.GetTotalBackoffTimes() == 3, IsTrue) + c.Assert(sender.leaderReplicaSelector.region.isValid(), IsFalse) + for _, store := range s.storeIDs { + s.cluster.StartStore(store) + } +} diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go index d57ab62a210ad..716011f8836ab 100644 --- a/store/tikv/retry/backoff.go +++ b/store/tikv/retry/backoff.go @@ -40,11 +40,12 @@ type Backoffer struct { fn map[string]backoffFn maxSleep int totalSleep int - errors []error - configs []*Config - vars *kv.Variables - noop bool + vars *kv.Variables + noop bool + + errors []error + configs []*Config backoffSleepMS map[string]int backoffTimes map[string]int } @@ -80,7 +81,7 @@ func (b *Backoffer) withVars(vars *kv.Variables) *Backoffer { } // maxSleep is the max sleep time in millisecond. // When it is multiplied by BackOffWeight, it should not be greater than MaxInt32. - if math.MaxInt32/b.vars.BackOffWeight >= b.maxSleep { + if b.maxSleep > 0 && math.MaxInt32/b.vars.BackOffWeight >= b.maxSleep { b.maxSleep *= b.vars.BackOffWeight } return b @@ -290,6 +291,15 @@ func (b *Backoffer) GetBackoffTimes() map[string]int { return b.backoffTimes } +// GetTotalBackoffTimes returns the total backoff times of the backoffer. +func (b *Backoffer) GetTotalBackoffTimes() int { + total := 0 + for _, time := range b.backoffTimes { + total += time + } + return total +} + // GetBackoffSleepMS returns a map contains backoff sleep time by type. func (b *Backoffer) GetBackoffSleepMS() map[string]int { return b.backoffSleepMS @@ -299,3 +309,20 @@ func (b *Backoffer) GetBackoffSleepMS() map[string]int { func (b *Backoffer) ErrorsNum() int { return len(b.errors) } + +// Reset resets the sleep state of the backoffer, so that following backoff +// can sleep shorter. The reason why we don't create a new backoffer is that +// backoffer is similar to context and it records some metrics that we +// want to record for an entire process which is composed of serveral stages. +func (b *Backoffer) Reset() { + b.fn = nil + b.totalSleep = 0 +} + +// ResetMaxSleep resets the sleep state and max sleep limit of the backoffer. +// It's used when switches to the next stage of the process. +func (b *Backoffer) ResetMaxSleep(maxSleep int) { + b.Reset() + b.maxSleep = maxSleep + b.withVars(b.vars) +} diff --git a/store/tikv/tests/2pc_fail_test.go b/store/tikv/tests/2pc_fail_test.go index d8b35e4b28c53..8b00dba08996e 100644 --- a/store/tikv/tests/2pc_fail_test.go +++ b/store/tikv/tests/2pc_fail_test.go @@ -93,6 +93,22 @@ func (s *testCommitterSuite) TestFailCommitPrimaryKeyError(c *C) { c.Assert(terror.ErrorNotEqual(err, terror.ErrResultUndetermined), IsTrue) } +// TestFailCommitPrimaryRPCErrorThenKeyError tests KeyError overwrites the undeterminedErr. +func (s *testCommitterSuite) TestFailCommitPrimaryRPCErrorThenKeyError(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitResult", `1*return("timeout")->return("keyError")`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitResult"), IsNil) + }() + // Ensure it returns the original error without wrapped to ErrResultUndetermined + // if it meets KeyError. + t3 := s.begin(c) + err := t3.Set([]byte("c"), []byte("c1")) + c.Assert(err, IsNil) + err = t3.Commit(context.Background()) + c.Assert(err, NotNil) + c.Assert(terror.ErrorEqual(err, terror.ErrResultUndetermined), IsFalse) +} + func (s *testCommitterSuite) TestFailCommitTimeout(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockstore/mocktikv/rpcCommitTimeout", `return(true)`), IsNil) defer func() { diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index d56436232e935..dc59e2b1bbcea 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -57,6 +57,7 @@ func (s *testCommitterSuite) SetUpSuite(c *C) { atomic.StoreUint64(&tikv.ManagedLockTTL, 3000) // 3s s.OneByOneSuite.SetUpSuite(c) atomic.StoreUint64(&tikv.CommitMaxBackoff, 1000) + atomic.StoreUint64(&tikv.VeryLongMaxBackoff, 1000) } func (s *testCommitterSuite) SetUpTest(c *C) { @@ -91,6 +92,7 @@ func (s *testCommitterSuite) SetUpTest(c *C) { func (s *testCommitterSuite) TearDownSuite(c *C) { atomic.StoreUint64(&tikv.CommitMaxBackoff, 20000) + atomic.StoreUint64(&tikv.VeryLongMaxBackoff, 600000) s.store.Close() s.OneByOneSuite.TearDownSuite(c) } diff --git a/store/tikv/tests/async_commit_fail_test.go b/store/tikv/tests/async_commit_fail_test.go index a791f16c54e86..aa65c534aa155 100644 --- a/store/tikv/tests/async_commit_fail_test.go +++ b/store/tikv/tests/async_commit_fail_test.go @@ -233,3 +233,25 @@ func (s *testAsyncCommitFailSuite) TestAsyncCommitContextCancelCausingUndetermin c.Assert(err, NotNil) c.Assert(txn.GetCommitter().GetUndeterminedErr(), NotNil) } + +// TestAsyncCommitRPCErrorThenWriteConflict verifies that the determined failure error overwrites undetermined error. +func (s *testAsyncCommitFailSuite) TestAsyncCommitRPCErrorThenWriteConflict(c *C) { + // This test doesn't support tikv mode because it needs setting failpoint in unistore. + if *WithTiKV { + return + } + + txn := s.beginAsyncCommit(c) + err := txn.Set([]byte("a"), []byte("va")) + c.Assert(err, IsNil) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/rpcPrewriteResult", `1*return("timeout")->return("writeConflict")`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/rpcPrewriteResult"), IsNil) + }() + + ctx := context.WithValue(context.Background(), util.SessionID, uint64(1)) + err = txn.Commit(ctx) + c.Assert(err, NotNil) + c.Assert(txn.GetCommitter().GetUndeterminedErr(), IsNil) +} From 9f8175743fbebda89e97eb0885a25dd1a9844eef Mon Sep 17 00:00:00 2001 From: wjHuang Date: Thu, 3 Jun 2021 21:14:26 +0800 Subject: [PATCH 270/343] *: refine some error messages (#24767) --- errno/errcode.go | 2 + errno/errname.go | 168 +++++++++++++++++++------------------- errors.toml | 12 ++- executor/admin_test.go | 39 +++++++-- executor/distsql.go | 28 ++++--- executor/errors.go | 12 +-- executor/executor_test.go | 2 +- util/admin/admin.go | 4 +- 8 files changed, 158 insertions(+), 109 deletions(-) diff --git a/errno/errcode.go b/errno/errcode.go index 9947a8a726b71..2a3fe2733b5bf 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1003,6 +1003,8 @@ const ( ErrMultiStatementDisabled = 8130 ErrPartitionStatsMissing = 8131 ErrNotSupportedWithSem = 8132 + ErrDataInConsistentExtraIndex = 8133 + ErrDataInConsistentMisMatchIndex = 8134 // Error codes used by TiDB ddl package ErrUnsupportedDDLOperation = 8200 diff --git a/errno/errname.go b/errno/errname.go index 868e5dcc5231e..05a8f476f835b 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -921,89 +921,91 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrSequenceInvalidTableStructure: mysql.Message("Sequence '%-.64s.%-.64s' table structure is invalid (%s)", nil), // TiDB errors. - ErrMemExceedThreshold: mysql.Message("%s holds %dB memory, exceeds threshold %dB.%s", nil), - ErrForUpdateCantRetry: mysql.Message("[%d] can not retry select for update statement", nil), - ErrAdminCheckTable: mysql.Message("TiDB admin check table failed.", nil), - ErrOptOnTemporaryTable: mysql.Message("`%s` is unsupported on temporary tables.", nil), - ErrTxnTooLarge: mysql.Message("Transaction is too large, size: %d", nil), - ErrWriteConflictInTiDB: mysql.Message("Write conflict, txnStartTS %d is stale", nil), - ErrInvalidPluginID: mysql.Message("Wrong plugin id: %s, valid plugin id is [name]-[version], both name and version should not contain '-'", nil), - ErrInvalidPluginManifest: mysql.Message("Cannot read plugin %s's manifest", nil), - ErrInvalidPluginName: mysql.Message("Plugin load with %s but got wrong name %s", nil), - ErrInvalidPluginVersion: mysql.Message("Plugin load with %s but got %s", nil), - ErrDuplicatePlugin: mysql.Message("Plugin [%s] is redeclared", nil), - ErrInvalidPluginSysVarName: mysql.Message("Plugin %s's sysVar %s must start with its plugin name %s", nil), - ErrRequireVersionCheckFail: mysql.Message("Plugin %s require %s be %v but got %v", nil), - ErrUnsupportedReloadPlugin: mysql.Message("Plugin %s isn't loaded so cannot be reloaded", nil), - ErrUnsupportedReloadPluginVar: mysql.Message("Reload plugin with different sysVar is unsupported %v", nil), - ErrTableLocked: mysql.Message("Table '%s' was locked in %s by %v", nil), - ErrNotExist: mysql.Message("Error: key not exist", nil), - ErrTxnRetryable: mysql.Message("Error: KV error safe to retry %s ", []int{0}), - ErrCannotSetNilValue: mysql.Message("can not set nil value", nil), - ErrInvalidTxn: mysql.Message("invalid transaction", nil), - ErrEntryTooLarge: mysql.Message("entry too large, the max entry size is %d, the size of data is %d", nil), - ErrNotImplemented: mysql.Message("not implemented", nil), - ErrInfoSchemaExpired: mysql.Message("Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV", nil), - ErrInfoSchemaChanged: mysql.Message("Information schema is changed during the execution of the statement(for example, table definition may be updated by other DDL ran in parallel). If you see this error often, try increasing `tidb_max_delta_schema_count`", nil), - ErrBadNumber: mysql.Message("Bad Number", nil), - ErrCastAsSignedOverflow: mysql.Message("Cast to signed converted positive out-of-range integer to it's negative complement", nil), - ErrCastNegIntAsUnsigned: mysql.Message("Cast to unsigned converted negative integer to it's positive complement", nil), - ErrInvalidYearFormat: mysql.Message("invalid year format", nil), - ErrInvalidYear: mysql.Message("invalid year", nil), - ErrIncorrectDatetimeValue: mysql.Message("Incorrect datetime value: '%s'", []int{0}), - ErrInvalidTimeFormat: mysql.Message("invalid time format: '%v'", []int{0}), - ErrInvalidWeekModeFormat: mysql.Message("invalid week mode format: '%v'", nil), - ErrFieldGetDefaultFailed: mysql.Message("Field '%s' get default value fail", nil), - ErrIndexOutBound: mysql.Message("Index column %s offset out of bound, offset: %d, row: %v", []int{2}), - ErrUnsupportedOp: mysql.Message("operation not supported", nil), - ErrRowNotFound: mysql.Message("can not find the row: %s", []int{0}), - ErrTableStateCantNone: mysql.Message("table %s can't be in none state", nil), - ErrColumnStateCantNone: mysql.Message("column %s can't be in none state", nil), - ErrColumnStateNonPublic: mysql.Message("can not use non-public column", nil), - ErrIndexStateCantNone: mysql.Message("index %s can't be in none state", nil), - ErrInvalidRecordKey: mysql.Message("invalid record key", nil), - ErrUnsupportedValueForVar: mysql.Message("variable '%s' does not yet support value: %s", nil), - ErrUnsupportedIsolationLevel: mysql.Message("The isolation level '%s' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error", nil), - ErrInvalidDDLWorker: mysql.Message("Invalid DDL worker", nil), - ErrUnsupportedDDLOperation: mysql.Message("Unsupported %s", nil), - ErrNotOwner: mysql.Message("TiDB server is not a DDL owner", nil), - ErrCantDecodeRecord: mysql.Message("Cannot decode %s value, because %v", nil), - ErrInvalidDDLJob: mysql.Message("Invalid DDL job", nil), - ErrInvalidDDLJobFlag: mysql.Message("Invalid DDL job flag", nil), - ErrWaitReorgTimeout: mysql.Message("Timeout waiting for data reorganization", nil), - ErrInvalidStoreVersion: mysql.Message("Invalid storage current version: %d", nil), - ErrUnknownTypeLength: mysql.Message("Unknown length for type %d", nil), - ErrUnknownFractionLength: mysql.Message("Unknown length for type %d and fraction %d", nil), - ErrInvalidDDLJobVersion: mysql.Message("Version %d of DDL job is greater than current one: %d", nil), - ErrInvalidSplitRegionRanges: mysql.Message("Failed to split region ranges: %s", nil), - ErrReorgPanic: mysql.Message("Reorg worker panic", nil), - ErrInvalidDDLState: mysql.Message("Invalid %s state: %v", nil), - ErrCancelledDDLJob: mysql.Message("Cancelled DDL job", nil), - ErrRepairTable: mysql.Message("Failed to repair table: %s", nil), - ErrLoadPrivilege: mysql.Message("Load privilege table fail: %s", nil), - ErrInvalidPrivilegeType: mysql.Message("unknown privilege type %s", nil), - ErrUnknownFieldType: mysql.Message("unknown field type", nil), - ErrInvalidSequence: mysql.Message("invalid sequence", nil), - ErrInvalidType: mysql.Message("invalid type", nil), - ErrCantGetValidID: mysql.Message("Cannot get a valid auto-ID when retrying the statement", nil), - ErrCantSetToNull: mysql.Message("cannot set variable to null", nil), - ErrSnapshotTooOld: mysql.Message("snapshot is older than GC safe point %s", nil), - ErrInvalidTableID: mysql.Message("invalid TableID", nil), - ErrInvalidAutoRandom: mysql.Message("Invalid auto random: %s", nil), - ErrInvalidHashKeyFlag: mysql.Message("invalid encoded hash key flag", nil), - ErrInvalidListIndex: mysql.Message("invalid list index", nil), - ErrInvalidListMetaData: mysql.Message("invalid list meta data", nil), - ErrWriteOnSnapshot: mysql.Message("write on snapshot", nil), - ErrInvalidKey: mysql.Message("invalid key", nil), - ErrInvalidIndexKey: mysql.Message("invalid index key", nil), - ErrDataInConsistent: mysql.Message("data isn't equal", nil), - ErrDDLReorgElementNotExist: mysql.Message("DDL reorg element does not exist", nil), - ErrDDLJobNotFound: mysql.Message("DDL Job:%v not found", nil), - ErrCancelFinishedDDLJob: mysql.Message("This job:%v is finished, so can't be cancelled", nil), - ErrCannotCancelDDLJob: mysql.Message("This job:%v is almost finished, can't be cancelled now", nil), - ErrUnknownAllocatorType: mysql.Message("Invalid allocator type", nil), - ErrAutoRandReadFailed: mysql.Message("Failed to read auto-random value from storage engine", nil), - ErrInvalidIncrementAndOffset: mysql.Message("Invalid auto_increment settings: auto_increment_increment: %d, auto_increment_offset: %d, both of them must be in range [1..65535]", nil), + ErrMemExceedThreshold: mysql.Message("%s holds %dB memory, exceeds threshold %dB.%s", nil), + ErrForUpdateCantRetry: mysql.Message("[%d] can not retry select for update statement", nil), + ErrAdminCheckTable: mysql.Message("TiDB admin check table failed.", nil), + ErrOptOnTemporaryTable: mysql.Message("`%s` is unsupported on temporary tables.", nil), + ErrTxnTooLarge: mysql.Message("Transaction is too large, size: %d", nil), + ErrWriteConflictInTiDB: mysql.Message("Write conflict, txnStartTS %d is stale", nil), + ErrInvalidPluginID: mysql.Message("Wrong plugin id: %s, valid plugin id is [name]-[version], both name and version should not contain '-'", nil), + ErrInvalidPluginManifest: mysql.Message("Cannot read plugin %s's manifest", nil), + ErrInvalidPluginName: mysql.Message("Plugin load with %s but got wrong name %s", nil), + ErrInvalidPluginVersion: mysql.Message("Plugin load with %s but got %s", nil), + ErrDuplicatePlugin: mysql.Message("Plugin [%s] is redeclared", nil), + ErrInvalidPluginSysVarName: mysql.Message("Plugin %s's sysVar %s must start with its plugin name %s", nil), + ErrRequireVersionCheckFail: mysql.Message("Plugin %s require %s be %v but got %v", nil), + ErrUnsupportedReloadPlugin: mysql.Message("Plugin %s isn't loaded so cannot be reloaded", nil), + ErrUnsupportedReloadPluginVar: mysql.Message("Reload plugin with different sysVar is unsupported %v", nil), + ErrTableLocked: mysql.Message("Table '%s' was locked in %s by %v", nil), + ErrNotExist: mysql.Message("Error: key not exist", nil), + ErrTxnRetryable: mysql.Message("Error: KV error safe to retry %s ", []int{0}), + ErrCannotSetNilValue: mysql.Message("can not set nil value", nil), + ErrInvalidTxn: mysql.Message("invalid transaction", nil), + ErrEntryTooLarge: mysql.Message("entry too large, the max entry size is %d, the size of data is %d", nil), + ErrNotImplemented: mysql.Message("not implemented", nil), + ErrInfoSchemaExpired: mysql.Message("Information schema is out of date: schema failed to update in 1 lease, please make sure TiDB can connect to TiKV", nil), + ErrInfoSchemaChanged: mysql.Message("Information schema is changed during the execution of the statement(for example, table definition may be updated by other DDL ran in parallel). If you see this error often, try increasing `tidb_max_delta_schema_count`", nil), + ErrBadNumber: mysql.Message("Bad Number", nil), + ErrCastAsSignedOverflow: mysql.Message("Cast to signed converted positive out-of-range integer to it's negative complement", nil), + ErrCastNegIntAsUnsigned: mysql.Message("Cast to unsigned converted negative integer to it's positive complement", nil), + ErrInvalidYearFormat: mysql.Message("invalid year format", nil), + ErrInvalidYear: mysql.Message("invalid year", nil), + ErrIncorrectDatetimeValue: mysql.Message("Incorrect datetime value: '%s'", []int{0}), + ErrInvalidTimeFormat: mysql.Message("invalid time format: '%v'", []int{0}), + ErrInvalidWeekModeFormat: mysql.Message("invalid week mode format: '%v'", nil), + ErrFieldGetDefaultFailed: mysql.Message("Field '%s' get default value fail", nil), + ErrIndexOutBound: mysql.Message("Index column %s offset out of bound, offset: %d, row: %v", []int{2}), + ErrUnsupportedOp: mysql.Message("operation not supported", nil), + ErrRowNotFound: mysql.Message("can not find the row: %s", []int{0}), + ErrTableStateCantNone: mysql.Message("table %s can't be in none state", nil), + ErrColumnStateCantNone: mysql.Message("column %s can't be in none state", nil), + ErrColumnStateNonPublic: mysql.Message("can not use non-public column", nil), + ErrIndexStateCantNone: mysql.Message("index %s can't be in none state", nil), + ErrInvalidRecordKey: mysql.Message("invalid record key", nil), + ErrUnsupportedValueForVar: mysql.Message("variable '%s' does not yet support value: %s", nil), + ErrUnsupportedIsolationLevel: mysql.Message("The isolation level '%s' is not supported. Set tidb_skip_isolation_level_check=1 to skip this error", nil), + ErrInvalidDDLWorker: mysql.Message("Invalid DDL worker", nil), + ErrUnsupportedDDLOperation: mysql.Message("Unsupported %s", nil), + ErrNotOwner: mysql.Message("TiDB server is not a DDL owner", nil), + ErrCantDecodeRecord: mysql.Message("Cannot decode %s value, because %v", nil), + ErrInvalidDDLJob: mysql.Message("Invalid DDL job", nil), + ErrInvalidDDLJobFlag: mysql.Message("Invalid DDL job flag", nil), + ErrWaitReorgTimeout: mysql.Message("Timeout waiting for data reorganization", nil), + ErrInvalidStoreVersion: mysql.Message("Invalid storage current version: %d", nil), + ErrUnknownTypeLength: mysql.Message("Unknown length for type %d", nil), + ErrUnknownFractionLength: mysql.Message("Unknown length for type %d and fraction %d", nil), + ErrInvalidDDLJobVersion: mysql.Message("Version %d of DDL job is greater than current one: %d", nil), + ErrInvalidSplitRegionRanges: mysql.Message("Failed to split region ranges: %s", nil), + ErrReorgPanic: mysql.Message("Reorg worker panic", nil), + ErrInvalidDDLState: mysql.Message("Invalid %s state: %v", nil), + ErrCancelledDDLJob: mysql.Message("Cancelled DDL job", nil), + ErrRepairTable: mysql.Message("Failed to repair table: %s", nil), + ErrLoadPrivilege: mysql.Message("Load privilege table fail: %s", nil), + ErrInvalidPrivilegeType: mysql.Message("unknown privilege type %s", nil), + ErrUnknownFieldType: mysql.Message("unknown field type", nil), + ErrInvalidSequence: mysql.Message("invalid sequence", nil), + ErrInvalidType: mysql.Message("invalid type", nil), + ErrCantGetValidID: mysql.Message("Cannot get a valid auto-ID when retrying the statement", nil), + ErrCantSetToNull: mysql.Message("cannot set variable to null", nil), + ErrSnapshotTooOld: mysql.Message("snapshot is older than GC safe point %s", nil), + ErrInvalidTableID: mysql.Message("invalid TableID", nil), + ErrInvalidAutoRandom: mysql.Message("Invalid auto random: %s", nil), + ErrInvalidHashKeyFlag: mysql.Message("invalid encoded hash key flag", nil), + ErrInvalidListIndex: mysql.Message("invalid list index", nil), + ErrInvalidListMetaData: mysql.Message("invalid list meta data", nil), + ErrWriteOnSnapshot: mysql.Message("write on snapshot", nil), + ErrInvalidKey: mysql.Message("invalid key", nil), + ErrInvalidIndexKey: mysql.Message("invalid index key", nil), + ErrDataInConsistent: mysql.Message("index:%#v != record:%#v", []int{0, 1}), + ErrDDLReorgElementNotExist: mysql.Message("DDL reorg element does not exist", nil), + ErrDDLJobNotFound: mysql.Message("DDL Job:%v not found", nil), + ErrCancelFinishedDDLJob: mysql.Message("This job:%v is finished, so can't be cancelled", nil), + ErrCannotCancelDDLJob: mysql.Message("This job:%v is almost finished, can't be cancelled now", nil), + ErrUnknownAllocatorType: mysql.Message("Invalid allocator type", nil), + ErrAutoRandReadFailed: mysql.Message("Failed to read auto-random value from storage engine", nil), + ErrInvalidIncrementAndOffset: mysql.Message("Invalid auto_increment settings: auto_increment_increment: %d, auto_increment_offset: %d, both of them must be in range [1..65535]", nil), + ErrDataInConsistentExtraIndex: mysql.Message("handle %#v, index:%#v != record:%#v", []int{0, 1, 2}), + ErrDataInConsistentMisMatchIndex: mysql.Message("col %s, handle %#v, index:%#v != record:%#v, compare err:%#v", []int{1, 2, 3, 4}), ErrWarnOptimizerHintInvalidInteger: mysql.Message("integer value is out of range in '%s'", nil), ErrWarnOptimizerHintUnsupportedHint: mysql.Message("Optimizer hint %s is not supported by TiDB and is ignored", nil), diff --git a/errors.toml b/errors.toml index ce6be545d6ab5..ce6ce2e58206c 100644 --- a/errors.toml +++ b/errors.toml @@ -8,7 +8,7 @@ TiDB admin check table failed. ["admin:8223"] error = ''' -data isn't equal +index:%#v != record:%#v ''' ["admin:8224"] @@ -636,6 +636,16 @@ error = ''' Export failed: %s ''' +["executor:8133"] +error = ''' +handle %#v, index:%#v != record:%#v +''' + +["executor:8134"] +error = ''' +col %s, handle %#v, index:%#v != record:%#v, compare err:%#v +''' + ["executor:8212"] error = ''' Failed to split region ranges: %s diff --git a/executor/admin_test.go b/executor/admin_test.go index f23680e789e56..34d22ae58b2f4 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -793,7 +793,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test_p") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, fmt.Sprintf("handle %d, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:", i+8, i+8)) + c.Assert(err.Error(), Equals, fmt.Sprintf("[executor:8133]handle %d, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:", i+8, i+8)) // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) @@ -816,7 +816,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test_p") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, fmt.Sprintf("col c2, handle %d, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}", i, i+8, i)) + c.Assert(err.Error(), Equals, fmt.Sprintf("[executor:8134]col c2, handle %d, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:%d, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:", i, i+8, i)) // TODO: fix admin recover for partition table. txn, err = s.store.Begin() c.Assert(err, IsNil) @@ -828,7 +828,7 @@ func (s *testSuite3) TestAdminCheckPartitionTableFailed(c *C) { } } -func (s *testSuite5) TestAdminCheckTableFailed(c *C) { +func (s *testSuiteJoinSerial) TestAdminCheckTableFailed(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists admin_test") @@ -864,6 +864,11 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { c.Assert(err.Error(), Equals, "[executor:8003]admin_test err:[admin:8223]index: != record:&admin.RecordData{Handle:-1, Values:[]types.Datum{types.Datum{k:0x1, decimal:0x0, length:0x0, i:-10, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}}}") c.Assert(executor.ErrAdminCheckTable.Equal(err), IsTrue) + tk.MustExec("set @@tidb_redact_log=1;") + err = tk.ExecToErr("admin check table admin_test") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[executor:8003]admin_test err:[admin:8223]index:\"?\" != record:\"?\"") + tk.MustExec("set @@tidb_redact_log=0;") r := tk.MustQuery("admin recover index admin_test c2") r.Check(testkit.Rows("1 7")) tk.MustExec("admin check table admin_test") @@ -879,7 +884,12 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "handle 0, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:0, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:") + c.Assert(err.Error(), Equals, "[executor:8133]handle 0, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:0, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:") + tk.MustExec("set @@tidb_redact_log=1;") + err = tk.ExecToErr("admin check table admin_test") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[executor:8133]handle \"?\", index:\"?\" != record:\"?\"") + tk.MustExec("set @@tidb_redact_log=0;") // Add one row of index. // Table count < index count. @@ -897,7 +907,12 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "col c2, handle 2, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:13, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:12, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}") + c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle 2, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:13, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:12, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:") + tk.MustExec("set @@tidb_redact_log=1;") + err = tk.ExecToErr("admin check table admin_test") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") + tk.MustExec("set @@tidb_redact_log=0;") // Table count = index count. // Two indices have the same handle. @@ -911,7 +926,12 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "col c2, handle 10, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:19, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:20, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}") + c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle 10, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:19, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:20, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:") + tk.MustExec("set @@tidb_redact_log=1;") + err = tk.ExecToErr("admin check table admin_test") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") + tk.MustExec("set @@tidb_redact_log=0;") // Table count = index count. // Index c2 has one line of data is 19, the corresponding table data is 20. @@ -925,7 +945,12 @@ func (s *testSuite5) TestAdminCheckTableFailed(c *C) { c.Assert(err, IsNil) err = tk.ExecToErr("admin check table admin_test") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "col c2, handle 10, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:19, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:20, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}") + c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle 10, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:19, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:types.Datum{k:0x1, decimal:0x0, length:0x0, i:20, collation:\"\", b:[]uint8(nil), x:interface {}(nil)}, compare err:") + tk.MustExec("set @@tidb_redact_log=1;") + err = tk.ExecToErr("admin check table admin_test") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[executor:8134]col c2, handle \"?\", index:\"?\" != record:\"?\", compare err:\"?\"") + tk.MustExec("set @@tidb_redact_log=0;") // Recover records. txn, err = s.store.Begin() diff --git a/executor/distsql.go b/executor/distsql.go index a4d71f45a6a9e..759ec6cb35e85 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -1116,7 +1116,7 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta if chk.NumRows() == 0 { task.indexOrder.Range(func(h kv.Handle, val interface{}) bool { idxRow := task.idxRows.GetRow(val.(int)) - err = errors.Errorf("handle %#v, index:%#v != record:%#v", h, idxRow.GetDatum(0, w.idxColTps[0]), nil) + err = ErrDataInConsistentExtraIndex.GenWithStackByArgs(h, idxRow.GetDatum(0, w.idxColTps[0]), nil) return false }) if err != nil { @@ -1151,12 +1151,12 @@ func (w *tableWorker) compareData(ctx context.Context, task *lookupTableTask, ta tablecodec.TruncateIndexValue(&idxVal, w.idxLookup.index.Columns[i], col.ColumnInfo) cmpRes, err := idxVal.CompareDatum(sctx, &val) if err != nil { - return errors.Errorf("col %s, handle %#v, index:%#v != record:%#v, compare err:%#v", col.Name, + return ErrDataInConsistentMisMatchIndex.GenWithStackByArgs(col.Name, handle, idxRow.GetDatum(i, tp), val, err) } if cmpRes != 0 { - return errors.Errorf("col %s, handle %#v, index:%#v != record:%#v", col.Name, - handle, idxRow.GetDatum(i, tp), val) + return ErrDataInConsistentMisMatchIndex.GenWithStackByArgs(col.Name, + handle, idxRow.GetDatum(i, tp), val, err) } } } @@ -1234,12 +1234,20 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er obtainedHandlesMap.Set(handle, true) } - logutil.Logger(ctx).Error("inconsistent index handles", - zap.String("table_name", w.idxLookup.index.Table.O), - zap.String("index", w.idxLookup.index.Name.O), - zap.Int("index_cnt", handleCnt), zap.Int("table_cnt", len(task.rows)), - zap.String("missing_handles", fmt.Sprint(GetLackHandles(task.handles, obtainedHandlesMap))), - zap.String("total_handles", fmt.Sprint(task.handles))) + if w.idxLookup.ctx.GetSessionVars().EnableRedactLog { + logutil.Logger(ctx).Error("inconsistent index handles", + zap.String("table_name", w.idxLookup.index.Table.O), + zap.String("index", w.idxLookup.index.Name.O), + zap.Int("index_cnt", handleCnt), + zap.Int("table_cnt", len(task.rows))) + } else { + logutil.Logger(ctx).Error("inconsistent index handles", + zap.String("table_name", w.idxLookup.index.Table.O), + zap.String("index", w.idxLookup.index.Name.O), + zap.Int("index_cnt", handleCnt), zap.Int("table_cnt", len(task.rows)), + zap.String("missing_handles", fmt.Sprint(GetLackHandles(task.handles, obtainedHandlesMap))), + zap.String("total_handles", fmt.Sprint(task.handles))) + } // table scan in double read can never has conditions according to convertToIndexScan. // if this table scan has no condition, the number of rows it returns must equal to the length of handles. diff --git a/executor/errors.go b/executor/errors.go index 94237808d1562..05f889e5c9ff8 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -50,9 +50,11 @@ var ( ErrIllegalPrivilegeLevel = dbterror.ClassExecutor.NewStd(mysql.ErrIllegalPrivilegeLevel) ErrInvalidSplitRegionRanges = dbterror.ClassExecutor.NewStd(mysql.ErrInvalidSplitRegionRanges) - ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed) - ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed) - ErrBRIEImportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEImportFailed) - ErrBRIEExportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEExportFailed) - ErrCTEMaxRecursionDepth = dbterror.ClassExecutor.NewStd(mysql.ErrCTEMaxRecursionDepth) + ErrBRIEBackupFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEBackupFailed) + ErrBRIERestoreFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIERestoreFailed) + ErrBRIEImportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEImportFailed) + ErrBRIEExportFailed = dbterror.ClassExecutor.NewStd(mysql.ErrBRIEExportFailed) + ErrCTEMaxRecursionDepth = dbterror.ClassExecutor.NewStd(mysql.ErrCTEMaxRecursionDepth) + ErrDataInConsistentExtraIndex = dbterror.ClassExecutor.NewStd(mysql.ErrDataInConsistentExtraIndex) + ErrDataInConsistentMisMatchIndex = dbterror.ClassExecutor.NewStd(mysql.ErrDataInConsistentMisMatchIndex) ) diff --git a/executor/executor_test.go b/executor/executor_test.go index 450e83f152e39..744f4f29c7b95 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3847,7 +3847,7 @@ func (s *testSuite) TestCheckIndex(c *C) { c.Assert(err, IsNil) _, err = se.Execute(context.Background(), "admin check index t c") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "handle 3, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:30, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:") + c.Assert(err.Error(), Equals, "[executor:8133]handle 3, index:types.Datum{k:0x1, decimal:0x0, length:0x0, i:30, collation:\"\", b:[]uint8(nil), x:interface {}(nil)} != record:") // set data to: // index data (handle, data): (1, 10), (2, 20), (3, 30), (4, 40) diff --git a/util/admin/admin.go b/util/admin/admin.go index 608040a89d6b0..84481d006abb8 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -393,14 +393,14 @@ func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table if kv.ErrKeyExists.Equal(err) { record1 := &RecordData{Handle: h1, Values: vals1} record2 := &RecordData{Handle: h2, Values: vals1} - return false, ErrDataInConsistent.GenWithStack("index:%#v != record:%#v", record2, record1) + return false, ErrDataInConsistent.GenWithStackByArgs(record2, record1) } if err != nil { return false, errors.Trace(err) } if !isExist { record := &RecordData{Handle: h1, Values: vals1} - return false, ErrDataInConsistent.GenWithStack("index:%#v != record:%#v", nil, record) + return false, ErrDataInConsistent.GenWithStackByArgs(nil, record) } return true, nil From 0999df9223217e7bf7f8cbe8b37679ddd0b9174f Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 3 Jun 2021 21:36:26 +0800 Subject: [PATCH 271/343] planner: support explain analyze for mpp task with union (#24898) (#25081) --- distsql/select_result.go | 52 ++++++++++++++++++++++++++------- util/execdetails/execdetails.go | 3 +- 2 files changed, 43 insertions(+), 12 deletions(-) diff --git a/distsql/select_result.go b/distsql/select_result.go index d5a2d65de2622..fa4315b55a041 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -290,13 +290,6 @@ func (r *selectResult) updateCopRuntimeStats(ctx context.Context, copStats *copr if r.rootPlanID <= 0 || r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl == nil || callee == "" { return } - if len(r.selectResp.GetExecutionSummaries()) != len(r.copPlanIDs) { - logutil.Logger(ctx).Error("invalid cop task execution summaries length", - zap.Int("expected", len(r.copPlanIDs)), - zap.Int("received", len(r.selectResp.GetExecutionSummaries()))) - - return - } if r.stats == nil { id := r.rootPlanID r.stats = &selectResultRuntimeStats{ @@ -311,12 +304,49 @@ func (r *selectResult) updateCopRuntimeStats(ctx context.Context, copStats *copr r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RecordScanDetail(r.copPlanIDs[len(r.copPlanIDs)-1], r.storeType.Name(), copStats.ScanDetail) } - for i, detail := range r.selectResp.GetExecutionSummaries() { + // If hasExecutor is true, it means the summary is returned from TiFlash. + hasExecutor := false + for _, detail := range r.selectResp.GetExecutionSummaries() { if detail != nil && detail.TimeProcessedNs != nil && detail.NumProducedRows != nil && detail.NumIterations != nil { - planID := r.copPlanIDs[i] - r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl. - RecordOneCopTask(planID, r.storeType.Name(), callee, detail) + if detail.ExecutorId != nil { + hasExecutor = true + } + break + } + } + if hasExecutor { + var recorededPlanIDs = make(map[int]int) + for i, detail := range r.selectResp.GetExecutionSummaries() { + if detail != nil && detail.TimeProcessedNs != nil && + detail.NumProducedRows != nil && detail.NumIterations != nil { + planID := r.copPlanIDs[i] + recorededPlanIDs[r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl. + RecordOneCopTask(planID, r.storeType.Name(), callee, detail)] = 0 + } + } + num := uint64(0) + dummySummary := &tipb.ExecutorExecutionSummary{TimeProcessedNs: &num, NumProducedRows: &num, NumIterations: &num, ExecutorId: nil} + for _, planID := range r.copPlanIDs { + if _, ok := recorededPlanIDs[planID]; !ok { + r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RecordOneCopTask(planID, r.storeType.Name(), callee, dummySummary) + } + } + } else { + // For cop task cases, we still need this protection. + if len(r.selectResp.GetExecutionSummaries()) != len(r.copPlanIDs) { + logutil.Logger(ctx).Error("invalid cop task execution summaries length", + zap.Int("expected", len(r.copPlanIDs)), + zap.Int("received", len(r.selectResp.GetExecutionSummaries()))) + return + } + for i, detail := range r.selectResp.GetExecutionSummaries() { + if detail != nil && detail.TimeProcessedNs != nil && + detail.NumProducedRows != nil && detail.NumIterations != nil { + planID := r.copPlanIDs[i] + r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl. + RecordOneCopTask(planID, r.storeType.Name(), callee, detail) + } } } } diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 5f19b30158284..faed7f8c9eef7 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -640,7 +640,7 @@ func getPlanIDFromExecutionSummary(summary *tipb.ExecutorExecutionSummary) (int, } // RecordOneCopTask records a specific cop tasks's execution detail. -func (e *RuntimeStatsColl) RecordOneCopTask(planID int, storeType string, address string, summary *tipb.ExecutorExecutionSummary) { +func (e *RuntimeStatsColl) RecordOneCopTask(planID int, storeType string, address string, summary *tipb.ExecutorExecutionSummary) int { // for TiFlash cop response, ExecutorExecutionSummary contains executor id, so if there is a valid executor id in // summary, use it overwrite the planID if id, valid := getPlanIDFromExecutionSummary(summary); valid { @@ -648,6 +648,7 @@ func (e *RuntimeStatsColl) RecordOneCopTask(planID int, storeType string, addres } copStats := e.GetOrCreateCopStats(planID, storeType) copStats.RecordOneCopTask(address, summary) + return planID } // RecordScanDetail records a specific cop tasks's cop detail. From fce4dde1cd0399853aafa616842ec327a02458d7 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Thu, 3 Jun 2021 08:20:26 -0600 Subject: [PATCH 272/343] variable, main: make sure default values for sysvars validate (#24807) --- server/http_handler_test.go | 4 +++- sessionctx/variable/noop.go | 8 ++++---- sessionctx/variable/sysvar.go | 4 ++-- sessionctx/variable/sysvar_test.go | 25 +++++++++++++++++++++++++ tidb-server/main.go | 2 +- tidb-server/main_test.go | 4 ++-- 6 files changed, 37 insertions(+), 10 deletions(-) diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 1dc3d896bf14a..abb466f897080 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -1144,7 +1144,7 @@ func dummyRecord() *deadlockhistory.DeadlockRecord { return &deadlockhistory.DeadlockRecord{} } -func (ts *HTTPHandlerTestSuite) TestPostSettings(c *C) { +func (ts *HTTPHandlerTestSerialSuite) TestPostSettings(c *C) { ts.startServer(c) ts.prepareData(c) defer ts.stopServer(c) @@ -1252,6 +1252,8 @@ func (ts *HTTPHandlerTestSuite) TestPostSettings(c *C) { c.Assert(len(deadlockhistory.GlobalDeadlockHistory.GetAll()), Equals, 6) c.Assert(deadlockhistory.GlobalDeadlockHistory.GetAll()[0].ID, Equals, uint64(7)) c.Assert(deadlockhistory.GlobalDeadlockHistory.GetAll()[5].ID, Equals, uint64(12)) + // restore original value. + config.GetGlobalConfig().CheckMb4ValueInUTF8 = true } func (ts *HTTPHandlerTestSuite) TestPprof(c *C) { diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 053d028641016..93b3c1c9ce084 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -76,7 +76,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "log_backward_compatible_user_definitions", Value: ""}, {Scope: ScopeNone, Name: "lc_messages_dir", Value: "/usr/local/mysql-5.6.25-osx10.8-x86_64/share/"}, {Scope: ScopeGlobal, Name: "ft_boolean_syntax", Value: "+ -><()~*:\"\"&|"}, - {Scope: ScopeGlobal, Name: TableDefinitionCache, Value: "-1", Type: TypeUnsigned, MinValue: 400, MaxValue: 524288, AutoConvertOutOfRange: true}, + {Scope: ScopeGlobal, Name: TableDefinitionCache, Value: "2000", Type: TypeUnsigned, MinValue: 400, MaxValue: 524288, AutoConvertOutOfRange: true}, {Scope: ScopeNone, Name: SkipNameResolve, Value: Off, Type: TypeBool}, {Scope: ScopeNone, Name: "performance_schema_max_file_handles", Value: "32768"}, {Scope: ScopeSession, Name: "transaction_allow_batching", Value: ""}, @@ -141,7 +141,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: "innodb_file_format_max", Value: "Antelope"}, {Scope: ScopeGlobal | ScopeSession, Name: "debug", Value: ""}, {Scope: ScopeGlobal, Name: "log_warnings", Value: "1"}, - {Scope: ScopeGlobal | ScopeSession, Name: InnodbStrictMode, Value: "1", Type: TypeBool, AutoConvertNegativeBool: true}, + {Scope: ScopeGlobal | ScopeSession, Name: InnodbStrictMode, Value: On, Type: TypeBool, AutoConvertNegativeBool: true}, {Scope: ScopeGlobal, Name: "innodb_rollback_segments", Value: "128"}, {Scope: ScopeGlobal | ScopeSession, Name: "join_buffer_size", Value: "262144", IsHintUpdatable: true}, {Scope: ScopeNone, Name: "innodb_mirrored_log_groups", Value: "1"}, @@ -473,7 +473,7 @@ var noopSysVars = []*SysVar{ // for compatibility purpose, we should leave them alone. // TODO: Follow the Terminology Updates of MySQL after their changes arrived. // https://mysqlhighavailability.com/mysql-terminology-updates/ - {Scope: ScopeSession, Name: PseudoSlaveMode, Value: "", Type: TypeInt}, + {Scope: ScopeSession, Name: PseudoSlaveMode, Value: Off, Type: TypeBool}, {Scope: ScopeGlobal, Name: "slave_pending_jobs_size_max", Value: "16777216"}, {Scope: ScopeGlobal, Name: "slave_transaction_retries", Value: "10"}, {Scope: ScopeGlobal, Name: "slave_checkpoint_period", Value: "300"}, @@ -491,7 +491,7 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "slave_load_tmpdir", Value: "/var/tmp/"}, {Scope: ScopeGlobal, Name: "slave_parallel_type", Value: ""}, {Scope: ScopeGlobal, Name: "slave_parallel_workers", Value: "0"}, - {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_timeout", Value: "10000", Type: TypeInt}, + {Scope: ScopeGlobal, Name: "rpl_semi_sync_master_timeout", Value: "10000", Type: TypeInt, MaxValue: math.MaxInt64}, {Scope: ScopeNone, Name: "slave_skip_errors", Value: Off}, {Scope: ScopeGlobal, Name: "sql_slave_skip_counter", Value: "0"}, {Scope: ScopeGlobal, Name: "rpl_semi_sync_slave_enabled", Value: Off, Type: TypeBool}, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 98e8ca0c94ed3..6e4079ca3cc03 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1195,7 +1195,7 @@ var defaultSysVars = []*SysVar{ SetDDLReorgRowFormat(tidbOptInt64(val, DefTiDBRowFormatV2)) return nil }}, - {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), skipInit: true, Type: TypeUnsigned, MinValue: 1, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBOptimizerSelectivityLevel, Value: strconv.Itoa(DefTiDBOptimizerSelectivityLevel), skipInit: true, Type: TypeUnsigned, MinValue: 0, MaxValue: math.MaxInt32, SetSession: func(s *SessionVars, val string) error { s.OptimizerSelectivityLevel = tidbOptPositiveInt32(val, DefTiDBOptimizerSelectivityLevel) return nil }}, @@ -1459,7 +1459,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskMaxTime, Value: strconv.Itoa(DefTiDBEvolvePlanTaskMaxTime), Type: TypeInt, MinValue: -1, MaxValue: math.MaxInt64}, {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskStartTime, Value: DefTiDBEvolvePlanTaskStartTime, Type: TypeTime}, {Scope: ScopeGlobal, Name: TiDBEvolvePlanTaskEndTime, Value: DefTiDBEvolvePlanTaskEndTime, Type: TypeTime}, - {Scope: ScopeSession, Name: TiDBIsolationReadEngines, Value: strings.Join(config.GetGlobalConfig().IsolationRead.Engines, ", "), Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeSession, Name: TiDBIsolationReadEngines, Value: strings.Join(config.GetGlobalConfig().IsolationRead.Engines, ","), Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { engines := strings.Split(normalizedValue, ",") var formatVal string for i, engine := range engines { diff --git a/sessionctx/variable/sysvar_test.go b/sessionctx/variable/sysvar_test.go index 05f8fc83d97de..58f6ac803c563 100644 --- a/sessionctx/variable/sysvar_test.go +++ b/sessionctx/variable/sysvar_test.go @@ -594,3 +594,28 @@ func (*testSysVarSuite) TestDeepCopyGetSysVars(c *C) { c.Assert(GetSysVar("datarace").Name, Equals, "datarace") UnregisterSysVar("datarace") } + +// Test that sysvars defaults are logically valid. i.e. +// the default itself must validate without error provided the scope and read-only is correct. +// The default values should also be normalized for consistency. +func (*testSysVarSuite) TestDefaultValuesAreSettable(c *C) { + vars := NewSessionVars() + for _, sv := range GetSysVars() { + if sv.HasSessionScope() && !sv.ReadOnly { + val, err := sv.Validate(vars, sv.Value, ScopeSession) + c.Assert(sv.Value, Equals, val) + c.Assert(err, IsNil) + } + + if sv.HasGlobalScope() && !sv.ReadOnly { + if sv.Name == TiDBEnableNoopFuncs { + // TODO: this requires access to the global var accessor, + // which is not available in this test. + continue + } + val, err := sv.Validate(vars, sv.Value, ScopeGlobal) + c.Assert(sv.Value, Equals, val) + c.Assert(err, IsNil) + } + } +} diff --git a/tidb-server/main.go b/tidb-server/main.go index 344df22865577..4dcf7296e7dcd 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -536,7 +536,7 @@ func setGlobalVars() { variable.SetSysVar(variable.Socket, cfg.Socket) variable.SetSysVar(variable.DataDir, cfg.Path) variable.SetSysVar(variable.TiDBSlowQueryFile, cfg.Log.SlowQueryFile) - variable.SetSysVar(variable.TiDBIsolationReadEngines, strings.Join(cfg.IsolationRead.Engines, ", ")) + variable.SetSysVar(variable.TiDBIsolationReadEngines, strings.Join(cfg.IsolationRead.Engines, ",")) variable.MemoryUsageAlarmRatio.Store(cfg.Performance.MemoryUsageAlarmRatio) if hostname, err := os.Hostname(); err != nil { variable.SetSysVar(variable.Hostname, hostname) diff --git a/tidb-server/main_test.go b/tidb-server/main_test.go index def1048adb719..f504e9bd47656 100644 --- a/tidb-server/main_test.go +++ b/tidb-server/main_test.go @@ -40,7 +40,7 @@ var _ = Suite(&testMainSuite{}) type testMainSuite struct{} func (t *testMainSuite) TestSetGlobalVars(c *C) { - c.Assert(variable.GetSysVar(variable.TiDBIsolationReadEngines).Value, Equals, "tikv, tiflash, tidb") + c.Assert(variable.GetSysVar(variable.TiDBIsolationReadEngines).Value, Equals, "tikv,tiflash,tidb") c.Assert(variable.GetSysVar(variable.TiDBMemQuotaQuery).Value, Equals, "1073741824") config.UpdateGlobal(func(conf *config.Config) { conf.IsolationRead.Engines = []string{"tikv", "tidb"} @@ -48,6 +48,6 @@ func (t *testMainSuite) TestSetGlobalVars(c *C) { }) setGlobalVars() - c.Assert(variable.GetSysVar(variable.TiDBIsolationReadEngines).Value, Equals, "tikv, tidb") + c.Assert(variable.GetSysVar(variable.TiDBIsolationReadEngines).Value, Equals, "tikv,tidb") c.Assert(variable.GetSysVar(variable.TiDBMemQuotaQuery).Value, Equals, "9999999") } From 6563219318b335f3f4aca06e4851226f596c5d87 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 3 Jun 2021 22:50:26 +0800 Subject: [PATCH 273/343] executor: fix ifnull bug when arg is enum/set (#25110) --- planner/core/expression_rewriter.go | 6 +++++- planner/core/expression_rewriter_test.go | 10 ++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index c228ee957c90b..4259f690c6985 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1665,9 +1665,13 @@ func (er *expressionRewriter) rewriteFuncCall(v *ast.FuncCallExpr) bool { stackLen := len(er.ctxStack) arg1 := er.ctxStack[stackLen-2] col, isColumn := arg1.(*expression.Column) + var isEnumSet bool + if arg1.GetType().Tp == mysql.TypeEnum || arg1.GetType().Tp == mysql.TypeSet { + isEnumSet = true + } // if expr1 is a column and column has not null flag, then we can eliminate ifnull on // this column. - if isColumn && mysql.HasNotNullFlag(col.RetType.Flag) { + if isColumn && !isEnumSet && mysql.HasNotNullFlag(col.RetType.Flag) { name := er.ctxNameStk[stackLen-2] newCol := col.Clone().(*expression.Column) er.ctxStackPop(len(v.Args)) diff --git a/planner/core/expression_rewriter_test.go b/planner/core/expression_rewriter_test.go index 13b7355eb7ea4..fd23efe8f3dd3 100644 --- a/planner/core/expression_rewriter_test.go +++ b/planner/core/expression_rewriter_test.go @@ -43,6 +43,16 @@ func (s *testExpressionRewriterSuite) TestIfNullEliminateColName(c *C) { c.Assert(err, IsNil) fields := rs.Fields() c.Assert(fields[0].Column.Name.L, Equals, "ifnull(a,b)") + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(e int not null, b int)") + tk.MustExec("insert into t values(1, 1)") + tk.MustExec("create table t1(e int not null, b int)") + tk.MustExec("insert into t1 values(1, 1)") + rows := tk.MustQuery("select b from t where ifnull(e, b)") + rows.Check(testkit.Rows("1")) + rows = tk.MustQuery("select b from t1 where ifnull(e, b)") + rows.Check(testkit.Rows("1")) } func (s *testExpressionRewriterSuite) TestBinaryOpFunction(c *C) { From 41e1e814ff869cb5e03c2e02c4dc018e36226620 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Thu, 3 Jun 2021 23:08:26 +0800 Subject: [PATCH 274/343] executor: parallel some part of the sampling-based analyze (#24860) --- executor/analyze.go | 318 +++++++++++++++++++++++++++++---------- executor/analyze_test.go | 19 +++ 2 files changed, 261 insertions(+), 76 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index 180700bbc85b6..c4e3141b2fdd2 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -625,18 +625,20 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { // AnalyzeColumnsExec represents Analyze columns push down executor. type AnalyzeColumnsExec struct { - ctx sessionctx.Context - tableID core.AnalyzeTableID - colsInfo []*model.ColumnInfo - handleCols core.HandleCols - concurrency int - analyzePB *tipb.AnalyzeReq - commonHandle *model.IndexInfo - resultHandler *tableResultHandler - opts map[ast.AnalyzeOptionType]uint64 - job *statistics.AnalyzeJob - analyzeVer int - indexes []*model.IndexInfo + ctx sessionctx.Context + tableID core.AnalyzeTableID + colsInfo []*model.ColumnInfo + handleCols core.HandleCols + concurrency int + analyzePB *tipb.AnalyzeReq + commonHandle *model.IndexInfo + resultHandler *tableResultHandler + opts map[ast.AnalyzeOptionType]uint64 + job *statistics.AnalyzeJob + analyzeVer int + indexes []*model.IndexInfo + samplingBuilderWg *sync.WaitGroup + samplingMergeWg *sync.WaitGroup } func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { @@ -711,6 +713,17 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtS rootRowCollector.FMSketches = append(rootRowCollector.FMSketches, statistics.NewFMSketch(maxSketchSize)) } sc := e.ctx.GetSessionVars().StmtCtx + statsConcurrency, err := getBuildStatsConcurrency(e.ctx) + if err != nil { + return 0, nil, nil, nil, nil, err + } + mergeResultCh := make(chan *samplingMergeResult, statsConcurrency) + mergeTaskCh := make(chan []byte, statsConcurrency) + e.samplingMergeWg = &sync.WaitGroup{} + e.samplingMergeWg.Add(statsConcurrency) + for i := 0; i < statsConcurrency; i++ { + go e.subMergeWorker(mergeResultCh, mergeTaskCh, l, i == 0) + } for { data, err1 := e.resultHandler.nextRaw(context.TODO()) if err1 != nil { @@ -719,17 +732,26 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtS if data == nil { break } - colResp := &tipb.AnalyzeColumnsResp{} - err = colResp.Unmarshal(data) - if err != nil { - return 0, nil, nil, nil, nil, err + mergeTaskCh <- data + } + close(mergeTaskCh) + mergeWorkerPanicCnt := 0 + for mergeWorkerPanicCnt < statsConcurrency { + mergeResult, ok := <-mergeResultCh + if !ok { + break } - subCollector := &statistics.RowSampleCollector{ - MaxSampleSize: int(e.analyzePB.ColReq.SampleSize), + if mergeResult.err != nil { + err = mergeResult.err + if mergeResult.err == errAnalyzeWorkerPanic { + mergeWorkerPanicCnt++ + } + continue } - subCollector.FromProto(colResp.RowCollector) - e.job.Update(subCollector.Count) - rootRowCollector.MergeCollector(subCollector) + rootRowCollector.MergeCollector(mergeResult.collector) + } + if err != nil { + return 0, nil, nil, nil, nil, err } for _, sample := range rootRowCollector.Samples { for i := range sample.Columns { @@ -754,70 +776,57 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtS return rootRowCollector.Samples[i].Handle.Compare(rootRowCollector.Samples[j].Handle) < 0 }) - hists = make([]*statistics.Histogram, 0, len(e.colsInfo)) - topns = make([]*statistics.TopN, 0, len(e.colsInfo)) - fmSketches = make([]*statistics.FMSketch, 0, len(e.colsInfo)) - sampleCollectors := make([]*statistics.SampleCollector, 0, len(e.colsInfo)) + totalLen := len(e.colsInfo) + len(e.indexes) + hists = make([]*statistics.Histogram, totalLen) + topns = make([]*statistics.TopN, totalLen) + fmSketches = make([]*statistics.FMSketch, 0, totalLen) + buildResultChan := make(chan error, totalLen) + buildTaskChan := make(chan *samplingBuildTask, totalLen) + e.samplingBuilderWg = &sync.WaitGroup{} + e.samplingBuilderWg.Add(statsConcurrency) + sampleCollectors := make([]*statistics.SampleCollector, len(e.colsInfo)) + for i := 0; i < statsConcurrency; i++ { + go e.subBuildWorker(buildResultChan, buildTaskChan, hists, topns, sampleCollectors, i == 0) + } for i, col := range e.colsInfo { - sampleItems := make([]*statistics.SampleItem, 0, rootRowCollector.MaxSampleSize) - for j, row := range rootRowCollector.Samples { - if row.Columns[i].IsNull() { - continue - } - sampleItems = append(sampleItems, &statistics.SampleItem{ - Value: row.Columns[i], - Ordinal: j, - }) - } - collector := &statistics.SampleCollector{ - Samples: sampleItems, - NullCount: rootRowCollector.NullCount[i], - Count: rootRowCollector.Count - rootRowCollector.NullCount[i], - FMSketch: rootRowCollector.FMSketches[i], - TotalSize: rootRowCollector.TotalSizes[i], - } - sampleCollectors = append(sampleCollectors, collector) - hg, topn, err := statistics.BuildHistAndTopN(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), col.ID, collector, &col.FieldType, true) - if err != nil { - return 0, nil, nil, nil, nil, err + buildTaskChan <- &samplingBuildTask{ + id: col.ID, + rootRowCollector: rootRowCollector, + tp: &col.FieldType, + isColumn: true, + slicePos: i, } - hists = append(hists, hg) - topns = append(topns, topn) fmSketches = append(fmSketches, rootRowCollector.FMSketches[i]) } colLen := len(e.colsInfo) for i, idx := range e.indexes { - sampleItems := make([]*statistics.SampleItem, 0, rootRowCollector.MaxSampleSize) - for _, row := range rootRowCollector.Samples { - if len(idx.Columns) == 1 && row.Columns[idx.Columns[0].Offset].IsNull() { - continue - } - b := make([]byte, 0, 8) - for _, col := range idx.Columns { - b, err = codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, b, row.Columns[col.Offset]) - if err != nil { - return 0, nil, nil, nil, nil, err - } - } - sampleItems = append(sampleItems, &statistics.SampleItem{ - Value: types.NewBytesDatum(b), - }) - } - collector := &statistics.SampleCollector{ - Samples: sampleItems, - NullCount: rootRowCollector.NullCount[colLen+i], - Count: rootRowCollector.Count - rootRowCollector.NullCount[colLen+i], - FMSketch: rootRowCollector.FMSketches[colLen+i], - TotalSize: rootRowCollector.TotalSizes[colLen+i], - } - hg, topn, err := statistics.BuildHistAndTopN(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), idx.ID, collector, types.NewFieldType(mysql.TypeBlob), false) - if err != nil { - return 0, nil, nil, nil, nil, err + buildTaskChan <- &samplingBuildTask{ + id: idx.ID, + rootRowCollector: rootRowCollector, + tp: types.NewFieldType(mysql.TypeBlob), + isColumn: false, + slicePos: colLen + i, } - hists = append(hists, hg) - topns = append(topns, topn) fmSketches = append(fmSketches, rootRowCollector.FMSketches[colLen+i]) } + close(buildTaskChan) + panicCnt := 0 + for panicCnt < statsConcurrency { + err1, ok := <-buildResultChan + if !ok { + break + } + if err1 != nil { + err = err1 + if err1 == errAnalyzeWorkerPanic { + panicCnt++ + } + continue + } + } + if err != nil { + return 0, nil, nil, nil, nil, err + } count = rootRowCollector.Count if needExtStats { statsHandle := domain.GetDomain(e.ctx).StatsHandle() @@ -829,6 +838,163 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtS return } +type samplingMergeResult struct { + collector *statistics.RowSampleCollector + err error +} + +func (e *AnalyzeColumnsExec) subMergeWorker(resultCh chan<- *samplingMergeResult, taskCh <-chan []byte, l int, isClosedChanThread bool) { + defer func() { + if r := recover(); r != nil { + buf := make([]byte, 4096) + stackSize := runtime.Stack(buf, false) + buf = buf[:stackSize] + logutil.BgLogger().Error("analyze worker panicked", zap.String("stack", string(buf))) + metrics.PanicCounter.WithLabelValues(metrics.LabelAnalyze).Inc() + resultCh <- &samplingMergeResult{err: errAnalyzeWorkerPanic} + } + // Consume the remaining things. + for { + _, ok := <-taskCh + if !ok { + break + } + } + e.samplingMergeWg.Done() + if isClosedChanThread { + e.samplingMergeWg.Wait() + close(resultCh) + } + }() + failpoint.Inject("mockAnalyzeSamplingMergeWorkerPanic", func() { + panic("failpoint triggered") + }) + retCollector := &statistics.RowSampleCollector{ + NullCount: make([]int64, l), + FMSketches: make([]*statistics.FMSketch, 0, l), + TotalSizes: make([]int64, l), + Samples: make(statistics.WeightedRowSampleHeap, 0, e.analyzePB.ColReq.SampleSize), + MaxSampleSize: int(e.analyzePB.ColReq.SampleSize), + } + for i := 0; i < l; i++ { + retCollector.FMSketches = append(retCollector.FMSketches, statistics.NewFMSketch(maxSketchSize)) + } + for { + data, ok := <-taskCh + if !ok { + break + } + colResp := &tipb.AnalyzeColumnsResp{} + err := colResp.Unmarshal(data) + if err != nil { + resultCh <- &samplingMergeResult{err: err} + return + } + subCollector := &statistics.RowSampleCollector{ + MaxSampleSize: int(e.analyzePB.ColReq.SampleSize), + } + subCollector.FromProto(colResp.RowCollector) + e.job.Update(subCollector.Count) + retCollector.MergeCollector(subCollector) + } + resultCh <- &samplingMergeResult{collector: retCollector} +} + +type samplingBuildTask struct { + id int64 + rootRowCollector *statistics.RowSampleCollector + tp *types.FieldType + isColumn bool + slicePos int +} + +func (e *AnalyzeColumnsExec) subBuildWorker(resultCh chan error, taskCh chan *samplingBuildTask, hists []*statistics.Histogram, topns []*statistics.TopN, collectors []*statistics.SampleCollector, isClosedChanThread bool) { + defer func() { + if r := recover(); r != nil { + buf := make([]byte, 4096) + stackSize := runtime.Stack(buf, false) + buf = buf[:stackSize] + logutil.BgLogger().Error("analyze worker panicked", zap.String("stack", string(buf))) + metrics.PanicCounter.WithLabelValues(metrics.LabelAnalyze).Inc() + resultCh <- errAnalyzeWorkerPanic + } + e.samplingBuilderWg.Done() + if isClosedChanThread { + e.samplingBuilderWg.Wait() + close(resultCh) + } + }() + failpoint.Inject("mockAnalyzeSamplingBuildWorkerPanic", func() { + panic("failpoint triggered") + }) + colLen := len(e.colsInfo) +workLoop: + for { + task, ok := <-taskCh + if !ok { + break + } + var collector *statistics.SampleCollector + if task.isColumn { + sampleItems := make([]*statistics.SampleItem, 0, task.rootRowCollector.MaxSampleSize) + for j, row := range task.rootRowCollector.Samples { + if row.Columns[task.slicePos].IsNull() { + continue + } + sampleItems = append(sampleItems, &statistics.SampleItem{ + Value: row.Columns[task.slicePos], + Ordinal: j, + }) + } + collector = &statistics.SampleCollector{ + Samples: sampleItems, + NullCount: task.rootRowCollector.NullCount[task.slicePos], + Count: task.rootRowCollector.Count - task.rootRowCollector.NullCount[task.slicePos], + FMSketch: task.rootRowCollector.FMSketches[task.slicePos], + TotalSize: task.rootRowCollector.TotalSizes[task.slicePos], + } + } else { + idx := e.indexes[task.slicePos-colLen] + sampleItems := make([]*statistics.SampleItem, 0, task.rootRowCollector.MaxSampleSize) + for _, row := range task.rootRowCollector.Samples { + if len(idx.Columns) == 1 && row.Columns[idx.Columns[0].Offset].IsNull() { + continue + } + b := make([]byte, 0, 8) + var err error + for _, col := range idx.Columns { + b, err = codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, b, row.Columns[col.Offset]) + if err != nil { + resultCh <- err + continue workLoop + } + } + sampleItems = append(sampleItems, &statistics.SampleItem{ + Value: types.NewBytesDatum(b), + }) + } + collector = &statistics.SampleCollector{ + Samples: sampleItems, + NullCount: task.rootRowCollector.NullCount[task.slicePos], + Count: task.rootRowCollector.Count - task.rootRowCollector.NullCount[task.slicePos], + FMSketch: task.rootRowCollector.FMSketches[task.slicePos], + TotalSize: task.rootRowCollector.TotalSizes[task.slicePos], + } + } + if task.isColumn { + collectors[task.slicePos] = collector + } + hist, topn, err := statistics.BuildHistAndTopN(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), task.id, collector, task.tp, task.isColumn) + if err != nil { + resultCh <- err + continue + } + hists[task.slicePos] = hist + topns[task.slicePos] = topn + resultCh <- nil + } +} + func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range, needExtStats bool) (hists []*statistics.Histogram, cms []*statistics.CMSketch, topNs []*statistics.TopN, fms []*statistics.FMSketch, extStats *statistics.ExtendedStatsColl, err error) { if err = e.open(ranges); err != nil { return nil, nil, nil, nil, nil, err diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 45be004a2578a..6b1e37b743f74 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -963,3 +963,22 @@ func (s *testSuite1) TestAnalyzeClusteredIndexPrimary(c *C) { "test t0 PRIMARY 1 0 1 1 1111 1111 0", "test t1 PRIMARY 1 0 1 1 1111 1111 0")) } + +func (s *testSuite2) TestAnalyzeSamplingWorkPanic(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("set @@session.tidb_analyze_version = 3") + tk.MustExec("create table t(a int)") + tk.MustExec("insert into t values(1), (2), (3), (4), (5), (6), (7), (8), (9), (10), (11), (12)") + tk.MustExec("split table t between (-9223372036854775808) and (9223372036854775807) regions 12") + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingBuildWorkerPanic", "return(1)"), IsNil) + err := tk.ExecToErr("analyze table t") + c.Assert(err, NotNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingBuildWorkerPanic"), IsNil) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingMergeWorkerPanic", "return(1)"), IsNil) + err = tk.ExecToErr("analyze table t") + c.Assert(err, NotNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeSamplingMergeWorkerPanic"), IsNil) +} From 057250c7ab41ca32f5a3ac5238ec0d8903543235 Mon Sep 17 00:00:00 2001 From: Lei Zhao Date: Thu, 3 Jun 2021 23:52:26 +0800 Subject: [PATCH 275/343] store/tikv: better handle undetermined error for committing primary key (#25115) --- store/tikv/commit.go | 28 +++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/store/tikv/commit.go b/store/tikv/commit.go index 73ed6bc0b422a..b6456bbe8d3cb 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -42,7 +42,7 @@ func (actionCommit) tiKVTxnRegionsNumHistogram() prometheus.Observer { return metrics.TxnRegionsNumHistogramCommit } -func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) (err error) { +func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { keys := batch.mutations.GetKeys() req := tikvrpc.NewRequest(tikvrpc.CmdCommit, &pb.CommitRequest{ StartVersion: c.startTS, @@ -54,18 +54,6 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch attempts := 0 sender := NewRegionRequestSender(c.store.regionCache, c.store.GetTiKVClient()) - defer func() { - if err != nil { - // If we fail to receive response for the request that commits primary key, it will be undetermined whether this - // transaction has been successfully committed. - // Under this circumstance, we can not declare the commit is complete (may lead to data lost), nor can we throw - // an error (may lead to the duplicated key error when upper level restarts the transaction). Currently the best - // solution is to populate this error and let upper layer drop the connection to the corresponding mysql client. - if batch.isPrimary && sender.rpcError != nil && !c.isAsyncCommit() { - c.setUndeterminedErr(errors.Trace(sender.rpcError)) - } - } - }() for { attempts++ if time.Since(tBegin) > slowRequestThreshold { @@ -74,6 +62,15 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch } resp, err := sender.SendReq(bo, req, batch.region, client.ReadTimeoutShort) + // If we fail to receive response for the request that commits primary key, it will be undetermined whether this + // transaction has been successfully committed. + // Under this circumstance, we can not declare the commit is complete (may lead to data lost), nor can we throw + // an error (may lead to the duplicated key error when upper level restarts the transaction). Currently the best + // solution is to populate this error and let upper layer drop the connection to the corresponding mysql client. + if batch.isPrimary && sender.rpcError != nil && !c.isAsyncCommit() { + c.setUndeterminedErr(errors.Trace(sender.rpcError)) + } + // Unexpected error occurs, return it. if err != nil { return errors.Trace(err) @@ -108,6 +105,11 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch return errors.Trace(tikverr.ErrBodyMissing) } commitResp := resp.Resp.(*pb.CommitResponse) + // Here we can make sure tikv has processed the commit primary key request. So + // we can clean undetermined error. + if batch.isPrimary && !c.isAsyncCommit() { + c.setUndeterminedErr(nil) + } if keyErr := commitResp.GetError(); keyErr != nil { if rejected := keyErr.GetCommitTsExpired(); rejected != nil { logutil.Logger(bo.GetCtx()).Info("2PC commitTS rejected by TiKV, retry with a newer commitTS", From 66e68be2965864637e01e32bb3510185cbfb37b6 Mon Sep 17 00:00:00 2001 From: MyonKeminta <9948422+MyonKeminta@users.noreply.github.com> Date: Fri, 4 Jun 2021 00:16:27 +0800 Subject: [PATCH 276/343] sessionctx: Fix tidb_gc_scan_lock_mode sysvar doesn't show correct default value on new clusters (#25112) --- sessionctx/variable/sysvar.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 6e4079ca3cc03..61ea0cdb17344 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1667,7 +1667,7 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBGCRunInterval, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64}, {Scope: ScopeGlobal, Name: TiDBGCLifetime, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64}, {Scope: ScopeGlobal, Name: TiDBGCConcurrency, Value: "-1", Type: TypeInt, MinValue: 1, MaxValue: 128, AllowAutoValue: true}, - {Scope: ScopeGlobal, Name: TiDBGCScanLockMode, Value: "PHYSICAL", Type: TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}}, + {Scope: ScopeGlobal, Name: TiDBGCScanLockMode, Value: "LEGACY", Type: TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}}, // variable for top SQL feature. {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { From 6154335ac01731bdcfdabbb978cd121f656956d2 Mon Sep 17 00:00:00 2001 From: Zhi Qi <30543181+LittleFall@users.noreply.github.com> Date: Fri, 4 Jun 2021 10:22:27 +0800 Subject: [PATCH 277/343] expression: Support push function year,day,datediff,datesub,castTimeAsString,concat_ws down to TiFlash. (#25097) --- expression/expr_to_pb_test.go | 44 +++++++++++++++++++++++++++++++++++ expression/expression.go | 23 +++++++++++------- 2 files changed, 58 insertions(+), 9 deletions(-) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index c7b97e0227626..6fb3a152f8f76 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -769,6 +769,50 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_UnixTimestampDec) exprs = append(exprs, function) + // Year + function, err = NewFunction(mock.NewContext(), ast.Year, types.NewFieldType(mysql.TypeLonglong), datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_Year) + exprs = append(exprs, function) + + // Day + function, err = NewFunction(mock.NewContext(), ast.Day, types.NewFieldType(mysql.TypeLonglong), datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_DayOfMonth) + exprs = append(exprs, function) + + // Datediff + function, err = NewFunction(mock.NewContext(), ast.DateDiff, types.NewFieldType(mysql.TypeLonglong), datetimeColumn, datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_DateDiff) + exprs = append(exprs, function) + + // Datesub + function, err = NewFunction(mock.NewContext(), ast.DateSub, types.NewFieldType(mysql.TypeDatetime), datetimeColumn, intColumn, stringColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_SubDateDatetimeInt) + exprs = append(exprs, function) + function, err = NewFunction(mock.NewContext(), ast.DateSub, types.NewFieldType(mysql.TypeDatetime), stringColumn, intColumn, stringColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_SubDateStringInt) + exprs = append(exprs, function) + function, err = NewFunction(mock.NewContext(), ast.SubDate, types.NewFieldType(mysql.TypeDatetime), datetimeColumn, intColumn, stringColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_SubDateDatetimeInt) + exprs = append(exprs, function) + + // castTimeAsString: + function, err = NewFunction(mock.NewContext(), ast.Cast, types.NewFieldType(mysql.TypeString), datetimeColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_CastTimeAsString) + exprs = append(exprs, function) + + // concat_ws + function, err = NewFunction(mock.NewContext(), ast.ConcatWS, types.NewFieldType(mysql.TypeString), stringColumn, stringColumn, stringColumn) + c.Assert(err, IsNil) + c.Assert(function.(*ScalarFunction).Function.PbCode(), Equals, tipb.ScalarFuncSig_ConcatWS) + exprs = append(exprs, function) + // StrToDateDateTime function, err = NewFunction(mock.NewContext(), ast.StrToDate, types.NewFieldType(mysql.TypeDatetime), stringColumn, stringColumn) c.Assert(err, IsNil) diff --git a/expression/expression.go b/expression/expression.go index cbbc200811e23..12dccd4a7dae5 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1001,9 +1001,9 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { ast.GE, ast.LE, ast.EQ, ast.NE, ast.LT, ast.GT, ast.In, ast.IsNull, ast.Like, ast.Plus, ast.Minus, ast.Div, ast.Mul, /*ast.Mod,*/ ast.If, ast.Ifnull, ast.Case, - ast.Concat, - ast.Month, - ast.TimestampDiff, ast.DateFormat, ast.FromUnixTime, + ast.Concat, ast.ConcatWS, + ast.Year, ast.Month, ast.Day, + ast.DateDiff, ast.TimestampDiff, ast.DateFormat, ast.FromUnixTime, ast.JSONLength: return true case ast.Substr, ast.Substring: @@ -1015,18 +1015,23 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { } case ast.Cast: switch function.Function.PbCode() { - case tipb.ScalarFuncSig_CastIntAsInt, tipb.ScalarFuncSig_CastIntAsDecimal, tipb.ScalarFuncSig_CastIntAsString, tipb.ScalarFuncSig_CastIntAsTime, - tipb.ScalarFuncSig_CastRealAsInt, tipb.ScalarFuncSig_CastRealAsDecimal, tipb.ScalarFuncSig_CastRealAsString, tipb.ScalarFuncSig_CastRealAsTime, - tipb.ScalarFuncSig_CastStringAsInt, tipb.ScalarFuncSig_CastStringAsDecimal, tipb.ScalarFuncSig_CastStringAsString, tipb.ScalarFuncSig_CastStringAsTime, tipb.ScalarFuncSig_CastStringAsReal, - tipb.ScalarFuncSig_CastDecimalAsInt, tipb.ScalarFuncSig_CastDecimalAsDecimal, tipb.ScalarFuncSig_CastDecimalAsString, tipb.ScalarFuncSig_CastDecimalAsTime, - tipb.ScalarFuncSig_CastTimeAsInt, tipb.ScalarFuncSig_CastTimeAsDecimal, tipb.ScalarFuncSig_CastTimeAsTime, tipb.ScalarFuncSig_CastIntAsReal, tipb.ScalarFuncSig_CastRealAsReal: + case tipb.ScalarFuncSig_CastIntAsInt, tipb.ScalarFuncSig_CastIntAsReal, tipb.ScalarFuncSig_CastIntAsDecimal, tipb.ScalarFuncSig_CastIntAsString, tipb.ScalarFuncSig_CastIntAsTime, + tipb.ScalarFuncSig_CastRealAsInt, tipb.ScalarFuncSig_CastRealAsReal, tipb.ScalarFuncSig_CastRealAsDecimal, tipb.ScalarFuncSig_CastRealAsString, tipb.ScalarFuncSig_CastRealAsTime, + tipb.ScalarFuncSig_CastStringAsInt, tipb.ScalarFuncSig_CastStringAsReal, tipb.ScalarFuncSig_CastStringAsDecimal, tipb.ScalarFuncSig_CastStringAsString, tipb.ScalarFuncSig_CastStringAsTime, + tipb.ScalarFuncSig_CastDecimalAsInt /*, tipb.ScalarFuncSig_CastDecimalAsReal*/, tipb.ScalarFuncSig_CastDecimalAsDecimal, tipb.ScalarFuncSig_CastDecimalAsString, tipb.ScalarFuncSig_CastDecimalAsTime, + tipb.ScalarFuncSig_CastTimeAsInt /*, tipb.ScalarFuncSig_CastTimeAsReal*/, tipb.ScalarFuncSig_CastTimeAsDecimal, tipb.ScalarFuncSig_CastTimeAsTime, tipb.ScalarFuncSig_CastTimeAsString: return true } - case ast.DateAdd: + case ast.DateAdd, ast.AddDate: switch function.Function.PbCode() { case tipb.ScalarFuncSig_AddDateDatetimeInt, tipb.ScalarFuncSig_AddDateStringInt: return true } + case ast.DateSub, ast.SubDate: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_SubDateDatetimeInt, tipb.ScalarFuncSig_SubDateStringInt: + return true + } case ast.UnixTimestamp: switch function.Function.PbCode() { case tipb.ScalarFuncSig_UnixTimestampInt, tipb.ScalarFuncSig_UnixTimestampDec: From 5a74d52e4687c26c0d39f3a28c133bccc3bb0f66 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Fri, 4 Jun 2021 11:38:27 +0800 Subject: [PATCH 278/343] store/tikv: clear undetermined error if txn determines to fail (#25120) --- store/tikv/2pc.go | 10 ++++++--- store/tikv/prewrite.go | 8 ++++++- store/tikv/tests/async_commit_fail_test.go | 26 ++++++++++++++++++++++ 3 files changed, 40 insertions(+), 4 deletions(-) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 66957629e2473..049b944799aa6 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -101,6 +101,7 @@ type twoPhaseCommitter struct { maxCommitTS uint64 prewriteStarted bool prewriteCancelled uint32 + prewriteFailed uint32 useOnePC uint32 onePCCommitTS uint64 @@ -1043,10 +1044,13 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { start := time.Now() err = c.prewriteMutations(bo, c.mutations) + // Return an undetermined error only if we don't know the transaction fails. + // If it fails due to a write conflict or a already existed unique key, we + // needn't return an undetermined error even if such an error is set. + if atomic.LoadUint32(&c.prewriteFailed) == 1 { + c.setUndeterminedErr(nil) + } if err != nil { - // TODO: Now we return an undetermined error as long as one of the prewrite - // RPCs fails. However, if there are multiple errors and some of the errors - // are not RPC failures, we can return the actual error instead of undetermined. if undeterminedErr := c.getUndeterminedErr(); undeterminedErr != nil { logutil.Logger(ctx).Error("2PC commit result undetermined", zap.Error(err), diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index e5a3c0c8419da..0e6afa96010b2 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -203,6 +203,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff if err != nil { return errors.Trace(err) } + failpoint.Inject("forceRecursion", func() { same = false }) if same { continue } @@ -276,12 +277,17 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff // Check already exists error if alreadyExist := keyErr.GetAlreadyExist(); alreadyExist != nil { e := &tikverr.ErrKeyExist{AlreadyExist: alreadyExist} - return c.extractKeyExistsErr(e) + err = c.extractKeyExistsErr(e) + if err != nil { + atomic.StoreUint32(&c.prewriteFailed, 1) + } + return err } // Extract lock from key error lock, err1 := extractLockFromKeyErr(keyErr) if err1 != nil { + atomic.StoreUint32(&c.prewriteFailed, 1) return errors.Trace(err1) } logutil.BgLogger().Info("prewrite encounters lock", diff --git a/store/tikv/tests/async_commit_fail_test.go b/store/tikv/tests/async_commit_fail_test.go index aa65c534aa155..cfe1ad0595eb1 100644 --- a/store/tikv/tests/async_commit_fail_test.go +++ b/store/tikv/tests/async_commit_fail_test.go @@ -255,3 +255,29 @@ func (s *testAsyncCommitFailSuite) TestAsyncCommitRPCErrorThenWriteConflict(c *C c.Assert(err, NotNil) c.Assert(txn.GetCommitter().GetUndeterminedErr(), IsNil) } + +// TestAsyncCommitRPCErrorThenWriteConflictInChild verifies that the determined failure error in a child recursion +// overwrites the undetermined error in the parent. +func (s *testAsyncCommitFailSuite) TestAsyncCommitRPCErrorThenWriteConflictInChild(c *C) { + // This test doesn't support tikv mode because it needs setting failpoint in unistore. + if *WithTiKV { + return + } + + txn := s.beginAsyncCommit(c) + err := txn.Set([]byte("a"), []byte("va")) + c.Assert(err, IsNil) + + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/rpcPrewriteResult", `1*return("timeout")->return("writeConflict")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/forceRecursion", `return`), IsNil) + + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/rpcPrewriteResult"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/forceRecursion"), IsNil) + }() + + ctx := context.WithValue(context.Background(), util.SessionID, uint64(1)) + err = txn.Commit(ctx) + c.Assert(err, NotNil) + c.Assert(txn.GetCommitter().GetUndeterminedErr(), IsNil) +} From 99b1fa56b604ab9056e859e5a4f3628d43aab7cd Mon Sep 17 00:00:00 2001 From: Zhuhe Fang Date: Fri, 4 Jun 2021 11:54:27 +0800 Subject: [PATCH 279/343] expression: push down left/right/abs to tiflash (#25018) --- expression/expression.go | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/expression/expression.go b/expression/expression.go index 12dccd4a7dae5..5adb4af16a5ea 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -999,16 +999,19 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { case ast.LogicOr, ast.LogicAnd, ast.UnaryNot, ast.BitNeg, ast.Xor, ast.And, ast.Or, ast.GE, ast.LE, ast.EQ, ast.NE, ast.LT, ast.GT, ast.In, ast.IsNull, ast.Like, - ast.Plus, ast.Minus, ast.Div, ast.Mul, /*ast.Mod,*/ + ast.Plus, ast.Minus, ast.Div, ast.Mul, ast.Abs, /*ast.Mod,*/ ast.If, ast.Ifnull, ast.Case, ast.Concat, ast.ConcatWS, ast.Year, ast.Month, ast.Day, ast.DateDiff, ast.TimestampDiff, ast.DateFormat, ast.FromUnixTime, ast.JSONLength: return true - case ast.Substr, ast.Substring: + case ast.Substr, ast.Substring, ast.Left, ast.Right, ast.CharLength: switch function.Function.PbCode() { case + tipb.ScalarFuncSig_LeftUTF8, + tipb.ScalarFuncSig_RightUTF8, + tipb.ScalarFuncSig_CharLengthUTF8, tipb.ScalarFuncSig_Substring2ArgsUTF8, tipb.ScalarFuncSig_Substring3ArgsUTF8: return true From c59b3bcea6ef0b47ff4b358043ddd7432b4d1996 Mon Sep 17 00:00:00 2001 From: xufei Date: Fri, 4 Jun 2021 12:44:27 +0800 Subject: [PATCH 280/343] planner: Mpp outer join build side (#25130) --- planner/core/exhaust_physical_plans.go | 16 ++- planner/core/integration_test.go | 132 ++++++++++++++++++ .../testdata/integration_serial_suite_in.json | 21 +++ .../integration_serial_suite_out.json | 107 ++++++++++++++ sessionctx/variable/session.go | 4 + sessionctx/variable/sysvar.go | 9 +- sessionctx/variable/tidb_vars.go | 3 + 7 files changed, 290 insertions(+), 2 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 6ad716c93cac2..a0c0072e94a33 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1808,9 +1808,23 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC if p.children[0].statsInfo().Count() > p.children[1].statsInfo().Count() { preferredBuildIndex = 1 } - } else if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin || p.JoinType == LeftOuterJoin { + } else if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin { preferredBuildIndex = 1 } + if p.JoinType == LeftOuterJoin || p.JoinType == RightOuterJoin { + // TiFlash does not requires that the build side must be the inner table for outer join + // so we can choose the build side based on the row count, except that + // 1. it is a broadcast join(for broadcast join, it make sense to use the broadcast side as the build side) + // 2. or session variable MPPOuterJoinFixedBuildSide is set to true + // 3. or there are otherConditions for this join + if useBCJ || p.ctx.GetSessionVars().MPPOuterJoinFixedBuildSide || len(p.OtherConditions) > 0 { + if p.JoinType == LeftOuterJoin { + preferredBuildIndex = 1 + } + } else if p.children[0].statsInfo().Count() > p.children[1].statsInfo().Count() { + preferredBuildIndex = 1 + } + } baseJoin.InnerChildIdx = preferredBuildIndex childrenProps := make([]*property.PhysicalProperty, 2) if useBCJ { diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index bbfe52a0ee710..d441cd673e3b9 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -554,6 +554,138 @@ func (s *testIntegrationSerialSuite) TestMPPJoin(c *C) { } } +func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForBroadcastJoin(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists a") + tk.MustExec("create table a(id int, value int)") + tk.MustExec("insert into a values(1,2),(2,3)") + tk.MustExec("analyze table a") + tk.MustExec("drop table if exists b") + tk.MustExec("create table b(id int, value int)") + tk.MustExec("insert into b values(1,2),(2,3),(3,4)") + tk.MustExec("analyze table b") + // 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 == "a" || tblInfo.Name.L == "b" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 10000") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 10000") + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists a") + tk.MustExec("create table a(id int, value int)") + tk.MustExec("insert into a values(1,2),(2,3)") + tk.MustExec("analyze table a") + tk.MustExec("drop table if exists b") + tk.MustExec("create table b(id int, value int)") + tk.MustExec("insert into b values(1,2),(2,3),(3,4)") + tk.MustExec("analyze table b") + // 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 == "a" || tblInfo.Name.L == "b" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 1") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + +func (s *testIntegrationSerialSuite) TestMPPOuterJoinBuildSideForShuffleJoin(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists a") + tk.MustExec("create table a(id int, value int)") + tk.MustExec("insert into a values(1,2),(2,3)") + tk.MustExec("analyze table a") + tk.MustExec("drop table if exists b") + tk.MustExec("create table b(id int, value int)") + tk.MustExec("insert into b values(1,2),(2,3),(3,4)") + tk.MustExec("analyze table b") + // 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 == "a" || tblInfo.Name.L == "b" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + tk.MustExec("set @@session.tidb_opt_mpp_outer_join_fixed_build_side = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_size = 0") + tk.MustExec("set @@session.tidb_broadcast_join_threshold_count = 0") + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } +} + func (s *testIntegrationSerialSuite) TestMPPShuffledJoin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 8d6f3dee0bc53..0cc21a8c36e4b 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -59,6 +59,27 @@ "explain format = 'brief' select count(*) from fact_t where d1_k not in (select d1_k from d1_t)" ] }, + { + "name": "TestMPPOuterJoinBuildSideForBroadcastJoin", + "cases": [ + "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "explain format = 'brief' select count(*) from b right join a on a.id = b.id" + ] + }, + { + "name": "TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide", + "cases": [ + "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "explain format = 'brief' select count(*) from b right join a on a.id = b.id" + ] + }, + { + "name": "TestMPPOuterJoinBuildSideForShuffleJoin", + "cases": [ + "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "explain format = 'brief' select count(*) from b right join a on a.id = b.id" + ] + }, { "name": "TestMPPShuffledJoin", "cases": [ diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 46f04a2af0340..a2bb282dc3b4a 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -522,6 +522,113 @@ } ] }, + { + "Name": "TestMPPOuterJoinBuildSideForBroadcastJoin", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─TableFullScan(Probe) 2.00 cop[tiflash] table:a keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─TableFullScan(Probe) 2.00 cop[tiflash] table:a keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPOuterJoinBuildSideForShuffleJoinWithFixedBuildSide", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.b.id", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.a.id", + " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 3.00 cop[tiflash] ", + " │ └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.b.id", + " │ └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " │ └─TableFullScan 3.00 cop[tiflash] table:b keep order:false", + " └─ExchangeReceiver(Probe) 2.00 cop[tiflash] ", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.a.id", + " └─TableFullScan 2.00 cop[tiflash] table:a keep order:false" + ] + } + ] + }, + { + "Name": "TestMPPOuterJoinBuildSideForShuffleJoin", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] left outer join, equal:[eq(test.a.id, test.b.id)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.a.id", + " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 3.00 cop[tiflash] ", + " └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.b.id", + " └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " └─TableFullScan 3.00 cop[tiflash] table:b keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from b right join a on a.id = b.id", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#7", + "└─TableReader 2.00 root data:ExchangeSender", + " └─ExchangeSender 2.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 2.00 cop[tiflash] right outer join, equal:[eq(test.b.id, test.a.id)]", + " ├─ExchangeReceiver(Build) 2.00 cop[tiflash] ", + " │ └─ExchangeSender 2.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.a.id", + " │ └─TableFullScan 2.00 cop[tiflash] table:a keep order:false", + " └─ExchangeReceiver(Probe) 3.00 cop[tiflash] ", + " └─ExchangeSender 3.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.b.id", + " └─Selection 3.00 cop[tiflash] not(isnull(test.b.id))", + " └─TableFullScan 3.00 cop[tiflash] table:b keep order:false" + ] + } + ] + }, { "Name": "TestMPPShuffledJoin", "Cases": [ diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 16172ca1ed213..93db60de55e50 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -492,6 +492,9 @@ type SessionVars struct { // size exceeds the broadcast threshold AllowCartesianBCJ int + // MPPOuterJoinFixedBuildSide means in MPP plan, always use right(left) table as build side for left(right) out join + MPPOuterJoinFixedBuildSide bool + // AllowDistinctAggPushDown can be set true to allow agg with distinct push down to tikv/tiflash. AllowDistinctAggPushDown bool @@ -978,6 +981,7 @@ func NewSessionVars() *SessionVars { AllowAggPushDown: false, AllowBCJ: false, AllowCartesianBCJ: DefOptCartesianBCJ, + MPPOuterJoinFixedBuildSide: DefOptMPPOuterJoinFixedBuildSide, BroadcastJoinThresholdSize: DefBroadcastJoinThresholdSize, BroadcastJoinThresholdCount: DefBroadcastJoinThresholdSize, OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 61ea0cdb17344..295401b0a97c3 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -877,7 +877,14 @@ var defaultSysVars = []*SysVar{ return nil }}, {Scope: ScopeGlobal | ScopeSession, Name: TiDBBuildStatsConcurrency, skipInit: true, Value: strconv.Itoa(DefBuildStatsConcurrency)}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCartesianBCJ, Value: strconv.Itoa(DefOptCartesianBCJ), Type: TypeInt, MinValue: 0, MaxValue: 2}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptCartesianBCJ, Value: strconv.Itoa(DefOptCartesianBCJ), Type: TypeInt, MinValue: 0, MaxValue: 2, SetSession: func(s *SessionVars, val string) error { + s.AllowCartesianBCJ = tidbOptInt(val, DefOptCartesianBCJ) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptMPPOuterJoinFixedBuildSide, Value: BoolToOnOff(DefOptMPPOuterJoinFixedBuildSide), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.MPPOuterJoinFixedBuildSide = TiDBOptOn(val) + return nil + }}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeRatio, Value: strconv.FormatFloat(DefAutoAnalyzeRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0, MaxValue: math.MaxUint64}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeStartTime, Value: DefAutoAnalyzeStartTime, Type: TypeTime}, {Scope: ScopeGlobal, Name: TiDBAutoAnalyzeEndTime, Value: DefAutoAnalyzeEndTime, Type: TypeTime}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 29f10bda8cdb8..68c5be3ee6910 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -52,6 +52,8 @@ const ( // TiDBOptCartesianBCJ is used to disable/enable broadcast cartesian join in MPP mode TiDBOptCartesianBCJ = "tidb_opt_broadcast_cartesian_join" + TiDBOptMPPOuterJoinFixedBuildSide = "tidb_opt_mpp_outer_join_fixed_build_side" + // tidb_opt_distinct_agg_push_down is used to decide whether agg with distinct should be pushed to tikv/tiflash. TiDBOptDistinctAggPushDown = "tidb_opt_distinct_agg_push_down" @@ -594,6 +596,7 @@ const ( DefOptAggPushDown = false DefOptBCJ = false DefOptCartesianBCJ = 1 + DefOptMPPOuterJoinFixedBuildSide = true DefOptWriteRowID = false DefOptCorrelationThreshold = 0.9 DefOptCorrelationExpFactor = 1 From 7602a2774c3ed44409cb40cec1bf2a28b6eb77e3 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 4 Jun 2021 13:16:27 +0800 Subject: [PATCH 281/343] planner: add more test cases about tiflash and dynamic mode (#25111) --- executor/tiflash_test.go | 138 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 138 insertions(+) diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index e442a219ac1d2..736720e1a7348 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -15,6 +15,8 @@ package executor_test import ( "fmt" + "math/rand" + "strings" "sync" "sync/atomic" "time" @@ -248,6 +250,142 @@ func (s *tiflashTestSuite) TestInjectExtraProj(c *C) { tk.MustQuery("select avg(a), a from t group by a").Check(testkit.Rows("9223372036854775807.0000 9223372036854775807")) } +func (s *tiflashTestSuite) TestTiFlashPartitionTableShuffledHashJoin(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`create database tiflash_partition_SHJ`) + tk.MustExec("use tiflash_partition_SHJ") + tk.MustExec(`create table thash (a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int) partition by range(a) ( + partition p0 values less than (100), partition p1 values less than (200), + partition p2 values less than (300), partition p3 values less than (400))`) + listPartitions := make([]string, 4) + for i := 0; i < 400; i++ { + idx := i % 4 + if listPartitions[idx] != "" { + listPartitions[idx] += ", " + } + listPartitions[idx] = listPartitions[idx] + fmt.Sprintf("%v", i) + } + tk.MustExec(`create table tlist (a int, b int) partition by list(a) ( + partition p0 values in (` + listPartitions[0] + `), partition p1 values in (` + listPartitions[1] + `), + partition p2 values in (` + listPartitions[2] + `), partition p3 values in (` + listPartitions[3] + `))`) + tk.MustExec(`create table tnormal (a int, b int)`) + + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec("alter table " + tbl + " set tiflash replica 1") + tb := testGetTableByName(c, tk.Se, "tiflash_partition_SHJ", tbl) + err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + } + + vals := make([]string, 0, 100) + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(400), rand.Intn(400))) + } + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec(fmt.Sprintf("insert into %v values %v", tbl, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf("analyze table %v", tbl)) + } + + tk.MustExec("SET tidb_allow_mpp=2") + tk.MustExec("SET tidb_opt_broadcast_join=0") + tk.MustExec("SET tidb_broadcast_join_threshold_count=0") + tk.MustExec("SET tidb_broadcast_join_threshold_size=0") + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") + + lr := func() (int, int) { + l, r := rand.Intn(400), rand.Intn(400) + if l > r { + l, r = r, l + } + return l, r + } + for i := 0; i < 2; i++ { + l1, r1 := lr() + l2, r2 := lr() + cond := fmt.Sprintf("t1.b>=%v and t1.b<=%v and t2.b>=%v and t2.b<=%v", l1, r1, l2, r2) + var res [][]interface{} + for _, mode := range []string{"static", "dynamic"} { + tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", mode)) + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + q := fmt.Sprintf("select count(*) from %v t1 join %v t2 on t1.a=t2.a where %v", tbl, tbl, cond) + if res == nil { + res = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Check(res) + } + } + } + } +} + +func (s *tiflashTestSuite) TestTiFlashPartitionTableReader(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`create database tiflash_partition_tablereader`) + tk.MustExec("use tiflash_partition_tablereader") + tk.MustExec(`create table thash (a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int) partition by range(a) ( + partition p0 values less than (100), partition p1 values less than (200), + partition p2 values less than (300), partition p3 values less than (400))`) + listPartitions := make([]string, 4) + for i := 0; i < 400; i++ { + idx := i % 4 + if listPartitions[idx] != "" { + listPartitions[idx] += ", " + } + listPartitions[idx] = listPartitions[idx] + fmt.Sprintf("%v", i) + } + tk.MustExec(`create table tlist (a int, b int) partition by list(a) ( + partition p0 values in (` + listPartitions[0] + `), partition p1 values in (` + listPartitions[1] + `), + partition p2 values in (` + listPartitions[2] + `), partition p3 values in (` + listPartitions[3] + `))`) + tk.MustExec(`create table tnormal (a int, b int)`) + + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec("alter table " + tbl + " set tiflash replica 1") + tb := testGetTableByName(c, tk.Se, "tiflash_partition_tablereader", tbl) + err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + } + + vals := make([]string, 0, 500) + for i := 0; i < 500; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(400), rand.Intn(400))) + } + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec(fmt.Sprintf("insert into %v values %v", tbl, strings.Join(vals, ", "))) + } + + tk.MustExec("SET tidb_allow_mpp=2") + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") + for i := 0; i < 100; i++ { + l, r := rand.Intn(400), rand.Intn(400) + if l > r { + l, r = r, l + } + cond := fmt.Sprintf("a>=%v and a<=%v", l, r) + var res [][]interface{} + for _, mode := range []string{"static", "dynamic"} { + tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", mode)) + for _, tbl := range []string{"thash", "trange", "tlist", "tnormal"} { + q := fmt.Sprintf("select * from %v where %v", tbl, cond) + if res == nil { + res = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Sort().Check(res) + } + } + } + } +} + func (s *tiflashTestSuite) TestPartitionTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") From b07942836fa9ecca299727908f5d75e9dc89f158 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 4 Jun 2021 13:54:27 +0800 Subject: [PATCH 282/343] planner: fix a panic caused by sinking a Limit with inlined Proj into IndexLookUp when accessing a partition table (#25063) --- cmd/explaintest/r/explain_easy.result | 11 +++--- cmd/explaintest/r/explain_easy_stats.result | 11 +++--- executor/partition_table_test.go | 36 +++++++++++++++++++ planner/core/exhaust_physical_plans.go | 1 + planner/core/task.go | 10 ++++++ .../core/testdata/integration_suite_out.json | 11 +++--- 6 files changed, 65 insertions(+), 15 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 214e51b366de1..c2c61bf79b58e 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -124,11 +124,12 @@ Projection 10000.00 root eq(test.t1.c2, test.t2.c2)->Column#11 └─Apply 10000.00 root CARTESIAN left outer join ├─TableReader(Build) 10000.00 root data:TableFullScan │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo - └─Projection(Probe) 1.00 root test.t2.c1, test.t2.c2 - └─IndexLookUp 1.00 root limit embedded(offset:0, count:1) - ├─Limit(Build) 1.00 cop[tikv] offset:0, count:1 - │ └─IndexRangeScan 1.00 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo - └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─Limit(Probe) 1.00 root offset:0, count:1 + └─Projection 1.00 root test.t2.c1, test.t2.c2 + └─IndexLookUp 1.00 root + ├─Limit(Build) 1.00 cop[tikv] offset:0, count:1 + │ └─IndexRangeScan 1.00 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true, stats:pseudo + └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo explain format = 'brief' select * from t1 order by c1 desc limit 1; id estRows task access object operator info Limit 1.00 root offset:0, count:1 diff --git a/cmd/explaintest/r/explain_easy_stats.result b/cmd/explaintest/r/explain_easy_stats.result index e76b0d4d233a6..8305dfa929e2b 100644 --- a/cmd/explaintest/r/explain_easy_stats.result +++ b/cmd/explaintest/r/explain_easy_stats.result @@ -106,11 +106,12 @@ Projection 1999.00 root eq(test.t1.c2, test.t2.c2)->Column#11 └─Apply 1999.00 root CARTESIAN left outer join ├─TableReader(Build) 1999.00 root data:TableFullScan │ └─TableFullScan 1999.00 cop[tikv] table:t1 keep order:false - └─Projection(Probe) 1.00 root test.t2.c1, test.t2.c2 - └─IndexLookUp 1.00 root limit embedded(offset:0, count:1) - ├─Limit(Build) 1.00 cop[tikv] offset:0, count:1 - │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true - └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo + └─Limit(Probe) 1.00 root offset:0, count:1 + └─Projection 1.00 root test.t2.c1, test.t2.c2 + └─IndexLookUp 1.00 root + ├─Limit(Build) 1.00 cop[tikv] offset:0, count:1 + │ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true + └─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo explain format = 'brief' select * from t1 order by c1 desc limit 1; id estRows task access object operator info Limit 1.00 root offset:0, count:1 diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 34066a301bff4..84bf7513a6514 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -2403,6 +2403,42 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { } } +func (s *partitionTableSuite) TestIssue24636(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_issue_24636") + tk.MustExec("use test_issue_24636") + + tk.MustExec(`CREATE TABLE t (a int, b date, c int, PRIMARY KEY (a,b)) + PARTITION BY RANGE ( TO_DAYS(b) ) ( + PARTITION p0 VALUES LESS THAN (737821), + PARTITION p1 VALUES LESS THAN (738289) + )`) + tk.MustExec(`INSERT INTO t (a, b, c) VALUES(0, '2021-05-05', 0)`) + tk.MustQuery(`select c from t use index(primary) where a=0 limit 1`).Check(testkit.Rows("0")) + + tk.MustExec(` + CREATE TABLE test_partition ( + a varchar(100) NOT NULL, + b date NOT NULL, + c varchar(100) NOT NULL, + d datetime DEFAULT NULL, + e datetime DEFAULT NULL, + f bigint(20) DEFAULT NULL, + g bigint(20) DEFAULT NULL, + h bigint(20) DEFAULT NULL, + i bigint(20) DEFAULT NULL, + j bigint(20) DEFAULT NULL, + k bigint(20) DEFAULT NULL, + l bigint(20) DEFAULT NULL, + PRIMARY KEY (a,b,c) /*T![clustered_index] NONCLUSTERED */ + ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin + PARTITION BY RANGE ( TO_DAYS(b) ) ( + PARTITION pmin VALUES LESS THAN (737821), + PARTITION p20200601 VALUES LESS THAN (738289))`) + tk.MustExec(`INSERT INTO test_partition (a, b, c, d, e, f, g, h, i, j, k, l) VALUES('aaa', '2021-05-05', '428ff6a1-bb37-42ac-9883-33d7a29961e6', '2021-05-06 08:13:38', '2021-05-06 13:28:08', 0, 8, 3, 0, 9, 1, 0)`) + tk.MustQuery(`select c,j,l from test_partition where c='428ff6a1-bb37-42ac-9883-33d7a29961e6' and a='aaa' limit 0, 200`).Check(testkit.Rows("428ff6a1-bb37-42ac-9883-33d7a29961e6 9 0")) +} + func (s *partitionTableSuite) TestIdexMerge(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index a0c0072e94a33..0d8b183a6900a 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2061,6 +2061,7 @@ func (lt *LogicalTopN) getPhysLimits(prop *property.PhysicalProperty) []Physical Count: lt.Count, Offset: lt.Offset, }.Init(lt.ctx, lt.stats, lt.blockOffset, resultProp) + limit.SetSchema(lt.Schema()) ret = append(ret, limit) } return ret diff --git a/planner/core/task.go b/planner/core/task.go index d3b8b84d78ead..c1b925451ca1b 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1127,6 +1127,16 @@ func (p *PhysicalLimit) sinkIntoIndexLookUp(t task) bool { return false } } + + // If this happens, some Projection Operator must be inlined into this Limit. (issues/14428) + // For example, if the original plan is `IndexLookUp(col1, col2) -> Limit(col1, col2) -> Project(col1)`, + // then after inlining the Project, it will be `IndexLookUp(col1, col2) -> Limit(col1)` here. + // If the Limit is sunk into the IndexLookUp, the IndexLookUp's schema needs to be updated as well, + // but updating it here is not safe, so do not sink Limit into this IndexLookUp in this case now. + if p.Schema().Len() != reader.Schema().Len() { + return false + } + // We can sink Limit into IndexLookUpReader only if tablePlan contains no Selection. ts, isTableScan := reader.tablePlan.(*PhysicalTableScan) if !isTableScan { diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 77aa5b1494da7..72d5c968d7468 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -14,11 +14,12 @@ { "SQL": "explain format = 'brief' select * from tbl use index(idx_b_c) where b > 1 order by b desc limit 2,1", "Plan": [ - "Projection 1.00 root test.tbl.a, test.tbl.b, test.tbl.c", - "└─IndexLookUp 1.00 root limit embedded(offset:2, count:1)", - " ├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", - " │ └─IndexRangeScan 3.00 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:true, desc", - " └─TableRowIDScan(Probe) 1.00 cop[tikv] table:tbl keep order:false, stats:pseudo" + "Limit 1.00 root offset:2, count:1", + "└─Projection 3.00 root test.tbl.a, test.tbl.b, test.tbl.c", + " └─IndexLookUp 3.00 root ", + " ├─Limit(Build) 3.00 cop[tikv] offset:0, count:3", + " │ └─IndexRangeScan 3.00 cop[tikv] table:tbl, index:idx_b_c(b, c) range:(1,+inf], keep order:true, desc", + " └─TableRowIDScan(Probe) 3.00 cop[tikv] table:tbl keep order:false, stats:pseudo" ] }, { From b1beb1b67dc982b75f09b97fdf903f9fb7407349 Mon Sep 17 00:00:00 2001 From: rebelice Date: Fri, 4 Jun 2021 14:24:17 +0800 Subject: [PATCH 283/343] planner: add some tiflash test cases on dynamic partition prune mode (#25131) --- executor/tiflash_test.go | 139 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 139 insertions(+) diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index 736720e1a7348..f6d478649887c 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -664,3 +664,142 @@ func (s *tiflashTestSuite) TestTiFlashVirtualColumn(c *C) { tk.MustQuery("select /*+ hash_agg() */ count(*) from t2 where c > 1").Check(testkit.Rows("2")) tk.MustQuery("select /*+ hash_agg() */ count(*) from t3 where c > b'01'").Check(testkit.Rows("3")) } + +func (s *tiflashTestSuite) TestTiFlashPartitionTableShuffledHashAggregation(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database tiflash_partition_AGG") + tk.MustExec("use tiflash_partition_AGG") + tk.MustExec(`create table thash (a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int) partition by range(a) ( + partition p0 values less than (100), partition p1 values less than (200), + partition p2 values less than (300), partition p3 values less than (400))`) + listPartitions := make([]string, 4) + for i := 0; i < 400; i++ { + idx := i % 4 + if listPartitions[idx] != "" { + listPartitions[idx] += ", " + } + listPartitions[idx] = listPartitions[idx] + fmt.Sprintf("%v", i) + } + tk.MustExec(`create table tlist (a int, b int) partition by list(a) ( + partition p0 values in (` + listPartitions[0] + `), partition p1 values in (` + listPartitions[1] + `), + partition p2 values in (` + listPartitions[2] + `), partition p3 values in (` + listPartitions[3] + `))`) + tk.MustExec(`create table tnormal (a int, b int) partition by hash(a) partitions 4`) + + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec("alter table " + tbl + " set tiflash replica 1") + tb := testGetTableByName(c, tk.Se, "tiflash_partition_AGG", tbl) + err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + } + + vals := make([]string, 0, 100) + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(400), rand.Intn(400))) + } + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec(fmt.Sprintf("insert into %v values %v", tbl, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf("analyze table %v", tbl)) + } + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp=2") + + lr := func() (int, int) { + l, r := rand.Intn(400), rand.Intn(400) + if l > r { + l, r = r, l + } + return l, r + } + for i := 0; i < 2; i++ { + l1, r1 := lr() + cond := fmt.Sprintf("t1.b>=%v and t1.b<=%v", l1, r1) + var res [][]interface{} + for _, mode := range []string{"static", "dynamic"} { + tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", mode)) + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + q := fmt.Sprintf("select /*+ HASH_AGG() */ count(*) from %v t1 where %v", tbl, cond) + c.Assert(tk.HasPlan(q, "HashAgg"), IsTrue) + if res == nil { + res = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Check(res) + } + } + } + } +} + +func (s *tiflashTestSuite) TestTiFlashPartitionTableBroadcastJoin(c *C) { + if israce.RaceEnabled { + c.Skip("exhaustive types test, skip race test") + } + + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database tiflash_partition_BCJ") + tk.MustExec("use tiflash_partition_BCJ") + tk.MustExec(`create table thash (a int, b int) partition by hash(a) partitions 4`) + tk.MustExec(`create table trange (a int, b int) partition by range(a) ( + partition p0 values less than (100), partition p1 values less than (200), + partition p2 values less than (300), partition p3 values less than (400))`) + listPartitions := make([]string, 4) + for i := 0; i < 400; i++ { + idx := i % 4 + if listPartitions[idx] != "" { + listPartitions[idx] += ", " + } + listPartitions[idx] = listPartitions[idx] + fmt.Sprintf("%v", i) + } + tk.MustExec(`create table tlist (a int, b int) partition by list(a) ( + partition p0 values in (` + listPartitions[0] + `), partition p1 values in (` + listPartitions[1] + `), + partition p2 values in (` + listPartitions[2] + `), partition p3 values in (` + listPartitions[3] + `))`) + tk.MustExec(`create table tnormal (a int, b int) partition by hash(a) partitions 4`) + + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec("alter table " + tbl + " set tiflash replica 1") + tb := testGetTableByName(c, tk.Se, "tiflash_partition_BCJ", tbl) + err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) + c.Assert(err, IsNil) + } + + vals := make([]string, 0, 100) + for i := 0; i < 100; i++ { + vals = append(vals, fmt.Sprintf("(%v, %v)", rand.Intn(400), rand.Intn(400))) + } + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + tk.MustExec(fmt.Sprintf("insert into %v values %v", tbl, strings.Join(vals, ", "))) + tk.MustExec(fmt.Sprintf("analyze table %v", tbl)) + } + tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") + tk.MustExec("set @@session.tidb_allow_mpp=2") + tk.MustExec("set @@session.tidb_opt_broadcast_join=ON") + + lr := func() (int, int) { + l, r := rand.Intn(400), rand.Intn(400) + if l > r { + l, r = r, l + } + return l, r + } + for i := 0; i < 2; i++ { + l1, r1 := lr() + l2, r2 := lr() + cond := fmt.Sprintf("t1.b>=%v and t1.b<=%v and t2.b>=%v and t2.b<=%v", l1, r1, l2, r2) + var res [][]interface{} + for _, mode := range []string{"static", "dynamic"} { + tk.MustExec(fmt.Sprintf("set @@tidb_partition_prune_mode = '%v'", mode)) + for _, tbl := range []string{`thash`, `trange`, `tlist`, `tnormal`} { + q := fmt.Sprintf("select count(*) from %v t1 join %v t2 on t1.a=t2.a where %v", tbl, tbl, cond) + if res == nil { + res = tk.MustQuery(q).Sort().Rows() + } else { + tk.MustQuery(q).Check(res) + } + } + } + } +} From e7d4c3054ba4308e1c41f751a60477c6f5794ec9 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 4 Jun 2021 15:10:28 +0800 Subject: [PATCH 284/343] planner: enable the dynamic mode of partition tables by default (#25143) --- sessionctx/variable/tidb_vars.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 68c5be3ee6910..8e60d2a9e3492 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -691,7 +691,7 @@ const ( DefTiDBEnableTelemetry = true DefTiDBEnableParallelApply = false DefTiDBEnableAmendPessimisticTxn = false - DefTiDBPartitionPruneMode = "static" + DefTiDBPartitionPruneMode = "dynamic" DefTiDBEnableRateLimitAction = true DefTiDBEnableAsyncCommit = false DefTiDBEnable1PC = false From 693708d627858f9f4d8e769800b037edd0b1ace5 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 4 Jun 2021 15:36:38 +0800 Subject: [PATCH 285/343] planner/core: remove unstable test (#25138) Co-authored-by: Ti Chi Robot --- planner/core/integration_test.go | 74 -------------------------------- 1 file changed, 74 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index d441cd673e3b9..b2861ccc6ec60 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -746,80 +746,6 @@ func (s *testIntegrationSerialSuite) TestMPPShuffledJoin(c *C) { } } -func (s *testIntegrationSerialSuite) TestBroadcastJoin(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("set session tidb_allow_mpp = OFF") - tk.MustExec("drop table if exists d1_t") - tk.MustExec("create table d1_t(d1_k int, value int)") - tk.MustExec("insert into d1_t values(1,2),(2,3)") - tk.MustExec("analyze table d1_t") - tk.MustExec("drop table if exists d2_t") - tk.MustExec("create table d2_t(d2_k decimal(10,2), value int)") - tk.MustExec("insert into d2_t values(10.11,2),(10.12,3)") - tk.MustExec("analyze table d2_t") - tk.MustExec("drop table if exists d3_t") - tk.MustExec("create table d3_t(d3_k date, value int)") - tk.MustExec("insert into d3_t values(date'2010-01-01',2),(date'2010-01-02',3)") - tk.MustExec("analyze table d3_t") - tk.MustExec("drop table if exists fact_t") - tk.MustExec("create table fact_t(d1_k int, d2_k decimal(10,2), d3_k date, col1 int, col2 int, col3 int)") - tk.MustExec("insert into fact_t values(1,10.11,date'2010-01-01',1,2,3),(1,10.11,date'2010-01-02',1,2,3),(1,10.12,date'2010-01-01',1,2,3),(1,10.12,date'2010-01-02',1,2,3)") - tk.MustExec("insert into fact_t values(2,10.11,date'2010-01-01',1,2,3),(2,10.11,date'2010-01-02',1,2,3),(2,10.12,date'2010-01-01',1,2,3),(2,10.12,date'2010-01-02',1,2,3)") - tk.MustExec("analyze table fact_t") - - // 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 == "fact_t" || tblInfo.Name.L == "d1_t" || tblInfo.Name.L == "d2_t" || tblInfo.Name.L == "d3_t" { - tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ - Count: 1, - Available: true, - } - } - } - - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - tk.MustExec("set @@session.tidb_allow_batch_cop = 1") - tk.MustExec("set @@session.tidb_opt_broadcast_join = 1") - // make cbo force choose broadcast join since sql hint does not work for semi/anti-semi join - tk.MustExec("set @@session.tidb_opt_cpu_factor=10000000;") - var input []string - var output []struct { - SQL string - Plan []string - } - s.testData.GetTestCases(c, &input, &output) - for i, tt := range input { - s.testData.OnRecord(func() { - output[i].SQL = tt - output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) - }) - res := tk.MustQuery(tt) - res.Check(testkit.Rows(output[i].Plan...)) - } - - // out table of out join should not be global - _, err := tk.Exec("explain format = 'brief' select /*+ broadcast_join(fact_t, d1_t), broadcast_join_local(d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") - // nullEQ not supported - _, err = tk.Exec("explain format = 'brief' select /*+ broadcast_join(fact_t, d1_t) */ count(*) from fact_t join d1_t on fact_t.d1_k <=> d1_t.d1_k") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") - // not supported if join condition has unsupported expr - _, err = tk.Exec("explain format = 'brief' select /*+ broadcast_join(fact_t, d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and sqrt(fact_t.col1) > 2") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") - // cartsian join not supported - _, err = tk.Exec("explain format = 'brief' select /*+ broadcast_join(fact_t, d1_t) */ count(*) from fact_t join d1_t") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[planner:1815]Internal : Can't find a proper physical plan for this query") -} - func (s *testIntegrationSerialSuite) TestJoinNotSupportedByTiFlash(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") From 2e083b40e562feb964986ba5a2580915de723489 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Fri, 4 Jun 2021 16:18:28 +0800 Subject: [PATCH 286/343] statistics: support indexes containing virtual column for full sampling (#24999) --- executor/analyze.go | 416 +++++++++++++++++++-- executor/analyze_test.go | 29 ++ executor/builder.go | 34 +- planner/core/common_plans.go | 8 +- planner/core/planbuilder.go | 28 +- planner/core/stats_test.go | 1 + planner/core/testdata/stats_suite_out.json | 36 +- statistics/builder.go | 2 +- 8 files changed, 462 insertions(+), 92 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index c4e3141b2fdd2..5a2ed6e28e212 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/distsql" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" @@ -53,6 +54,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) @@ -146,6 +148,10 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } statisticsID := result.TableID.GetStatisticsID() for i, hg := range result.Hist { + // It's normal virtual column, skip. + if hg == nil { + continue + } if result.TableID.IsPartitionTable() && needGlobalStats { // If it does not belong to the statistics of index, we need to set it to -1 to distinguish. idxID := int64(-1) @@ -330,6 +336,36 @@ func analyzeIndexPushdown(idxExec *AnalyzeIndexExec) analyzeResult { return result } +func analyzeIndexNDVPushDown(idxExec *AnalyzeIndexExec) analyzeResult { + ranges := ranger.FullRange() + // For single-column index, we do not load null rows from TiKV, so the built histogram would not include + // null values, and its `NullCount` would be set by result of another distsql call to get null rows. + // For multi-column index, we cannot define null for the rows, so we still use full range, and the rows + // containing null fields would exist in built histograms. Note that, the `NullCount` of histograms for + // multi-column index is always 0 then. + if len(idxExec.idxInfo.Columns) == 1 { + ranges = ranger.FullNotNullRange() + } + fms, nullHist, err := idxExec.buildSimpleStats(ranges, len(idxExec.idxInfo.Columns) == 1) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + result := analyzeResult{ + TableID: idxExec.tableID, + Fms: []*statistics.FMSketch{fms}, + // We use histogram to get the Index's ID. + Hist: []*statistics.Histogram{statistics.NewHistogram(idxExec.idxInfo.ID, 0, 0, statistics.Version1, types.NewFieldType(mysql.TypeBlob), 0, 0)}, + IsIndex: 1, + job: idxExec.job, + // TODO: avoid reusing Version1. + StatsVer: statistics.Version1, + } + if nullHist != nil && nullHist.Len() > 0 { + result.Count = nullHist.Buckets[nullHist.Len()-1].Count + } + return result +} + // AnalyzeIndexExec represents analyze index push down executor. type AnalyzeIndexExec struct { ctx sessionctx.Context @@ -512,6 +548,29 @@ func (e *AnalyzeIndexExec) buildStats(ranges []*ranger.Range, considerNull bool) return hist, cms, fms, topN, nil } +func (e *AnalyzeIndexExec) buildSimpleStats(ranges []*ranger.Range, considerNull bool) (fms *statistics.FMSketch, nullHist *statistics.Histogram, err error) { + if err = e.open(ranges, considerNull); err != nil { + return nil, nil, err + } + defer func() { + err1 := closeAll(e.result, e.countNullRes) + if err == nil { + err = err1 + } + }() + _, _, fms, _, err = e.buildStatsFromResult(e.result, false) + if e.countNullRes != nil { + nullHist, _, _, _, err := e.buildStatsFromResult(e.countNullRes, false) + if err != nil { + return nil, nil, err + } + if l := nullHist.Len(); l > 0 { + return fms, nullHist, nil + } + } + return fms, nil, nil +} + func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { var ranges []*ranger.Range if hc := colExec.handleCols; hc != nil { @@ -524,19 +583,39 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { ranges = ranger.FullIntRange(false) } collExtStats := colExec.ctx.GetSessionVars().EnableExtendedStats - if colExec.analyzeVer == statistics.Version3 { - count, hists, topns, fmSketches, extStats, err := colExec.buildSamplingStats(ranges, collExtStats) + if colExec.StatsVersion == statistics.Version3 { + specialIndexes := make([]*model.IndexInfo, 0, len(colExec.indexes)) + specialIndexesOffsets := make([]int, 0, len(colExec.indexes)) + for i, idx := range colExec.indexes { + isSpecial := false + for _, col := range idx.Columns { + colInfo := colExec.colsInfo[col.Offset] + isVirtualCol := colInfo.IsGenerated() && !colInfo.GeneratedStored + isPrefixCol := col.Length != types.UnspecifiedLength + if isVirtualCol || isPrefixCol { + isSpecial = true + break + } + } + if isSpecial { + specialIndexesOffsets = append(specialIndexesOffsets, i) + specialIndexes = append(specialIndexes, idx) + } + } + idxNDVPushDownCh := make(chan analyzeIndexNDVTotalResult, 1) + go colExec.handleNDVForSpecialIndexes(specialIndexes, idxNDVPushDownCh) + count, hists, topns, fmSketches, extStats, err := colExec.buildSamplingStats(ranges, collExtStats, specialIndexesOffsets, idxNDVPushDownCh) if err != nil { return []analyzeResult{{Err: err, job: colExec.job}} } cLen := len(colExec.analyzePB.ColReq.ColumnsInfo) colGroupResult := analyzeResult{ - TableID: colExec.tableID, + TableID: colExec.TableID, Hist: hists[cLen:], TopNs: topns[cLen:], Fms: fmSketches[cLen:], job: colExec.job, - StatsVer: colExec.analyzeVer, + StatsVer: colExec.StatsVersion, Count: count, IsIndex: 1, } @@ -544,17 +623,18 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { // Because the process of analyzing will keep the order of results be the same as the colsInfo in the analyze task, // and in `buildAnalyzeFullSamplingTask` we always place the _tidb_rowid at the last of colsInfo, so if there are // stats for _tidb_rowid, it must be at the end of the column stats. - if hists[cLen-1].ID == -1 { + // Virtual column has no histogram yet. So we check nil here. + if hists[cLen-1] != nil && hists[cLen-1].ID == -1 { cLen -= 1 } colResult := analyzeResult{ - TableID: colExec.tableID, + TableID: colExec.TableID, Hist: hists[:cLen], TopNs: topns[:cLen], Fms: fmSketches[:cLen], ExtStats: extStats, job: colExec.job, - StatsVer: colExec.analyzeVer, + StatsVer: colExec.StatsVersion, Count: count, } @@ -567,7 +647,7 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { if hasPkHist(colExec.handleCols) { PKresult := analyzeResult{ - TableID: colExec.tableID, + TableID: colExec.TableID, Hist: hists[:1], Cms: cms[:1], TopNs: topNs[:1], @@ -578,14 +658,14 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { } PKresult.Count = int64(PKresult.Hist[0].TotalRowCount()) restResult := analyzeResult{ - TableID: colExec.tableID, + TableID: colExec.TableID, Hist: hists[1:], Cms: cms[1:], TopNs: topNs[1:], Fms: fms[1:], ExtStats: extStats, job: colExec.job, - StatsVer: colExec.analyzeVer, + StatsVer: colExec.StatsVersion, } restResult.Count = PKresult.Count return []analyzeResult{PKresult, restResult} @@ -593,28 +673,28 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { var result []analyzeResult if colExec.analyzePB.Tp == tipb.AnalyzeType_TypeMixed { result = append(result, analyzeResult{ - TableID: colExec.tableID, + TableID: colExec.TableID, Hist: []*statistics.Histogram{hists[0]}, Cms: []*statistics.CMSketch{cms[0]}, TopNs: []*statistics.TopN{topNs[0]}, Fms: []*statistics.FMSketch{nil}, IsIndex: 1, job: colExec.job, - StatsVer: colExec.analyzeVer, + StatsVer: colExec.StatsVersion, }) hists = hists[1:] cms = cms[1:] topNs = topNs[1:] } colResult := analyzeResult{ - TableID: colExec.tableID, + TableID: colExec.TableID, Hist: hists, Cms: cms, TopNs: topNs, Fms: fms, ExtStats: extStats, job: colExec.job, - StatsVer: colExec.analyzeVer, + StatsVer: colExec.StatsVersion, } colResult.Count = int64(colResult.Hist[0].TotalRowCount()) if colResult.StatsVer >= statistics.Version2 { @@ -625,20 +705,24 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { // AnalyzeColumnsExec represents Analyze columns push down executor. type AnalyzeColumnsExec struct { - ctx sessionctx.Context - tableID core.AnalyzeTableID - colsInfo []*model.ColumnInfo - handleCols core.HandleCols - concurrency int - analyzePB *tipb.AnalyzeReq - commonHandle *model.IndexInfo - resultHandler *tableResultHandler - opts map[ast.AnalyzeOptionType]uint64 - job *statistics.AnalyzeJob - analyzeVer int - indexes []*model.IndexInfo + ctx sessionctx.Context + tableInfo *model.TableInfo + colsInfo []*model.ColumnInfo + handleCols core.HandleCols + concurrency int + analyzePB *tipb.AnalyzeReq + commonHandle *model.IndexInfo + resultHandler *tableResultHandler + opts map[ast.AnalyzeOptionType]uint64 + job *statistics.AnalyzeJob + indexes []*model.IndexInfo + core.AnalyzeInfo + + subIndexWorkerWg *sync.WaitGroup samplingBuilderWg *sync.WaitGroup samplingMergeWg *sync.WaitGroup + + schemaForVirtualColEval *expression.Schema } func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { @@ -664,7 +748,7 @@ func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectResult, error) { var builder distsql.RequestBuilder - reqBuilder := builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.tableID.GetStatisticsID()}, e.handleCols != nil && !e.handleCols.IsInt(), ranges, nil) + reqBuilder := builder.SetHandleRangesForTables(e.ctx.GetSessionVars().StmtCtx, []int64{e.TableID.GetStatisticsID()}, e.handleCols != nil && !e.handleCols.IsInt(), ranges, nil) builder.SetResourceGroupTag(e.ctx.GetSessionVars().StmtCtx) // Always set KeepOrder of the request to be true, in order to compute // correct `correlation` of columns. @@ -685,7 +769,49 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe return result, nil } -func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtStats bool) ( +// decodeSampleDataWithVirtualColumn constructs the virtual column by evaluating from the deocded normal columns. +// If it failed, it would return false to trigger normal decoding way without the virtual column. +func (e AnalyzeColumnsExec) decodeSampleDataWithVirtualColumn( + collector *statistics.RowSampleCollector, + fieldTps []*types.FieldType, + virtualColIdx []int, + schema *expression.Schema, +) error { + totFts := make([]*types.FieldType, 0, e.schemaForVirtualColEval.Len()) + for _, col := range e.schemaForVirtualColEval.Columns { + totFts = append(totFts, col.RetType) + } + chk := chunk.NewChunkWithCapacity(totFts, len(collector.Samples)) + decoder := codec.NewDecoder(chk, e.ctx.GetSessionVars().TimeZone) + for _, sample := range collector.Samples { + for i := range sample.Columns { + if schema.Columns[i].VirtualExpr != nil { + continue + } + _, err := decoder.DecodeOne(sample.Columns[i].GetBytes(), i, e.schemaForVirtualColEval.Columns[i].RetType) + if err != nil { + return err + } + } + } + err := FillVirtualColumnValue(fieldTps, virtualColIdx, schema, e.colsInfo, e.ctx, chk) + if err != nil { + return err + } + iter := chunk.NewIterator4Chunk(chk) + for row, i := iter.Begin(), 0; row != iter.End(); row, i = iter.Next(), i+1 { + datums := row.GetDatumRow(totFts) + collector.Samples[i].Columns = datums + } + return nil +} + +func (e *AnalyzeColumnsExec) buildSamplingStats( + ranges []*ranger.Range, + needExtStats bool, + indexesWithVirtualColOffsets []int, + idxNDVPushDownCh chan analyzeIndexNDVTotalResult, +) ( count int64, hists []*statistics.Histogram, topns []*statistics.TopN, @@ -701,6 +827,7 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtS err = err1 } }() + l := len(e.analyzePB.ColReq.ColumnsInfo) + len(e.analyzePB.ColReq.ColumnGroups) rootRowCollector := &statistics.RowSampleCollector{ NullCount: make([]int64, l), @@ -753,12 +880,32 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtS if err != nil { return 0, nil, nil, nil, nil, err } + + // handling virtual columns + virtualColIdx := buildVirtualColumnIndex(e.schemaForVirtualColEval, e.colsInfo) + if len(virtualColIdx) > 0 { + fieldTps := make([]*types.FieldType, 0, len(virtualColIdx)) + for _, colOffset := range virtualColIdx { + fieldTps = append(fieldTps, e.schemaForVirtualColEval.Columns[colOffset].RetType) + } + err = e.decodeSampleDataWithVirtualColumn(rootRowCollector, fieldTps, virtualColIdx, e.schemaForVirtualColEval) + if err != nil { + return 0, nil, nil, nil, nil, err + } + } else { + // If there's no virtual column or we meet error during eval virtual column, we fallback to normal decode otherwise. + for _, sample := range rootRowCollector.Samples { + for i := range sample.Columns { + sample.Columns[i], err = tablecodec.DecodeColumnValue(sample.Columns[i].GetBytes(), &e.colsInfo[i].FieldType, sc.TimeZone) + if err != nil { + return 0, nil, nil, nil, nil, err + } + } + } + } + for _, sample := range rootRowCollector.Samples { for i := range sample.Columns { - sample.Columns[i], err = tablecodec.DecodeColumnValue(sample.Columns[i].GetBytes(), &e.colsInfo[i].FieldType, sc.TimeZone) - if err != nil { - return 0, nil, nil, nil, nil, err - } if sample.Columns[i].Kind() == types.KindBytes { sample.Columns[i].SetBytes(sample.Columns[i].GetBytes()) } @@ -770,6 +917,8 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtS } } + colLen := len(e.colsInfo) + // The order of the samples are broken when merging samples from sub-collectors. // So now we need to sort the samples according to the handle in order to calculate correlation. sort.Slice(rootRowCollector.Samples, func(i, j int) bool { @@ -798,7 +947,18 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtS } fmSketches = append(fmSketches, rootRowCollector.FMSketches[i]) } - colLen := len(e.colsInfo) + + indexPushedDownResult := <-idxNDVPushDownCh + if indexPushedDownResult.err != nil { + return 0, nil, nil, nil, nil, indexPushedDownResult.err + } + for _, offset := range indexesWithVirtualColOffsets { + ret := indexPushedDownResult.results[e.indexes[offset].ID] + rootRowCollector.NullCount[colLen+offset] = ret.Count + rootRowCollector.FMSketches[colLen+offset] = ret.Fms[0] + } + + // build index stats for i, idx := range e.indexes { buildTaskChan <- &samplingBuildTask{ id: idx.ID, @@ -830,7 +990,7 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtS count = rootRowCollector.Count if needExtStats { statsHandle := domain.GetDomain(e.ctx).StatsHandle() - extStats, err = statsHandle.BuildExtendedStats(e.tableID.GetStatisticsID(), e.colsInfo, sampleCollectors) + extStats, err = statsHandle.BuildExtendedStats(e.TableID.GetStatisticsID(), e.colsInfo, sampleCollectors) if err != nil { return 0, nil, nil, nil, nil, err } @@ -838,6 +998,167 @@ func (e *AnalyzeColumnsExec) buildSamplingStats(ranges []*ranger.Range, needExtS return } +type analyzeIndexNDVTotalResult struct { + results map[int64]analyzeResult + err error +} + +// handleNDVForSpecialIndexes deals with the logic to analyze the index containing the virtual column when the mode is full sampling. +func (e *AnalyzeColumnsExec) handleNDVForSpecialIndexes(indexInfos []*model.IndexInfo, totalResultCh chan analyzeIndexNDVTotalResult) { + defer func() { + if r := recover(); r != nil { + buf := make([]byte, 4096) + stackSize := runtime.Stack(buf, false) + buf = buf[:stackSize] + logutil.BgLogger().Error("analyze ndv for special index panicked", zap.String("stack", string(buf))) + metrics.PanicCounter.WithLabelValues(metrics.LabelAnalyze).Inc() + totalResultCh <- analyzeIndexNDVTotalResult{ + err: errAnalyzeWorkerPanic, + } + } + }() + tasks := e.buildSubIndexJobForSpecialIndex(indexInfos) + statsConcurrncy, err := getBuildStatsConcurrency(e.ctx) + taskCh := make(chan *analyzeTask, len(tasks)) + for _, task := range tasks { + statistics.AddNewAnalyzeJob(task.job) + } + resultCh := make(chan analyzeResult, len(tasks)) + e.subIndexWorkerWg = &sync.WaitGroup{} + e.subIndexWorkerWg.Add(statsConcurrncy) + for i := 0; i < statsConcurrncy; i++ { + go e.subIndexWorkerForNDV(taskCh, resultCh, i == 0) + } + for _, task := range tasks { + taskCh <- task + } + close(taskCh) + panicCnt := 0 + totalResult := analyzeIndexNDVTotalResult{ + results: make(map[int64]analyzeResult, len(indexInfos)), + } + for panicCnt < statsConcurrncy { + result, ok := <-resultCh + if !ok { + break + } + if result.Err != nil { + result.job.Finish(true) + err = result.Err + if err == errAnalyzeWorkerPanic { + panicCnt++ + } + continue + } + result.job.Finish(false) + statistics.MoveToHistory(result.job) + totalResult.results[result.Hist[0].ID] = result + } + if err != nil { + totalResult.err = err + } + totalResultCh <- totalResult +} + +// subIndexWorker receive the task for each index and return the result for them. +func (e *AnalyzeColumnsExec) subIndexWorkerForNDV(taskCh chan *analyzeTask, resultCh chan analyzeResult, isFirstToCloseCh bool) { + var task *analyzeTask + defer func() { + if r := recover(); r != nil { + buf := make([]byte, 4096) + stackSize := runtime.Stack(buf, false) + buf = buf[:stackSize] + logutil.BgLogger().Error("analyze worker panicked", zap.String("stack", string(buf))) + metrics.PanicCounter.WithLabelValues(metrics.LabelAnalyze).Inc() + resultCh <- analyzeResult{ + Err: errAnalyzeWorkerPanic, + job: task.job, + } + } + e.subIndexWorkerWg.Done() + if isFirstToCloseCh { + e.subIndexWorkerWg.Wait() + close(resultCh) + } + }() + for { + var ok bool + task, ok = <-taskCh + if !ok { + break + } + task.job.Start() + if task.taskType != idxTask { + resultCh <- analyzeResult{ + Err: errors.Errorf("incorrect analyze type"), + job: task.job, + } + continue + } + task.idxExec.job = task.job + resultCh <- analyzeIndexNDVPushDown(task.idxExec) + } +} + +// buildSubIndexJobForSpecialIndex builds sub index pushed down task to calculate the NDV information for indexes containing virtual column. +// This is because we cannot push the calculation of the virtual column down to the tikv side. +func (e *AnalyzeColumnsExec) buildSubIndexJobForSpecialIndex(indexInfos []*model.IndexInfo) []*analyzeTask { + _, offset := timeutil.Zone(e.ctx.GetSessionVars().Location()) + tasks := make([]*analyzeTask, 0, len(indexInfos)) + sc := e.ctx.GetSessionVars().StmtCtx + for _, indexInfo := range indexInfos { + idxExec := &AnalyzeIndexExec{ + ctx: e.ctx, + tableID: e.TableID, + isCommonHandle: e.tableInfo.IsCommonHandle, + idxInfo: indexInfo, + concurrency: e.ctx.GetSessionVars().IndexSerialScanConcurrency(), + analyzePB: &tipb.AnalyzeReq{ + Tp: tipb.AnalyzeType_TypeIndex, + Flags: sc.PushDownFlags(), + TimeZoneOffset: offset, + }, + } + idxExec.opts = make(map[ast.AnalyzeOptionType]uint64, len(ast.AnalyzeOptionString)) + idxExec.opts[ast.AnalyzeOptNumTopN] = 0 + idxExec.opts[ast.AnalyzeOptCMSketchDepth] = 0 + idxExec.opts[ast.AnalyzeOptCMSketchWidth] = 0 + idxExec.opts[ast.AnalyzeOptNumSamples] = 0 + idxExec.opts[ast.AnalyzeOptNumBuckets] = 1 + statsVersion := new(int32) + *statsVersion = statistics.Version1 + // No Top-N + topnSize := int32(0) + idxExec.analyzePB.IdxReq = &tipb.AnalyzeIndexReq{ + // One bucket to store the null for null histogram. + BucketSize: 1, + NumColumns: int32(len(indexInfo.Columns)), + TopNSize: &topnSize, + Version: statsVersion, + SketchSize: maxSketchSize, + } + if idxExec.isCommonHandle && indexInfo.Primary { + idxExec.analyzePB.Tp = tipb.AnalyzeType_TypeCommonHandle + } + // No CM-Sketch. + depth := int32(0) + width := int32(0) + idxExec.analyzePB.IdxReq.CmsketchDepth = &depth + idxExec.analyzePB.IdxReq.CmsketchWidth = &width + autoAnalyze := "" + if e.ctx.GetSessionVars().InRestrictedSQL { + autoAnalyze = "auto " + } + job := &statistics.AnalyzeJob{DBName: e.job.DBName, TableName: e.job.TableName, PartitionName: e.job.PartitionName, JobInfo: autoAnalyze + "analyze ndv for index " + indexInfo.Name.O} + tasks = append(tasks, &analyzeTask{ + taskType: idxTask, + idxExec: idxExec, + job: job, + }) + } + return tasks +} + type samplingMergeResult struct { collector *statistics.RowSampleCollector err error @@ -936,6 +1257,11 @@ workLoop: } var collector *statistics.SampleCollector if task.isColumn { + if e.colsInfo[task.slicePos].IsGenerated() && !e.colsInfo[task.slicePos].GeneratedStored { + hists[task.slicePos] = nil + topns[task.slicePos] = nil + continue + } sampleItems := make([]*statistics.SampleItem, 0, task.rootRowCollector.MaxSampleSize) for j, row := range task.rootRowCollector.Samples { if row.Columns[task.slicePos].IsNull() { @@ -954,15 +1280,27 @@ workLoop: TotalSize: task.rootRowCollector.TotalSizes[task.slicePos], } } else { + var tmpDatum types.Datum + var err error idx := e.indexes[task.slicePos-colLen] sampleItems := make([]*statistics.SampleItem, 0, task.rootRowCollector.MaxSampleSize) for _, row := range task.rootRowCollector.Samples { if len(idx.Columns) == 1 && row.Columns[idx.Columns[0].Offset].IsNull() { continue } + b := make([]byte, 0, 8) - var err error for _, col := range idx.Columns { + if col.Length != types.UnspecifiedLength { + row.Columns[col.Offset].Copy(&tmpDatum) + ranger.CutDatumByPrefixLen(&tmpDatum, col.Length, &e.colsInfo[col.Offset].FieldType) + b, err = codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, b, tmpDatum) + if err != nil { + resultCh <- err + continue workLoop + } + continue + } b, err = codec.EncodeKey(e.ctx.GetSessionVars().StmtCtx, b, row.Columns[col.Offset]) if err != nil { resultCh <- err @@ -1089,7 +1427,7 @@ func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range, needExtStats boo fms = append(fms, nil) } for i, col := range e.colsInfo { - if e.analyzeVer < 2 { + if e.StatsVersion < 2 { // In analyze version 2, we don't collect TopN this way. We will collect TopN from samples in `BuildColumnHistAndTopN()` below. err := collectors[i].ExtractTopN(uint32(e.opts[ast.AnalyzeOptNumTopN]), e.ctx.GetSessionVars().StmtCtx, &col.FieldType, timeZone) if err != nil { @@ -1112,7 +1450,7 @@ func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range, needExtStats boo var hg *statistics.Histogram var err error var topn *statistics.TopN - if e.analyzeVer < 2 { + if e.StatsVersion < 2 { hg, err = statistics.BuildColumn(e.ctx, int64(e.opts[ast.AnalyzeOptNumBuckets]), col.ID, collectors[i], &col.FieldType) } else { hg, topn, err = statistics.BuildHistAndTopN(e.ctx, int(e.opts[ast.AnalyzeOptNumBuckets]), int(e.opts[ast.AnalyzeOptNumTopN]), col.ID, collectors[i], &col.FieldType, true) @@ -1128,7 +1466,7 @@ func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range, needExtStats boo } if needExtStats { statsHandle := domain.GetDomain(e.ctx).StatsHandle() - extStats, err = statsHandle.BuildExtendedStats(e.tableID.GetStatisticsID(), e.colsInfo, collectors) + extStats, err = statsHandle.BuildExtendedStats(e.TableID.GetStatisticsID(), e.colsInfo, collectors) if err != nil { return nil, nil, nil, nil, nil, err } @@ -1846,7 +2184,7 @@ func analyzePKIncremental(colExec *analyzePKIncrementalExec) analyzeResult { return analyzeResult{Err: err, job: colExec.job} } result := analyzeResult{ - TableID: colExec.tableID, + TableID: colExec.TableID, Hist: []*statistics.Histogram{hist}, Cms: []*statistics.CMSketch{nil}, TopNs: []*statistics.TopN{nil}, diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 6b1e37b743f74..58ee27bf47d79 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -964,6 +964,35 @@ func (s *testSuite1) TestAnalyzeClusteredIndexPrimary(c *C) { "test t1 PRIMARY 1 0 1 1 1111 1111 0")) } +func (s *testSuite1) TestAnalyzeFullSamplingOnIndexWithVirtualColumnOrPrefixColumn(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists sampling_index_virtual_col") + tk.MustExec("create table sampling_index_virtual_col(a int, b int as (a+1), index idx(b))") + tk.MustExec("insert into sampling_index_virtual_col (a) values (1), (2), (null), (3), (4), (null), (5), (5), (5), (5)") + tk.MustExec("set @@session.tidb_analyze_version = 3") + tk.MustExec("analyze table sampling_index_virtual_col with 1 topn") + tk.MustQuery("show stats_buckets where table_name = 'sampling_index_virtual_col' and column_name = 'idx'").Check(testkit.Rows( + "test sampling_index_virtual_col idx 1 0 1 1 2 2 0", + "test sampling_index_virtual_col idx 1 1 2 1 3 3 0", + "test sampling_index_virtual_col idx 1 2 3 1 4 4 0", + "test sampling_index_virtual_col idx 1 3 4 1 5 5 0")) + tk.MustQuery("show stats_topn where table_name = 'sampling_index_virtual_col' and column_name = 'idx'").Check(testkit.Rows("test sampling_index_virtual_col idx 1 6 4")) + row := tk.MustQuery(`show stats_histograms where db_name = "test" and table_name = "sampling_index_virtual_col"`).Rows()[0] + // The NDV. + c.Assert(row[6], Equals, "5") + // The NULLs. + c.Assert(row[7], Equals, "2") + tk.MustExec("drop table if exists sampling_index_prefix_col") + tk.MustExec("create table sampling_index_prefix_col(a varchar(3), index idx(a(1)))") + tk.MustExec("insert into sampling_index_prefix_col (a) values ('aa'), ('ab'), ('ac'), ('bb')") + tk.MustExec("analyze table sampling_index_prefix_col with 1 topn") + tk.MustQuery("show stats_buckets where table_name = 'sampling_index_prefix_col' and column_name = 'idx'").Check(testkit.Rows( + "test sampling_index_prefix_col idx 1 0 1 1 b b 0", + )) + tk.MustQuery("show stats_topn where table_name = 'sampling_index_prefix_col' and column_name = 'idx'").Check(testkit.Rows("test sampling_index_prefix_col idx 1 a 3")) +} + func (s *testSuite2) TestAnalyzeSamplingWorkPanic(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/builder.go b/executor/builder.go index 5e24ae8b880c0..6886dafc13d3c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2049,11 +2049,7 @@ func (b *executorBuilder) buildAnalyzeIndexIncremental(task plannercore.AnalyzeI return analyzeTask } -func (b *executorBuilder) buildAnalyzeSamplingPushdown( - task plannercore.AnalyzeColumnsTask, - opts map[ast.AnalyzeOptionType]uint64, - autoAnalyze string, -) *analyzeTask { +func (b *executorBuilder) buildAnalyzeSamplingPushdown(task plannercore.AnalyzeColumnsTask, opts map[ast.AnalyzeOptionType]uint64, autoAnalyze string, schemaForVirtualColEval *expression.Schema) *analyzeTask { availableIdx := make([]*model.IndexInfo, 0, len(task.Indexes)) colGroups := make([]*tipb.AnalyzeColumnGroup, 0, len(task.Indexes)) if len(task.Indexes) > 0 { @@ -2073,7 +2069,7 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown( sc := b.ctx.GetSessionVars().StmtCtx e := &AnalyzeColumnsExec{ ctx: b.ctx, - tableID: task.TableID, + tableInfo: task.TblInfo, colsInfo: task.ColsInfo, handleCols: task.HandleCols, concurrency: b.ctx.GetSessionVars().DistSQLScanConcurrency(), @@ -2082,9 +2078,10 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown( Flags: sc.PushDownFlags(), TimeZoneOffset: offset, }, - opts: opts, - analyzeVer: task.StatsVersion, - indexes: availableIdx, + opts: opts, + indexes: availableIdx, + AnalyzeInfo: task.AnalyzeInfo, + schemaForVirtualColEval: schemaForVirtualColEval, } e.analyzePB.ColReq = &tipb.AnalyzeColumnsReq{ BucketSize: int64(opts[ast.AnalyzeOptNumBuckets]), @@ -2104,9 +2101,9 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown( return &analyzeTask{taskType: colTask, colExec: e, job: job} } -func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeColumnsTask, opts map[ast.AnalyzeOptionType]uint64, autoAnalyze string) *analyzeTask { +func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeColumnsTask, opts map[ast.AnalyzeOptionType]uint64, autoAnalyze string, schemaForVirtualColEval *expression.Schema) *analyzeTask { if task.StatsVersion == statistics.Version3 { - return b.buildAnalyzeSamplingPushdown(task, opts, autoAnalyze) + return b.buildAnalyzeSamplingPushdown(task, opts, autoAnalyze, schemaForVirtualColEval) } cols := task.ColsInfo if hasPkHist(task.HandleCols) { @@ -2125,7 +2122,6 @@ func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeCo sc := b.ctx.GetSessionVars().StmtCtx e := &AnalyzeColumnsExec{ ctx: b.ctx, - tableID: task.TableID, colsInfo: task.ColsInfo, handleCols: task.HandleCols, concurrency: b.ctx.GetSessionVars().DistSQLScanConcurrency(), @@ -2134,8 +2130,8 @@ func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeCo Flags: sc.PushDownFlags(), TimeZoneOffset: offset, }, - opts: opts, - analyzeVer: task.StatsVersion, + opts: opts, + AnalyzeInfo: task.AnalyzeInfo, } depth := int32(opts[ast.AnalyzeOptCMSketchDepth]) width := int32(opts[ast.AnalyzeOptCMSketchWidth]) @@ -2181,7 +2177,7 @@ func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeCo func (b *executorBuilder) buildAnalyzePKIncremental(task plannercore.AnalyzeColumnsTask, opts map[ast.AnalyzeOptionType]uint64) *analyzeTask { h := domain.GetDomain(b.ctx).StatsHandle() statsTbl := h.GetPartitionStats(&model.TableInfo{}, task.TableID.GetStatisticsID()) - analyzeTask := b.buildAnalyzeColumnsPushdown(task, opts, "") + analyzeTask := b.buildAnalyzeColumnsPushdown(task, opts, "", nil) if statsTbl.Pseudo { return analyzeTask } @@ -2307,7 +2303,13 @@ func (b *executorBuilder) buildAnalyze(v *plannercore.Analyze) Executor { if enableFastAnalyze { b.buildAnalyzeFastColumn(e, task, v.Opts) } else { - e.tasks = append(e.tasks, b.buildAnalyzeColumnsPushdown(task, v.Opts, autoAnalyze)) + columns, _, err := expression.ColumnInfos2ColumnsAndNames(b.ctx, model.NewCIStr(task.AnalyzeInfo.DBName), task.TblInfo.Name, task.ColsInfo, task.TblInfo) + if err != nil { + b.err = err + return nil + } + schema := expression.NewSchema(columns...) + e.tasks = append(e.tasks, b.buildAnalyzeColumnsPushdown(task, v.Opts, autoAnalyze, schema)) } } if b.err != nil { diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 46aa621f427b4..8e3baf262101c 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -834,8 +834,8 @@ func (h *AnalyzeTableID) Equals(t *AnalyzeTableID) bool { return h.TableID == t.TableID && h.PartitionID == t.PartitionID } -// analyzeInfo is used to store the database name, table name and partition name of analyze task. -type analyzeInfo struct { +// AnalyzeInfo is used to store the database name, table name and partition name of analyze task. +type AnalyzeInfo struct { DBName string TableName string PartitionName string @@ -851,14 +851,14 @@ type AnalyzeColumnsTask struct { ColsInfo []*model.ColumnInfo TblInfo *model.TableInfo Indexes []*model.IndexInfo - analyzeInfo + AnalyzeInfo } // AnalyzeIndexTask is used for analyze index. type AnalyzeIndexTask struct { IndexInfo *model.IndexInfo TblInfo *model.TableInfo - analyzeInfo + AnalyzeInfo } // Analyze represents an analyze plan diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index b287c42c8b4f7..81b9f14d55bb2 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1691,7 +1691,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( if id == tbl.TableInfo.ID { id = -1 } - info := analyzeInfo{ + info := AnalyzeInfo{ DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], @@ -1702,7 +1702,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( newTask := AnalyzeColumnsTask{ HandleCols: BuildHandleColsForAnalyze(b.ctx, tbl.TableInfo), ColsInfo: tbl.TableInfo.Columns, - analyzeInfo: info, + AnalyzeInfo: info, TblInfo: tbl.TableInfo, Indexes: idxInfos, } @@ -1758,7 +1758,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A if id == tbl.TableInfo.ID { id = -1 } - info := analyzeInfo{ + info := AnalyzeInfo{ DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], @@ -1768,7 +1768,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A } p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{ IndexInfo: idx, - analyzeInfo: info, + AnalyzeInfo: info, TblInfo: tbl.TableInfo, }) } @@ -1779,7 +1779,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A if id == tbl.TableInfo.ID { id = -1 } - info := analyzeInfo{ + info := AnalyzeInfo{ DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], @@ -1791,7 +1791,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A HandleCols: handleCols, CommonHandleInfo: commonHandleInfo, ColsInfo: colInfo, - analyzeInfo: info, + AnalyzeInfo: info, TblInfo: tbl.TableInfo, }) } @@ -1831,14 +1831,14 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A if id == tblInfo.ID { id = -1 } - info := analyzeInfo{ + info := AnalyzeInfo{ DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], TableID: AnalyzeTableID{TableID: tblInfo.ID, PartitionID: id}, Incremental: as.Incremental, StatsVersion: version, } - p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{HandleCols: handleCols, analyzeInfo: info, TblInfo: tblInfo}) + p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{HandleCols: handleCols, AnalyzeInfo: info, TblInfo: tblInfo}) } continue } @@ -1851,7 +1851,7 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A if id == tblInfo.ID { id = -1 } - info := analyzeInfo{ + info := AnalyzeInfo{ DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], @@ -1859,7 +1859,7 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A Incremental: as.Incremental, StatsVersion: version, } - p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info, TblInfo: tblInfo}) + p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, AnalyzeInfo: info, TblInfo: tblInfo}) } } return p, nil @@ -1893,7 +1893,7 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as if id == tblInfo.ID { id = -1 } - info := analyzeInfo{ + info := AnalyzeInfo{ DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], @@ -1901,7 +1901,7 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as Incremental: as.Incremental, StatsVersion: version, } - p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info, TblInfo: tblInfo}) + p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, AnalyzeInfo: info, TblInfo: tblInfo}) } } } @@ -1911,7 +1911,7 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as if id == tblInfo.ID { id = -1 } - info := analyzeInfo{ + info := AnalyzeInfo{ DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], @@ -1919,7 +1919,7 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as Incremental: as.Incremental, StatsVersion: version, } - p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{HandleCols: handleCols, analyzeInfo: info, TblInfo: tblInfo}) + p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{HandleCols: handleCols, AnalyzeInfo: info, TblInfo: tblInfo}) } } return p, nil diff --git a/planner/core/stats_test.go b/planner/core/stats_test.go index 6767a9b910b7d..8999d3208621f 100644 --- a/planner/core/stats_test.go +++ b/planner/core/stats_test.go @@ -77,6 +77,7 @@ func (s *testStatsSuite) TestGroupNDVs(c *C) { ret := &core.PreprocessorReturn{} err = core.Preprocess(tk.Se, stmt, core.WithPreprocessorReturn(ret)) c.Assert(err, IsNil) + tk.Se.GetSessionVars().PlanColumnID = 0 builder, _ := core.NewPlanBuilder(tk.Se, ret.InfoSchema, &hint.BlockHintProcessor{}) p, err := builder.Build(ctx, stmt) c.Assert(err, IsNil, comment) diff --git a/planner/core/testdata/stats_suite_out.json b/planner/core/testdata/stats_suite_out.json index c782b4658d542..ee73ce1a878a4 100644 --- a/planner/core/testdata/stats_suite_out.json +++ b/planner/core/testdata/stats_suite_out.json @@ -10,16 +10,16 @@ { "SQL": "select * from t1, t2 where t1.a = t2.a and t1.b = t2.b", "AggInput": "", - "JoinInput": "[{[5 6] 4}];[{[8 9] 9}]" + "JoinInput": "[{[1 2] 4}];[{[4 5] 9}]" }, { "SQL": "select count(1) from t1 where a > 0 group by a, b", - "AggInput": "[{[11 12] 4}]", + "AggInput": "[{[1 2] 4}]", "JoinInput": "" }, { "SQL": "select count(1) from t1 where b > 0 group by a, b", - "AggInput": "[{[15 16] 4}]", + "AggInput": "[{[1 2] 4}]", "JoinInput": "" }, { @@ -29,7 +29,7 @@ }, { "SQL": "select count(c3) from (select a as c1, b as c2, a+1 as c3 from t1) as tmp group by c2, c1", - "AggInput": "[{[23 24] 4}]", + "AggInput": "[{[1 2] 4}]", "JoinInput": "" }, { @@ -39,7 +39,7 @@ }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b > (select t2.b from t2 where t2.a = t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", - "AggInput": "[{[37 38] 4}]", + "AggInput": "[{[1 2] 4}]", "JoinInput": "" }, { @@ -49,12 +49,12 @@ }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b in (select t2.b from t2 where t2.a = t1.a limit 3)) as cmp from t1) tmp group by tmp.a, tmp.b", - "AggInput": "[{[56 57] 4}]", + "AggInput": "[{[1 2] 4}]", "JoinInput": "" }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b not in (select t2.b from t2 where t2.a = t1.a limit 3)) as cmp from t1) tmp group by tmp.a, tmp.b", - "AggInput": "[{[67 68] 4}]", + "AggInput": "[{[1 2] 4}]", "JoinInput": "" }, { @@ -74,8 +74,8 @@ }, { "SQL": "select count(1) from t1 left join t2 on t1.a = t2.a group by t1.a, t1.b", - "AggInput": "[{[99 100] 4}]", - "JoinInput": "[{[99 100] 4}];[]" + "AggInput": "[{[1 2] 4}]", + "JoinInput": "[{[1 2] 4}];[]" }, { "SQL": "select count(1) from t1 left join t2 on t1.a = t2.a group by t2.a, t2.b", @@ -89,18 +89,18 @@ }, { "SQL": "select count(1) from t1 right join t2 on t1.a = t2.a group by t2.a, t2.b", - "AggInput": "[{[123 124] 9}]", - "JoinInput": "[];[{[123 124] 9}]" + "AggInput": "[{[4 5] 9}]", + "JoinInput": "[];[{[4 5] 9}]" }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b in (select t2.b from t2 where t2.a > t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", - "AggInput": "[{[127 128] 4}]", - "JoinInput": "[{[127 128] 4}];[]" + "AggInput": "[{[1 2] 4}]", + "JoinInput": "[{[1 2] 4}];[]" }, { "SQL": "select count(tmp.cmp) from (select t1.a as a, t1.b as b, (t1.b not in (select t2.b from t2 where t2.a > t1.a)) as cmp from t1) tmp group by tmp.a, tmp.b", - "AggInput": "[{[138 139] 4}]", - "JoinInput": "[{[138 139] 4}];[]" + "AggInput": "[{[1 2] 4}]", + "JoinInput": "[{[1 2] 4}];[]" }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 where t1.b in (select t2.b from t2 where t2.a > t1.a)) tmp group by tmp.a, tmp.b", @@ -114,8 +114,8 @@ }, { "SQL": "select * from t1 left join (select t2.a as a, t2.b as b, count(1) as cnt from t2 group by t2.a, t2.b) as tmp on t1.a = tmp.a and t1.b = tmp.b", - "AggInput": "[{[166 167] 9}]", - "JoinInput": "[{[163 164] 4}];[{[166 167] 9}]" + "AggInput": "[{[4 5] 9}]", + "JoinInput": "[{[1 2] 4}];[{[4 5] 9}]" }, { "SQL": "select count(1) from (select t1.a as a, t1.b as b from t1 limit 3) tmp group by tmp.a, tmp.b", @@ -124,7 +124,7 @@ }, { "SQL": "select count(tmp.a_sum) from (select t1.a as a, t1.b as b, sum(a) over() as a_sum from t1) tmp group by tmp.a, tmp.b", - "AggInput": "[{[174 175] 4}]", + "AggInput": "[{[1 2] 4}]", "JoinInput": "" } ] diff --git a/statistics/builder.go b/statistics/builder.go index 2c36b0c27917a..d3d66294650b2 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -236,7 +236,7 @@ func BuildHistAndTopN( if ndv > count { ndv = count } - if count == 0 || len(collector.Samples) == 0 { + if count == 0 || len(collector.Samples) == 0 || ndv == 0 { return NewHistogram(id, ndv, nullCount, 0, tp, 0, collector.TotalSize), nil, nil } sc := ctx.GetSessionVars().StmtCtx From c4d6d8e924f4fe50bd355a1f1138fa039465875b Mon Sep 17 00:00:00 2001 From: Shirly Date: Fri, 4 Jun 2021 16:28:28 +0800 Subject: [PATCH 287/343] go.mod: update br to the new version (#25127) --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 74ef9b0bb07d6..b125020a771b1 100644 --- a/go.mod +++ b/go.mod @@ -37,7 +37,7 @@ require ( github.com/opentracing/opentracing-go v1.1.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 - github.com/pingcap/br v5.1.0-alpha.0.20210526054934-d5f5f9df24f5+incompatible + github.com/pingcap/br v5.1.0-alpha.0.20210604015827-db22c6d284a0+incompatible github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3 github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd diff --git a/go.sum b/go.sum index b2583c595388f..92e5a0aae6ee3 100644 --- a/go.sum +++ b/go.sum @@ -407,8 +407,8 @@ github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5e github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 h1:IXpGy7y9HyoShAFmzW2OPF0xCA5EOoSTyZHwsgYk9Ro= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= -github.com/pingcap/br v5.1.0-alpha.0.20210526054934-d5f5f9df24f5+incompatible h1:JXkYrdHpoW0Ht6fI+pl9SC7OcNpYvJg8hc/6i+V60Eo= -github.com/pingcap/br v5.1.0-alpha.0.20210526054934-d5f5f9df24f5+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= +github.com/pingcap/br v5.1.0-alpha.0.20210604015827-db22c6d284a0+incompatible h1:+kzI8gD8buDo3y/IQiifiMHD5uIBB0GQJf3Myoj14uY= +github.com/pingcap/br v5.1.0-alpha.0.20210604015827-db22c6d284a0+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= From 878c246b08936623c8ce7878ce5904e82f5b76cd Mon Sep 17 00:00:00 2001 From: Shirly Date: Fri, 4 Jun 2021 16:42:28 +0800 Subject: [PATCH 288/343] store/tikv: move tests of rawkv from region_cache_test to rawkv_test.go (#25128) --- store/tikv/rawkv_test.go | 169 ++++++++++++++++++++++++++++++++ store/tikv/region_cache_test.go | 117 ---------------------- 2 files changed, 169 insertions(+), 117 deletions(-) create mode 100644 store/tikv/rawkv_test.go diff --git a/store/tikv/rawkv_test.go b/store/tikv/rawkv_test.go new file mode 100644 index 0000000000000..2783db1e9945f --- /dev/null +++ b/store/tikv/rawkv_test.go @@ -0,0 +1,169 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tikv + +import ( + "context" + "fmt" + + . "github.com/pingcap/check" + "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/mockstore/mocktikv" + "github.com/pingcap/tidb/store/tikv/retry" +) + +type testRawkvSuite struct { + OneByOneSuite + cluster *mocktikv.Cluster + store1 uint64 // store1 is leader + store2 uint64 // store2 is follower + peer1 uint64 // peer1 is leader + peer2 uint64 // peer2 is follower + region1 uint64 + bo *retry.Backoffer +} + +var _ = Suite(&testRawkvSuite{}) + +func (s *testRawkvSuite) SetUpTest(c *C) { + s.cluster = mocktikv.NewCluster(mocktikv.MustNewMVCCStore()) + storeIDs, peerIDs, regionID, _ := mocktikv.BootstrapWithMultiStores(s.cluster, 2) + s.region1 = regionID + s.store1 = storeIDs[0] + s.store2 = storeIDs[1] + s.peer1 = peerIDs[0] + s.peer2 = peerIDs[1] + s.bo = retry.NewBackofferWithVars(context.Background(), 5000, nil) +} + +func (s *testRawkvSuite) storeAddr(id uint64) string { + return fmt.Sprintf("store%d", id) +} + +func (s *testRawkvSuite) TestReplaceAddrWithNewStore(c *C) { + mvccStore := mocktikv.MustNewMVCCStore() + defer mvccStore.Close() + + client := &RawKVClient{ + clusterID: 0, + regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), + rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), + } + defer client.Close() + testKey := []byte("test_key") + testValue := []byte("test_value") + err := client.Put(testKey, testValue) + c.Assert(err, IsNil) + + // make store2 using store1's addr and store1 offline + store1Addr := s.storeAddr(s.store1) + s.cluster.UpdateStoreAddr(s.store1, s.storeAddr(s.store2)) + s.cluster.UpdateStoreAddr(s.store2, store1Addr) + s.cluster.RemoveStore(s.store1) + s.cluster.ChangeLeader(s.region1, s.peer2) + s.cluster.RemovePeer(s.region1, s.peer1) + + getVal, err := client.Get(testKey) + + c.Assert(err, IsNil) + c.Assert(getVal, BytesEquals, testValue) +} + +func (s *testRawkvSuite) TestUpdateStoreAddr(c *C) { + mvccStore := mocktikv.MustNewMVCCStore() + defer mvccStore.Close() + + client := &RawKVClient{ + clusterID: 0, + regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), + rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), + } + defer client.Close() + testKey := []byte("test_key") + testValue := []byte("test_value") + err := client.Put(testKey, testValue) + c.Assert(err, IsNil) + // tikv-server reports `StoreNotMatch` And retry + store1Addr := s.storeAddr(s.store1) + s.cluster.UpdateStoreAddr(s.store1, s.storeAddr(s.store2)) + s.cluster.UpdateStoreAddr(s.store2, store1Addr) + + getVal, err := client.Get(testKey) + + c.Assert(err, IsNil) + c.Assert(getVal, BytesEquals, testValue) +} + +func (s *testRawkvSuite) TestReplaceNewAddrAndOldOfflineImmediately(c *C) { + mvccStore := mocktikv.MustNewMVCCStore() + defer mvccStore.Close() + + client := &RawKVClient{ + clusterID: 0, + regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), + rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), + } + defer client.Close() + testKey := []byte("test_key") + testValue := []byte("test_value") + err := client.Put(testKey, testValue) + c.Assert(err, IsNil) + + // pre-load store2's address into cache via follower-read. + loc, err := client.regionCache.LocateKey(s.bo, testKey) + c.Assert(err, IsNil) + fctx, err := client.regionCache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, 0) + c.Assert(err, IsNil) + c.Assert(fctx.Store.storeID, Equals, s.store2) + c.Assert(fctx.Addr, Equals, "store2") + + // make store2 using store1's addr and store1 offline + store1Addr := s.storeAddr(s.store1) + s.cluster.UpdateStoreAddr(s.store1, s.storeAddr(s.store2)) + s.cluster.UpdateStoreAddr(s.store2, store1Addr) + s.cluster.RemoveStore(s.store1) + s.cluster.ChangeLeader(s.region1, s.peer2) + s.cluster.RemovePeer(s.region1, s.peer1) + + getVal, err := client.Get(testKey) + c.Assert(err, IsNil) + c.Assert(getVal, BytesEquals, testValue) +} + +func (s *testRawkvSuite) TestReplaceStore(c *C) { + mvccStore := mocktikv.MustNewMVCCStore() + defer mvccStore.Close() + + client := &RawKVClient{ + clusterID: 0, + regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), + rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), + } + defer client.Close() + testKey := []byte("test_key") + testValue := []byte("test_value") + err := client.Put(testKey, testValue) + c.Assert(err, IsNil) + + s.cluster.MarkTombstone(s.store1) + store3 := s.cluster.AllocID() + peer3 := s.cluster.AllocID() + s.cluster.AddStore(store3, s.storeAddr(s.store1)) + s.cluster.AddPeer(s.region1, store3, peer3) + s.cluster.RemovePeer(s.region1, s.peer1) + s.cluster.ChangeLeader(s.region1, peer3) + + err = client.Put(testKey, testValue) + c.Assert(err, IsNil) +} diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index 5ad0721c75c00..ccb685a680004 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -1077,123 +1077,6 @@ func loadRegionsToCache(cache *RegionCache, regionCnt int) { } } -func (s *testRegionCacheSuite) TestUpdateStoreAddr(c *C) { - mvccStore := mocktikv.MustNewMVCCStore() - defer mvccStore.Close() - - client := &RawKVClient{ - clusterID: 0, - regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), - } - defer client.Close() - testKey := []byte("test_key") - testValue := []byte("test_value") - err := client.Put(testKey, testValue) - c.Assert(err, IsNil) - // tikv-server reports `StoreNotMatch` And retry - store1Addr := s.storeAddr(s.store1) - s.cluster.UpdateStoreAddr(s.store1, s.storeAddr(s.store2)) - s.cluster.UpdateStoreAddr(s.store2, store1Addr) - - getVal, err := client.Get(testKey) - - c.Assert(err, IsNil) - c.Assert(getVal, BytesEquals, testValue) -} - -func (s *testRegionCacheSuite) TestReplaceAddrWithNewStore(c *C) { - mvccStore := mocktikv.MustNewMVCCStore() - defer mvccStore.Close() - - client := &RawKVClient{ - clusterID: 0, - regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), - } - defer client.Close() - testKey := []byte("test_key") - testValue := []byte("test_value") - err := client.Put(testKey, testValue) - c.Assert(err, IsNil) - - // make store2 using store1's addr and store1 offline - store1Addr := s.storeAddr(s.store1) - s.cluster.UpdateStoreAddr(s.store1, s.storeAddr(s.store2)) - s.cluster.UpdateStoreAddr(s.store2, store1Addr) - s.cluster.RemoveStore(s.store1) - s.cluster.ChangeLeader(s.region1, s.peer2) - s.cluster.RemovePeer(s.region1, s.peer1) - - getVal, err := client.Get(testKey) - - c.Assert(err, IsNil) - c.Assert(getVal, BytesEquals, testValue) -} - -func (s *testRegionCacheSuite) TestReplaceNewAddrAndOldOfflineImmediately(c *C) { - mvccStore := mocktikv.MustNewMVCCStore() - defer mvccStore.Close() - - client := &RawKVClient{ - clusterID: 0, - regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), - } - defer client.Close() - testKey := []byte("test_key") - testValue := []byte("test_value") - err := client.Put(testKey, testValue) - c.Assert(err, IsNil) - - // pre-load store2's address into cache via follower-read. - loc, err := client.regionCache.LocateKey(s.bo, testKey) - c.Assert(err, IsNil) - fctx, err := client.regionCache.GetTiKVRPCContext(s.bo, loc.Region, kv.ReplicaReadFollower, 0) - c.Assert(err, IsNil) - c.Assert(fctx.Store.storeID, Equals, s.store2) - c.Assert(fctx.Addr, Equals, "store2") - - // make store2 using store1's addr and store1 offline - store1Addr := s.storeAddr(s.store1) - s.cluster.UpdateStoreAddr(s.store1, s.storeAddr(s.store2)) - s.cluster.UpdateStoreAddr(s.store2, store1Addr) - s.cluster.RemoveStore(s.store1) - s.cluster.ChangeLeader(s.region1, s.peer2) - s.cluster.RemovePeer(s.region1, s.peer1) - - getVal, err := client.Get(testKey) - c.Assert(err, IsNil) - c.Assert(getVal, BytesEquals, testValue) -} - -func (s *testRegionCacheSuite) TestReplaceStore(c *C) { - mvccStore := mocktikv.MustNewMVCCStore() - defer mvccStore.Close() - - client := &RawKVClient{ - clusterID: 0, - regionCache: NewRegionCache(mocktikv.NewPDClient(s.cluster)), - rpcClient: mocktikv.NewRPCClient(s.cluster, mvccStore, nil), - } - defer client.Close() - testKey := []byte("test_key") - testValue := []byte("test_value") - err := client.Put(testKey, testValue) - c.Assert(err, IsNil) - - s.cluster.MarkTombstone(s.store1) - store3 := s.cluster.AllocID() - peer3 := s.cluster.AllocID() - s.cluster.AddStore(store3, s.storeAddr(s.store1)) - s.cluster.AddPeer(s.region1, store3, peer3) - s.cluster.RemovePeer(s.region1, s.peer1) - s.cluster.ChangeLeader(s.region1, peer3) - - err = client.Put(testKey, testValue) - c.Assert(err, IsNil) -} - func (s *testRegionCacheSuite) TestListRegionIDsInCache(c *C) { // ['' - 'm' - 'z'] region2 := s.cluster.AllocID() From 64b469623c87a25b686a324fae776af9a1735c9e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 4 Jun 2021 16:56:28 +0800 Subject: [PATCH 289/343] planner/core: support limit push down (#24757) --- planner/core/exhaust_physical_plans.go | 6 +- planner/core/integration_test.go | 4 +- planner/core/task.go | 12 ++- .../testdata/integration_serial_suite_in.json | 5 +- .../integration_serial_suite_out.json | 88 +++++++++++++++---- 5 files changed, 91 insertions(+), 24 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 0d8b183a6900a..51fb32086a3aa 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1868,7 +1868,8 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC EqualConditions: p.EqualConditions, storeTp: kv.TiFlash, mppShuffleJoin: !useBCJ, - }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, childrenProps...) + // Mpp Join has quite heavy cost. Even limit might not suspend it in time, so we dont scale the count. + }.Init(p.ctx, p.stats, p.blockOffset, childrenProps...) return []PhysicalPlan{join} } @@ -2520,6 +2521,9 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] if !p.limitHints.preferLimitToCop { allTaskTypes = append(allTaskTypes, property.RootTaskType) } + if p.canPushToCop(kv.TiFlash) && p.ctx.GetSessionVars().IsMPPAllowed() { + allTaskTypes = append(allTaskTypes, property.MppTaskType) + } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) for _, tp := range allTaskTypes { resultProp := &property.PhysicalProperty{TaskTp: tp, ExpectedCnt: float64(p.Count + p.Offset)} diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index b2861ccc6ec60..700b9fd8d9c23 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3925,8 +3925,8 @@ func (s *testIntegrationSerialSuite) TestEnforceMPP(c *C) { // read from tiflash, mpp with large cost tk.MustQuery("explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1").Check(testkit.Rows( - "HashAgg_21 1.00 11910.68 root funcs:count(Column#5)->Column#3", - "└─TableReader_23 1.00 11877.08 root data:ExchangeSender_22", + "HashAgg_21 1.00 11910.73 root funcs:count(Column#5)->Column#3", + "└─TableReader_23 1.00 11877.13 root data:ExchangeSender_22", " └─ExchangeSender_22 1.00 285050.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg_9 1.00 285050.00 batchCop[tiflash] funcs:count(1)->Column#5", " └─Selection_20 10.00 285020.00 batchCop[tiflash] eq(test.t.a, 1)", diff --git a/planner/core/task.go b/planner/core/task.go index c1b925451ca1b..8559afa8ab6a3 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1106,6 +1106,15 @@ func (p *PhysicalLimit) attach2Task(tasks ...task) task { } t = cop.convertToRootTask(p.ctx) sunk = p.sinkIntoIndexLookUp(t) + } else if mpp, ok := t.(*mppTask); ok { + newCount := p.Offset + p.Count + childProfile := mpp.plan().statsInfo() + stats := deriveLimitStats(childProfile, float64(newCount)) + pushedDownLimit := PhysicalLimit{Count: newCount}.Init(p.ctx, stats, p.blockOffset) + mpp = attachPlan2Task(pushedDownLimit, mpp).(*mppTask) + pushedDownLimit.SetSchema(pushedDownLimit.children[0].Schema()) + pushedDownLimit.cost = mpp.cost() + t = mpp.convertToRootTask(p.ctx) } p.cost = t.cost() if sunk { @@ -2067,7 +2076,8 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { }.Init(ctx, t.p.SelectBlockOffset()) p.stats = t.p.statsInfo() - p.cost = t.cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor + cst := t.cst + t.count()*ctx.GetSessionVars().GetNetworkFactor(nil) + p.cost = cst / p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor if p.ctx.GetSessionVars().IsMPPEnforced() { p.cost = 0 } diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 0cc21a8c36e4b..57d2ffa9068f1 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -301,7 +301,10 @@ "desc format = 'brief' select * from (select id from t group by id) C join (select sum(b),id from (select t.id, t1.id as b from t join (select id, count(*) as c from t group by id) t1 on t.id=t1.id)A group by id)B on C.id=b.id", "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value limit 1", "desc format = 'brief' select * from t join t t1 on t.id = t1.id order by t.value % 100 limit 1", - "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1" + "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1", + "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", + "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", + "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id limit 20) v group by v.v1" ] }, { diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index a2bb282dc3b4a..373a09470a69c 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -102,7 +102,7 @@ { "SQL": "explain format = 'verbose' select count(*) from t2 group by a", "Plan": [ - "TableReader_24 3.00 3.21 root data:ExchangeSender_23", + "TableReader_24 3.00 3.33 root data:ExchangeSender_23", "└─ExchangeSender_23 3.00 77.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection_22 3.00 0.00 batchCop[tiflash] Column#4", " └─HashAgg_8 3.00 77.00 batchCop[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", @@ -152,8 +152,8 @@ { "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a", "Plan": [ - "StreamAgg_12 1.00 18.81 root funcs:count(1)->Column#7", - "└─TableReader_44 3.00 9.81 root data:ExchangeSender_43", + "StreamAgg_12 1.00 18.93 root funcs:count(1)->Column#7", + "└─TableReader_44 3.00 9.93 root data:ExchangeSender_43", " └─ExchangeSender_43 3.00 235.38 cop[tiflash] ExchangeType: PassThrough", " └─HashJoin_40 3.00 235.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─ExchangeReceiver_19(Build) 3.00 77.00 cop[tiflash] ", @@ -167,11 +167,11 @@ { "SQL": "explain format = 'verbose' select count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", "Plan": [ - "StreamAgg_15 1.00 60.48 root funcs:count(1)->Column#10", - "└─HashJoin_65 3.00 51.48 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + "StreamAgg_15 1.00 60.60 root funcs:count(1)->Column#10", + "└─HashJoin_65 3.00 51.60 root inner join, equal:[eq(test.t1.b, test.t3.b)]", " ├─IndexReader_53(Build) 3.00 11.66 root index:IndexFullScan_52", " │ └─IndexFullScan_52 3.00 150.50 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_39(Probe) 3.00 11.02 root data:ExchangeSender_38", + " └─TableReader_39(Probe) 3.00 11.14 root data:ExchangeSender_38", " └─ExchangeSender_38 3.00 264.38 cop[tiflash] ExchangeType: PassThrough", " └─HashJoin_29 3.00 264.38 cop[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─ExchangeReceiver_35(Build) 3.00 106.00 cop[tiflash] ", @@ -185,21 +185,22 @@ { "SQL": "explain format = 'verbose' select (2) in (select count(*) from t1) from (select t.b < (select t.b from t2 limit 1 ) from t3 t) t", "Plan": [ - "HashJoin_19 3.00 133.41 root CARTESIAN left outer semi join", - "├─Selection_39(Build) 0.80 11.18 root eq(2, Column#18)", - "│ └─StreamAgg_60 1.00 69.50 root funcs:count(Column#32)->Column#18", - "│ └─TableReader_61 1.00 5.17 root data:StreamAgg_44", - "│ └─StreamAgg_44 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#32", - "│ └─TableFullScan_59 3.00 60.50 batchCop[tiflash] table:t1 keep order:false", - "└─Projection_20(Probe) 3.00 101.83 root 1->Column#28", - " └─Apply_22 3.00 82.03 root CARTESIAN left outer join", + "HashJoin_19 3.00 127.40 root CARTESIAN left outer semi join", + "├─Selection_44(Build) 0.80 11.18 root eq(2, Column#18)", + "│ └─StreamAgg_65 1.00 69.50 root funcs:count(Column#32)->Column#18", + "│ └─TableReader_66 1.00 5.17 root data:StreamAgg_49", + "│ └─StreamAgg_49 1.00 8.18 batchCop[tiflash] funcs:count(1)->Column#32", + "│ └─TableFullScan_64 3.00 60.50 batchCop[tiflash] table:t1 keep order:false", + "└─Projection_20(Probe) 3.00 95.82 root 1->Column#28", + " └─Apply_22 3.00 76.02 root CARTESIAN left outer join", " ├─TableReader_24(Build) 3.00 10.16 root data:TableFullScan_23", " │ └─TableFullScan_23 3.00 128.00 cop[tikv] table:t keep order:false", - " └─Projection_27(Probe) 1.00 23.96 root 1->Column#26", - " └─Limit_28 1.00 5.36 root offset:0, count:1", - " └─TableReader_34 1.00 5.36 root data:Limit_33", - " └─Limit_33 1.00 56.00 cop[tikv] offset:0, count:1", - " └─TableFullScan_31 1.00 56.00 cop[tikv] table:t2 keep order:false" + " └─Projection_27(Probe) 1.00 21.95 root 1->Column#26", + " └─Limit_31 1.00 3.35 root offset:0, count:1", + " └─TableReader_43 1.00 3.35 root data:ExchangeSender_42", + " └─ExchangeSender_42 1.00 79.50 cop[tiflash] ExchangeType: PassThrough", + " └─Limit_41 1.00 79.50 cop[tiflash] offset:0, count:1", + " └─TableFullScan_40 1.00 79.50 cop[tiflash] table:t2 keep order:false" ] }, { @@ -2669,6 +2670,55 @@ " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo" ] + }, + { + "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", + "Plan": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─Limit 1.00 cop[tiflash] offset:0, count:1", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 0.80 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select * from t join t t1 on t.id = t1.id limit 1", + "Plan": [ + "Limit 1.00 root offset:0, count:1", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─Limit 1.00 cop[tiflash] offset:0, count:1", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 0.80 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id limit 20) v group by v.v1", + "Plan": [ + "HashAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", + "└─Limit 20.00 root offset:0, count:20", + " └─TableReader 20.00 root data:ExchangeSender", + " └─ExchangeSender 20.00 cop[tiflash] ExchangeType: PassThrough", + " └─Limit 20.00 cop[tiflash] offset:0, count:20", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 16.02 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] } ] }, From be636811610b46b8650fd0fef4428169c1ac31ad Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Fri, 4 Jun 2021 17:28:28 +0800 Subject: [PATCH 290/343] expression: processing empty string for enum index correctly (#25101) --- executor/index_lookup_join.go | 5 +-- expression/distsql_builtin.go | 10 +++-- expression/integration_test.go | 49 ++++++++++++++++++---- planner/core/physical_plan_test.go | 4 +- planner/core/testdata/plan_suite_in.json | 6 ++- planner/core/testdata/plan_suite_out.json | 51 +++++++++++++++++++---- util/ranger/points.go | 11 +++-- 7 files changed, 109 insertions(+), 27 deletions(-) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 976ca336d088d..95f7d19f3616a 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -577,14 +577,11 @@ func (iw *innerWorker) constructDatumLookupKey(task *lookUpJoinTask, chkIdx, row } innerColType := iw.rowTypes[iw.hashCols[i]] innerValue, err := outerValue.ConvertTo(sc, innerColType) - if err != nil { + if err != nil && !(terror.ErrorEqual(err, types.ErrTruncated) && (innerColType.Tp == mysql.TypeSet || innerColType.Tp == mysql.TypeEnum)) { // If the converted outerValue overflows, we don't need to lookup it. if terror.ErrorEqual(err, types.ErrOverflow) { return nil, nil, nil } - if terror.ErrorEqual(err, types.ErrTruncated) && (innerColType.Tp == mysql.TypeSet || innerColType.Tp == mysql.TypeEnum) { - return nil, nil, nil - } return nil, nil, err } cmp, err := outerValue.CompareDatum(sc, &innerValue) diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index 0350a0b5acf4d..7fd61b30fa3fa 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -1268,9 +1268,13 @@ func convertEnum(val []byte, tp *tipb.FieldType) (*Constant, error) { if err != nil { return nil, errors.Errorf("invalid enum % x", val) } - e, err := types.ParseEnumValue(tp.Elems, uVal) - if err != nil { - return nil, err + // If uVal is 0, it should return Enum{} + var e = types.Enum{} + if uVal != 0 { + e, err = types.ParseEnumValue(tp.Elems, uVal) + if err != nil { + return nil, err + } } d := types.NewMysqlEnumDatum(e) return &Constant{Value: d, RetType: FieldTypeFromPB(tp)}, nil diff --git a/expression/integration_test.go b/expression/integration_test.go index a26600ec98e54..6a2de6f4526b8 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -9570,6 +9570,41 @@ func (s *testIntegrationSuite) TestEnumIndex(c *C) { testkit.Rows("2")) tk.MustQuery("select /*+ use_index(t,idx) */ col3 from t where col2 = 'b' and col1 is not null;").Check( testkit.Rows("2")) + + // issue25099 + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(e enum(\"a\",\"b\",\"c\"), index idx(e));") + tk.MustExec("insert ignore into t values(0),(1),(2),(3);") + tk.MustQuery("select * from t where e = '';").Check( + testkit.Rows("")) + tk.MustQuery("select * from t where e != 'a';").Sort().Check( + testkit.Rows("", "b", "c")) + tk.MustExec("alter table t drop index idx;") + tk.MustQuery("select * from t where e = '';").Check( + testkit.Rows("")) + tk.MustQuery("select * from t where e != 'a';").Sort().Check( + testkit.Rows("", "b", "c")) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(e enum(\"\"), index idx(e));") + tk.MustExec("insert ignore into t values(0),(1);") + tk.MustQuery("select * from t where e = '';").Check( + testkit.Rows("", "")) + tk.MustExec("alter table t drop index idx;") + tk.MustQuery("select * from t where e = '';").Check( + testkit.Rows("", "")) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(e enum(\"a\",\"b\",\"c\"), index idx(e));") + tk.MustExec("insert ignore into t values(0);") + tk.MustExec("select * from t t1 join t t2 on t1.e=t2.e;") + tk.MustQuery("select /*+ inl_join(t1,t2) */ * from t t1 join t t2 on t1.e=t2.e;").Check( + testkit.Rows(" ")) + tk.MustQuery("select /*+ hash_join(t1,t2) */ * from t t1 join t t2 on t1.e=t2.e;").Check( + testkit.Rows(" ")) + tk.MustQuery("select /*+ inl_hash_join(t1,t2) */ * from t t1 join t t2 on t1.e=t2.e;").Check( + testkit.Rows(" ")) } // Previously global values were cached. This is incorrect. @@ -9675,13 +9710,13 @@ func (s *testIntegrationSuite) TestComplexShowVariables(c *C) { // and 16 rows in MySQL 8.0 (the aliases for tx_isolation is removed, along with query cache) // In the event that we hide noop sysvars in future, we must keep these variables. tk := testkit.NewTestKit(c, s.store) - c.Assert(tk.MustQuery(`SHOW VARIABLES WHERE Variable_name ='language' OR Variable_name = 'net_write_timeout' OR Variable_name = 'interactive_timeout' -OR Variable_name = 'wait_timeout' OR Variable_name = 'character_set_client' OR Variable_name = 'character_set_connection' -OR Variable_name = 'character_set' OR Variable_name = 'character_set_server' OR Variable_name = 'tx_isolation' -OR Variable_name = 'transaction_isolation' OR Variable_name = 'character_set_results' OR Variable_name = 'timezone' -OR Variable_name = 'time_zone' OR Variable_name = 'system_time_zone' -OR Variable_name = 'lower_case_table_names' OR Variable_name = 'max_allowed_packet' OR Variable_name = 'net_buffer_length' -OR Variable_name = 'sql_mode' OR Variable_name = 'query_cache_type' OR Variable_name = 'query_cache_size' + c.Assert(tk.MustQuery(`SHOW VARIABLES WHERE Variable_name ='language' OR Variable_name = 'net_write_timeout' OR Variable_name = 'interactive_timeout' +OR Variable_name = 'wait_timeout' OR Variable_name = 'character_set_client' OR Variable_name = 'character_set_connection' +OR Variable_name = 'character_set' OR Variable_name = 'character_set_server' OR Variable_name = 'tx_isolation' +OR Variable_name = 'transaction_isolation' OR Variable_name = 'character_set_results' OR Variable_name = 'timezone' +OR Variable_name = 'time_zone' OR Variable_name = 'system_time_zone' +OR Variable_name = 'lower_case_table_names' OR Variable_name = 'max_allowed_packet' OR Variable_name = 'net_buffer_length' +OR Variable_name = 'sql_mode' OR Variable_name = 'query_cache_type' OR Variable_name = 'query_cache_size' OR Variable_name = 'license' OR Variable_name = 'init_connect'`).Rows(), HasLen, 19) } diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 6bfda74d6ab1d..419c12715713c 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -1720,8 +1720,8 @@ func (s *testPlanSuite) TestEnumIndex(c *C) { tk := testkit.NewTestKit(c, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") - tk.MustExec("create table t(e enum('c','b','a'), index idx(e))") - tk.MustExec("insert into t values(1),(2),(3);") + tk.MustExec("create table t(e enum('c','b','a',''), index idx(e))") + tk.MustExec("insert ignore into t values(0),(1),(2),(3),(4);") for i, ts := range input { s.testData.OnRecord(func() { diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index 64a5c973e89cd..dc420972aaa50 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -697,7 +697,11 @@ "select e from t where e > ''", "select e from t where e > 'd'", "select e from t where e > -1", - "select e from t where e > 5" + "select e from t where e > 5", + + // zero-value + "select e from t where e = ''", + "select e from t where e != ''" ] } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index ce33bec905757..9e2d5b248a7ac 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -2297,10 +2297,12 @@ { "SQL": "select e from t where e != 'b'", "Plan": [ - "IndexReader 20.00 root index:IndexRangeScan", - "└─IndexRangeScan 20.00 cop[tikv] table:t, index:idx(e) range:[\"c\",\"c\"], [\"a\",\"a\"], keep order:false, stats:pseudo" + "IndexReader 40.00 root index:IndexRangeScan", + "└─IndexRangeScan 40.00 cop[tikv] table:t, index:idx(e) range:[\"\",\"\"], [\"c\",\"c\"], [\"a\",\"a\"], [\"\",\"\"], keep order:false, stats:pseudo" ], "Result": [ + "", + "", "a", "c" ] @@ -2329,20 +2331,24 @@ { "SQL": "select e from t where e < 'b'", "Plan": [ - "IndexReader 10.00 root index:IndexRangeScan", - "└─IndexRangeScan 10.00 cop[tikv] table:t, index:idx(e) range:[\"a\",\"a\"], keep order:false, stats:pseudo" + "IndexReader 30.00 root index:IndexRangeScan", + "└─IndexRangeScan 30.00 cop[tikv] table:t, index:idx(e) range:[\"\",\"\"], [\"a\",\"a\"], [\"\",\"\"], keep order:false, stats:pseudo" ], "Result": [ + "", + "", "a" ] }, { "SQL": "select e from t where e <= 'b'", "Plan": [ - "IndexReader 20.00 root index:IndexRangeScan", - "└─IndexRangeScan 20.00 cop[tikv] table:t, index:idx(e) range:[\"b\",\"b\"], [\"a\",\"a\"], keep order:false, stats:pseudo" + "IndexReader 40.00 root index:IndexRangeScan", + "└─IndexRangeScan 40.00 cop[tikv] table:t, index:idx(e) range:[\"\",\"\"], [\"b\",\"b\"], [\"a\",\"a\"], [\"\",\"\"], keep order:false, stats:pseudo" ], "Result": [ + "", + "", "a", "b" ] @@ -2364,6 +2370,8 @@ "└─IndexRangeScan 6656.67 cop[tikv] table:t, index:idx(e) range:[-inf,\"b\"), (\"b\",+inf], keep order:false, stats:pseudo" ], "Result": [ + "", + "", "a", "c" ] @@ -2375,6 +2383,7 @@ "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:(\"b\",+inf], keep order:false, stats:pseudo" ], "Result": [ + "", "a" ] }, @@ -2385,6 +2394,7 @@ "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:[\"b\",+inf], keep order:false, stats:pseudo" ], "Result": [ + "", "a", "b" ] @@ -2396,6 +2406,7 @@ "└─IndexRangeScan 3323.33 cop[tikv] table:t, index:idx(e) range:[-inf,\"b\"), keep order:false, stats:pseudo" ], "Result": [ + "", "c" ] }, @@ -2406,6 +2417,7 @@ "└─IndexRangeScan 3323.33 cop[tikv] table:t, index:idx(e) range:[-inf,\"b\"], keep order:false, stats:pseudo" ], "Result": [ + "", "b", "c" ] @@ -2436,6 +2448,8 @@ "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:[\"\",+inf], keep order:false, stats:pseudo" ], "Result": [ + "", + "", "a", "b", "c" @@ -2445,9 +2459,32 @@ "SQL": "select e from t where e > 5", "Plan": [ "IndexReader 3333.33 root index:IndexRangeScan", - "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:(\"a\",+inf], keep order:false, stats:pseudo" + "└─IndexRangeScan 3333.33 cop[tikv] table:t, index:idx(e) range:(\"\",+inf], keep order:false, stats:pseudo" ], "Result": null + }, + { + "SQL": "select e from t where e = ''", + "Plan": [ + "IndexReader 20.00 root index:IndexRangeScan", + "└─IndexRangeScan 20.00 cop[tikv] table:t, index:idx(e) range:[\"\",\"\"], [\"\",\"\"], keep order:false, stats:pseudo" + ], + "Result": [ + "", + "" + ] + }, + { + "SQL": "select e from t where e != ''", + "Plan": [ + "IndexReader 30.00 root index:IndexRangeScan", + "└─IndexRangeScan 30.00 cop[tikv] table:t, index:idx(e) range:[\"c\",\"c\"], [\"b\",\"b\"], [\"a\",\"a\"], keep order:false, stats:pseudo" + ], + "Result": [ + "a", + "b", + "c" + ] } ] } diff --git a/util/ranger/points.go b/util/ranger/points.go index 3931a2c42b9fe..03b35e367cb00 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -457,9 +457,14 @@ func handleEnumFromBinOp(sc *stmtctx.StatementContext, ft *types.FieldType, val } tmpEnum := types.Enum{} - for i := range ft.Elems { - tmpEnum.Name = ft.Elems[i] - tmpEnum.Value = uint64(i) + 1 + for i := 0; i <= len(ft.Elems); i++ { + if i == 0 { + tmpEnum = types.Enum{} + } else { + tmpEnum.Name = ft.Elems[i-1] + tmpEnum.Value = uint64(i) + } + d := types.NewCollateMysqlEnumDatum(tmpEnum, ft.Collate) if v, err := d.CompareDatum(sc, &val); err == nil { switch op { From 11f68a9d283b77d4514447442ce0c43f32f51562 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Fri, 4 Jun 2021 17:33:19 +0800 Subject: [PATCH 291/343] planner: skip some unstable test cases (#25155) * fixup --- server/tidb_test.go | 1 + statistics/handle/handle_test.go | 9 ++------- tests/globalkilltest/global_kill_test.go | 1 + 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/server/tidb_test.go b/server/tidb_test.go index 09025e8bec95b..a5649033d1e06 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -1174,6 +1174,7 @@ func (ts *tidbTestSerialSuite) TestPrepareCount(c *C) { } func (ts *tidbTestTopSQLSuite) TestTopSQLCPUProfile(c *C) { + c.Skip("unstable") db, err := sql.Open("mysql", ts.getDSN()) c.Assert(err, IsNil, Commentf("Error connecting")) defer func() { diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 234e5f3cae0cc..37d3d18479e7c 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" - "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" ) @@ -874,9 +873,7 @@ func (s *testStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, } func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts(c *C) { - if israce.RaceEnabled { - c.Skip("unstable, skip race test") - } + c.Skip("unstable, skip race test") defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) s.prepareForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt") @@ -912,9 +909,7 @@ func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts(c *C) { } func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts2(c *C) { - if israce.RaceEnabled { - c.Skip("unstable, skip race test") - } + c.Skip("unstable, skip race test") defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) s.prepareForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2") diff --git a/tests/globalkilltest/global_kill_test.go b/tests/globalkilltest/global_kill_test.go index f424a4b52d167..164e5fbf02525 100644 --- a/tests/globalkilltest/global_kill_test.go +++ b/tests/globalkilltest/global_kill_test.go @@ -535,6 +535,7 @@ func (s *TestGlobalKillSuite) TestMultipleTiDB(c *C) { } func (s *TestGlobalKillSuite) TestLostConnection(c *C) { + c.Skip("unstable, skip race test") c.Assert(s.pdErr, IsNil, Commentf(msgErrConnectPD, s.pdErr)) // tidb1 From 2f44a7111037ff212461e74fbcd2952dde7314b2 Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Fri, 4 Jun 2021 18:02:28 +0800 Subject: [PATCH 292/343] cast: fix cast bit to binary and varbinary (#25064) --- ddl/column_type_change_test.go | 37 +++++++++++++++++++++++++++++++--- ddl/ddl_api.go | 18 ++++++++++++++++- types/datum.go | 15 +++++++++++++- 3 files changed, 65 insertions(+), 5 deletions(-) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 38e2cc83ebc5a..f1e5b9d08632e 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -817,7 +817,7 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromNumericToOthers(c *C // MySQL will get "ERROR 1406 (22001): Data truncation: Data too long for column 'f64' at row 1". tk.MustExec("alter table t modify f64 char(20)") tk.MustExec("alter table t modify b char(20)") - tk.MustQuery("select * from t").Check(testkit.Rows("-258.1234500 333.33 2000000.20000002 323232323.32323235 -111.111115 -222222222222.22223 \x15")) + tk.MustQuery("select * from t").Check(testkit.Rows("-258.1234500 333.33 2000000.20000002 323232323.32323235 -111.111115 -222222222222.22223 21")) // varchar reset(tk) @@ -844,7 +844,7 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromNumericToOthers(c *C tk.MustGetErrCode("alter table t modify f32 binary(10)", mysql.ErrDataTooLong) tk.MustGetErrCode("alter table t modify f64 binary(10)", mysql.ErrDataTooLong) tk.MustExec("alter table t modify b binary(10)") - tk.MustQuery("select * from t").Check(testkit.Rows("-258.1234500 333.33\x00\x00\x00\x00 2000000.20000002 323232323.32323235 -111.111115 -222222222222.22223 \x15\x00\x00\x00\x00\x00\x00\x00\x00\x00")) + tk.MustQuery("select * from t").Check(testkit.Rows("-258.1234500 333.33\x00\x00\x00\x00 2000000.20000002 323232323.32323235 -111.111115 -222222222222.22223 21\x00\x00\x00\x00\x00\x00\x00\x00")) // varbinary reset(tk) @@ -858,7 +858,7 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromNumericToOthers(c *C // MySQL will get "ERROR 1406 (22001): Data truncation: Data too long for column 'f64' at row 1". tk.MustExec("alter table t modify f64 varbinary(30)") tk.MustExec("alter table t modify b varbinary(30)") - tk.MustQuery("select * from t").Check(testkit.Rows("-258.1234500 333.33 2000000.20000002 323232323.32323235 -111.111115 -222222222222.22223 \x15")) + tk.MustQuery("select * from t").Check(testkit.Rows("-258.1234500 333.33 2000000.20000002 323232323.32323235 -111.111115 -222222222222.22223 21")) // blob reset(tk) @@ -1913,3 +1913,34 @@ func (s *testColumnTypeChangeSuite) TestCTCShouldCastTheDefaultValue(c *C) { tk.MustGetErrCode("alter table tbl change column col_279 col_287 int", mysql.ErrTruncatedWrongValue) tk.MustQuery("select col_279 from tbl").Check(testkit.Rows("RAWTdm\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00")) } + +// Close issue #25037 +// 1: for default value of binary of create-table, it should append the \0 as the suffix to meet flen. +// 2: when cast the bit to binary, we should consider to convert it to uint then cast uint to string, rather than taking the bit to string directly. +func (s *testColumnTypeChangeSuite) TestCTCCastBitToBinary(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + // Enable column change variable. + tk.Se.GetSessionVars().EnableChangeColumnType = true + + // For point 1: + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a binary(10) default 't')") + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `a` binary(10) DEFAULT 't\\0\\0\\0\\0\\0\\0\\0\\0\\0'\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + + // For point 2 with binary: + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a bit(13) not null) collate utf8mb4_general_ci") + tk.MustExec("insert into t values ( 4047 )") + tk.MustExec("alter table t change column a a binary(248) collate binary default 't'") + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `a` binary(248) DEFAULT 't\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0\\0'\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci")) + tk.MustQuery("select * from t").Check(testkit.Rows("4047\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00")) + + // For point 2 with varbinary: + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a bit(13) not null) collate utf8mb4_general_ci") + tk.MustExec("insert into t values ( 4047 )") + tk.MustExec("alter table t change column a a varbinary(248) collate binary default 't'") + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `a` varbinary(248) DEFAULT 't'\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci")) + tk.MustQuery("select * from t").Check(testkit.Rows("4047")) +} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 30ad5bb6f2878..d565fa5f81cff 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3663,13 +3663,29 @@ func setDefaultValue(ctx sessionctx.Context, col *table.Column, option *ast.Colu if err != nil { return hasDefaultValue, errors.Trace(err) } - err = col.SetDefaultValue(value) + err = setDefaultValueWithBinaryPadding(col, value) if err != nil { return hasDefaultValue, errors.Trace(err) } return hasDefaultValue, nil } +func setDefaultValueWithBinaryPadding(col *table.Column, value interface{}) error { + err := col.SetDefaultValue(value) + if err != nil { + return err + } + // https://dev.mysql.com/doc/refman/8.0/en/binary-varbinary.html + // Set the default value for binary type should append the paddings. + if value != nil { + if col.Tp == mysql.TypeString && types.IsBinaryStr(&col.FieldType) && len(value.(string)) < col.Flen { + padding := make([]byte, col.Flen-len(value.(string))) + col.DefaultValue = string(append([]byte(col.DefaultValue.(string)), padding...)) + } + } + return nil +} + func setColumnComment(ctx sessionctx.Context, col *table.Column, option *ast.ColumnOption) error { value, err := expression.EvalAstExpr(ctx, option.Expr) if err != nil { diff --git a/types/datum.go b/types/datum.go index a1f8c33416b4c..c744f514e0a9b 100644 --- a/types/datum.go +++ b/types/datum.go @@ -958,8 +958,21 @@ func (d *Datum) convertToString(sc *stmtctx.StatementContext, target *FieldType) s = d.GetMysqlEnum().String() case KindMysqlSet: s = d.GetMysqlSet().String() - case KindBinaryLiteral, KindMysqlBit: + case KindBinaryLiteral: s = d.GetBinaryLiteral().ToString() + case KindMysqlBit: + // issue #25037 + // bit to binary/varbinary. should consider transferring to uint first. + if target.Tp == mysql.TypeString || (target.Tp == mysql.TypeVarchar && target.Collate == charset.CollationBin) { + val, err := d.GetBinaryLiteral().ToInt(sc) + if err != nil { + s = d.GetBinaryLiteral().ToString() + } else { + s = strconv.FormatUint(val, 10) + } + } else { + s = d.GetBinaryLiteral().ToString() + } case KindMysqlJSON: s = d.GetMysqlJSON().String() default: From 36d11d165e95ee262ce12e60ab7d65c558a5aafc Mon Sep 17 00:00:00 2001 From: Lynn Date: Fri, 4 Jun 2021 18:32:28 +0800 Subject: [PATCH 293/343] ddl: forbidden conversion decimal to date/datetime/timestamp (#25136) --- ddl/column_type_change_test.go | 18 +++++++-------- ddl/ddl_api.go | 42 ++++++++++++---------------------- 2 files changed, 24 insertions(+), 36 deletions(-) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index f1e5b9d08632e..10f21a0e8bbca 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -915,15 +915,15 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromNumericToOthers(c *C reset(tk) tk.MustExec("insert into t values (200805.11, 307.333, 20200805.11111111, 20200805111307.11111111, 200805111307.11111111, 20200805111307.11111111, b'10101')") // MySQL will get "ERROR 1292 (22001) Data truncation: Incorrect datetime value: '200805.1100000' for column 'd' at row 1". - tk.MustExec("alter table t modify d datetime") + tk.MustGetErrCode("alter table t modify d datetime", mysql.ErrUnsupportedDDLOperation) // MySQL will get "ERROR 1292 (22001) Data truncation: Incorrect datetime value: '307.33' for column 'n' at row 1". - tk.MustExec("alter table t modify n datetime") + tk.MustGetErrCode("alter table t modify n datetime", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify r datetime", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify db datetime", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify f32 datetime", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify f64 datetime", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify b datetime", mysql.ErrUnsupportedDDLOperation) - tk.MustQuery("select * from t").Check(testkit.Rows("2020-08-05 00:00:00 2000-03-07 00:00:00 20200805.11111111 20200805111307.11 200805100000 20200805111307.11 \x15")) + tk.MustQuery("select * from t").Check(testkit.Rows("200805.1100000 307.33 20200805.11111111 20200805111307.11 200805100000 20200805111307.11 \x15")) // time reset(tk) tk.MustExec("insert into t values (200805.11, 307.333, 20200805.11111111, 20200805111307.11111111, 200805111307.11111111, 20200805111307.11111111, b'10101')") @@ -939,28 +939,28 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromNumericToOthers(c *C reset(tk) tk.MustExec("insert into t values (200805.11, 307.333, 20200805.11111111, 20200805111307.11111111, 200805111307.11111111, 20200805111307.11111111, b'10101')") // MySQL will get "ERROR 1292 (22001) Data truncation: Incorrect date value: '200805.1100000' for column 'd' at row 1". - tk.MustExec("alter table t modify d date") + tk.MustGetErrCode("alter table t modify d date", mysql.ErrUnsupportedDDLOperation) // MySQL will get "ERROR 1292 (22001) Data truncation: Incorrect date value: '307.33' for column 'n' at row 1". - tk.MustExec("alter table t modify n date") + tk.MustGetErrCode("alter table t modify n date", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify r date", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify db date", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify f32 date", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify f64 date", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify b date", mysql.ErrUnsupportedDDLOperation) - tk.MustQuery("select * from t").Check(testkit.Rows("2020-08-05 2000-03-07 20200805.11111111 20200805111307.11 200805100000 20200805111307.11 \x15")) + tk.MustQuery("select * from t").Check(testkit.Rows("200805.1100000 307.33 20200805.11111111 20200805111307.11 200805100000 20200805111307.11 \x15")) // timestamp reset(tk) tk.MustExec("insert into t values (200805.11, 307.333, 20200805.11111111, 20200805111307.11111111, 200805111307.11111111, 20200805111307.11111111, b'10101')") // MySQL will get "ERROR 1292 (22001) Data truncation: Incorrect datetime value: '200805.1100000' for column 'd' at row 1". - tk.MustExec("alter table t modify d timestamp") + tk.MustGetErrCode("alter table t modify d timestamp", mysql.ErrUnsupportedDDLOperation) // MySQL will get "ERROR 1292 (22001) Data truncation: Incorrect datetime value: '307.33' for column 'n' at row 1". - tk.MustExec("alter table t modify n timestamp") + tk.MustGetErrCode("alter table t modify n timestamp", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify r timestamp", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify db timestamp", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify f32 timestamp", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify f64 timestamp", mysql.ErrUnsupportedDDLOperation) tk.MustGetErrCode("alter table t modify b timestamp", mysql.ErrUnsupportedDDLOperation) - tk.MustQuery("select * from t").Check(testkit.Rows("2020-08-05 00:00:00 2000-03-07 00:00:00 20200805.11111111 20200805111307.11 200805100000 20200805111307.11 \x15")) + tk.MustQuery("select * from t").Check(testkit.Rows("200805.1100000 307.33 20200805.11111111 20200805111307.11 200805100000 20200805111307.11 \x15")) // year reset(tk) tk.MustExec("insert into t values (200805.11, 307.333, 2.55555, 98.1111111, 2154.00001, 20200805111307.11111111, b'10101')") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d565fa5f81cff..e98e68313c8c6 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3573,42 +3573,30 @@ func needReorgToChange(origin *types.FieldType, to *types.FieldType) (needReorg } func checkTypeChangeSupported(origin *types.FieldType, to *types.FieldType) bool { - if types.IsString(origin.Tp) && to.Tp == mysql.TypeBit { - // TODO: Currently string data type cast to bit are not compatible with mysql, should fix here after compatible. + if (types.IsTypeTime(origin.Tp) || origin.Tp == mysql.TypeDuration || origin.Tp == mysql.TypeYear || + types.IsString(origin.Tp) || origin.Tp == mysql.TypeJSON) && + to.Tp == mysql.TypeBit { + // TODO: Currently date/datetime/timestamp/time/year/string/json data type cast to bit are not compatible with mysql, should fix here after compatible. return false } - if (origin.Tp == mysql.TypeEnum || origin.Tp == mysql.TypeSet) && - (types.IsTypeTime(to.Tp) || to.Tp == mysql.TypeDuration) { - // TODO: Currently enum/set cast to date/datetime/timestamp/time/bit are not support yet, should fix here after supported. + if (types.IsTypeTime(origin.Tp) || origin.Tp == mysql.TypeDuration || origin.Tp == mysql.TypeYear || + origin.Tp == mysql.TypeNewDecimal || origin.Tp == mysql.TypeFloat || origin.Tp == mysql.TypeDouble || origin.Tp == mysql.TypeJSON || origin.Tp == mysql.TypeBit) && + (to.Tp == mysql.TypeEnum || to.Tp == mysql.TypeSet) { + // TODO: Currently date/datetime/timestamp/time/year/decimal/float/double/json/bit cast to enum/set are not support yet, should fix here after supported. return false } - if (types.IsTypeTime(origin.Tp) || origin.Tp == mysql.TypeDuration || origin.Tp == mysql.TypeYear) && - (to.Tp == mysql.TypeEnum || to.Tp == mysql.TypeSet || to.Tp == mysql.TypeBit) { - // TODO: Currently date and time cast to enum/set/bit are not support yet, should fix here after supported. + if (origin.Tp == mysql.TypeEnum || origin.Tp == mysql.TypeSet || origin.Tp == mysql.TypeBit || + origin.Tp == mysql.TypeNewDecimal || origin.Tp == mysql.TypeFloat || origin.Tp == mysql.TypeDouble) && + (types.IsTypeTime(to.Tp)) { + // TODO: Currently enum/set/bit/decimal/float/double cast to date/datetime/timestamp type are not support yet, should fix here after supported. return false } - if (origin.Tp == mysql.TypeFloat || origin.Tp == mysql.TypeDouble) && - (types.IsTypeTime(to.Tp) || to.Tp == mysql.TypeEnum || to.Tp == mysql.TypeSet) { - // TODO: Currently float/double cast to date/datetime/timestamp/enum/set type are not support yet, should fix here after supported. - return false - } - - if origin.Tp == mysql.TypeBit && - (types.IsTypeTime(to.Tp) || to.Tp == mysql.TypeDuration || to.Tp == mysql.TypeEnum || to.Tp == mysql.TypeSet) { - // TODO: Currently bit cast to date/datetime/timestamp/time/enum/set are not support yet, should fix here after supported. - return false - } - - if origin.Tp == mysql.TypeNewDecimal && (to.Tp == mysql.TypeEnum || to.Tp == mysql.TypeSet) { - // TODO: Currently decimal cast to enum/set are not support yet, should fix here after supported. - return false - } - - if origin.Tp == mysql.TypeJSON && (to.Tp == mysql.TypeEnum || to.Tp == mysql.TypeSet || to.Tp == mysql.TypeBit) { - // TODO: Currently json cast to enum/set/bit are not support yet, should fix here after supported. + if (origin.Tp == mysql.TypeEnum || origin.Tp == mysql.TypeSet || origin.Tp == mysql.TypeBit) && + to.Tp == mysql.TypeDuration { + // TODO: Currently enum/set/bit cast to time are not support yet, should fix here after supported. return false } From eaa426cb6af6de3618a59fe50efc962d5480a2e5 Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Fri, 4 Jun 2021 18:56:28 +0800 Subject: [PATCH 294/343] sessionctx: add skip-init for variable tidb_memory_usage_alarm_ratio (#25166) --- sessionctx/variable/sysvar.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 295401b0a97c3..e5e63ab86952f 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1366,7 +1366,7 @@ var defaultSysVars = []*SysVar{ }, GetSession: func(s *SessionVars) (string, error) { return fmt.Sprintf("%d", atomic.LoadUint64(&ExpensiveQueryTimeThreshold)), nil }}, - {Scope: ScopeSession, Name: TiDBMemoryUsageAlarmRatio, Value: strconv.FormatFloat(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0.0, MaxValue: 1.0, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeSession, Name: TiDBMemoryUsageAlarmRatio, Value: strconv.FormatFloat(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio, 'f', -1, 64), Type: TypeFloat, MinValue: 0.0, MaxValue: 1.0, skipInit: true, SetSession: func(s *SessionVars, val string) error { MemoryUsageAlarmRatio.Store(tidbOptFloat64(val, 0.8)) return nil }, GetSession: func(s *SessionVars) (string, error) { From a591671a30c96110105f9ae645896e45067a70a9 Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Fri, 4 Jun 2021 19:27:28 +0800 Subject: [PATCH 295/343] fix column type change error message (#25154) --- ddl/ddl_api.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e98e68313c8c6..8458ef70e3f6f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3527,7 +3527,7 @@ func CheckModifyTypeCompatible(origin *types.FieldType, to *types.FieldType) (ca // Deal with the different type. if !checkTypeChangeSupported(origin, to) { - unsupportedMsg := fmt.Sprintf("change from original type %v to %v is currently unsupported yet", to.CompactStr(), origin.CompactStr()) + unsupportedMsg := fmt.Sprintf("change from original type %v to %v is currently unsupported yet", origin.CompactStr(), to.CompactStr()) return false, unsupportedMsg, errUnsupportedModifyColumn.GenWithStackByArgs(unsupportedMsg) } From 9a1e57c0219a0a5f0fdce936e70c82ee91f577f8 Mon Sep 17 00:00:00 2001 From: Fu Zhe Date: Fri, 4 Jun 2021 20:00:27 +0800 Subject: [PATCH 296/343] expression: Support push function replace down to TiFlash (#25084) --- expression/expr_to_pb_test.go | 5 +++++ expression/expression.go | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index 6fb3a152f8f76..c767b5ad88992 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -734,6 +734,11 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) + // Replace + function, err = NewFunction(mock.NewContext(), ast.Replace, types.NewFieldType(mysql.TypeString), stringColumn, stringColumn, stringColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + // ScalarFuncSig_RoundReal function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeDouble), realColumn) c.Assert(err, IsNil) diff --git a/expression/expression.go b/expression/expression.go index 5adb4af16a5ea..e1ec7c5b311d4 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1050,6 +1050,11 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { case tipb.ScalarFuncSig_ExtractDatetime: return true } + case ast.Replace: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_Replace: + return true + } case ast.StrToDate: switch function.Function.PbCode() { case From 5c350bef3b73187a6004b68f721d4502d9d85247 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 4 Jun 2021 20:30:28 +0800 Subject: [PATCH 297/343] meta,executor: fix show create table panic for temporary table with auto_increment columns (#25140) --- executor/show_test.go | 16 ++++++++++++++++ meta/autoid/memid.go | 2 +- 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/executor/show_test.go b/executor/show_test.go index 778f9452b5347..4109559b30928 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -1337,4 +1337,20 @@ func (s *testSuite5) TestShowTemporaryTable(c *C) { // Verify that the `show create table` result can be used to build the table. createTable := strings.ReplaceAll(expect, "t3", "t4") tk.MustExec(createTable) + + // Cover auto increment column. + tk.MustExec(`CREATE GLOBAL TEMPORARY TABLE t5 ( + id int(11) NOT NULL AUTO_INCREMENT, + b int(11) NOT NULL, + pad varbinary(255) DEFAULT NULL, + PRIMARY KEY (id), + KEY b (b)) ON COMMIT DELETE ROWS`) + expect = "CREATE GLOBAL TEMPORARY TABLE `t5` (\n" + + " `id` int(11) NOT NULL AUTO_INCREMENT,\n" + + " `b` int(11) NOT NULL,\n" + + " `pad` varbinary(255) DEFAULT NULL,\n" + + " PRIMARY KEY (`id`) /*T![clustered_index] CLUSTERED */,\n" + + " KEY `b` (`b`)\n" + + ") ENGINE=memory DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin ON COMMIT DELETE ROWS" + tk.MustQuery("show create table t5").Check(testkit.Rows("t5 " + expect)) } diff --git a/meta/autoid/memid.go b/meta/autoid/memid.go index 703ffe1db4bf8..d9a6be9f6058a 100644 --- a/meta/autoid/memid.go +++ b/meta/autoid/memid.go @@ -64,7 +64,7 @@ func (alloc *inMemoryAllocator) GetType() AllocatorType { // NextGlobalAutoID implements autoid.Allocator NextGlobalAutoID interface. func (alloc *inMemoryAllocator) NextGlobalAutoID(tableID int64) (int64, error) { - return 0, errNotImplemented.GenWithStackByArgs() + return alloc.base, nil } func (alloc *inMemoryAllocator) Alloc(ctx context.Context, tableID int64, n uint64, increment, offset int64) (int64, int64, error) { From 38e2faea90b57885979c74a0c1e8e819893a1973 Mon Sep 17 00:00:00 2001 From: bb7133 Date: Fri, 4 Jun 2021 21:02:15 +0800 Subject: [PATCH 298/343] executor: skip unstable test case TestMergeJoinInDisk and TestAnalyzeIndexExtractTopN. (#25170) * executor: skip unstable test case TestMergeJoinInDisk. --- executor/analyze_test.go | 4 +--- executor/merge_join_test.go | 2 ++ 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 58ee27bf47d79..bc5e2c9ba868c 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -238,9 +238,7 @@ func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) { } func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { - if israce.RaceEnabled { - c.Skip("unstable, skip race test") - } + c.Skip("unstable, skip it and fix it before 20210618") store, err := mockstore.NewMockStore() c.Assert(err, IsNil) defer func() { diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index cf5328fc93263..c58eb89181ef5 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -279,6 +279,8 @@ func (s *testSerialSuite1) TestShuffleMergeJoinInDisk(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.DiskTracker.MaxConsumed(), Greater, int64(0)) } func (s *testSerialSuite1) TestMergeJoinInDisk(c *C) { + c.Skip("unstable, skip it and fix it before 20210618") + defer config.RestoreFunc()() config.UpdateGlobal(func(conf *config.Config) { conf.OOMUseTmpStorage = true From 5ca65a883499453e84526bb91fb4c76bce5b2fee Mon Sep 17 00:00:00 2001 From: xhe Date: Fri, 4 Jun 2021 21:32:28 +0800 Subject: [PATCH 299/343] *: concurrent session access leads to race (#25173) --- session/session.go | 26 +++++++++++++++++--------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/session/session.go b/session/session.go index 7435ef5882aaa..fe6fef23bf019 100644 --- a/session/session.go +++ b/session/session.go @@ -2430,14 +2430,22 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { } if !config.GetGlobalConfig().Security.SkipGrantTable { - err = dom.LoadPrivilegeLoop(se) + se4, err := createSession(store) + if err != nil { + return nil, err + } + err = dom.LoadPrivilegeLoop(se4) if err != nil { return nil, err } } // Rebuild sysvar cache in a loop - err = dom.LoadSysVarCacheLoop(se) + se5, err := createSession(store) + if err != nil { + return nil, err + } + err = dom.LoadSysVarCacheLoop(se5) if err != nil { return nil, err } @@ -2448,28 +2456,28 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { return nil, err } } - se4, err := createSession(store) + se6, err := createSession(store) if err != nil { return nil, err } - err = executor.LoadExprPushdownBlacklist(se4) + err = executor.LoadExprPushdownBlacklist(se6) if err != nil { return nil, err } - err = executor.LoadOptRuleBlacklist(se4) + err = executor.LoadOptRuleBlacklist(se6) if err != nil { return nil, err } - dom.TelemetryReportLoop(se4) - dom.TelemetryRotateSubWindowLoop(se4) + dom.TelemetryReportLoop(se6) + dom.TelemetryRotateSubWindowLoop(se6) - se5, err := createSession(store) + se7, err := createSession(store) if err != nil { return nil, err } - err = dom.UpdateTableStatsLoop(se5) + err = dom.UpdateTableStatsLoop(se7) if err != nil { return nil, err } From 4457ba88e66924f852674cd391c03e88e9ff1a78 Mon Sep 17 00:00:00 2001 From: Howie Date: Fri, 4 Jun 2021 22:20:28 +0800 Subject: [PATCH 300/343] ddl: add virtual columns compatibility for temporary table (#25065) --- ddl/db_test.go | 31 +++++++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/ddl/db_test.go b/ddl/db_test.go index fb4e38f37f109..2a114b49d205f 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -3514,6 +3514,37 @@ out: tk.MustExec("drop table tnn") } +func (s *testDBSuite3) TestVirtualColumnDDL(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("set tidb_enable_global_temporary_table=true") + tk.MustExec("use test") + tk.MustExec("drop table if exists test_gv_ddl") + tk.MustExec(`create global temporary table test_gv_ddl(a int, b int as (a+8) virtual, c int as (b + 2) stored) on commit delete rows;`) + defer tk.MustExec("drop table if exists test_gv_ddl") + is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) + table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("test_gv_ddl")) + c.Assert(err, IsNil) + testCases := []struct { + generatedExprString string + generatedStored bool + }{ + {"", false}, + {"`a` + 8", false}, + {"`b` + 2", true}, + } + for i, column := range table.Meta().Columns { + c.Assert(column.GeneratedExprString, Equals, testCases[i].generatedExprString) + c.Assert(column.GeneratedStored, Equals, testCases[i].generatedStored) + } + result := tk.MustQuery(`DESC test_gv_ddl`) + result.Check(testkit.Rows(`a int(11) YES `, `b int(11) YES VIRTUAL GENERATED`, `c int(11) YES STORED GENERATED`)) + tk.MustExec("begin;") + tk.MustExec("insert into test_gv_ddl values (1, default, default)") + tk.MustQuery("select * from test_gv_ddl").Check(testkit.Rows("1 9 11")) + _, err = tk.Exec("commit") + c.Assert(err, IsNil) +} + func (s *testDBSuite3) TestGeneratedColumnDDL(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") From a8b1f687e0474a3487327c50d085ac3ee08d241b Mon Sep 17 00:00:00 2001 From: rebelice Date: Fri, 4 Jun 2021 22:40:28 +0800 Subject: [PATCH 301/343] session: fix a bug when updating duplicate bindings (#25168) --- session/bootstrap.go | 8 +++++++- session/bootstrap_test.go | 22 ++++++++++++++++++++-- 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/session/bootstrap.go b/session/bootstrap.go index 6c03e5ae73e34..2a310e162ab7e 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -1407,6 +1407,12 @@ func updateBindInfo(iter *chunk.Iterator4Chunk, p *parser.Parser, bindMap map[st for row := iter.Begin(); row != iter.End(); row = iter.Next() { bind := row.GetString(0) db := row.GetString(1) + status := row.GetString(2) + + if status != "using" && status != "builtin" { + continue + } + charset := row.GetString(4) collation := row.GetString(5) stmt, err := p.ParseOneStmt(bind, charset, collation) @@ -1425,7 +1431,7 @@ func updateBindInfo(iter *chunk.Iterator4Chunk, p *parser.Parser, bindMap map[st } bindMap[originWithDB] = bindInfo{ bindSQL: utilparser.RestoreWithDefaultDB(stmt, db, bind), - status: row.GetString(2), + status: status, createTime: row.GetTime(3), charset: charset, collation: collation, diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index a27dd7a3ea7ba..35c4495f3b74e 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -592,18 +592,36 @@ func (s *testBootstrapSuite) TestUpdateDuplicateBindInfo(c *C) { // The latest one. mustExecSQL(c, se, `insert into mysql.bind_info values('select * from test . t', 'select /*+ use_index(t, idx_b)*/ * from test.t', 'test', 'using', '2021-01-04 14:50:58.257', '2021-01-09 14:50:58.257', 'utf8', 'utf8_general_ci', 'manual')`) + mustExecSQL(c, se, `insert into mysql.bind_info values('select * from t where a < ?', 'select * from t use index(idx) where a < 1', 'test', 'deleted', '2021-06-04 17:04:43.333', '2021-06-04 17:04:43.335', 'utf8', 'utf8_general_ci', 'manual')`) + mustExecSQL(c, se, `insert into mysql.bind_info values('select * from t where a < ?', 'select * from t ignore index(idx) where a < 1', 'test', 'using', '2021-06-04 17:04:43.335', '2021-06-04 17:04:43.335', 'utf8', 'utf8_general_ci', 'manual')`) + mustExecSQL(c, se, `insert into mysql.bind_info values('select * from test . t where a <= ?', 'select * from test.t use index(idx) where a <= 1', '', 'deleted', '2021-06-04 17:04:43.345', '2021-06-04 17:04:45.334', 'utf8', 'utf8_general_ci', 'manual')`) + mustExecSQL(c, se, `insert into mysql.bind_info values('select * from test . t where a <= ?', 'select * from test.t ignore index(idx) where a <= 1', '', 'using', '2021-06-04 17:04:45.334', '2021-06-04 17:04:45.334', 'utf8', 'utf8_general_ci', 'manual')`) + upgradeToVer67(se, version66) - r := mustExecSQL(c, se, `select original_sql, bind_sql, default_db, status, create_time from mysql.bind_info where source != 'builtin'`) + r := mustExecSQL(c, se, `select original_sql, bind_sql, default_db, status, create_time from mysql.bind_info where source != 'builtin' order by create_time`) req := r.NewChunk() c.Assert(r.Next(ctx, req), IsNil) - c.Assert(req.NumRows(), Equals, 1) + c.Assert(req.NumRows(), Equals, 3) row := req.GetRow(0) c.Assert(row.GetString(0), Equals, "select * from `test` . `t`") c.Assert(row.GetString(1), Equals, "SELECT /*+ use_index(`t` `idx_b`)*/ * FROM `test`.`t`") c.Assert(row.GetString(2), Equals, "") c.Assert(row.GetString(3), Equals, "using") c.Assert(row.GetTime(4).String(), Equals, "2021-01-04 14:50:58.257") + row = req.GetRow(1) + c.Assert(row.GetString(0), Equals, "select * from `test` . `t` where `a` < ?") + c.Assert(row.GetString(1), Equals, "SELECT * FROM `test`.`t` IGNORE INDEX (`idx`) WHERE `a` < 1") + c.Assert(row.GetString(2), Equals, "") + c.Assert(row.GetString(3), Equals, "using") + c.Assert(row.GetTime(4).String(), Equals, "2021-06-04 17:04:43.335") + row = req.GetRow(2) + c.Assert(row.GetString(0), Equals, "select * from `test` . `t` where `a` <= ?") + c.Assert(row.GetString(1), Equals, "SELECT * FROM `test`.`t` IGNORE INDEX (`idx`) WHERE `a` <= 1") + c.Assert(row.GetString(2), Equals, "") + c.Assert(row.GetString(3), Equals, "using") + c.Assert(row.GetTime(4).String(), Equals, "2021-06-04 17:04:45.334") + c.Assert(r.Close(), IsNil) mustExecSQL(c, se, "delete from mysql.bind_info where original_sql = 'select * from test . t'") } From acdacf186a31d1e8b82ef550cfab67f2c31d2405 Mon Sep 17 00:00:00 2001 From: Yilong Li Date: Fri, 4 Jun 2021 22:56:28 +0800 Subject: [PATCH 302/343] util: add top sql collector (#24988) --- executor/set_test.go | 19 + go.mod | 7 +- go.sum | 14 +- sessionctx/variable/sysvar.go | 18 + sessionctx/variable/tidb_vars.go | 18 +- store/mockstore/unistore/tikv/server.go | 6 - store/tikv/region_request_test.go | 5 - tidb-server/main.go | 1 + util/topsql/collector/collector.go | 24 -- util/topsql/reporter/client.go | 190 +++++++++ util/topsql/reporter/mock/server.go | 144 +++++++ util/topsql/reporter/reporter.go | 368 ++++++++++++++++++ util/topsql/reporter/reporter_test.go | 192 +++++++++ util/topsql/topsql.go | 19 +- .../{tracecpu_test.go => topsql_test.go} | 83 +++- util/topsql/tracecpu/mock/mock.go | 21 +- util/topsql/tracecpu/profile.go | 25 +- 17 files changed, 1082 insertions(+), 72 deletions(-) delete mode 100644 util/topsql/collector/collector.go create mode 100644 util/topsql/reporter/client.go create mode 100644 util/topsql/reporter/mock/server.go create mode 100644 util/topsql/reporter/reporter.go create mode 100644 util/topsql/reporter/reporter_test.go rename util/topsql/{tracecpu_test.go => topsql_test.go} (60%) diff --git a/executor/set_test.go b/executor/set_test.go index 4134cb128d990..9684c51012bd8 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -1444,4 +1444,23 @@ func (s *testSerialSuite) TestSetTopSQLVariables(c *C) { c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_max_statement_count' can't be set to the value of '5001'") tk.MustQuery("select @@global.tidb_top_sql_precision_seconds;").Check(testkit.Rows("2")) c.Assert(variable.TopSQLVariable.MaxStatementCount.Load(), Equals, int64(2)) + + tk.MustExec("set @@tidb_top_sql_report_interval_seconds=10;") + tk.MustQuery("select @@tidb_top_sql_report_interval_seconds;").Check(testkit.Rows("10")) + c.Assert(variable.TopSQLVariable.ReportIntervalSeconds.Load(), Equals, int64(10)) + _, err = tk.Exec("set @@tidb_top_sql_report_interval_seconds='abc';") + c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_report_interval_seconds'") + _, err = tk.Exec("set @@tidb_top_sql_report_interval_seconds='5000';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_report_interval_seconds' can't be set to the value of '5000'") + tk.MustQuery("select @@tidb_top_sql_report_interval_seconds;").Check(testkit.Rows("10")) + c.Assert(variable.TopSQLVariable.ReportIntervalSeconds.Load(), Equals, int64(10)) + tk.MustExec("set @@global.tidb_top_sql_report_interval_seconds=120;") + tk.MustQuery("select @@global.tidb_top_sql_report_interval_seconds;").Check(testkit.Rows("120")) + c.Assert(variable.TopSQLVariable.ReportIntervalSeconds.Load(), Equals, int64(120)) + _, err = tk.Exec("set @@global.tidb_top_sql_report_interval_seconds='abc';") + c.Assert(err.Error(), Equals, "[variable:1232]Incorrect argument type to variable 'tidb_top_sql_report_interval_seconds'") + _, err = tk.Exec("set @@global.tidb_top_sql_report_interval_seconds='5000';") + c.Assert(err.Error(), Equals, "[variable:1231]Variable 'tidb_top_sql_report_interval_seconds' can't be set to the value of '5000'") + tk.MustQuery("select @@global.tidb_top_sql_report_interval_seconds;").Check(testkit.Rows("120")) + c.Assert(variable.TopSQLVariable.ReportIntervalSeconds.Load(), Equals, int64(120)) } diff --git a/go.mod b/go.mod index b125020a771b1..7004ac7b657a2 100644 --- a/go.mod +++ b/go.mod @@ -22,7 +22,7 @@ require ( github.com/golang/protobuf v1.3.4 github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf github.com/google/btree v1.0.0 - github.com/google/go-cmp v0.5.2 // indirect + github.com/google/go-cmp v0.5.5 // indirect github.com/google/pprof v0.0.0-20200407044318-7d83b28da2e9 github.com/google/uuid v1.1.1 github.com/gorilla/mux v1.7.4 @@ -43,12 +43,12 @@ require ( github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20210531063847-f42e582bf0bb + github.com/pingcap/kvproto v0.0.0-20210602120243-804ac0a6ce21 github.com/pingcap/log v0.0.0-20210317133921-96f4fcab92a4 github.com/pingcap/parser v0.0.0-20210525032559-c37778aff307 github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3 github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible - github.com/pingcap/tipb v0.0.0-20210601083426-79a378b6d1c4 + github.com/pingcap/tipb v0.0.0-20210603161937-cfb5a9225f95 github.com/prometheus/client_golang v1.5.1 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.9.1 @@ -62,6 +62,7 @@ require ( github.com/uber-go/atomic v1.4.0 github.com/uber/jaeger-client-go v2.22.1+incompatible github.com/uber/jaeger-lib v2.4.0+incompatible // indirect + github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457 // indirect go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b go.uber.org/atomic v1.7.0 diff --git a/go.sum b/go.sum index 92e5a0aae6ee3..be3ebf435d139 100644 --- a/go.sum +++ b/go.sum @@ -237,8 +237,8 @@ github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5a 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= github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.2 h1:X2ev0eStA3AbceY54o37/0PQ/UWqKEiiO2dKL5OPaFM= -github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= @@ -433,8 +433,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17Xtb github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210219064844-c1844a4775d6/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20210531063847-f42e582bf0bb h1:6isHwZRl1fc9i1Mggiza2iQcfvVvYAAerFIs5t9msXg= -github.com/pingcap/kvproto v0.0.0-20210531063847-f42e582bf0bb/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20210602120243-804ac0a6ce21 h1:WLViPiCazVwK0wULKpmwLVP/aA8NvyyfOUqQy0bPEPk= +github.com/pingcap/kvproto v0.0.0-20210602120243-804ac0a6ce21/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20201112100606-8f1e84a3abc8/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= @@ -448,8 +448,8 @@ github.com/pingcap/sysutil v0.0.0-20210315073920-cc0985d983a3/go.mod h1:tckvA041 github.com/pingcap/tidb-dashboard v0.0.0-20210312062513-eef5d6404638/go.mod h1:OzFN8H0EDMMqeulPhPMw2i2JaiZWOKFQ7zdRPhENNgo= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible h1:ceznmu/lLseGHP/jKyOa/3u/5H3wtLLLqkH2V3ssSjg= github.com/pingcap/tidb-tools v4.0.9-0.20201127090955-2707c97b3853+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20210601083426-79a378b6d1c4 h1:n47+OwdI/uxKenfBT8Y2/be11MwbeLKNLdzOWnxNQKg= -github.com/pingcap/tipb v0.0.0-20210601083426-79a378b6d1c4/go.mod h1:nsEhnMokcn7MRqd2J60yxpn/ac3ZH8A6GOJ9NslabUo= +github.com/pingcap/tipb v0.0.0-20210603161937-cfb5a9225f95 h1:Cj7FhGvYn8hrXDNcaHi0aTl0KdV67KTL+P5gBp3vqT4= +github.com/pingcap/tipb v0.0.0-20210603161937-cfb5a9225f95/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= @@ -586,6 +586,8 @@ github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKn github.com/vmihailenco/msgpack/v4 v4.3.11/go.mod h1:gborTTJjAo/GWTqqRjrLCn9pgNN+NXzzngzBKDPIqw4= github.com/vmihailenco/msgpack/v5 v5.0.0-beta.1/go.mod h1:xlngVLeyQ/Qi05oQxhQ+oTuqa03RjMwMfk/7/TCs+QI= github.com/vmihailenco/tagparser v0.1.1/go.mod h1:OeAg3pn3UbLjkWt+rN9oFYB6u/cQgqMEUPoW2WPyhdI= +github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f h1:9DDCDwOyEy/gId+IEMrFHLuQ5R/WV0KNxWLler8X2OY= +github.com/wangjohn/quickselect v0.0.0-20161129230411-ed8402a42d5f/go.mod h1:8sdOQnirw1PrcnTJYkmW1iOHtUmblMmGdUOHyWYycLI= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 h1:eY9dn8+vbi4tKz5Qo6v2eYzo7kUS51QINcR5jNpbZS8= github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/xitongsys/parquet-go v1.5.1/go.mod h1:xUxwM8ELydxh4edHGegYq1pA8NnMKDx0K/GyB0o2bww= diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index e5e63ab86952f..4b4688727a1db 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1730,6 +1730,24 @@ var defaultSysVars = []*SysVar{ TopSQLVariable.MaxStatementCount.Store(val) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBTopSQLReportIntervalSeconds, Value: strconv.Itoa(DefTiDBTopSQLReportIntervalSeconds), Type: TypeInt, MinValue: 1, MaxValue: 1 * 60 * 60, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { + return strconv.FormatInt(TopSQLVariable.ReportIntervalSeconds.Load(), 10), nil + }, SetSession: func(vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TopSQLVariable.ReportIntervalSeconds.Store(val) + return nil + }, SetGlobal: func(vars *SessionVars, s string) error { + val, err := strconv.ParseInt(s, 10, 64) + if err != nil { + return err + } + TopSQLVariable.ReportIntervalSeconds.Store(val) + return nil + }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableGlobalTemporaryTable, Value: BoolToOnOff(DefTiDBEnableGlobalTemporaryTable), Hidden: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableGlobalTemporaryTable = TiDBOptOn(val) return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 8e60d2a9e3492..d62e2a0f76809 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -554,6 +554,8 @@ const ( // TiDBTopSQLMaxStatementCount indicates the max number of statements been collected. TiDBTopSQLMaxStatementCount = "tidb_top_sql_max_statement_count" + // TiDBTopSQLReportIntervalSeconds indicates the top SQL report interval seconds. + TiDBTopSQLReportIntervalSeconds = "tidb_top_sql_report_interval_seconds" // TiDBEnableGlobalTemporaryTable indicates whether to enable global temporary table TiDBEnableGlobalTemporaryTable = "tidb_enable_global_temporary_table" ) @@ -705,6 +707,7 @@ const ( DefTiDBTopSQLAgentAddress = "" DefTiDBTopSQLPrecisionSeconds = 1 DefTiDBTopSQLMaxStatementCount = 200 + DefTiDBTopSQLReportIntervalSeconds = 60 DefTiDBEnableGlobalTemporaryTable = false ) @@ -731,23 +734,26 @@ var ( DefExecutorConcurrency = 5 MemoryUsageAlarmRatio = atomic.NewFloat64(config.GetGlobalConfig().Performance.MemoryUsageAlarmRatio) TopSQLVariable = TopSQL{ - Enable: atomic.NewBool(DefTiDBTopSQLEnable), - AgentAddress: atomic.NewString(DefTiDBTopSQLAgentAddress), - PrecisionSeconds: atomic.NewInt64(DefTiDBTopSQLPrecisionSeconds), - MaxStatementCount: atomic.NewInt64(DefTiDBTopSQLMaxStatementCount), + Enable: atomic.NewBool(DefTiDBTopSQLEnable), + AgentAddress: atomic.NewString(DefTiDBTopSQLAgentAddress), + PrecisionSeconds: atomic.NewInt64(DefTiDBTopSQLPrecisionSeconds), + MaxStatementCount: atomic.NewInt64(DefTiDBTopSQLMaxStatementCount), + ReportIntervalSeconds: atomic.NewInt64(DefTiDBTopSQLReportIntervalSeconds), } ) // TopSQL is the variable for control top sql feature. type TopSQL struct { - // Enable statement summary or not. + // Enable top-sql or not. Enable *atomic.Bool // AgentAddress indicate the collect agent address. AgentAddress *atomic.String - // The refresh interval of statement summary. + // The refresh interval of top-sql. PrecisionSeconds *atomic.Int64 // The maximum number of statements kept in memory. MaxStatementCount *atomic.Int64 + // The report data interval of top-sql. + ReportIntervalSeconds *atomic.Int64 } // TopSQLEnabled uses to check whether enabled the top SQL feature. diff --git a/store/mockstore/unistore/tikv/server.go b/store/mockstore/unistore/tikv/server.go index 1b8bfa163ec85..8d8d2553fc823 100644 --- a/store/mockstore/unistore/tikv/server.go +++ b/store/mockstore/unistore/tikv/server.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/kvproto/pkg/coprocessor_v2" deadlockPb "github.com/pingcap/kvproto/pkg/deadlock" "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" @@ -967,11 +966,6 @@ func (svr *Server) RawCompareAndSwap(context.Context, *kvrpcpb.RawCASRequest) (* panic("implement me") } -// CoprocessorV2 implements the tikvpb.TikvServer interface. -func (svr *Server) CoprocessorV2(context.Context, *coprocessor_v2.RawCoprocessorRequest) (*coprocessor_v2.RawCoprocessorResponse, error) { - panic("implement me") -} - // GetStoreSafeTS implements the tikvpb.TikvServer interface. func (svr *Server) GetStoreSafeTS(context.Context, *kvrpcpb.StoreSafeTSRequest) (*kvrpcpb.StoreSafeTSResponse, error) { return &kvrpcpb.StoreSafeTSResponse{}, nil diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 7b8016e50abc9..5f880d26c7f67 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -26,7 +26,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/coprocessor" - "github.com/pingcap/kvproto/pkg/coprocessor_v2" "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" @@ -586,10 +585,6 @@ func (s *mockTikvGrpcServer) RawCompareAndSwap(context.Context, *kvrpcpb.RawCASR return nil, errors.New("unreachable") } -func (s *mockTikvGrpcServer) CoprocessorV2(context.Context, *coprocessor_v2.RawCoprocessorRequest) (*coprocessor_v2.RawCoprocessorResponse, error) { - return nil, errors.New("unreachable") -} - func (s *mockTikvGrpcServer) GetLockWaitInfo(context.Context, *kvrpcpb.GetLockWaitInfoRequest) (*kvrpcpb.GetLockWaitInfoResponse, error) { return nil, errors.New("unreachable") } diff --git a/tidb-server/main.go b/tidb-server/main.go index 4dcf7296e7dcd..aecae4e924192 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -666,6 +666,7 @@ func cleanup(svr *server.Server, storage kv.Storage, dom *domain.Domain, gracefu plugin.Shutdown(context.Background()) closeDomainAndStorage(storage, dom) disk.CleanUp() + topsql.Close() } func stringToList(repairString string) []string { diff --git a/util/topsql/collector/collector.go b/util/topsql/collector/collector.go deleted file mode 100644 index 9cc2a429ea990..0000000000000 --- a/util/topsql/collector/collector.go +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package collector - -import "github.com/pingcap/tidb/util/topsql/tracecpu" - -// TopSQLCollector uses to collect SQL stats. -// TODO: add a collector to collect and store the SQL stats. -type TopSQLCollector interface { - tracecpu.Collector - RegisterSQL(sqlDigest []byte, normalizedSQL string) - RegisterPlan(planDigest []byte, normalizedPlan string) -} diff --git a/util/topsql/reporter/client.go b/util/topsql/reporter/client.go new file mode 100644 index 0000000000000..dc5854611bb51 --- /dev/null +++ b/util/topsql/reporter/client.go @@ -0,0 +1,190 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package reporter + +import ( + "context" + "math" + "sync" + "time" + + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tipb/go-tipb" + "go.uber.org/zap" + "google.golang.org/grpc" + "google.golang.org/grpc/backoff" +) + +// ReportClient send data to the target server. +type ReportClient interface { + Send(ctx context.Context, addr string, sqlMetas []*tipb.SQLMeta, planMetas []*tipb.PlanMeta, records []*tipb.CPUTimeRecord) error + Close() +} + +// GRPCReportClient reports data to grpc servers. +type GRPCReportClient struct { + curRPCAddr string + conn *grpc.ClientConn +} + +// NewGRPCReportClient returns a new GRPCReportClient +func NewGRPCReportClient() *GRPCReportClient { + return &GRPCReportClient{} +} + +// Send implements the ReportClient interface. +func (r *GRPCReportClient) Send( + ctx context.Context, targetRPCAddr string, + sqlMetas []*tipb.SQLMeta, planMetas []*tipb.PlanMeta, records []*tipb.CPUTimeRecord) error { + if targetRPCAddr == "" { + return nil + } + err := r.tryEstablishConnection(ctx, targetRPCAddr) + if err != nil { + return err + } + + var wg sync.WaitGroup + errCh := make(chan error, 3) + wg.Add(3) + + go func() { + defer wg.Done() + errCh <- r.sendBatchSQLMeta(ctx, sqlMetas) + }() + go func() { + defer wg.Done() + errCh <- r.sendBatchPlanMeta(ctx, planMetas) + }() + go func() { + defer wg.Done() + errCh <- r.sendBatchCPUTimeRecord(ctx, records) + }() + wg.Wait() + close(errCh) + for err := range errCh { + if err != nil { + return err + } + } + return nil +} + +// Close uses to close grpc connection. +func (r *GRPCReportClient) Close() { + if r.conn == nil { + return + } + err := r.conn.Close() + if err != nil { + logutil.BgLogger().Warn("[top-sql] grpc client close connection failed", zap.Error(err)) + } + r.conn = nil +} + +// sendBatchCPUTimeRecord sends a batch of TopSQL records by stream. +func (r *GRPCReportClient) sendBatchCPUTimeRecord(ctx context.Context, records []*tipb.CPUTimeRecord) error { + if len(records) == 0 { + return nil + } + client := tipb.NewTopSQLAgentClient(r.conn) + stream, err := client.ReportCPUTimeRecords(ctx) + if err != nil { + return err + } + for _, record := range records { + if err := stream.Send(record); err != nil { + break + } + } + // See https://pkg.go.dev/google.golang.org/grpc#ClientConn.NewStream for how to avoid leaking the stream + _, err = stream.CloseAndRecv() + return err +} + +// sendBatchSQLMeta sends a batch of SQL metas by stream. +func (r *GRPCReportClient) sendBatchSQLMeta(ctx context.Context, metas []*tipb.SQLMeta) error { + if len(metas) == 0 { + return nil + } + client := tipb.NewTopSQLAgentClient(r.conn) + stream, err := client.ReportSQLMeta(ctx) + if err != nil { + return err + } + for _, meta := range metas { + if err := stream.Send(meta); err != nil { + break + } + } + _, err = stream.CloseAndRecv() + return err +} + +// sendBatchSQLMeta sends a batch of SQL metas by stream. +func (r *GRPCReportClient) sendBatchPlanMeta(ctx context.Context, metas []*tipb.PlanMeta) error { + if len(metas) == 0 { + return nil + } + client := tipb.NewTopSQLAgentClient(r.conn) + stream, err := client.ReportPlanMeta(ctx) + if err != nil { + return err + } + for _, meta := range metas { + if err := stream.Send(meta); err != nil { + break + } + } + _, err = stream.CloseAndRecv() + return err +} + +// tryEstablishConnection establishes the gRPC connection if connection is not established. +func (r *GRPCReportClient) tryEstablishConnection(ctx context.Context, targetRPCAddr string) (err error) { + if r.curRPCAddr == targetRPCAddr && r.conn != nil { + // Address is not changed, skip. + return nil + } + r.conn, err = r.dial(ctx, targetRPCAddr) + if err != nil { + return err + } + r.curRPCAddr = targetRPCAddr + return nil +} + +func (r *GRPCReportClient) dial(ctx context.Context, targetRPCAddr string) (*grpc.ClientConn, error) { + dialCtx, cancel := context.WithTimeout(ctx, dialTimeout) + defer cancel() + return grpc.DialContext( + dialCtx, + targetRPCAddr, + grpc.WithBlock(), + grpc.WithInsecure(), + grpc.WithInitialWindowSize(grpcInitialWindowSize), + grpc.WithInitialConnWindowSize(grpcInitialConnWindowSize), + grpc.WithDefaultCallOptions( + grpc.MaxCallRecvMsgSize(math.MaxInt64), + ), + grpc.WithConnectParams(grpc.ConnectParams{ + Backoff: backoff.Config{ + BaseDelay: 100 * time.Millisecond, // Default was 1s. + Multiplier: 1.6, // Default + Jitter: 0.2, // Default + MaxDelay: 3 * time.Second, // Default was 120s. + }, + }), + ) +} diff --git a/util/topsql/reporter/mock/server.go b/util/topsql/reporter/mock/server.go new file mode 100644 index 0000000000000..42479bc7a6201 --- /dev/null +++ b/util/topsql/reporter/mock/server.go @@ -0,0 +1,144 @@ +package mock + +import ( + "fmt" + "io" + "net" + "sync" + "time" + + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tipb/go-tipb" + "go.uber.org/zap" + "google.golang.org/grpc" +) + +type mockAgentServer struct { + sync.Mutex + addr string + grpcServer *grpc.Server + sqlMetas map[string]string + planMetas map[string]string + records []*tipb.CPUTimeRecord +} + +// StartMockAgentServer starts the mock agent server. +func StartMockAgentServer() (*mockAgentServer, error) { + addr := "127.0.0.1:0" + lis, err := net.Listen("tcp", addr) + if err != nil { + return nil, err + } + server := grpc.NewServer() + agentServer := &mockAgentServer{ + addr: fmt.Sprintf("127.0.0.1:%d", lis.Addr().(*net.TCPAddr).Port), + grpcServer: server, + sqlMetas: make(map[string]string, 5000), + planMetas: make(map[string]string, 5000), + } + tipb.RegisterTopSQLAgentServer(server, agentServer) + + go func() { + err := server.Serve(lis) + if err != nil { + logutil.BgLogger().Warn("[top-sql] mock agent server serve failed", zap.Error(err)) + } + }() + + return agentServer, nil +} + +func (svr *mockAgentServer) ReportCPUTimeRecords(stream tipb.TopSQLAgent_ReportCPUTimeRecordsServer) error { + for { + req, err := stream.Recv() + if err == io.EOF { + break + } else if err != nil { + return err + } + svr.Lock() + svr.records = append(svr.records, req) + svr.Unlock() + } + return stream.SendAndClose(&tipb.EmptyResponse{}) +} + +func (svr *mockAgentServer) ReportSQLMeta(stream tipb.TopSQLAgent_ReportSQLMetaServer) error { + for { + req, err := stream.Recv() + if err == io.EOF { + break + } else if err != nil { + return err + } + svr.Lock() + svr.sqlMetas[string(req.SqlDigest)] = req.NormalizedSql + svr.Unlock() + } + return stream.SendAndClose(&tipb.EmptyResponse{}) +} + +func (svr *mockAgentServer) ReportPlanMeta(stream tipb.TopSQLAgent_ReportPlanMetaServer) error { + for { + req, err := stream.Recv() + if err == io.EOF { + break + } else if err != nil { + return err + } + svr.Lock() + svr.planMetas[string(req.PlanDigest)] = req.NormalizedPlan + svr.Unlock() + } + return stream.SendAndClose(&tipb.EmptyResponse{}) +} + +func (svr *mockAgentServer) WaitServerCollect(recordCount int, timeout time.Duration) { + start := time.Now() + for { + svr.Lock() + if len(svr.records) >= recordCount { + svr.Unlock() + return + } + svr.Unlock() + if time.Since(start) > timeout { + return + } + time.Sleep(time.Millisecond) + } +} + +func (svr *mockAgentServer) GetSQLMetas() map[string]string { + svr.Lock() + m := svr.sqlMetas + svr.sqlMetas = make(map[string]string) + svr.Unlock() + return m +} + +func (svr *mockAgentServer) GetPlanMetas() map[string]string { + svr.Lock() + m := svr.planMetas + svr.planMetas = make(map[string]string) + svr.Unlock() + return m +} + +func (svr *mockAgentServer) GetRecords() []*tipb.CPUTimeRecord { + svr.Lock() + records := svr.records + svr.records = []*tipb.CPUTimeRecord{} + svr.Unlock() + return records +} + +func (svr *mockAgentServer) Address() string { + return svr.addr +} + +func (svr *mockAgentServer) Stop() { + if svr.grpcServer != nil { + svr.grpcServer.Stop() + } +} diff --git a/util/topsql/reporter/reporter.go b/util/topsql/reporter/reporter.go new file mode 100644 index 0000000000000..7723a62d6aa2a --- /dev/null +++ b/util/topsql/reporter/reporter.go @@ -0,0 +1,368 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package reporter + +import ( + "bytes" + "context" + "sync" + "sync/atomic" + "time" + + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/topsql/tracecpu" + "github.com/pingcap/tipb/go-tipb" + "github.com/wangjohn/quickselect" + "go.uber.org/zap" +) + +const ( + dialTimeout = 5 * time.Second + reportTimeout = 40 * time.Second + grpcInitialWindowSize = 1 << 30 + grpcInitialConnWindowSize = 1 << 30 +) + +var _ TopSQLReporter = &RemoteTopSQLReporter{} + +// TopSQLReporter collects Top SQL metrics. +type TopSQLReporter interface { + tracecpu.Collector + RegisterSQL(sqlDigest []byte, normalizedSQL string) + RegisterPlan(planDigest []byte, normalizedPlan string) + Close() +} + +type cpuData struct { + timestamp uint64 + records []tracecpu.SQLCPUTimeRecord +} + +// dataPoints represents the cumulative SQL plan CPU time in current minute window +type dataPoints struct { + SQLDigest []byte + PlanDigest []byte + TimestampList []uint64 + CPUTimeMsList []uint32 + CPUTimeMsTotal uint64 +} + +// cpuTimeSort is used to sort TopSQL records by total CPU time +type cpuTimeSort struct { + Key string + SQLDigest []byte + PlanDigest []byte + CPUTimeMsTotal uint64 // The sorting field +} + +type cpuTimeSortSlice []cpuTimeSort + +func (t cpuTimeSortSlice) Len() int { + return len(t) +} + +func (t cpuTimeSortSlice) Less(i, j int) bool { + // We need find the kth largest value, so here should use > + return t[i].CPUTimeMsTotal > t[j].CPUTimeMsTotal +} +func (t cpuTimeSortSlice) Swap(i, j int) { + t[i], t[j] = t[j], t[i] +} + +type planBinaryDecodeFunc func(string) (string, error) + +// RemoteTopSQLReporter implements a TopSQL reporter that sends data to a remote agent +// This should be called periodically to collect TopSQL resource usage metrics +type RemoteTopSQLReporter struct { + ctx context.Context + cancel context.CancelFunc + client ReportClient + + // normalizedSQLMap is an map, whose keys are SQL digest strings and values are normalized SQL strings + normalizedSQLMap atomic.Value // sync.Map + + // normalizedPlanMap is an map, whose keys are plan digest strings and values are normalized plans **in binary**. + // The normalized plans in binary can be decoded to string using the `planBinaryDecoder`. + normalizedPlanMap atomic.Value // sync.Map + + // calling this can take a while, so should not block critical paths + planBinaryDecoder planBinaryDecodeFunc + + collectCPUDataChan chan cpuData + reportDataChan chan reportData +} + +// NewRemoteTopSQLReporter creates a new TopSQL reporter +// +// planBinaryDecoder is a decoding function which will be called asynchronously to decode the plan binary to string +// MaxStatementsNum is the maximum SQL and plan number, which will restrict the memory usage of the internal LFU cache +func NewRemoteTopSQLReporter(client ReportClient, planDecodeFn planBinaryDecodeFunc) *RemoteTopSQLReporter { + + ctx, cancel := context.WithCancel(context.Background()) + tsr := &RemoteTopSQLReporter{ + ctx: ctx, + cancel: cancel, + client: client, + planBinaryDecoder: planDecodeFn, + collectCPUDataChan: make(chan cpuData, 1), + reportDataChan: make(chan reportData, 1), + } + tsr.normalizedSQLMap.Store(&sync.Map{}) + tsr.normalizedPlanMap.Store(&sync.Map{}) + + go tsr.collectWorker() + go tsr.reportWorker() + + return tsr +} + +// RegisterSQL registers a normalized SQL string to a SQL digest. +// This function is thread-safe and efficient. +// +// Note that the normalized SQL string can be of >1M long. +// This function should be thread-safe, which means parallelly calling it in several goroutines should be fine. +// It should also return immediately, and do any CPU-intensive job asynchronously. +func (tsr *RemoteTopSQLReporter) RegisterSQL(sqlDigest []byte, normalizedSQL string) { + m := tsr.normalizedSQLMap.Load().(*sync.Map) + key := string(sqlDigest) + m.LoadOrStore(key, normalizedSQL) +} + +// RegisterPlan is like RegisterSQL, but for normalized plan strings. +// This function is thread-safe and efficient. +func (tsr *RemoteTopSQLReporter) RegisterPlan(planDigest []byte, normalizedBinaryPlan string) { + m := tsr.normalizedPlanMap.Load().(*sync.Map) + key := string(planDigest) + m.LoadOrStore(key, normalizedBinaryPlan) +} + +// Collect receives CPU time records for processing. WARN: It will drop the records if the processing is not in time. +// This function is thread-safe and efficient. +func (tsr *RemoteTopSQLReporter) Collect(timestamp uint64, records []tracecpu.SQLCPUTimeRecord) { + if len(records) == 0 { + return + } + select { + case tsr.collectCPUDataChan <- cpuData{ + timestamp: timestamp, + records: records, + }: + default: + // ignore if chan blocked + } +} + +// Close uses to close and release the reporter resource. +func (tsr *RemoteTopSQLReporter) Close() { + tsr.cancel() + tsr.client.Close() +} + +func (tsr *RemoteTopSQLReporter) collectWorker() { + defer util.Recover("top-sql", "collectWorker", nil, false) + + collectedData := make(map[string]*dataPoints) + + currentReportInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load() + reportTicker := time.NewTicker(time.Second * time.Duration(currentReportInterval)) + + for { + select { + case data := <-tsr.collectCPUDataChan: + // On receiving data to collect: Write to local data array, and retain records with most CPU time. + tsr.doCollect(collectedData, data.timestamp, data.records) + case <-reportTicker.C: + tsr.takeDataAndSendToReportChan(&collectedData) + + // Update `reportTicker` if report interval changed. + if newInterval := variable.TopSQLVariable.ReportIntervalSeconds.Load(); newInterval != currentReportInterval { + currentReportInterval = newInterval + reportTicker.Reset(time.Second * time.Duration(currentReportInterval)) + } + case <-tsr.ctx.Done(): + return + } + } +} + +func encodeKey(buf *bytes.Buffer, sqlDigest, planDigest []byte) string { + buf.Reset() + buf.Write(sqlDigest) + buf.Write(planDigest) + return buf.String() +} + +// doCollect uses a hashmap to store records in every second, and evict when necessary. +func (tsr *RemoteTopSQLReporter) doCollect( + collectTarget map[string]*dataPoints, timestamp uint64, records []tracecpu.SQLCPUTimeRecord) { + defer util.Recover("top-sql", "doCollect", nil, false) + + keyBuf := bytes.NewBuffer(make([]byte, 0, 64)) + listCapacity := int(variable.TopSQLVariable.ReportIntervalSeconds.Load()/variable.TopSQLVariable.PrecisionSeconds.Load() + 1) + if listCapacity < 1 { + listCapacity = 1 + } + for _, record := range records { + key := encodeKey(keyBuf, record.SQLDigest, record.PlanDigest) + entry, exist := collectTarget[key] + if !exist { + entry = &dataPoints{ + SQLDigest: record.SQLDigest, + PlanDigest: record.PlanDigest, + CPUTimeMsList: make([]uint32, 1, listCapacity), + TimestampList: make([]uint64, 1, listCapacity), + } + entry.CPUTimeMsList[0] = record.CPUTimeMs + entry.TimestampList[0] = timestamp + collectTarget[key] = entry + } else { + entry.CPUTimeMsList = append(entry.CPUTimeMsList, record.CPUTimeMs) + entry.TimestampList = append(entry.TimestampList, timestamp) + } + entry.CPUTimeMsTotal += uint64(record.CPUTimeMs) + } + + // evict records according to `MaxStatementCount` variable. + // TODO: Better to pass in the variable in the constructor, instead of referencing directly. + maxStmt := int(variable.TopSQLVariable.MaxStatementCount.Load()) + if len(collectTarget) <= maxStmt { + return + } + + // find the max CPUTimeMsTotal that should be evicted + digestCPUTimeList := make([]cpuTimeSort, len(collectTarget)) + idx := 0 + for key, value := range collectTarget { + digestCPUTimeList[idx] = cpuTimeSort{ + Key: key, + SQLDigest: value.SQLDigest, + PlanDigest: value.PlanDigest, + CPUTimeMsTotal: value.CPUTimeMsTotal, + } + idx++ + } + + // QuickSelect will only return error when the second parameter is out of range + if err := quickselect.QuickSelect(cpuTimeSortSlice(digestCPUTimeList), maxStmt); err != nil { + // skip eviction + return + } + + itemsToEvict := digestCPUTimeList[maxStmt:] + normalizedSQLMap := tsr.normalizedSQLMap.Load().(*sync.Map) + normalizedPlanMap := tsr.normalizedPlanMap.Load().(*sync.Map) + for _, evict := range itemsToEvict { + delete(collectTarget, evict.Key) + normalizedSQLMap.Delete(string(evict.SQLDigest)) + normalizedPlanMap.Delete(string(evict.PlanDigest)) + } +} + +// takeDataAndSendToReportChan takes out (resets) collected data. These data will be send to a report channel +// for reporting later. +func (tsr *RemoteTopSQLReporter) takeDataAndSendToReportChan(collectedDataPtr *map[string]*dataPoints) { + data := reportData{ + collectedData: *collectedDataPtr, + normalizedSQLMap: tsr.normalizedSQLMap.Load().(*sync.Map), + normalizedPlanMap: tsr.normalizedPlanMap.Load().(*sync.Map), + } + + // Reset data for next report. + *collectedDataPtr = make(map[string]*dataPoints) + tsr.normalizedSQLMap.Store(&sync.Map{}) + tsr.normalizedPlanMap.Store(&sync.Map{}) + + // Send to report channel. When channel is full, data will be dropped. + select { + case tsr.reportDataChan <- data: + default: + } +} + +type reportData struct { + collectedData map[string]*dataPoints + normalizedSQLMap *sync.Map + normalizedPlanMap *sync.Map +} + +// prepareReportDataForSending prepares the data that need to reported. +func (tsr *RemoteTopSQLReporter) prepareReportDataForSending(data reportData) (sqlMetas []*tipb.SQLMeta, planMetas []*tipb.PlanMeta, records []*tipb.CPUTimeRecord) { + sqlMetas = make([]*tipb.SQLMeta, 0, len(data.collectedData)) + data.normalizedSQLMap.Range(func(key, value interface{}) bool { + sqlMetas = append(sqlMetas, &tipb.SQLMeta{ + SqlDigest: []byte(key.(string)), + NormalizedSql: value.(string), + }) + return true + }) + + planMetas = make([]*tipb.PlanMeta, 0, len(data.collectedData)) + data.normalizedPlanMap.Range(func(key, value interface{}) bool { + planDecoded, err := tsr.planBinaryDecoder(value.(string)) + if err != nil { + logutil.BgLogger().Warn("[top-sql] decode plan failed", zap.Error(err)) + return true + } + planMetas = append(planMetas, &tipb.PlanMeta{ + PlanDigest: []byte(key.(string)), + NormalizedPlan: planDecoded, + }) + return true + }) + + records = make([]*tipb.CPUTimeRecord, 0, len(data.collectedData)) + for _, value := range data.collectedData { + records = append(records, &tipb.CPUTimeRecord{ + TimestampList: value.TimestampList, + CpuTimeMsList: value.CPUTimeMsList, + SqlDigest: value.SQLDigest, + PlanDigest: value.PlanDigest, + }) + } + return sqlMetas, planMetas, records +} + +// reportWorker sends data to the gRPC endpoint from the `reportDataChan` one by one. +func (tsr *RemoteTopSQLReporter) reportWorker() { + defer util.Recover("top-sql", "reportWorker", nil, false) + + for { + select { + case data := <-tsr.reportDataChan: + // When `reportDataChan` receives something, there could be ongoing `RegisterSQL` and `RegisterPlan` running, + // who writes to the data structure that `data` contains. So we wait for a little while to ensure that + // these writes are finished. + time.Sleep(time.Millisecond * 100) + tsr.doReport(data) + case <-tsr.ctx.Done(): + return + } + } +} + +func (tsr *RemoteTopSQLReporter) doReport(data reportData) { + defer util.Recover("top-sql", "doReport", nil, false) + + sqlMetas, planMetas, records := tsr.prepareReportDataForSending(data) + agentAddr := variable.TopSQLVariable.AgentAddress.Load() + + ctx, cancel := context.WithTimeout(tsr.ctx, reportTimeout) + err := tsr.client.Send(ctx, agentAddr, sqlMetas, planMetas, records) + if err != nil { + logutil.BgLogger().Warn("[top-sql] client failed to send data", zap.Error(err)) + } + cancel() +} diff --git a/util/topsql/reporter/reporter_test.go b/util/topsql/reporter/reporter_test.go new file mode 100644 index 0000000000000..7962d970b97af --- /dev/null +++ b/util/topsql/reporter/reporter_test.go @@ -0,0 +1,192 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package reporter + +import ( + "strconv" + "strings" + "testing" + "time" + + . "github.com/pingcap/check" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/topsql/reporter/mock" + "github.com/pingcap/tidb/util/topsql/tracecpu" +) + +const ( + maxSQLNum = 5000 +) + +func TestT(t *testing.T) { + TestingT(t) +} + +var _ = SerialSuites(&testTopSQLReporter{}) + +type testTopSQLReporter struct{} + +func (s *testTopSQLReporter) SetUpSuite(c *C) {} + +func (s *testTopSQLReporter) SetUpTest(c *C) {} + +func testPlanBinaryDecoderFunc(plan string) (string, error) { + return plan, nil +} + +func populateCache(tsr *RemoteTopSQLReporter, begin, end int, timestamp uint64) { + // register normalized sql + for i := begin; i < end; i++ { + key := []byte("sqlDigest" + strconv.Itoa(i+1)) + value := "sqlNormalized" + strconv.Itoa(i+1) + tsr.RegisterSQL(key, value) + } + // register normalized plan + for i := begin; i < end; i++ { + key := []byte("planDigest" + strconv.Itoa(i+1)) + value := "planNormalized" + strconv.Itoa(i+1) + tsr.RegisterPlan(key, value) + } + // collect + var records []tracecpu.SQLCPUTimeRecord + for i := begin; i < end; i++ { + records = append(records, tracecpu.SQLCPUTimeRecord{ + SQLDigest: []byte("sqlDigest" + strconv.Itoa(i+1)), + PlanDigest: []byte("planDigest" + strconv.Itoa(i+1)), + CPUTimeMs: uint32(i + 1), + }) + } + tsr.Collect(timestamp, records) + // sleep a while for the asynchronous collect + time.Sleep(100 * time.Millisecond) +} + +func setupRemoteTopSQLReporter(maxStatementsNum, interval int, addr string) *RemoteTopSQLReporter { + variable.TopSQLVariable.MaxStatementCount.Store(int64(maxStatementsNum)) + variable.TopSQLVariable.ReportIntervalSeconds.Store(int64(interval)) + variable.TopSQLVariable.AgentAddress.Store(addr) + + rc := NewGRPCReportClient() + ts := NewRemoteTopSQLReporter(rc, testPlanBinaryDecoderFunc) + return ts +} + +func initializeCache(maxStatementsNum, interval int, addr string) *RemoteTopSQLReporter { + ts := setupRemoteTopSQLReporter(maxStatementsNum, interval, addr) + populateCache(ts, 0, maxStatementsNum, 1) + return ts +} + +func (s *testTopSQLReporter) TestCollectAndSendBatch(c *C) { + agentServer, err := mock.StartMockAgentServer() + c.Assert(err, IsNil) + defer agentServer.Stop() + + tsr := setupRemoteTopSQLReporter(maxSQLNum, 1, agentServer.Address()) + defer tsr.Close() + populateCache(tsr, 0, maxSQLNum, 1) + + agentServer.WaitServerCollect(maxSQLNum, time.Second*5) + + c.Assert(agentServer.GetRecords(), HasLen, maxSQLNum) + + // check for equality of server received batch and the original data + records := agentServer.GetRecords() + sqlMetas := agentServer.GetSQLMetas() + planMetas := agentServer.GetPlanMetas() + for _, req := range records { + id := 0 + prefix := "sqlDigest" + if strings.HasPrefix(string(req.SqlDigest), prefix) { + n, err := strconv.Atoi(string(req.SqlDigest)[len(prefix):]) + c.Assert(err, IsNil) + id = n + } + c.Assert(req.CpuTimeMsList, HasLen, 1) + for i := range req.CpuTimeMsList { + c.Assert(req.CpuTimeMsList[i], Equals, uint32(id)) + } + c.Assert(req.TimestampList, HasLen, 1) + for i := range req.TimestampList { + c.Assert(req.TimestampList[i], Equals, uint64(1)) + } + normalizedSQL, exist := sqlMetas[string(req.SqlDigest)] + c.Assert(exist, IsTrue) + c.Assert(normalizedSQL, Equals, "sqlNormalized"+strconv.Itoa(id)) + normalizedPlan, exist := planMetas[string(req.PlanDigest)] + c.Assert(exist, IsTrue) + c.Assert(normalizedPlan, Equals, "planNormalized"+strconv.Itoa(id)) + } +} + +func (s *testTopSQLReporter) TestCollectAndEvicted(c *C) { + agentServer, err := mock.StartMockAgentServer() + c.Assert(err, IsNil) + defer agentServer.Stop() + + tsr := setupRemoteTopSQLReporter(maxSQLNum, 1, agentServer.Address()) + defer tsr.Close() + populateCache(tsr, 0, maxSQLNum*2, 2) + + agentServer.WaitServerCollect(maxSQLNum, time.Second*10) + + c.Assert(agentServer.GetRecords(), HasLen, maxSQLNum) + + // check for equality of server received batch and the original data + records := agentServer.GetRecords() + sqlMetas := agentServer.GetSQLMetas() + planMetas := agentServer.GetPlanMetas() + for _, req := range records { + id := 0 + prefix := "sqlDigest" + if strings.HasPrefix(string(req.SqlDigest), prefix) { + n, err := strconv.Atoi(string(req.SqlDigest)[len(prefix):]) + c.Assert(err, IsNil) + id = n + } + c.Assert(id >= maxSQLNum, IsTrue) + c.Assert(req.CpuTimeMsList, HasLen, 1) + for i := range req.CpuTimeMsList { + c.Assert(req.CpuTimeMsList[i], Equals, uint32(id)) + } + c.Assert(req.TimestampList, HasLen, 1) + for i := range req.TimestampList { + c.Assert(req.TimestampList[i], Equals, uint64(2)) + } + normalizedSQL, exist := sqlMetas[string(req.SqlDigest)] + c.Assert(exist, IsTrue) + c.Assert(normalizedSQL, Equals, "sqlNormalized"+strconv.Itoa(id)) + normalizedPlan, exist := planMetas[string(req.PlanDigest)] + c.Assert(exist, IsTrue) + c.Assert(normalizedPlan, Equals, "planNormalized"+strconv.Itoa(id)) + } +} + +func BenchmarkTopSQL_CollectAndIncrementFrequency(b *testing.B) { + tsr := initializeCache(maxSQLNum, 120, ":23333") + for i := 0; i < b.N; i++ { + populateCache(tsr, 0, maxSQLNum, uint64(i)) + } +} + +func BenchmarkTopSQL_CollectAndEvict(b *testing.B) { + tsr := initializeCache(maxSQLNum, 120, ":23333") + begin := 0 + end := maxSQLNum + for i := 0; i < b.N; i++ { + begin += maxSQLNum + end += maxSQLNum + populateCache(tsr, begin, end, uint64(i)) + } +} diff --git a/util/topsql/topsql.go b/util/topsql/topsql.go index 255ece7256c09..39e26b085a6fa 100644 --- a/util/topsql/topsql.go +++ b/util/topsql/topsql.go @@ -18,15 +18,28 @@ import ( "runtime/pprof" "github.com/pingcap/parser" - "github.com/pingcap/tidb/util/topsql/collector" + "github.com/pingcap/tidb/util/plancodec" + "github.com/pingcap/tidb/util/topsql/reporter" "github.com/pingcap/tidb/util/topsql/tracecpu" ) +var globalTopSQLReport reporter.TopSQLReporter + // SetupTopSQL sets up the top-sql worker. func SetupTopSQL() { + rc := reporter.NewGRPCReportClient() + globalTopSQLReport = reporter.NewRemoteTopSQLReporter(rc, plancodec.DecodeNormalizedPlan) + tracecpu.GlobalSQLCPUProfiler.SetCollector(globalTopSQLReport) tracecpu.GlobalSQLCPUProfiler.Run() } +// Close uses to close and release the top sql resource. +func Close() { + if globalTopSQLReport != nil { + globalTopSQLReport.Close() + } +} + // AttachSQLInfo attach the sql information info top sql. func AttachSQLInfo(ctx context.Context, normalizedSQL string, sqlDigest *parser.Digest, normalizedPlan string, planDigest *parser.Digest) context.Context { if len(normalizedSQL) == 0 || sqlDigest == nil || len(sqlDigest.Bytes()) == 0 { @@ -54,7 +67,7 @@ func linkSQLTextWithDigest(sqlDigest []byte, normalizedSQL string) { if c == nil { return } - topc, ok := c.(collector.TopSQLCollector) + topc, ok := c.(reporter.TopSQLReporter) if ok { topc.RegisterSQL(sqlDigest, normalizedSQL) } @@ -65,7 +78,7 @@ func linkPlanTextWithDigest(planDigest []byte, normalizedPlan string) { if c == nil { return } - topc, ok := c.(collector.TopSQLCollector) + topc, ok := c.(reporter.TopSQLReporter) if ok { topc.RegisterPlan(planDigest, normalizedPlan) } diff --git a/util/topsql/tracecpu_test.go b/util/topsql/topsql_test.go similarity index 60% rename from util/topsql/tracecpu_test.go rename to util/topsql/topsql_test.go index 6091457a9db07..8d7500900cb42 100644 --- a/util/topsql/tracecpu_test.go +++ b/util/topsql/topsql_test.go @@ -24,6 +24,8 @@ import ( "github.com/pingcap/parser" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/topsql" + "github.com/pingcap/tidb/util/topsql/reporter" + mockServer "github.com/pingcap/tidb/util/topsql/reporter/mock" "github.com/pingcap/tidb/util/topsql/tracecpu" "github.com/pingcap/tidb/util/topsql/tracecpu/mock" ) @@ -37,6 +39,10 @@ var _ = SerialSuites(&testSuite{}) type testSuite struct{} +type collectorWrapper struct { + reporter.TopSQLReporter +} + func (s *testSuite) SetUpSuite(c *C) { variable.TopSQLVariable.Enable.Store(true) variable.TopSQLVariable.AgentAddress.Store("mock") @@ -46,7 +52,7 @@ func (s *testSuite) SetUpSuite(c *C) { func (s *testSuite) TestTopSQLCPUProfile(c *C) { collector := mock.NewTopSQLCollector() - tracecpu.GlobalSQLCPUProfiler.SetCollector(collector) + tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{collector}) reqs := []struct { sql string plan string @@ -110,6 +116,81 @@ func (s *testSuite) TestIsEnabled(c *C) { c.Assert(tracecpu.GlobalSQLCPUProfiler.IsEnabled(), IsTrue) } +func (s *testSuite) TestTopSQLReporter(c *C) { + server, err := mockServer.StartMockAgentServer() + c.Assert(err, IsNil) + variable.TopSQLVariable.MaxStatementCount.Store(200) + variable.TopSQLVariable.ReportIntervalSeconds.Store(1) + variable.TopSQLVariable.AgentAddress.Store(server.Address()) + + client := reporter.NewGRPCReportClient() + report := reporter.NewRemoteTopSQLReporter(client, func(s string) (string, error) { + return s, nil + }) + defer report.Close() + + tracecpu.GlobalSQLCPUProfiler.SetCollector(&collectorWrapper{report}) + reqs := []struct { + sql string + plan string + }{ + {"select * from t where a=?", "point-get"}, + {"select * from t where a>?", "table-scan"}, + {"insert into t values (?)", ""}, + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + sqlMap := make(map[string]string) + sql2plan := make(map[string]string) + for _, req := range reqs { + sql2plan[req.sql] = req.plan + sqlDigest := mock.GenSQLDigest(req.sql) + sqlMap[string(sqlDigest.Bytes())] = req.sql + + go func(sql, plan string) { + for { + select { + case <-ctx.Done(): + return + default: + s.mockExecuteSQL(sql, plan) + } + } + }(req.sql, req.plan) + } + + server.WaitServerCollect(6, time.Second*5) + + records := server.GetRecords() + sqlMetas := server.GetSQLMetas() + planMetas := server.GetPlanMetas() + + checkSQLPlanMap := map[string]struct{}{} + for _, req := range records { + c.Assert(len(req.CpuTimeMsList) > 0, IsTrue) + c.Assert(req.CpuTimeMsList[0] > 0, IsTrue) + c.Assert(req.TimestampList[0] > 0, IsTrue) + normalizedSQL, exist := sqlMetas[string(req.SqlDigest)] + c.Assert(exist, IsTrue) + expectedNormalizedSQL, exist := sqlMap[string(req.SqlDigest)] + c.Assert(exist, IsTrue) + c.Assert(normalizedSQL, Equals, expectedNormalizedSQL) + + expectedNormalizedPlan := sql2plan[expectedNormalizedSQL] + if expectedNormalizedPlan == "" || len(req.PlanDigest) == 0 { + c.Assert(len(req.PlanDigest), Equals, 0) + continue + } + normalizedPlan, exist := planMetas[string(req.PlanDigest)] + c.Assert(exist, IsTrue) + c.Assert(normalizedPlan, Equals, expectedNormalizedPlan) + checkSQLPlanMap[expectedNormalizedSQL] = struct{}{} + } + c.Assert(len(checkSQLPlanMap) == 2, IsTrue) +} + func (s *testSuite) setTopSQLEnable(enabled bool) { variable.TopSQLVariable.Enable.Store(enabled) } diff --git a/util/topsql/tracecpu/mock/mock.go b/util/topsql/tracecpu/mock/mock.go index 2737306f1a096..a48100eb2eebb 100644 --- a/util/topsql/tracecpu/mock/mock.go +++ b/util/topsql/tracecpu/mock/mock.go @@ -32,7 +32,7 @@ type TopSQLCollector struct { // plan_digest -> normalized plan planMap map[string]string // (sql + plan_digest) -> sql stats - sqlStatsMap map[string]*tracecpu.SQLCPUResult + sqlStatsMap map[string]*tracecpu.SQLCPUTimeRecord collectCnt atomic.Int64 } @@ -41,12 +41,12 @@ func NewTopSQLCollector() *TopSQLCollector { return &TopSQLCollector{ sqlMap: make(map[string]string), planMap: make(map[string]string), - sqlStatsMap: make(map[string]*tracecpu.SQLCPUResult), + sqlStatsMap: make(map[string]*tracecpu.SQLCPUTimeRecord), } } // Collect uses for testing. -func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.SQLCPUResult) { +func (c *TopSQLCollector) Collect(ts uint64, stats []tracecpu.SQLCPUTimeRecord) { defer c.collectCnt.Inc() if len(stats) == 0 { return @@ -57,7 +57,7 @@ func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.SQLCPUResult) { hash := c.hash(stmt) stats, ok := c.sqlStatsMap[hash] if !ok { - stats = &tracecpu.SQLCPUResult{ + stats = &tracecpu.SQLCPUTimeRecord{ SQLDigest: stmt.SQLDigest, PlanDigest: stmt.PlanDigest, } @@ -68,7 +68,7 @@ func (c *TopSQLCollector) Collect(ts int64, stats []tracecpu.SQLCPUResult) { } // GetSQLStatsBySQLWithRetry uses for testing. -func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.SQLCPUResult { +func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bool) []*tracecpu.SQLCPUTimeRecord { after := time.After(time.Second * 10) for { select { @@ -85,8 +85,8 @@ func (c *TopSQLCollector) GetSQLStatsBySQLWithRetry(sql string, planIsNotNull bo } // GetSQLStatsBySQL uses for testing. -func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*tracecpu.SQLCPUResult { - stats := make([]*tracecpu.SQLCPUResult, 0, 2) +func (c *TopSQLCollector) GetSQLStatsBySQL(sql string, planIsNotNull bool) []*tracecpu.SQLCPUTimeRecord { + stats := make([]*tracecpu.SQLCPUTimeRecord, 0, 2) sqlDigest := GenSQLDigest(sql) c.Lock() for _, stmt := range c.sqlStatsMap { @@ -149,7 +149,7 @@ func (c *TopSQLCollector) WaitCollectCnt(count int64) { timeout := time.After(time.Second * 10) end := c.collectCnt.Load() + count for { - // Wait for collector collect sql stats count >= expected count + // Wait for reporter to collect sql stats count >= expected count if c.collectCnt.Load() >= end { break } @@ -162,7 +162,10 @@ func (c *TopSQLCollector) WaitCollectCnt(count int64) { } } -func (c *TopSQLCollector) hash(stat tracecpu.SQLCPUResult) string { +// Close implements the interface. +func (c *TopSQLCollector) Close() {} + +func (c *TopSQLCollector) hash(stat tracecpu.SQLCPUTimeRecord) string { return string(stat.SQLDigest) + string(stat.PlanDigest) } diff --git a/util/topsql/tracecpu/profile.go b/util/topsql/tracecpu/profile.go index c0fdbe5fee52f..c068278b26169 100644 --- a/util/topsql/tracecpu/profile.go +++ b/util/topsql/tracecpu/profile.go @@ -47,11 +47,15 @@ var GlobalSQLCPUProfiler = newSQLCPUProfiler() type Collector interface { // Collect uses to collect the SQL execution cpu time. // ts is a Unix time, unit is second. - Collect(ts int64, stats []SQLCPUResult) + Collect(ts uint64, stats []SQLCPUTimeRecord) } -// SQLCPUResult contains the SQL meta and cpu time. -type SQLCPUResult struct { +// SQLCPUTimeRecord represents a single record of how much cpu time a sql plan consumes in one second. +// +// PlanDigest can be empty, because: +// 1. some sql statements has no plan, like `COMMIT` +// 2. when a sql statement is being compiled, there's no plan yet +type SQLCPUTimeRecord struct { SQLDigest []byte PlanDigest []byte CPUTimeMs uint32 @@ -125,6 +129,9 @@ func (sp *sqlCPUProfiler) doCPUProfile() { time.Sleep(time.Nanosecond * time.Duration(ns)) pprof.StopCPUProfile() task.end = time.Now().Unix() + if task.end < 0 { + task.end = 0 + } sp.taskCh <- task } @@ -141,7 +148,7 @@ func (sp *sqlCPUProfiler) startAnalyzeProfileWorker() { stats := sp.parseCPUProfileBySQLLabels(p) sp.handleExportProfileTask(p) if c := sp.GetCollector(); c != nil { - c.Collect(task.end, stats) + c.Collect(uint64(task.end), stats) } sp.putTaskToBuffer(task) } @@ -165,12 +172,12 @@ func (sp *sqlCPUProfiler) putTaskToBuffer(task *profileData) { } // parseCPUProfileBySQLLabels uses to aggregate the cpu-profile sample data by sql_digest and plan_digest labels, -// output the SQLCPUResult slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ +// output the TopSQLCPUTimeRecord slice. Want to know more information about profile labels, see https://rakyll.org/profiler-labels/ // The sql_digest label is been set by `SetSQLLabels` function after parse the SQL. // The plan_digest label is been set by `SetSQLAndPlanLabels` function after build the SQL plan. // Since `sqlCPUProfiler` only care about the cpu time that consume by (sql_digest,plan_digest), the other sample data // without those label will be ignore. -func (sp *sqlCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLCPUResult { +func (sp *sqlCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLCPUTimeRecord { sqlMap := make(map[string]*sqlStats) idx := len(p.SampleType) - 1 for _, s := range p.Sample { @@ -198,12 +205,12 @@ func (sp *sqlCPUProfiler) parseCPUProfileBySQLLabels(p *profile.Profile) []SQLCP return sp.createSQLStats(sqlMap) } -func (sp *sqlCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []SQLCPUResult { - stats := make([]SQLCPUResult, 0, len(sqlMap)) +func (sp *sqlCPUProfiler) createSQLStats(sqlMap map[string]*sqlStats) []SQLCPUTimeRecord { + stats := make([]SQLCPUTimeRecord, 0, len(sqlMap)) for sqlDigest, stmt := range sqlMap { stmt.tune() for planDigest, val := range stmt.plans { - stats = append(stats, SQLCPUResult{ + stats = append(stats, SQLCPUTimeRecord{ SQLDigest: []byte(sqlDigest), PlanDigest: []byte(planDigest), CPUTimeMs: uint32(time.Duration(val).Milliseconds()), From 5e55c2e847ef2dd1d9b1ea2631f0c1d920bd7391 Mon Sep 17 00:00:00 2001 From: chAnge <50198008+chAngeZhaoZhanBo@users.noreply.github.com> Date: Fri, 4 Jun 2021 23:18:28 +0800 Subject: [PATCH 303/343] expression: Support Sqrt, Ceil, Floor and CastIntAsReal push down to TiFlash (#25085) --- expression/expr_to_pb_test.go | 45 +++++++++++++++++++++++++++++++++++ expression/expression.go | 8 +++++++ 2 files changed, 53 insertions(+) diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index c767b5ad88992..f9b2a4d76978d 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -734,6 +734,51 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) + // sqrt + function, err = NewFunction(mock.NewContext(), ast.Sqrt, types.NewFieldType(mysql.TypeDouble), realColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_CeilReal + function, err = NewFunction(mock.NewContext(), ast.Ceil, types.NewFieldType(mysql.TypeDouble), realColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_CeilIntToInt + function, err = NewFunction(mock.NewContext(), ast.Ceil, types.NewFieldType(mysql.TypeLonglong), intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_CeilDecimalToInt + function, err = NewFunction(mock.NewContext(), ast.Ceil, types.NewFieldType(mysql.TypeLonglong), decimalColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_CeilDecimalToDecimal + function, err = NewFunction(mock.NewContext(), ast.Ceil, types.NewFieldType(mysql.TypeNewDecimal), decimalColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_FloorReal + function, err = NewFunction(mock.NewContext(), ast.Floor, types.NewFieldType(mysql.TypeDouble), realColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_FloorIntToInt + function, err = NewFunction(mock.NewContext(), ast.Floor, types.NewFieldType(mysql.TypeLonglong), intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_FloorDecimalToInt + function, err = NewFunction(mock.NewContext(), ast.Floor, types.NewFieldType(mysql.TypeLonglong), decimalColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_FloorDecimalToDecimal + function, err = NewFunction(mock.NewContext(), ast.Floor, types.NewFieldType(mysql.TypeNewDecimal), decimalColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + // Replace function, err = NewFunction(mock.NewContext(), ast.Replace, types.NewFieldType(mysql.TypeString), stringColumn, stringColumn, stringColumn) c.Assert(err, IsNil) diff --git a/expression/expression.go b/expression/expression.go index e1ec7c5b311d4..daa3be56a9959 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -996,6 +996,13 @@ func scalarExprSupportedByTiKV(sf *ScalarFunction) bool { func scalarExprSupportedByFlash(function *ScalarFunction) bool { switch function.FuncName.L { + case ast.Floor, ast.Ceil, ast.Ceiling: + switch function.Function.PbCode() { + case tipb.ScalarFuncSig_FloorIntToDec, tipb.ScalarFuncSig_CeilIntToDec: + return false + default: + return true + } case ast.LogicOr, ast.LogicAnd, ast.UnaryNot, ast.BitNeg, ast.Xor, ast.And, ast.Or, ast.GE, ast.LE, ast.EQ, ast.NE, ast.LT, ast.GT, ast.In, ast.IsNull, ast.Like, @@ -1004,6 +1011,7 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { ast.Concat, ast.ConcatWS, ast.Year, ast.Month, ast.Day, ast.DateDiff, ast.TimestampDiff, ast.DateFormat, ast.FromUnixTime, + ast.Sqrt, ast.JSONLength: return true case ast.Substr, ast.Substring, ast.Left, ast.Right, ast.CharLength: From 3e69ac2ed395e519a15ee5b2ebc6d58fa64b4061 Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Sat, 5 Jun 2021 00:04:28 +0800 Subject: [PATCH 304/343] ddl: fix reorg goroutine leak after column-type-change is cancelled (#24616) --- ddl/column.go | 19 +++++++++- ddl/column_type_change_test.go | 63 ++++++++++++++++++++++++++++++++++ ddl/reorg.go | 34 ++++++++++++++++++ ddl/rollingback.go | 14 ++++++-- 4 files changed, 126 insertions(+), 4 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 7d93c4e15668d..41086b2d63497 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1044,10 +1044,11 @@ func (w *worker) doModifyColumnTypeWithData( if err1 := t.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil { logutil.BgLogger().Warn("[ddl] run modify column job failed, RemoveDDLReorgHandle failed, can't convert job to rollback", zap.String("job", job.String()), zap.Error(err1)) - return ver, errors.Trace(err) } logutil.BgLogger().Warn("[ddl] run modify column job failed, convert job to rollback", zap.String("job", job.String()), zap.Error(err)) job.State = model.JobStateRollingback + // Clean up the channel of notifyCancelReorgJob. Make sure it can't affect other jobs. + w.reorgCtx.cleanNotifyReorgCancel() return ver, errors.Trace(err) } // Clean up the channel of notifyCancelReorgJob. Make sure it can't affect other jobs. @@ -1109,8 +1110,24 @@ func (w *worker) updatePhysicalTableRow(t table.PhysicalTable, oldColInfo, colIn return w.writePhysicalTableRecord(t.(table.PhysicalTable), typeUpdateColumnWorker, nil, oldColInfo, colInfo, reorgInfo) } +// TestReorgGoroutineRunning is only used in test to indicate the reorg goroutine has been started. +var TestReorgGoroutineRunning = make(chan interface{}) + // updateColumnAndIndexes handles the modify column reorganization state for a table. func (w *worker) updateColumnAndIndexes(t table.Table, oldCol, col *model.ColumnInfo, idxes []*model.IndexInfo, reorgInfo *reorgInfo) error { + failpoint.Inject("mockInfiniteReorgLogic", func(val failpoint.Value) { + if val.(bool) { + a := new(interface{}) + TestReorgGoroutineRunning <- a + for { + time.Sleep(30 * time.Millisecond) + if w.reorgCtx.isReorgCanceled() { + // Job is cancelled. So it can't be done. + failpoint.Return(errCancelledDDLJob) + } + } + } + }) // TODO: Support partition tables. if bytes.Equal(reorgInfo.currElement.TypeKey, meta.ColumnElementKey) { err := w.updatePhysicalTableRow(t.(table.PhysicalTable), oldCol, col, reorgInfo) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 10f21a0e8bbca..78512a10515cf 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -16,6 +16,8 @@ package ddl_test import ( "context" "errors" + "strconv" + "sync" "time" . "github.com/pingcap/check" @@ -42,12 +44,17 @@ import ( ) var _ = SerialSuites(&testColumnTypeChangeSuite{}) +var _ = SerialSuites(&testCTCSerialSuiteWrapper{&testColumnTypeChangeSuite{}}) type testColumnTypeChangeSuite struct { store kv.Storage dom *domain.Domain } +type testCTCSerialSuiteWrapper struct { + *testColumnTypeChangeSuite +} + func (s *testColumnTypeChangeSuite) SetUpSuite(c *C) { var err error ddl.SetWaitTimeWhenErrorOccurred(1 * time.Microsecond) @@ -1884,6 +1891,62 @@ func (s *testColumnTypeChangeSuite) TestChangeIntToBitWillPanicInBackfillIndexes tk.MustQuery("select * from t").Check(testkit.Rows("\x13 1 1.00", "\x11 2 2.00")) } +// Close issue #24584 +func (s *testColumnTypeChangeSuite) TestCancelCTCInReorgStateWillCauseGoroutineLeak(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + // Enable column change variable. + tk.Se.GetSessionVars().EnableChangeColumnType = true + + failpoint.Enable("github.com/pingcap/tidb/ddl/mockInfiniteReorgLogic", `return(true)`) + defer func() { + failpoint.Disable("github.com/pingcap/tidb/ddl/mockInfiniteReorgLogic") + }() + + // set ddl hook + originalHook := s.dom.DDL().GetHook() + defer s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook) + + tk.MustExec("drop table if exists ctc_goroutine_leak") + tk.MustExec("create table ctc_goroutine_leak (a int)") + tk.MustExec("insert into ctc_goroutine_leak values(1),(2),(3)") + tbl := testGetTableByName(c, tk.Se, "test", "ctc_goroutine_leak") + + hook := &ddl.TestDDLCallback{} + var jobID int64 + hook.OnJobRunBeforeExported = func(job *model.Job) { + if jobID != 0 { + return + } + if tbl.Meta().ID != job.TableID { + return + } + if job.Query == "alter table ctc_goroutine_leak modify column a tinyint" { + jobID = job.ID + } + } + s.dom.DDL().(ddl.DDLForTest).SetHook(hook) + + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec("use test") + // Enable column change variable. + tk1.Se.GetSessionVars().EnableChangeColumnType = true + var ( + wg = sync.WaitGroup{} + alterErr error + ) + wg.Add(1) + go func() { + defer wg.Done() + // This ddl will be hang over in the failpoint loop, waiting for outside cancel. + _, alterErr = tk1.Exec("alter table ctc_goroutine_leak modify column a tinyint") + }() + <-ddl.TestReorgGoroutineRunning + tk.MustExec("admin cancel ddl jobs " + strconv.Itoa(int(jobID))) + wg.Wait() + c.Assert(alterErr.Error(), Equals, "[ddl:8214]Cancelled DDL job") +} + // Close issue #24971, #24973, #24974 func (s *testColumnTypeChangeSuite) TestCTCShouldCastTheDefaultValue(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/ddl/reorg.go b/ddl/reorg.go index 6b5048eaa1643..eb20f31ce5fe7 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -155,6 +155,40 @@ func (rc *reorgCtx) clean() { rc.doneCh = nil } +// runReorgJob is used as a portal to do the reorganization work. +// eg: +// 1: add index +// 2: alter column type +// 3: clean global index +// +// ddl goroutine >---------+ +// ^ | +// | | +// | | +// | | <---(doneCh)--- f() +// HandleDDLQueue(...) | <---(regular timeout) +// | | <---(ctx done) +// | | +// | | +// A more ddl round <-----+ +// +// How can we cancel reorg job? +// +// The background reorg is continuously running except for several factors, for instances, ddl owner change, +// logic error (kv duplicate when insert index / cast error when alter column), ctx done, and cancel signal. +// +// When `admin cancel ddl jobs xxx` takes effect, we will give this kind of reorg ddl one more round. +// because we should pull the result from doneCh out, otherwise, the reorg worker will hang on `f()` logic, +// which is a kind of goroutine leak. +// +// That's why we couldn't set the job to rollingback state directly in `convertJob2RollbackJob`, which is a +// cancelling portal for admin cancel action. +// +// In other words, the cancelling signal is informed from the bottom up, we set the atomic cancel variable +// in the cancelling portal to notify the lower worker goroutine, and fetch the cancel error from them in +// the additional ddl round. +// +// After that, we can make sure that the worker goroutine is correctly shut down. func (w *worker) runReorgJob(t *meta.Meta, reorgInfo *reorgInfo, tblInfo *model.TableInfo, lease time.Duration, f func() error) error { job := reorgInfo.Job // This is for tests compatible, because most of the early tests try to build the reorg job manually diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 845ef175f9204..bd9daab771c82 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -110,7 +110,15 @@ func convertNotStartAddIdxJob2RollbackJob(t *meta.Meta, job *model.Job, occuredE // Since modifying column job has two types: normal-type and reorg-type, we should handle it respectively. // normal-type has only two states: None -> Public // reorg-type has five states: None -> Delete-only -> Write-only -> Write-org -> Public -func rollingbackModifyColumn(t *meta.Meta, job *model.Job) (ver int64, err error) { +func rollingbackModifyColumn(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { + // If the value of SnapshotVer isn't zero, it means the reorg workers have been started. + if job.SchemaState == model.StateWriteReorganization && job.SnapshotVer != 0 { + // column type change workers are started. we have to ask them to exit. + logutil.Logger(w.logCtx).Info("[ddl] run the cancelling DDL job", zap.String("job", job.String())) + w.reorgCtx.notifyReorgCancel() + // Give the this kind of ddl one more round to run, the errCancelledDDLJob should be fetched from the bottom up. + return w.onModifyColumn(d, t, job) + } _, tblInfo, oldCol, jp, err := getModifyColumnInfo(t, job) if err != nil { return ver, err @@ -138,7 +146,7 @@ func rollingbackModifyColumn(t *meta.Meta, job *model.Job) (ver int64, err error job.State = model.JobStateCancelled return ver, errCancelledDDLJob } - // The job has been in it's middle state and we roll it back. + // The job has been in its middle state (but the reorg worker hasn't started) and we roll it back here. job.State = model.JobStateRollingback return ver, errCancelledDDLJob } @@ -424,7 +432,7 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) case model.ActionTruncateTable: ver, err = rollingbackTruncateTable(t, job) case model.ActionModifyColumn: - ver, err = rollingbackModifyColumn(t, job) + ver, err = rollingbackModifyColumn(w, d, t, job) case model.ActionRebaseAutoID, model.ActionShardRowID, model.ActionAddForeignKey, model.ActionDropForeignKey, model.ActionRenameTable, model.ActionRenameTables, model.ActionModifyTableCharsetAndCollate, model.ActionTruncateTablePartition, From fbbada3f0f3c2c18f8af62dbe04c93144d6d83cb Mon Sep 17 00:00:00 2001 From: lysu Date: Sat, 5 Jun 2021 01:00:28 +0800 Subject: [PATCH 305/343] telemetry: fix panic caused by assign nil map (#25129) --- telemetry/data_slow_query.go | 46 +++++++++++++++++++----------------- telemetry/telemetry.go | 2 -- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/telemetry/data_slow_query.go b/telemetry/data_slow_query.go index 1b66e20ca2670..c3da41c57efbf 100644 --- a/telemetry/data_slow_query.go +++ b/telemetry/data_slow_query.go @@ -55,11 +55,11 @@ func (bucketMap SlowQueryBucket) String() string { const slowQueryBucketNum = 29 //prometheus.ExponentialBuckets(0.001, 2, 28), and 1 more +Inf var ( - // LastSQBInfo records last statistic information of slow query buckets - LastSQBInfo SlowQueryBucket - // CurrentSQBInfo records current statitic information of slow query buckets - CurrentSQBInfo SlowQueryBucket - slowQueryLock = sync.RWMutex{} + // lastSQBInfo records last statistic information of slow query buckets + lastSQBInfo SlowQueryBucket + // currentSQBInfo records current statitic information of slow query buckets + currentSQBInfo SlowQueryBucket + slowQueryLock sync.Mutex ) func getSlowQueryStats(ctx sessionctx.Context) (*slowQueryStats, error) { @@ -74,7 +74,7 @@ func getSlowQueryStats(ctx sessionctx.Context) (*slowQueryStats, error) { // getSlowQueryBucket genenrates the delta SlowQueryBucket to report func getSlowQueryBucket(ctx sessionctx.Context) (*SlowQueryBucket, error) { - // update CurrentSQBInfo first, then gen delta + // update currentSQBInfo first, then gen delta if err := updateCurrentSQB(ctx); err != nil { return nil, err } @@ -95,7 +95,9 @@ func updateCurrentSQB(ctx sessionctx.Context) (err error) { logutil.BgLogger().Info("querySlowQueryMetric got error") return err } - + if value == nil { + return + } if value.Type() != pmodel.ValVector { return errors.New("Prom vector expected, got " + value.Type().String()) } @@ -104,7 +106,7 @@ func updateCurrentSQB(ctx sessionctx.Context) (err error) { for _, sample := range promVec { metric := sample.Metric bucketName := metric["le"] //hardcode bucket upper bound - CurrentSQBInfo[string(bucketName)] = int(sample.Value) + currentSQBInfo[string(bucketName)] = int(sample.Value) } slowQueryLock.Unlock() return nil @@ -151,37 +153,37 @@ func querySlowQueryMetric(sctx sessionctx.Context) (result pmodel.Value, err err func calculateDeltaSQB() *SlowQueryBucket { deltaMap := make(SlowQueryBucket) slowQueryLock.Lock() - for key, value := range CurrentSQBInfo { - deltaMap[key] = value - (LastSQBInfo)[key] + for key, value := range currentSQBInfo { + deltaMap[key] = value - (lastSQBInfo)[key] } slowQueryLock.Unlock() return &deltaMap } -// initSlowQueryStats Init LastSQBInfo, follow the definition of metrics/server.go +// init Init lastSQBInfo, follow the definition of metrics/server.go // Buckets: prometheus.ExponentialBuckets(0.001, 2, 28), // 1ms ~ 1.5days -func initSlowQueryStats() { - LastSQBInfo := make(SlowQueryBucket) - CurrentSQBInfo := make(SlowQueryBucket) +func init() { + lastSQBInfo = make(SlowQueryBucket) + currentSQBInfo = make(SlowQueryBucket) bucketBase := 0.001 // From 0.001 to 134217.728, total 28 float number; the 29th is +Inf for i := 0; i < slowQueryBucketNum-1; i++ { - LastSQBInfo[strconv.FormatFloat(bucketBase, 'f', 3, 32)] = 0 - CurrentSQBInfo[strconv.FormatFloat(bucketBase, 'f', 3, 32)] = 0 + lastSQBInfo[strconv.FormatFloat(bucketBase, 'f', 3, 32)] = 0 + currentSQBInfo[strconv.FormatFloat(bucketBase, 'f', 3, 32)] = 0 bucketBase += bucketBase } - LastSQBInfo["+Inf"] = 0 - CurrentSQBInfo["+Inf"] = 0 + lastSQBInfo["+Inf"] = 0 + currentSQBInfo["+Inf"] = 0 - logutil.BgLogger().Info("Telemetry slow query stats initialized", zap.String("CurrentSQBInfo", CurrentSQBInfo.String()), zap.String("LastSQBInfo", LastSQBInfo.String())) + logutil.BgLogger().Info("Telemetry slow query stats initialized", zap.String("currentSQBInfo", currentSQBInfo.String()), zap.String("lastSQBInfo", lastSQBInfo.String())) } -// postReportSlowQueryStats copy CurrentSQBInfo to LastSQBInfo to be ready for next report +// postReportSlowQueryStats copy currentSQBInfo to lastSQBInfo to be ready for next report // this function is designed for being compatible with preview telemetry func postReportSlowQueryStats() { slowQueryLock.Lock() - LastSQBInfo = CurrentSQBInfo - CurrentSQBInfo = make(SlowQueryBucket) + lastSQBInfo = currentSQBInfo + currentSQBInfo = make(SlowQueryBucket) slowQueryLock.Unlock() logutil.BgLogger().Info("Telemetry slow query stats, postReportSlowQueryStats finished") } diff --git a/telemetry/telemetry.go b/telemetry/telemetry.go index 23fe2462891d4..85e6554ab5296 100644 --- a/telemetry/telemetry.go +++ b/telemetry/telemetry.go @@ -170,8 +170,6 @@ func InitialRun(ctx sessionctx.Context, etcdClient *clientv3.Client) error { if err != nil { return err } - - initSlowQueryStats() logutil.BgLogger().Info("Telemetry configuration", zap.String("endpoint", apiEndpoint), zap.Duration("report_interval", ReportInterval), zap.Bool("enabled", enabled)) return ReportUsageData(ctx, etcdClient) } From f837bc770de2b5010614679b0113e6bd09aabdca Mon Sep 17 00:00:00 2001 From: xiongjiwei Date: Sat, 5 Jun 2021 14:02:27 +0800 Subject: [PATCH 306/343] planner: reset recorded cte plan in plan encoder (#25175) --- planner/core/encode.go | 1 + planner/core/plan_test.go | 14 ++++++++++++++ 2 files changed, 15 insertions(+) diff --git a/planner/core/encode.go b/planner/core/encode.go index 2418c502aa964..ba4fc643821b5 100644 --- a/planner/core/encode.go +++ b/planner/core/encode.go @@ -60,6 +60,7 @@ func EncodePlan(p Plan) string { func (pn *planEncoder) encodePlanTree(p Plan) string { pn.encodedPlans = make(map[int]bool) pn.buf.Reset() + pn.ctes = pn.ctes[:0] pn.encodePlan(p, true, kv.TiKV, 0) pn.encodeCTEPlan() return plancodec.Compress(pn.buf.Bytes()) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 6c29eef90f5ae..2f395c823542d 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -211,6 +211,20 @@ func (s *testPlanNormalize) TestEncodeDecodePlan(c *C) { c.Assert(strings.Contains(planTree, "Insert"), IsTrue) c.Assert(strings.Contains(planTree, "time"), IsTrue) c.Assert(strings.Contains(planTree, "loops"), IsTrue) + + tk.MustExec("with cte(a) as (select 1) select * from cte") + planTree = getPlanTree() + c.Assert(strings.Contains(planTree, "CTE"), IsTrue) + c.Assert(strings.Contains(planTree, "1->Column#1"), IsTrue) + c.Assert(strings.Contains(planTree, "time"), IsTrue) + c.Assert(strings.Contains(planTree, "loops"), IsTrue) + + tk.MustExec("with cte(a) as (select 2) select * from cte") + planTree = getPlanTree() + c.Assert(strings.Contains(planTree, "CTE"), IsTrue) + c.Assert(strings.Contains(planTree, "2->Column#1"), IsTrue) + c.Assert(strings.Contains(planTree, "time"), IsTrue) + c.Assert(strings.Contains(planTree, "loops"), IsTrue) } func (s *testPlanNormalize) TestNormalizedDigest(c *C) { From ce0e3c3dd52ec3bf2aced3eeb9e8bec2b39fd022 Mon Sep 17 00:00:00 2001 From: Lei Zhao Date: Sat, 5 Jun 2021 14:38:27 +0800 Subject: [PATCH 307/343] store/tikv: refine the retry mechanism of region errors (#25165) --- store/tikv/metrics/shortcuts.go | 18 +++-- store/tikv/region_cache.go | 1 + store/tikv/region_request.go | 126 +++++++++++++++++------------- store/tikv/region_request_test.go | 6 +- store/tikv/retry/config.go | 1 + 5 files changed, 88 insertions(+), 64 deletions(-) diff --git a/store/tikv/metrics/shortcuts.go b/store/tikv/metrics/shortcuts.go index 01d4da8983cc7..d38c891b4c649 100644 --- a/store/tikv/metrics/shortcuts.go +++ b/store/tikv/metrics/shortcuts.go @@ -33,14 +33,15 @@ var ( RawkvSizeHistogramWithKey prometheus.Observer RawkvSizeHistogramWithValue prometheus.Observer - BackoffHistogramRPC prometheus.Observer - BackoffHistogramLock prometheus.Observer - BackoffHistogramLockFast prometheus.Observer - BackoffHistogramPD prometheus.Observer - BackoffHistogramRegionMiss prometheus.Observer - BackoffHistogramServerBusy prometheus.Observer - BackoffHistogramStaleCmd prometheus.Observer - BackoffHistogramEmpty prometheus.Observer + BackoffHistogramRPC prometheus.Observer + BackoffHistogramLock prometheus.Observer + BackoffHistogramLockFast prometheus.Observer + BackoffHistogramPD prometheus.Observer + BackoffHistogramRegionMiss prometheus.Observer + BackoffHistogramRegionScheduling prometheus.Observer + BackoffHistogramServerBusy prometheus.Observer + BackoffHistogramStaleCmd prometheus.Observer + BackoffHistogramEmpty prometheus.Observer TxnRegionsNumHistogramWithSnapshot prometheus.Observer TxnRegionsNumHistogramPrewrite prometheus.Observer @@ -120,6 +121,7 @@ func initShortcuts() { BackoffHistogramLockFast = TiKVBackoffHistogram.WithLabelValues("tikvLockFast") BackoffHistogramPD = TiKVBackoffHistogram.WithLabelValues("pdRPC") BackoffHistogramRegionMiss = TiKVBackoffHistogram.WithLabelValues("regionMiss") + BackoffHistogramRegionScheduling = TiKVBackoffHistogram.WithLabelValues("regionScheduling") BackoffHistogramServerBusy = TiKVBackoffHistogram.WithLabelValues("serverBusy") BackoffHistogramStaleCmd = TiKVBackoffHistogram.WithLabelValues("staleCommand") BackoffHistogramEmpty = TiKVBackoffHistogram.WithLabelValues("") diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 6e6303b3ace99..12295947b2448 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -1550,6 +1550,7 @@ func (c *RegionCache) getStoresByLabels(labels []*metapb.StoreLabel) []*Store { } // OnRegionEpochNotMatch removes the old region and inserts new regions into the cache. +// It returns whether retries the request because it's possible the region epoch is ahead of TiKV's due to slow appling. func (c *RegionCache) OnRegionEpochNotMatch(bo *Backoffer, ctx *RPCContext, currentRegions []*metapb.Region) (bool, error) { if len(currentRegions) == 0 { c.InvalidateCachedRegionWithReason(ctx.Region, EpochNotMatch) diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 3efb3f4a5ec06..f823c11a71408 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -879,8 +879,11 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req if ctx.tryTimes < 1 && req != nil && req.TxnScope == oracle.GlobalTxnScope && req.GetStaleRead() { *opts = append(*opts, WithLeaderOnly()) } + seed := req.GetReplicaReadSeed() + // NOTE: Please add the region error handler in the same order of errorpb.Error. metrics.TiKVRegionErrorCounter.WithLabelValues(regionErrorToLabel(regionErr)).Inc() + if notLeader := regionErr.GetNotLeader(); notLeader != nil { // Retry if error is `NotLeader`. logutil.BgLogger().Debug("tikv reports `NotLeader` retry later", @@ -891,7 +894,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req leader := notLeader.GetLeader() if leader == nil { // The region may be during transferring leader. - if err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("no leader, ctx: %v", ctx)); err != nil { + if err = bo.Backoff(retry.BoRegionScheduling, errors.Errorf("no leader, ctx: %v", ctx)); err != nil { return false, errors.Trace(err) } } else { @@ -904,7 +907,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req // isolated and removed from the Raft group. So it's necessary to reload // the region from PD. s.regionCache.InvalidateCachedRegionWithReason(ctx.Region, NoLeader) - if err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("not leader: %v, ctx: %v", notLeader, ctx)); err != nil { + if err = bo.Backoff(retry.BoRegionScheduling, errors.Errorf("not leader: %v, ctx: %v", notLeader, ctx)); err != nil { return false, errors.Trace(err) } return false, nil @@ -915,17 +918,23 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req } } - if storeNotMatch := regionErr.GetStoreNotMatch(); storeNotMatch != nil { - // store not match - logutil.BgLogger().Debug("tikv reports `StoreNotMatch` retry later", - zap.Stringer("storeNotMatch", storeNotMatch), - zap.Stringer("ctx", ctx)) - ctx.Store.markNeedCheck(s.regionCache.notifyCheckCh) + // This peer is removed from the region. Invalidate the region since it's too stale. + if regionErr.GetRegionNotFound() != nil { + if seed != nil { + logutil.BgLogger().Debug("tikv reports `RegionNotFound` in follow-reader", + zap.Stringer("ctx", ctx), zap.Uint32("seed", *seed)) + *seed = *seed + 1 + } + s.regionCache.InvalidateCachedRegion(ctx.Region) + return false, nil + } + + if regionErr.GetKeyNotInRegion() != nil { + logutil.BgLogger().Debug("tikv reports `KeyNotInRegion`", zap.Stringer("ctx", ctx)) s.regionCache.InvalidateCachedRegion(ctx.Region) return false, nil } - seed := req.GetReplicaReadSeed() if epochNotMatch := regionErr.GetEpochNotMatch(); epochNotMatch != nil { logutil.BgLogger().Debug("tikv reports `EpochNotMatch` retry later", zap.Stringer("EpochNotMatch", epochNotMatch), @@ -939,6 +948,7 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req } return retry, errors.Trace(err) } + if regionErr.GetServerIsBusy() != nil { logutil.BgLogger().Warn("tikv reports `ServerIsBusy` retry later", zap.String("reason", regionErr.GetServerIsBusy().GetReason()), @@ -956,11 +966,15 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req } return true, nil } + + // StaleCommand error indicates the request is sent to the old leader and its term is changed. + // We can't know whether the request is committed or not, so it's an undetermined error too, + // but we don't handle it now. if regionErr.GetStaleCommand() != nil { logutil.BgLogger().Debug("tikv reports `StaleCommand`", zap.Stringer("ctx", ctx)) if s.leaderReplicaSelector != nil { - // Needn't backoff because stale command indicates the command is sent to the old leader. - // The new leader should be elected soon and the leaderReplicaSelector will try the next peer. + // Needn't backoff because the new leader should be elected soon + // and the leaderReplicaSelector will try the next peer. } else { err = bo.Backoff(retry.BoStaleCmd, errors.Errorf("stale command, ctx: %v", ctx)) if err != nil { @@ -969,36 +983,37 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req } return true, nil } + + if storeNotMatch := regionErr.GetStoreNotMatch(); storeNotMatch != nil { + // store not match + logutil.BgLogger().Debug("tikv reports `StoreNotMatch` retry later", + zap.Stringer("storeNotMatch", storeNotMatch), + zap.Stringer("ctx", ctx)) + ctx.Store.markNeedCheck(s.regionCache.notifyCheckCh) + s.regionCache.InvalidateCachedRegion(ctx.Region) + return false, nil + } + if regionErr.GetRaftEntryTooLarge() != nil { logutil.BgLogger().Warn("tikv reports `RaftEntryTooLarge`", zap.Stringer("ctx", ctx)) return false, errors.New(regionErr.String()) } - // A stale read request may be sent to a peer which the data is not ready yet, we should retry in this case. - if regionErr.GetDataIsNotReady() != nil { - logutil.BgLogger().Warn("tikv reports `DataIsNotReady` retry later", - zap.Uint64("store-id", ctx.Store.storeID), - zap.Uint64("peer-id", regionErr.GetDataIsNotReady().GetPeerId()), - zap.Uint64("region-id", regionErr.GetDataIsNotReady().GetRegionId()), - zap.Uint64("safe-ts", regionErr.GetDataIsNotReady().GetSafeTs()), - zap.Stringer("ctx", ctx)) - // Don't backoff if it's a replica-read. - if seed != nil { - *seed = *seed + 1 - } else { - // The region is merging or splitting. - err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("data is not ready, ctx: %v", ctx)) - if err != nil { - return false, errors.Trace(err) - } - if s.leaderReplicaSelector != nil { - s.leaderReplicaSelector.rewind() - } + + if regionErr.GetMaxTimestampNotSynced() != nil { + logutil.BgLogger().Debug("tikv reports `MaxTimestampNotSynced`", zap.Stringer("ctx", ctx)) + err = bo.Backoff(retry.BoMaxTsNotSynced, errors.Errorf("max timestamp not synced, ctx: %v", ctx)) + if err != nil { + return false, errors.Trace(err) + } + if s.leaderReplicaSelector != nil { + s.leaderReplicaSelector.rewind() } return true, nil } + // A read request may be sent to a peer which has not been initialized yet, we should retry in this case. if regionErr.GetRegionNotInitialized() != nil { - logutil.BgLogger().Warn("tikv reports `RegionNotInitialized` retry later", + logutil.BgLogger().Debug("tikv reports `RegionNotInitialized` retry later", zap.Uint64("store-id", ctx.Store.storeID), zap.Uint64("region-id", regionErr.GetRegionNotInitialized().GetRegionId()), zap.Stringer("ctx", ctx)) @@ -1007,21 +1022,18 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req } return true, nil } - if regionErr.GetMaxTimestampNotSynced() != nil { - logutil.BgLogger().Debug("tikv reports `MaxTimestampNotSynced`", zap.Stringer("ctx", ctx)) - err = bo.Backoff(retry.BoMaxTsNotSynced, errors.Errorf("max timestamp not synced, ctx: %v", ctx)) - if err != nil { - return false, errors.Trace(err) - } - if s.leaderReplicaSelector != nil { - s.leaderReplicaSelector.rewind() - } - return true, nil - } + + // The read-index can't be handled timely because the region is splitting or merging. if regionErr.GetReadIndexNotReady() != nil { - logutil.BgLogger().Debug("tikv reports `ReadIndexNotReady`", zap.Stringer("ctx", ctx)) - // The region is merging or splitting. - err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("read index not ready, ctx: %v", ctx)) + logutil.BgLogger().Debug("tikv reports `ReadIndexNotReady` retry later", + zap.Uint64("store-id", ctx.Store.storeID), + zap.Uint64("region-id", regionErr.GetRegionNotInitialized().GetRegionId()), + zap.Stringer("ctx", ctx)) + if seed != nil { + *seed = *seed + 1 + } + // The region can't provide service until split or merge finished, so backoff. + err = bo.Backoff(retry.BoRegionScheduling, errors.Errorf("read index not ready, ctx: %v", ctx)) if err != nil { return false, errors.Trace(err) } @@ -1030,10 +1042,11 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req } return true, nil } + if regionErr.GetProposalInMergingMode() != nil { logutil.BgLogger().Debug("tikv reports `ProposalInMergingMode`", zap.Stringer("ctx", ctx)) - // The region is merging. - err = bo.Backoff(retry.BoRegionMiss, errors.Errorf("region is merging, ctx: %v", ctx)) + // The region is merging and it can't provide service until merge finished, so backoff. + err = bo.Backoff(retry.BoRegionScheduling, errors.Errorf("region is merging, ctx: %v", ctx)) if err != nil { return false, errors.Trace(err) } @@ -1042,14 +1055,21 @@ func (s *RegionRequestSender) onRegionError(bo *Backoffer, ctx *RPCContext, req } return true, nil } - if regionErr.GetRegionNotFound() != nil { + + // A stale read request may be sent to a peer which the data is not ready yet, we should retry in this case. + // This error is specific to stale read and the target replica is randomly selected. If the request is sent + // to the leader, the data must be ready, so we don't backoff here. + if regionErr.GetDataIsNotReady() != nil { + logutil.BgLogger().Warn("tikv reports `DataIsNotReady` retry later", + zap.Uint64("store-id", ctx.Store.storeID), + zap.Uint64("peer-id", regionErr.GetDataIsNotReady().GetPeerId()), + zap.Uint64("region-id", regionErr.GetDataIsNotReady().GetRegionId()), + zap.Uint64("safe-ts", regionErr.GetDataIsNotReady().GetSafeTs()), + zap.Stringer("ctx", ctx)) if seed != nil { - logutil.BgLogger().Debug("tikv reports `RegionNotFound` in follow-reader", - zap.Stringer("ctx", ctx), zap.Uint32("seed", *seed)) *seed = *seed + 1 } - s.regionCache.InvalidateCachedRegion(ctx.Region) - return false, nil + return true, nil } logutil.BgLogger().Debug("tikv reports region failed", diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 5f880d26c7f67..2c0c39cc6f5bd 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -190,7 +190,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestStoreTokenLimit(c *C) { // Test whether the Stale Read request will retry the leader or other peers on error. func (s *testRegionRequestToThreeStoresSuite) TestStaleReadRetry(c *C) { - var seed uint32 = 0 + var seed uint32 req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, &kvrpcpb.GetRequest{}, kv.ReplicaReadMixed, &seed) req.EnableStaleRead() @@ -1140,14 +1140,14 @@ func (s *testRegionRequestToThreeStoresSuite) TestSendReqWithReplicaSelector(c * c.Assert(bo.GetTotalBackoffTimes(), Equals, maxReplicaAttempt+2) s.cluster.StartStore(s.storeIDs[0]) - // Verify that retry the same replica when meets ServerIsBusy/MaxTimestampNotSynced/ReadIndexNotReady/ProposalInMergingMode/DataIsNotReady. + // Verify that retry the same replica when meets ServerIsBusy/MaxTimestampNotSynced/ReadIndexNotReady/ProposalInMergingMode. for _, regionErr := range []*errorpb.Error{ // ServerIsBusy takes too much time to test. // {ServerIsBusy: &errorpb.ServerIsBusy{}}, {MaxTimestampNotSynced: &errorpb.MaxTimestampNotSynced{}}, {ReadIndexNotReady: &errorpb.ReadIndexNotReady{}}, {ProposalInMergingMode: &errorpb.ProposalInMergingMode{}}, - {DataIsNotReady: &errorpb.DataIsNotReady{}}} { + } { func() { oc := sender.client defer func() { diff --git a/store/tikv/retry/config.go b/store/tikv/retry/config.go index 6eb93eee81366..7695fe046a2c8 100644 --- a/store/tikv/retry/config.go +++ b/store/tikv/retry/config.go @@ -89,6 +89,7 @@ var ( BoPDRPC = NewConfig("pdRPC", &metrics.BackoffHistogramPD, NewBackoffFnCfg(500, 3000, EqualJitter), tikverr.NewErrPDServerTimeout("")) // change base time to 2ms, because it may recover soon. BoRegionMiss = NewConfig("regionMiss", &metrics.BackoffHistogramRegionMiss, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrRegionUnavailable) + BoRegionScheduling = NewConfig("regionScheduling", &metrics.BackoffHistogramRegionScheduling, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrRegionUnavailable) BoTiKVServerBusy = NewConfig("tikvServerBusy", &metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiKVServerBusy) BoTiFlashServerBusy = NewConfig("tiflashServerBusy", &metrics.BackoffHistogramServerBusy, NewBackoffFnCfg(2000, 10000, EqualJitter), tikverr.ErrTiFlashServerBusy) BoTxnNotFound = NewConfig("txnNotFound", &metrics.BackoffHistogramEmpty, NewBackoffFnCfg(2, 500, NoJitter), tikverr.ErrResolveLockTimeout) From a8ad9b7d3aa53d0181422d2b22fa8d57b7d56db1 Mon Sep 17 00:00:00 2001 From: tangenta Date: Sat, 5 Jun 2021 15:00:27 +0800 Subject: [PATCH 308/343] ddl: fix changing index prefixed column type to non-prefixed (#25004) --- ddl/column.go | 9 ++++++-- ddl/column_type_change_test.go | 39 ++++++++++++++++++++++++++++++++++ ddl/ddl_api.go | 8 ++++++- 3 files changed, 53 insertions(+), 3 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 41086b2d63497..757f914d62ec6 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -884,8 +884,13 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in newIdxInfo := idxInfo.Clone() newIdxInfo.Name = model.NewCIStr(genChangingIndexUniqueName(tblInfo, idxInfo)) newIdxInfo.ID = allocateIndexID(tblInfo) - newIdxInfo.Columns[offsets[i]].Name = newColName - newIdxInfo.Columns[offsets[i]].Offset = jobParam.changingCol.Offset + newIdxChangingCol := newIdxInfo.Columns[offsets[i]] + newIdxChangingCol.Name = newColName + newIdxChangingCol.Offset = jobParam.changingCol.Offset + canPrefix := types.IsTypePrefixable(jobParam.changingCol.Tp) + if !canPrefix || (canPrefix && jobParam.changingCol.Flen < newIdxChangingCol.Length) { + newIdxChangingCol.Length = types.UnspecifiedLength + } jobParam.changingIdxs = append(jobParam.changingIdxs, newIdxInfo) } tblInfo.Indices = append(tblInfo.Indices, jobParam.changingIdxs...) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 78512a10515cf..625917cb7e693 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -17,6 +17,7 @@ import ( "context" "errors" "strconv" + "strings" "sync" "time" @@ -2007,3 +2008,41 @@ func (s *testColumnTypeChangeSuite) TestCTCCastBitToBinary(c *C) { tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `a` varbinary(248) DEFAULT 't'\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_general_ci")) tk.MustQuery("select * from t").Check(testkit.Rows("4047")) } + +func (s *testColumnTypeChangeSuite) TestChangePrefixedIndexColumnToNonPrefixOne(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.Se.GetSessionVars().EnableChangeColumnType = true + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a text, unique index idx(a(2)));") + tk.MustExec("alter table t modify column a int;") + showCreateTable := tk.MustQuery("show create table t").Rows()[0][1].(string) + c.Assert(strings.Contains(showCreateTable, "UNIQUE KEY `idx` (`a`)"), IsTrue, + Commentf("%s", showCreateTable)) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a char(255), unique index idx(a(2)));") + tk.MustExec("alter table t modify column a float;") + showCreateTable = tk.MustQuery("show create table t").Rows()[0][1].(string) + c.Assert(strings.Contains(showCreateTable, "UNIQUE KEY `idx` (`a`)"), IsTrue, + Commentf("%s", showCreateTable)) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a char(255), b text, unique index idx(a(2), b(10)));") + tk.MustExec("alter table t modify column b int;") + showCreateTable = tk.MustQuery("show create table t").Rows()[0][1].(string) + c.Assert(strings.Contains(showCreateTable, "UNIQUE KEY `idx` (`a`(2),`b`)"), IsTrue, + Commentf("%s", showCreateTable)) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a char(250), unique key idx(a(10)));") + tk.MustExec("alter table t modify a char(9);") + showCreateTable = tk.MustQuery("show create table t").Rows()[0][1].(string) + c.Assert(strings.Contains(showCreateTable, "UNIQUE KEY `idx` (`a`)"), IsTrue, + Commentf("%s", showCreateTable)) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a varchar(700), key(a(700)));") + tk.MustGetErrCode("alter table t change column a a tinytext;", mysql.ErrBlobKeyWithoutLength) +} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 8458ef70e3f6f..11c5141572075 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4039,7 +4039,13 @@ func checkIndexInModifiableColumns(columns []*model.ColumnInfo, idxColumns []*mo return errKeyColumnDoesNotExits.GenWithStack("column does not exist: %s", ic.Name) } - if err := checkIndexColumn(col, ic.Length); err != nil { + prefixLength := types.UnspecifiedLength + if types.IsTypePrefixable(col.FieldType.Tp) && col.FieldType.Flen > ic.Length { + // When the index column is changed, prefix length is only valid + // if the type is still prefixable and larger than old prefix length. + prefixLength = ic.Length + } + if err := checkIndexColumn(col, prefixLength); err != nil { return err } } From 11716e7f4a4c7446679a4f8eb4224ec045c7e849 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Sun, 6 Jun 2021 00:34:27 +0800 Subject: [PATCH 309/343] *: use full sampling as default analyze (#25132) --- cmd/explaintest/r/subquery.result | 6 +- executor/analyze.go | 5 +- executor/analyze_test.go | 48 +++-- executor/builder.go | 2 +- executor/show_stats_test.go | 28 ++- executor/simple_test.go | 2 +- planner/core/cbo_test.go | 1 + planner/core/find_best_task.go | 10 +- planner/core/integration_test.go | 8 +- planner/core/planbuilder.go | 34 ++-- planner/core/testdata/analyze_suite_out.json | 4 +- .../core/testdata/integration_suite_out.json | 4 +- sessionctx/variable/sysvar.go | 4 +- sessionctx/variable/tidb_vars.go | 2 +- statistics/handle/ddl_test.go | 2 + statistics/handle/dump_test.go | 4 +- statistics/handle/handle.go | 2 +- statistics/handle/handle_test.go | 165 +++++++++--------- statistics/handle/update.go | 1 - statistics/handle/update_test.go | 5 +- statistics/histogram.go | 7 +- statistics/integration_test.go | 19 +- statistics/selectivity_test.go | 4 +- statistics/testdata/stats_suite_out.json | 99 ++++++----- util/ranger/testdata/ranger_suite_out.json | 40 ++--- util/sqlexec/restricted_sql_executor.go | 5 - 26 files changed, 283 insertions(+), 228 deletions(-) diff --git a/cmd/explaintest/r/subquery.result b/cmd/explaintest/r/subquery.result index 2bf71b91088cb..84bac87bb1d23 100644 --- a/cmd/explaintest/r/subquery.result +++ b/cmd/explaintest/r/subquery.result @@ -22,9 +22,9 @@ Projection 5.00 root Column#22 ├─TableReader(Build) 5.00 root data:TableFullScan │ └─TableFullScan 5.00 cop[tikv] table:t keep order:false └─StreamAgg(Probe) 1.00 root funcs:count(1)->Column#21 - └─IndexJoin 0.50 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a) - ├─IndexReader(Build) 1.00 root index:IndexRangeScan - │ └─IndexRangeScan 1.00 cop[tikv] table:s, index:idx(b, c, d) range: decided by [eq(test.t.b, 1) eq(test.t.c, 1) eq(test.t.d, test.t.a)], keep order:false + └─IndexJoin 0.22 root inner join, inner:TableReader, outer key:test.t.a, inner key:test.t.a, equal cond:eq(test.t.a, test.t.a) + ├─IndexReader(Build) 0.45 root index:IndexRangeScan + │ └─IndexRangeScan 0.45 cop[tikv] table:s, index:idx(b, c, d) range: decided by [eq(test.t.b, 1) eq(test.t.c, 1) eq(test.t.d, test.t.a)], keep order:false └─TableReader(Probe) 1.00 root data:TableRangeScan └─TableRangeScan 1.00 cop[tikv] table:t1 range: decided by [test.t.a], keep order:false drop table if exists t; diff --git a/executor/analyze.go b/executor/analyze.go index 5a2ed6e28e212..f905c2c3f5b84 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -164,7 +164,7 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { } } var err1 error - if result.StatsVer == statistics.Version3 { + if result.StatsVer == statistics.Version2 { err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, nil, result.TopNs[i], result.Fms[i], result.StatsVer, 1, result.TableID.IsPartitionTable() && needGlobalStats) } else { err1 = statsHandle.SaveStatsToStorage(statisticsID, result.Count, result.IsIndex, hg, result.Cms[i], result.TopNs[i], result.Fms[i], result.StatsVer, 1, result.TableID.IsPartitionTable() && needGlobalStats) @@ -583,7 +583,7 @@ func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) []analyzeResult { ranges = ranger.FullIntRange(false) } collExtStats := colExec.ctx.GetSessionVars().EnableExtendedStats - if colExec.StatsVersion == statistics.Version3 { + if colExec.StatsVersion == statistics.Version2 { specialIndexes := make([]*model.IndexInfo, 0, len(colExec.indexes)) specialIndexesOffsets := make([]int, 0, len(colExec.indexes)) for i, idx := range colExec.indexes { @@ -937,6 +937,7 @@ func (e *AnalyzeColumnsExec) buildSamplingStats( for i := 0; i < statsConcurrency; i++ { go e.subBuildWorker(buildResultChan, buildTaskChan, hists, topns, sampleCollectors, i == 0) } + for i, col := range e.colsInfo { buildTaskChan <- &samplingBuildTask{ id: col.ID, diff --git a/executor/analyze_test.go b/executor/analyze_test.go index bc5e2c9ba868c..53e7ab8bd2908 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -57,6 +57,7 @@ func (s *testSuite1) TestAnalyzePartition(c *C) { testkit.WithPruneMode(tk, variable.Static, func() { tk.MustExec("use test") tk.MustExec("drop table if exists t") + tk.MustExec("set @@tidb_analyze_version=2") createTable := `CREATE TABLE t (a int, b int, c varchar(10), primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( PARTITION p0 VALUES LESS THAN (6), @@ -84,10 +85,10 @@ PARTITION BY RANGE ( a ) ( c.Assert(len(statsTbl.Columns), Equals, 3) c.Assert(len(statsTbl.Indices), Equals, 1) for _, col := range statsTbl.Columns { - c.Assert(col.Len(), Greater, 0) + c.Assert(col.Len()+col.Num(), Greater, 0) } for _, idx := range statsTbl.Indices { - c.Assert(idx.Len(), Greater, 0) + c.Assert(idx.Len()+idx.Num(), Greater, 0) } } @@ -176,6 +177,7 @@ func (s *testSuite1) TestAnalyzeParameters(c *C) { tk.MustExec("insert into t values (19), (19), (19)") tk.MustExec("set @@tidb_enable_fast_analyze = 1") + tk.MustExec("set @@tidb_analyze_version = 1") tk.MustExec("analyze table t with 30 samples") is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -259,7 +261,7 @@ func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { tk.MustExec("create table t(a int, b int, index idx(a, b))") tk.MustExec("insert into t values(1, 1), (1, 1), (1, 2), (1, 2)") tk.MustExec("set @@session.tidb_analyze_version=2") - tk.MustExec("analyze table t with 10 cmsketch width") + tk.MustExec("analyze table t") is := tk.Se.(sessionctx.Context).GetInfoSchema().(infoschema.InfoSchema) table, err := is.TableByName(model.NewCIStr("test_index_extract_topn"), model.NewCIStr("t")) @@ -268,7 +270,6 @@ func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { tbl := dom.StatsHandle().GetTableStats(tableInfo) // Construct TopN, should be (1, 1) -> 2 and (1, 2) -> 2 - cms := statistics.NewCMSketch(5, 10) topn := statistics.NewTopN(2) { key1, err := codec.EncodeKey(tk.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(1), types.NewIntDatum(1)) @@ -277,19 +278,11 @@ func (s *testSuite1) TestAnalyzeIndexExtractTopN(c *C) { key2, err := codec.EncodeKey(tk.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(1), types.NewIntDatum(2)) c.Assert(err, IsNil) topn.AppendTopN(key2, 2) - prefixKey, err := codec.EncodeKey(tk.Se.GetSessionVars().StmtCtx, nil, types.NewIntDatum(1)) - c.Assert(err, IsNil) - cms.InsertBytes(prefixKey) - cms.InsertBytes(prefixKey) - cms.InsertBytes(prefixKey) - cms.InsertBytes(prefixKey) - cms.CalcDefaultValForAnalyze(2) } for _, idx := range tbl.Indices { ok, err := checkHistogram(tk.Se.GetSessionVars().StmtCtx, &idx.Histogram) c.Assert(err, IsNil) c.Assert(ok, IsTrue) - c.Assert(idx.CMSketch.Equal(cms), IsTrue) c.Assert(idx.TopN.Equal(topn), IsTrue) } } @@ -422,6 +415,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) { tk.MustExec("create table t(a int primary key, b int, c char(10), index index_b(b))") tk.MustExec("set @@session.tidb_enable_fast_analyze=1") tk.MustExec("set @@session.tidb_build_stats_concurrency=1") + tk.MustExec("set @@tidb_analyze_version = 1") // Should not panic. tk.MustExec("analyze table t") tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -537,6 +531,7 @@ func (s *testSuite1) TestIssue15993(c *C) { tk.MustExec("drop table if exists t0") tk.MustExec("CREATE TABLE t0(c0 INT PRIMARY KEY);") tk.MustExec("set @@tidb_enable_fast_analyze=1;") + tk.MustExec("set @@tidb_analyze_version = 1") tk.MustExec("ANALYZE TABLE t0 INDEX PRIMARY;") } @@ -547,6 +542,7 @@ func (s *testSuite1) TestIssue15751(c *C) { tk.MustExec("CREATE TABLE t0(c0 INT, c1 INT, PRIMARY KEY(c0, c1))") tk.MustExec("INSERT INTO t0 VALUES (0, 0)") tk.MustExec("set @@tidb_enable_fast_analyze=1") + tk.MustExec("set @@tidb_analyze_version = 1") tk.MustExec("ANALYZE TABLE t0") } @@ -558,6 +554,7 @@ func (s *testSuite1) TestIssue15752(c *C) { tk.MustExec("INSERT INTO t0 VALUES (0)") tk.MustExec("CREATE INDEX i0 ON t0(c0)") tk.MustExec("set @@tidb_enable_fast_analyze=1") + tk.MustExec("set @@tidb_analyze_version = 1") tk.MustExec("ANALYZE TABLE t0 INDEX i0") } @@ -567,13 +564,18 @@ func (s *testSuite1) TestAnalyzeIndex(c *C) { tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (id int, v int, primary key(id), index k(v))") tk.MustExec("insert into t1(id, v) values(1, 2), (2, 2), (3, 2), (4, 2), (5, 1), (6, 3), (7, 4)") + tk.MustExec("set @@tidb_analyze_version=1") tk.MustExec("analyze table t1 index k") c.Assert(len(tk.MustQuery("show stats_buckets where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), Greater, 0) + tk.MustExec("set @@tidb_analyze_version=default") + tk.MustExec("analyze table t1") + c.Assert(len(tk.MustQuery("show stats_topn where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), Greater, 0) func() { defer tk.MustExec("set @@session.tidb_enable_fast_analyze=0") tk.MustExec("drop stats t1") tk.MustExec("set @@session.tidb_enable_fast_analyze=1") + tk.MustExec("set @@tidb_analyze_version=1") tk.MustExec("analyze table t1 index k") c.Assert(len(tk.MustQuery("show stats_buckets where table_name = 't1' and column_name = 'k' and is_index = 1").Rows()), Greater, 1) }() @@ -582,6 +584,7 @@ func (s *testSuite1) TestAnalyzeIndex(c *C) { func (s *testSuite1) TestAnalyzeIncremental(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") + tk.MustExec("set @@tidb_analyze_version = 1") tk.Se.GetSessionVars().EnableStreaming = false s.testAnalyzeIncremental(tk, c) } @@ -728,6 +731,7 @@ func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) { c.Assert(dom.StatsHandle().Update(dom.InfoSchema()), IsNil) tk.MustExec("set @@session.tidb_enable_fast_analyze=1") tk.MustExec("set @@session.tidb_build_stats_concurrency=1") + tk.MustExec("set @@tidb_analyze_version = 1") for i := 0; i < 30; i++ { tk.MustExec(fmt.Sprintf("insert into retry_row_count values (%d)", i)) } @@ -745,6 +749,7 @@ func (s *testSuite10) TestFailedAnalyzeRequest(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, index index_b(b))") + tk.MustExec("set @@tidb_analyze_version = 1") c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/buildStatsFromResult", `return(true)`), IsNil) _, err := tk.Exec("analyze table t") c.Assert(err.Error(), Equals, "mock buildStatsFromResult error") @@ -782,7 +787,7 @@ func (s *testSuite1) TestExtractTopN(c *C) { idxItem := idxStats.TopN.TopN[0] c.Assert(idxItem.Count, Equals, uint64(11)) // The columns are: DBName, table name, column name, is index, value, count. - tk.MustQuery("show stats_topn").Sort().Check(testkit.Rows("test_extract_topn test_extract_topn b 0 0 11", + tk.MustQuery("show stats_topn where column_name in ('b', 'index_b')").Sort().Check(testkit.Rows("test_extract_topn test_extract_topn b 0 0 11", "test_extract_topn test_extract_topn b 0 1 1", "test_extract_topn test_extract_topn b 0 2 1", "test_extract_topn test_extract_topn b 0 3 1", @@ -818,6 +823,7 @@ func (s *testSuite1) TestHashInTopN(c *C) { for i := 0; i < 3; i++ { tk.MustExec("insert into t select * from t") } + tk.MustExec("set @@tidb_analyze_version = 1") // get stats of normal analyze tk.MustExec("analyze table t") is := s.dom.InfoSchema() @@ -853,6 +859,8 @@ func (s *testSuite1) TestNormalAnalyzeOnCommonHandle(c *C) { tk.MustExec("CREATE TABLE t3 (a int, b int, c int, primary key (a, b), key(c))") tk.MustExec("insert into t3 values(1,1,1), (2,2,2), (3,3,3)") + // Version2 is tested in TestStatsVer2. + tk.MustExec("set@@tidb_analyze_version=1") tk.MustExec("analyze table t1, t2, t3") tk.MustQuery(`show stats_buckets where table_name in ("t1", "t2", "t3")`).Sort().Check(testkit.Rows( @@ -931,6 +939,7 @@ func (s *testSerialSuite2) TestIssue20874(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (a char(10) collate utf8mb4_unicode_ci not null, b char(20) collate utf8mb4_general_ci not null, key idxa(a), key idxb(b))") tk.MustExec("insert into t values ('#', 'C'), ('$', 'c'), ('a', 'a')") + tk.MustExec("set @@tidb_analyze_version=1") tk.MustExec("analyze table t") tk.MustQuery("show stats_buckets where db_name = 'test' and table_name = 't'").Sort().Check(testkit.Rows( "test t a 0 0 1 1 \x02\xd2 \x02\xd2 0", @@ -955,11 +964,20 @@ func (s *testSuite1) TestAnalyzeClusteredIndexPrimary(c *C) { tk.MustExec("create table t1(a varchar(20), primary key(a))") tk.MustExec("insert into t0 values('1111')") tk.MustExec("insert into t1 values('1111')") + tk.MustExec("set @@session.tidb_analyze_version = 1") tk.MustExec("analyze table t0 index primary") tk.MustExec("analyze table t1 index primary") tk.MustQuery("show stats_buckets").Check(testkit.Rows( "test t0 PRIMARY 1 0 1 1 1111 1111 0", "test t1 PRIMARY 1 0 1 1 1111 1111 0")) + tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("analyze table t0") + tk.MustExec("analyze table t1") + tk.MustQuery("show stats_topn").Sort().Check(testkit.Rows(""+ + "test t0 PRIMARY 1 1111 1", + "test t0 a 0 1111 1", + "test t1 PRIMARY 1 1111 1", + "test t1 a 0 1111 1")) } func (s *testSuite1) TestAnalyzeFullSamplingOnIndexWithVirtualColumnOrPrefixColumn(c *C) { @@ -968,7 +986,7 @@ func (s *testSuite1) TestAnalyzeFullSamplingOnIndexWithVirtualColumnOrPrefixColu tk.MustExec("drop table if exists sampling_index_virtual_col") tk.MustExec("create table sampling_index_virtual_col(a int, b int as (a+1), index idx(b))") tk.MustExec("insert into sampling_index_virtual_col (a) values (1), (2), (null), (3), (4), (null), (5), (5), (5), (5)") - tk.MustExec("set @@session.tidb_analyze_version = 3") + tk.MustExec("set @@session.tidb_analyze_version = 2") tk.MustExec("analyze table sampling_index_virtual_col with 1 topn") tk.MustQuery("show stats_buckets where table_name = 'sampling_index_virtual_col' and column_name = 'idx'").Check(testkit.Rows( "test sampling_index_virtual_col idx 1 0 1 1 2 2 0", @@ -994,7 +1012,7 @@ func (s *testSuite1) TestAnalyzeFullSamplingOnIndexWithVirtualColumnOrPrefixColu func (s *testSuite2) TestAnalyzeSamplingWorkPanic(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.MustExec("set @@session.tidb_analyze_version = 3") + tk.MustExec("set @@session.tidb_analyze_version = 2") tk.MustExec("create table t(a int)") tk.MustExec("insert into t values(1), (2), (3), (4), (5), (6), (7), (8), (9), (10), (11), (12)") tk.MustExec("split table t between (-9223372036854775808) and (9223372036854775807) regions 12") diff --git a/executor/builder.go b/executor/builder.go index 6886dafc13d3c..4a1b45965a531 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -2102,7 +2102,7 @@ func (b *executorBuilder) buildAnalyzeSamplingPushdown(task plannercore.AnalyzeC } func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeColumnsTask, opts map[ast.AnalyzeOptionType]uint64, autoAnalyze string, schemaForVirtualColEval *expression.Schema) *analyzeTask { - if task.StatsVersion == statistics.Version3 { + if task.StatsVersion == statistics.Version2 { return b.buildAnalyzeSamplingPushdown(task, opts, autoAnalyze, schemaForVirtualColEval) } cols := task.ColsInfo diff --git a/executor/show_stats_test.go b/executor/show_stats_test.go index e9e0ed0ebe82a..8dddc2151a7fc 100644 --- a/executor/show_stats_test.go +++ b/executor/show_stats_test.go @@ -75,10 +75,12 @@ func (s *testShowStatsSuite) TestShowStatsBuckets(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") + // Simple behavior testing. Version=1 is enough. + tk.MustExec("set @@tidb_analyze_version=1") tk.MustExec("create table t (a int, b int)") tk.MustExec("create index idx on t(a,b)") tk.MustExec("insert into t values (1,1)") - tk.MustExec("analyze table t") + tk.MustExec("analyze table t with 0 topn") result := tk.MustQuery("show stats_buckets").Sort() result.Check(testkit.Rows("test t a 0 0 1 1 1 1 0", "test t b 0 0 1 1 1 1 0", "test t idx 1 0 1 1 (1, 1) (1, 1) 0")) result = tk.MustQuery("show stats_buckets where column_name = 'idx'") @@ -87,7 +89,7 @@ func (s *testShowStatsSuite) TestShowStatsBuckets(c *C) { tk.MustExec("drop table t") tk.MustExec("create table t (`a` datetime, `b` int, key `idx`(`a`, `b`))") tk.MustExec("insert into t values (\"2020-01-01\", 1)") - tk.MustExec("analyze table t") + tk.MustExec("analyze table t with 0 topn") result = tk.MustQuery("show stats_buckets").Sort() result.Check(testkit.Rows("test t a 0 0 1 1 2020-01-01 00:00:00 2020-01-01 00:00:00 0", "test t b 0 0 1 1 1 1 0", "test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1) 0")) result = tk.MustQuery("show stats_buckets where column_name = 'idx'") @@ -96,7 +98,7 @@ func (s *testShowStatsSuite) TestShowStatsBuckets(c *C) { tk.MustExec("drop table t") tk.MustExec("create table t (`a` date, `b` int, key `idx`(`a`, `b`))") tk.MustExec("insert into t values (\"2020-01-01\", 1)") - tk.MustExec("analyze table t") + tk.MustExec("analyze table t with 0 topn") result = tk.MustQuery("show stats_buckets").Sort() result.Check(testkit.Rows("test t a 0 0 1 1 2020-01-01 2020-01-01 0", "test t b 0 0 1 1 1 1 0", "test t idx 1 0 1 1 (2020-01-01, 1) (2020-01-01, 1) 0")) result = tk.MustQuery("show stats_buckets where column_name = 'idx'") @@ -105,7 +107,7 @@ func (s *testShowStatsSuite) TestShowStatsBuckets(c *C) { tk.MustExec("drop table t") tk.MustExec("create table t (`a` timestamp, `b` int, key `idx`(`a`, `b`))") tk.MustExec("insert into t values (\"2020-01-01\", 1)") - tk.MustExec("analyze table t") + tk.MustExec("analyze table t with 0 topn") result = tk.MustQuery("show stats_buckets").Sort() result.Check(testkit.Rows("test t a 0 0 1 1 2020-01-01 00:00:00 2020-01-01 00:00:00 0", "test t b 0 0 1 1 1 1 0", "test t idx 1 0 1 1 (2020-01-01 00:00:00, 1) (2020-01-01 00:00:00, 1) 0")) result = tk.MustQuery("show stats_buckets where column_name = 'idx'") @@ -118,6 +120,7 @@ func (s *testShowStatsSuite) TestShowStatsHasNullValue(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, index idx(a))") tk.MustExec("insert into t values(NULL)") + tk.MustExec("set @@session.tidb_analyze_version=1") tk.MustExec("analyze table t") // Null values are excluded from histogram for single-column index. tk.MustQuery("show stats_buckets").Check(testkit.Rows()) @@ -177,6 +180,8 @@ func (s *testShowStatsSuite) TestShowPartitionStats(c *C) { tk := testkit.NewTestKit(c, s.store) testkit.WithPruneMode(tk, variable.Static, func() { tk.MustExec("set @@session.tidb_enable_table_partition=1") + // Version2 is tested in TestGlobalStatsData1/2/3 and TestAnalyzeGlobalStatsWithOpts. + tk.MustExec("set @@session.tidb_analyze_version=1") tk.MustExec("use test") tk.MustExec("drop table if exists t") createTable := `CREATE TABLE t (a int, b int, primary key(a), index idx(b)) @@ -215,6 +220,7 @@ func (s *testShowStatsSuite) TestShowAnalyzeStatus(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b int, primary key(a), index idx(b))") tk.MustExec(`insert into t values (1, 1), (2, 2)`) + tk.MustExec("set @@tidb_analyze_version=1") tk.MustExec("analyze table t") result := tk.MustQuery("show analyze status").Sort() @@ -237,6 +243,20 @@ func (s *testShowStatsSuite) TestShowAnalyzeStatus(c *C) { c.Assert(result.Rows()[1][5], NotNil) c.Assert(result.Rows()[1][6], NotNil) c.Assert(result.Rows()[1][7], Equals, "finished") + + statistics.ClearHistoryJobs() + tk.MustExec("set @@tidb_analyze_version=2") + tk.MustExec("analyze table t") + result = tk.MustQuery("show analyze status").Sort() + c.Assert(len(result.Rows()), Equals, 1) + c.Assert(result.Rows()[0][0], Equals, "test") + c.Assert(result.Rows()[0][1], Equals, "t") + c.Assert(result.Rows()[0][2], Equals, "") + c.Assert(result.Rows()[0][3], Equals, "analyze table") + c.Assert(result.Rows()[0][4], Equals, "2") + c.Assert(result.Rows()[0][5], NotNil) + c.Assert(result.Rows()[0][6], NotNil) + c.Assert(result.Rows()[0][7], Equals, "finished") } func (s *testShowStatsSuite) TestShowStatusSnapshot(c *C) { diff --git a/executor/simple_test.go b/executor/simple_test.go index dc464c38b83c0..425e7d5204731 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -697,7 +697,7 @@ func (s *testSuite3) TestDropStatsFromKV(c *C) { tk.MustExec(`insert into t values("1","1"),("2","2"),("3","3"),("4","4")`) tk.MustExec("insert into t select * from t") tk.MustExec("insert into t select * from t") - tk.MustExec("analyze table t") + tk.MustExec("analyze table t with 2 topn") tblID := tk.MustQuery(`select tidb_table_id from information_schema.tables where table_name = "t" and table_schema = "test"`).Rows()[0][0].(string) tk.MustQuery("select modify_count, count from mysql.stats_meta where table_id = " + tblID).Check( testkit.Rows("0 16")) diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index ffa0b7ccde75e..84a42702e945e 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -578,6 +578,7 @@ func (s *testAnalyzeSuite) TestInconsistentEstimation(c *C) { for i := 0; i < 10; i++ { tk.MustExec("insert into t values (5,5,5), (10,10,10)") } + tk.MustExec("set @@tidb_analyze_version=1") tk.MustExec("analyze table t with 2 buckets") // Force using the histogram to estimate. tk.MustExec("update mysql.stats_histograms set stats_ver = 0") diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index a9d82d308acc5..55610cdf293e2 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1305,7 +1305,7 @@ func getMostCorrCol4Handle(exprs []expression.Expression, histColl *statistics.T } // getColumnRangeCounts estimates row count for each range respectively. -func getColumnRangeCounts(sc *stmtctx.StatementContext, colID int64, ranges []*ranger.Range, histColl *statistics.Table, idxID int64) ([]float64, bool) { +func getColumnRangeCounts(sc *stmtctx.StatementContext, colID int64, ranges []*ranger.Range, histColl *statistics.HistColl, idxID int64) ([]float64, bool) { var err error var count float64 rangeCounts := make([]float64, len(ranges)) @@ -1383,7 +1383,7 @@ func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, conds []expre if col == nil || len(path.AccessConds) > 0 { return 0, false, corr } - colInfoID, colID := col.ID, col.UniqueID + colID := col.UniqueID if corr < 0 { desc = !desc } @@ -1400,7 +1400,7 @@ func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, conds []expre if !idxExists { idxID = -1 } - rangeCounts, ok := getColumnRangeCounts(sc, colInfoID, ranges, ds.statisticTable, idxID) + rangeCounts, ok := getColumnRangeCounts(sc, colID, ranges, ds.tableStats.HistColl, idxID) if !ok { return 0, false, corr } @@ -1410,9 +1410,9 @@ func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, conds []expre } var rangeCount float64 if idxExists { - rangeCount, err = ds.statisticTable.GetRowCountByIndexRanges(sc, idxID, convertedRanges) + rangeCount, err = ds.tableStats.HistColl.GetRowCountByIndexRanges(sc, idxID, convertedRanges) } else { - rangeCount, err = ds.statisticTable.GetRowCountByColumnRanges(sc, colInfoID, convertedRanges) + rangeCount, err = ds.tableStats.HistColl.GetRowCountByColumnRanges(sc, colID, convertedRanges) } if err != nil { return 0, false, corr diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 700b9fd8d9c23..b9329eb741bf9 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3718,13 +3718,13 @@ func (s *testIntegrationSuite) TestIssue24281(c *C) { "UNION select 1 as v1, 2 as v2") } -func (s *testIntegrationSuite) TestIncrementalAnalyzeStatsVer3(c *C) { +func (s *testIntegrationSuite) TestIncrementalAnalyzeStatsVer2(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, index idx_b(b))") tk.MustExec("insert into t values(1,1),(2,2),(3,3)") - tk.MustExec("set @@session.tidb_analyze_version = 3") + tk.MustExec("set @@session.tidb_analyze_version = 2") tk.MustExec("analyze table t") is := tk.Se.GetInfoSchema().(infoschema.InfoSchema) tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -3736,8 +3736,8 @@ func (s *testIntegrationSuite) TestIncrementalAnalyzeStatsVer3(c *C) { tk.MustExec("insert into t values(4,4),(5,5),(6,6)") tk.MustExec("analyze incremental table t index idx_b") c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 2) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, "The version 3 would collect all statistics not only the selected indexes") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[1].Err.Error(), Equals, "The version 3 stats would ignore the INCREMENTAL keyword and do full sampling") + c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, "The version 2 would collect all statistics not only the selected indexes") + c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[1].Err.Error(), Equals, "The version 2 stats would ignore the INCREMENTAL keyword and do full sampling") rows = tk.MustQuery(fmt.Sprintf("select distinct_count from mysql.stats_histograms where table_id = %d and is_index = 1", tblID)).Rows() c.Assert(len(rows), Equals, 1) c.Assert(rows[0][0], Equals, "6") diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 81b9f14d55bb2..6b4d4118379f8 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1685,7 +1685,7 @@ func (b *PlanBuilder) buildAnalyzeFullSamplingTask( idxInfos = append(idxInfos, idx) } if as.Incremental { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 3 stats would ignore the INCREMENTAL keyword and do full sampling")) + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 2 stats would ignore the INCREMENTAL keyword and do full sampling")) } for i, id := range physicalIDs { if id == tbl.TableInfo.ID { @@ -1743,7 +1743,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt, opts map[ast.A "If you want to switch to version 2 statistics, please first disable query feedback by setting feedback-probability to 0.0 in the config file.")) } } - if version == statistics.Version3 { + if version == statistics.Version2 { p.ColTasks = b.buildAnalyzeFullSamplingTask(as, p.ColTasks, physicalIDs, names, tbl, version) continue } @@ -1818,8 +1818,8 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt, opts map[ast.A } b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) } - if version == statistics.Version3 { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 3 would collect all statistics not only the selected indexes")) + if version == statistics.Version2 { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 2 would collect all statistics not only the selected indexes")) return b.buildAnalyzeTable(as, opts, version) } for _, idxName := range as.IndexNames { @@ -1883,8 +1883,8 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt, opts map[as } b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) } - if version == statistics.Version3 { - b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 3 would collect all statistics not only the selected indexes")) + if version == statistics.Version2 { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("The version 2 would collect all statistics not only the selected indexes")) return b.buildAnalyzeTable(as, opts, version) } for _, idx := range tblInfo.Indices { @@ -1944,10 +1944,24 @@ var analyzeOptionDefault = map[ast.AnalyzeOptionType]uint64{ ast.AnalyzeOptNumSamples: 10000, } -func handleAnalyzeOptions(opts []ast.AnalyzeOpt) (map[ast.AnalyzeOptionType]uint64, error) { +var analyzeOptionDefaultV2 = map[ast.AnalyzeOptionType]uint64{ + ast.AnalyzeOptNumBuckets: 256, + ast.AnalyzeOptNumTopN: 500, + ast.AnalyzeOptCMSketchWidth: 2048, + ast.AnalyzeOptCMSketchDepth: 5, + ast.AnalyzeOptNumSamples: 100000, +} + +func handleAnalyzeOptions(opts []ast.AnalyzeOpt, statsVer int) (map[ast.AnalyzeOptionType]uint64, error) { optMap := make(map[ast.AnalyzeOptionType]uint64, len(analyzeOptionDefault)) - for key, val := range analyzeOptionDefault { - optMap[key] = val + if statsVer == statistics.Version1 { + for key, val := range analyzeOptionDefault { + optMap[key] = val + } + } else { + for key, val := range analyzeOptionDefaultV2 { + optMap[key] = val + } } for _, opt := range opts { if opt.Type == ast.AnalyzeOptNumTopN { @@ -1986,7 +2000,7 @@ func (b *PlanBuilder) buildAnalyze(as *ast.AnalyzeTableStmt) (Plan, error) { b.visitInfo = appendVisitInfo(b.visitInfo, mysql.InsertPriv, tbl.Schema.O, tbl.Name.O, "", insertErr) b.visitInfo = appendVisitInfo(b.visitInfo, mysql.SelectPriv, tbl.Schema.O, tbl.Name.O, "", selectErr) } - opts, err := handleAnalyzeOptions(as.AnalyzeOpts) + opts, err := handleAnalyzeOptions(as.AnalyzeOpts, statsVersion) if err != nil { return nil, err } diff --git a/planner/core/testdata/analyze_suite_out.json b/planner/core/testdata/analyze_suite_out.json index bdd31cd56f8cd..cb0dd2137c515 100644 --- a/planner/core/testdata/analyze_suite_out.json +++ b/planner/core/testdata/analyze_suite_out.json @@ -435,14 +435,14 @@ { "Name": "TestAnalyze", "Cases": [ - "Analyze{Index(a),Table(a, b)}", + "Analyze{Table(_tidb_rowid, a, b, _tidb_rowid)}", "TableReader(Table(t)->Sel([le(test.t.a, 2)]))", "IndexReader(Index(t.b)[[-inf,2)])", "TableReader(Table(t)->Sel([eq(test.t.a, 1) le(test.t.b, 2)]))", "TableReader(Table(t1)->Sel([le(test.t1.a, 2)]))", "IndexLookUp(Index(t1.a)[[1,1]], Table(t1)->Sel([le(test.t1.b, 2)]))", "TableReader(Table(t2)->Sel([le(test.t2.a, 2)]))", - "Analyze{Index(a),Index(b)}", + "Analyze{Table(_tidb_rowid, a, b, _tidb_rowid)}", "PartitionUnionAll{TableReader(Table(t4)->Sel([le(test.t4.a, 2)]))->TableReader(Table(t4)->Sel([le(test.t4.a, 2)]))}", "PartitionUnionAll{IndexReader(Index(t4.b)[[-inf,2)])->IndexReader(Index(t4.b)[[-inf,2)])}", "TableReader(Table(t4)->Sel([eq(test.t4.a, 1) le(test.t4.b, 2)]))" diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 72d5c968d7468..81458b413da90 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -978,8 +978,8 @@ { "SQL": "select * from t1 where t1.a = 1 and t1.b < \"333\"", "Plan": [ - "TableReader 0.67 root data:TableRangeScan", - "└─TableRangeScan 0.67 cop[tikv] table:t1 range:[1 -inf,1 \"333\"), keep order:false" + "TableReader 0.82 root data:TableRangeScan", + "└─TableRangeScan 0.82 cop[tikv] table:t1 range:[1 -inf,1 \"333\"), keep order:false" ], "Res": [ "1 111 1.1000000000 11" diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 4b4688727a1db..4c4bd38b14d5b 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1625,8 +1625,8 @@ var defaultSysVars = []*SysVar{ s.GuaranteeLinearizability = TiDBOptOn(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Hidden: true, Type: TypeInt, MinValue: 1, MaxValue: 3, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { - if normalizedValue == "2" && FeedbackProbability.Load() > 0 { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBAnalyzeVersion, Value: strconv.Itoa(DefTiDBAnalyzeVersion), Hidden: false, Type: TypeInt, MinValue: 1, MaxValue: 2, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + if normalizedValue == "2" && FeedbackProbability != nil && FeedbackProbability.Load() > 0 { var original string var err error if scope == ScopeGlobal { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index d62e2a0f76809..e93414bc5e26e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -698,7 +698,7 @@ const ( DefTiDBEnableAsyncCommit = false DefTiDBEnable1PC = false DefTiDBGuaranteeLinearizability = true - DefTiDBAnalyzeVersion = 1 + DefTiDBAnalyzeVersion = 2 DefTiDBEnableIndexMergeJoin = false DefTiDBTrackAggregateMemoryUsage = true DefTiDBEnableExchangePartition = false diff --git a/statistics/handle/ddl_test.go b/statistics/handle/ddl_test.go index c62e80d372766..6bdd897270b8d 100644 --- a/statistics/handle/ddl_test.go +++ b/statistics/handle/ddl_test.go @@ -179,6 +179,8 @@ func (s *testStatsSuite) TestDDLHistogram(c *C) { rs := testKit.MustQuery("select count(*) from mysql.stats_histograms where table_id = ? and hist_id = 1 and is_index =1", tableInfo.ID) rs.Check(testkit.Rows("1")) rs = testKit.MustQuery("select count(*) from mysql.stats_buckets where table_id = ? and hist_id = 1 and is_index = 1", tableInfo.ID) + rs.Check(testkit.Rows("0")) + rs = testKit.MustQuery("select count(*) from mysql.stats_top_n where table_id = ? and hist_id = 1 and is_index = 1", tableInfo.ID) rs.Check(testkit.Rows("2")) } diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index 58411e1669a60..f0c7c2fb09110 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -287,10 +287,10 @@ func (s *testStatsSuite) TestDumpExtendedStats(c *C) { assertTableEqual(c, loadTblInStorage, tbl) } -func (s *testStatsSuite) TestDumpVer3Stats(c *C) { +func (s *testStatsSuite) TestDumpVer2Stats(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) - tk.MustExec("set @@tidb_analyze_version = 3") + tk.MustExec("set @@tidb_analyze_version = 2") tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b varchar(10))") diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 8d4ca1bc678bb..4279d2bd9b18c 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -961,7 +961,7 @@ func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg defer h.mu.Unlock() ctx := context.TODO() exec := h.mu.ctx.(sqlexec.SQLExecutor) - _, err = exec.ExecuteInternal(ctx, "begin") + _, err = exec.ExecuteInternal(ctx, "begin pessimistic") if err != nil { return errors.Trace(err) } diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 37d3d18479e7c..3a2ee051680ea 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/testkit" ) @@ -279,7 +280,14 @@ func (s *testStatsSuite) TestColumnIDs(c *C) { tableInfo := tbl.Meta() statsTbl := do.StatsHandle().GetTableStats(tableInfo) sc := new(stmtctx.StatementContext) - count := statsTbl.ColumnLessRowCount(sc, types.NewDatum(2), tableInfo.Columns[0].ID) + ran := &ranger.Range{ + LowVal: []types.Datum{types.MinNotNullDatum()}, + HighVal: []types.Datum{types.NewIntDatum(2)}, + LowExclude: false, + HighExclude: true, + } + count, err := statsTbl.GetRowCountByColumnRanges(sc, tableInfo.Columns[0].ID, []*ranger.Range{ran}) + c.Assert(err, IsNil) c.Assert(count, Equals, float64(1)) // Drop a column and the offset changed, @@ -293,7 +301,8 @@ func (s *testStatsSuite) TestColumnIDs(c *C) { tableInfo = tbl.Meta() statsTbl = do.StatsHandle().GetTableStats(tableInfo) // At that time, we should get c2's stats instead of c1's. - count = statsTbl.ColumnLessRowCount(sc, types.NewDatum(2), tableInfo.Columns[0].ID) + count, err = statsTbl.GetRowCountByColumnRanges(sc, tableInfo.Columns[0].ID, []*ranger.Range{ran}) + c.Assert(err, IsNil) c.Assert(count, Equals, 0.0) } @@ -523,7 +532,7 @@ func (s *testStatsSuite) TestInitStatsVer2(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.MustExec("set @@session.tidb_analyze_version=3") + tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("create table t(a int, b int, c int, index idx(a), index idxab(a, b))") tk.MustExec("insert into t values(1, 1, 1), (2, 2, 2), (3, 3, 3), (4, 4, 4), (4, 4, 4), (4, 4, 4)") tk.MustExec("analyze table t with 2 topn, 3 buckets") @@ -628,7 +637,7 @@ func (s *testStatsSuite) TestCorrelation(c *C) { c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0") c.Assert(result.Rows()[1][9], Equals, "1") - testKit.MustExec("set @@session.tidb_analyze_version=3") + testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) @@ -641,7 +650,7 @@ func (s *testStatsSuite) TestCorrelation(c *C) { c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0") c.Assert(result.Rows()[1][9], Equals, "0.8285714285714286") - testKit.MustExec("set @@session.tidb_analyze_version=3") + testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) @@ -658,7 +667,7 @@ func (s *testStatsSuite) TestCorrelation(c *C) { c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0") c.Assert(result.Rows()[1][9], Equals, "-1") - testKit.MustExec("set @@session.tidb_analyze_version=3") + testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) @@ -671,7 +680,7 @@ func (s *testStatsSuite) TestCorrelation(c *C) { c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0") c.Assert(result.Rows()[1][9], Equals, "-0.9428571428571428") - testKit.MustExec("set @@session.tidb_analyze_version=3") + testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) @@ -686,7 +695,7 @@ func (s *testStatsSuite) TestCorrelation(c *C) { c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0") c.Assert(result.Rows()[1][9], Equals, "1") - testKit.MustExec("set @@session.tidb_analyze_version=3") + testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) @@ -702,7 +711,7 @@ func (s *testStatsSuite) TestCorrelation(c *C) { c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "1") c.Assert(result.Rows()[1][9], Equals, "0.8285714285714286") - testKit.MustExec("set @@session.tidb_analyze_version=3") + testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) @@ -717,7 +726,7 @@ func (s *testStatsSuite) TestCorrelation(c *C) { c.Assert(len(result.Rows()), Equals, 2) c.Assert(result.Rows()[0][9], Equals, "0.8285714285714286") c.Assert(result.Rows()[1][9], Equals, "1") - testKit.MustExec("set @@session.tidb_analyze_version=3") + testKit.MustExec("set @@session.tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't'").Sort() c.Assert(len(result.Rows()), Equals, 2) @@ -737,7 +746,7 @@ func (s *testStatsSuite) TestCorrelation(c *C) { result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 1").Sort() c.Assert(len(result.Rows()), Equals, 1) c.Assert(result.Rows()[0][9], Equals, "0") - testKit.MustExec("set @@tidb_analyze_version=3") + testKit.MustExec("set @@tidb_analyze_version=2") testKit.MustExec("analyze table t") result = testKit.MustQuery("show stats_histograms where Table_name = 't' and Is_index = 0").Sort() c.Assert(len(result.Rows()), Equals, 3) @@ -855,7 +864,7 @@ func (s *testStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.TestKit } tk.MustExec(buf1.String()) tk.MustExec(buf2.String()) - tk.MustExec("set @@tidb_analyze_version=3") + tk.MustExec("set @@tidb_analyze_version=2") tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") c.Assert(s.do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil) } @@ -1034,7 +1043,7 @@ partition by range (a) ( tk.MustQuery("select distinct_count, null_count, tot_col_size, correlation=0 from mysql.stats_histograms where is_index=0 order by table_id asc").Check( testkit.Rows("15 1 17 1", "6 1 7 0", "9 0 10 0")) tk.MustQuery("select distinct_count, null_count, tot_col_size, correlation=0 from mysql.stats_histograms where is_index=1 order by table_id asc").Check( - testkit.Rows("15 1 0 1", "6 1 0 1", "9 0 0 1")) + testkit.Rows("15 1 0 1", "6 1 6 1", "9 0 10 1")) tk.MustQuery("show stats_buckets where is_index=0").Check( // db table partition col is_idx bucket_id count repeats lower upper ndv @@ -1047,10 +1056,10 @@ partition by range (a) ( tk.MustQuery("show stats_buckets where is_index=1").Check( testkit.Rows("test t global a 1 0 7 2 1 6 0", "test t global a 1 1 17 2 6 19 0", - "test t p0 a 1 0 4 1 1 4 4", - "test t p0 a 1 1 7 2 5 6 2", - "test t p1 a 1 0 8 1 11 18 8", - "test t p1 a 1 1 10 2 19 19 1")) + "test t p0 a 1 0 4 1 1 4 0", + "test t p0 a 1 1 7 2 5 6 0", + "test t p1 a 1 0 6 1 11 16 0", + "test t p1 a 1 1 10 2 17 19 0")) } func (s *testStatsSuite) TestGlobalStatsData2(c *C) { @@ -1104,12 +1113,12 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { tk.MustQuery("show stats_buckets where is_index=1").Check(testkit.Rows( // db, tbl, part, col, isIdx, bucketID, count, repeat, lower, upper, ndv - "test tint global c 1 0 5 2 1 4 0", // 4 is popped from p0.TopN, so g.ndv = p0.ndv+1 - "test tint global c 1 1 12 2 4 17 0", - "test tint p0 c 1 0 3 0 1 4 3", - "test tint p0 c 1 1 3 0 5 5 0", - "test tint p1 c 1 0 5 0 11 16 5", - "test tint p1 c 1 1 5 0 17 17 0")) + "test tint global c 1 0 5 2 1 4 0", // 4 is popped from p0.TopN, so g.ndv = p0.ndv+1 + "test tint global c 1 1 12 2 17 17 0", // same with the column's + "test tint p0 c 1 0 2 1 1 2 0", + "test tint p0 c 1 1 3 1 3 3 0", + "test tint p1 c 1 0 3 1 11 13 0", + "test tint p1 c 1 1 5 1 14 15 0")) tk.MustQuery("select distinct_count, null_count from mysql.stats_histograms where is_index=1 order by table_id asc").Check( testkit.Rows("12 1", // global, g = p0 + p1 @@ -1167,11 +1176,11 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { tk.MustQuery("show stats_buckets where table_name='tdouble' and is_index=1 and column_name='c'").Check(testkit.Rows( // db, tbl, part, col, isIdx, bucketID, count, repeat, lower, upper, ndv "test tdouble global c 1 0 5 2 1 4 0", // 4 is popped from p0.TopN, so g.ndv = p0.ndv+1 - "test tdouble global c 1 1 12 2 4 17 0", - "test tdouble p0 c 1 0 3 0 1 4 3", - "test tdouble p0 c 1 1 3 0 5 5 0", - "test tdouble p1 c 1 0 5 0 11 16 5", - "test tdouble p1 c 1 1 5 0 17 17 0")) + "test tdouble global c 1 1 12 2 17 17 0", + "test tdouble p0 c 1 0 2 1 1 2 0", + "test tdouble p0 c 1 1 3 1 3 3 0", + "test tdouble p1 c 1 0 3 1 11 13 0", + "test tdouble p1 c 1 1 5 1 14 15 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdouble' and column_name='c' and is_index=1").Rows() c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 @@ -1232,11 +1241,11 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { tk.MustQuery("show stats_buckets where table_name='tdecimal' and is_index=1 and column_name='c'").Check(testkit.Rows( // db, tbl, part, col, isIdx, bucketID, count, repeat, lower, upper, ndv "test tdecimal global c 1 0 5 2 1.00 4.00 0", // 4 is popped from p0.TopN, so g.ndv = p0.ndv+1 - "test tdecimal global c 1 1 12 2 4.00 17.00 0", - "test tdecimal p0 c 1 0 3 0 1.00 4.00 3", - "test tdecimal p0 c 1 1 3 0 5.00 5.00 0", - "test tdecimal p1 c 1 0 5 0 11.00 16.00 5", - "test tdecimal p1 c 1 1 5 0 17.00 17.00 0")) + "test tdecimal global c 1 1 12 2 17.00 17.00 0", + "test tdecimal p0 c 1 0 2 1 1.00 2.00 0", + "test tdecimal p0 c 1 1 3 1 3.00 3.00 0", + "test tdecimal p1 c 1 0 3 1 11.00 13.00 0", + "test tdecimal p1 c 1 1 5 1 14.00 15.00 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdecimal' and column_name='c' and is_index=1").Rows() c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 @@ -1297,11 +1306,11 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { tk.MustQuery("show stats_buckets where table_name='tdatetime' and is_index=1 and column_name='c'").Check(testkit.Rows( // db, tbl, part, col, isIdx, bucketID, count, repeat, lower, upper, ndv "test tdatetime global c 1 0 5 2 2000-01-01 00:00:00 2000-01-04 00:00:00 0", // 4 is popped from p0.TopN, so g.ndv = p0.ndv+1 - "test tdatetime global c 1 1 12 2 2000-01-04 00:00:00 2000-01-17 00:00:00 0", - "test tdatetime p0 c 1 0 3 0 2000-01-01 00:00:00 2000-01-04 00:00:00 3", - "test tdatetime p0 c 1 1 3 0 2000-01-05 00:00:00 2000-01-05 00:00:00 0", - "test tdatetime p1 c 1 0 5 0 2000-01-11 00:00:00 2000-01-16 00:00:00 5", - "test tdatetime p1 c 1 1 5 0 2000-01-17 00:00:00 2000-01-17 00:00:00 0")) + "test tdatetime global c 1 1 12 2 2000-01-17 00:00:00 2000-01-17 00:00:00 0", + "test tdatetime p0 c 1 0 2 1 2000-01-01 00:00:00 2000-01-02 00:00:00 0", + "test tdatetime p0 c 1 1 3 1 2000-01-03 00:00:00 2000-01-03 00:00:00 0", + "test tdatetime p1 c 1 0 3 1 2000-01-11 00:00:00 2000-01-13 00:00:00 0", + "test tdatetime p1 c 1 1 5 1 2000-01-14 00:00:00 2000-01-15 00:00:00 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdatetime' and column_name='c' and is_index=1").Rows() c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 @@ -1362,11 +1371,11 @@ func (s *testStatsSuite) TestGlobalStatsData2(c *C) { tk.MustQuery("show stats_buckets where table_name='tstring' and is_index=1 and column_name='c'").Check(testkit.Rows( // db, tbl, part, col, isIdx, bucketID, count, repeat, lower, upper, ndv "test tstring global c 1 0 5 2 a1 a4 0", // 4 is popped from p0.TopN, so g.ndv = p0.ndv+1 - "test tstring global c 1 1 12 2 a4 b17 0", - "test tstring p0 c 1 0 3 0 a1 a4 3", - "test tstring p0 c 1 1 3 0 a5 a5 0", - "test tstring p1 c 1 0 5 0 b11 b16 5", - "test tstring p1 c 1 1 5 0 b17 b17 0")) + "test tstring global c 1 1 12 2 b17 b17 0", + "test tstring p0 c 1 0 2 1 a1 a2 0", + "test tstring p0 c 1 1 3 1 a3 a3 0", + "test tstring p1 c 1 0 3 1 b11 b13 0", + "test tstring p1 c 1 1 5 1 b14 b15 0")) rs = tk.MustQuery("show stats_histograms where table_name='tstring' and column_name='c' and is_index=1").Rows() c.Assert(rs[0][6].(string), Equals, "12") // g.ndv = p0 + p1 @@ -1406,12 +1415,12 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { "test tintint p1 a 1 (13, 2) 3")) tk.MustQuery("show stats_buckets where table_name='tintint' and is_index=1").Check(testkit.Rows( - "test tintint global a 1 0 6 2 (1, 1) (2, 3) 0", // (2, 3) is popped into it - "test tintint global a 1 1 11 2 (2, 3) (13, 1) 0", // (13, 1) is popped into it - "test tintint p0 a 1 0 4 1 (1, 1) (2, 2) 4", - "test tintint p0 a 1 1 4 0 (2, 3) (3, 1) 0", - "test tintint p1 a 1 0 3 0 (11, 1) (13, 1) 3", - "test tintint p1 a 1 1 3 0 (13, 2) (13, 2) 0")) + "test tintint global a 1 0 6 2 (1, 1) (2, 3) 0", // (2, 3) is popped into it + "test tintint global a 1 1 11 2 (13, 1) (13, 1) 0", // (13, 1) is popped into it + "test tintint p0 a 1 0 3 1 (1, 1) (2, 1) 0", + "test tintint p0 a 1 1 4 1 (2, 2) (2, 2) 0", + "test tintint p1 a 1 0 2 1 (11, 1) (12, 1) 0", + "test tintint p1 a 1 1 3 1 (12, 2) (12, 2) 0")) rs = tk.MustQuery("show stats_histograms where table_name='tintint' and is_index=1").Rows() c.Assert(rs[0][6].(string), Equals, "11") // g.ndv = p0.ndv + p1.ndv @@ -1440,12 +1449,12 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { "test tintstr p1 a 1 (13, 2) 3")) tk.MustQuery("show stats_buckets where table_name='tintstr' and is_index=1").Check(testkit.Rows( - "test tintstr global a 1 0 6 2 (1, 1) (2, 3) 0", // (2, 3) is popped into it - "test tintstr global a 1 1 11 2 (2, 3) (13, 1) 0", // (13, 1) is popped into it - "test tintstr p0 a 1 0 4 1 (1, 1) (2, 2) 4", - "test tintstr p0 a 1 1 4 0 (2, 3) (3, 1) 0", - "test tintstr p1 a 1 0 3 0 (11, 1) (13, 1) 3", - "test tintstr p1 a 1 1 3 0 (13, 2) (13, 2) 0")) + "test tintstr global a 1 0 6 2 (1, 1) (2, 3) 0", // (2, 3) is popped into it + "test tintstr global a 1 1 11 2 (13, 1) (13, 1) 0", // (13, 1) is popped into it + "test tintstr p0 a 1 0 3 1 (1, 1) (2, 1) 0", + "test tintstr p0 a 1 1 4 1 (2, 2) (2, 2) 0", + "test tintstr p1 a 1 0 2 1 (11, 1) (12, 1) 0", + "test tintstr p1 a 1 1 3 1 (12, 2) (12, 2) 0")) rs = tk.MustQuery("show stats_histograms where table_name='tintstr' and is_index=1").Rows() c.Assert(rs[0][6].(string), Equals, "11") // g.ndv = p0.ndv + p1.ndv @@ -1474,12 +1483,12 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { "test tintdouble p1 a 1 (13, 2) 3")) tk.MustQuery("show stats_buckets where table_name='tintdouble' and is_index=1").Check(testkit.Rows( - "test tintdouble global a 1 0 6 2 (1, 1) (2, 3) 0", // (2, 3) is popped into it - "test tintdouble global a 1 1 11 2 (2, 3) (13, 1) 0", // (13, 1) is popped into it - "test tintdouble p0 a 1 0 4 1 (1, 1) (2, 2) 4", - "test tintdouble p0 a 1 1 4 0 (2, 3) (3, 1) 0", - "test tintdouble p1 a 1 0 3 0 (11, 1) (13, 1) 3", - "test tintdouble p1 a 1 1 3 0 (13, 2) (13, 2) 0")) + "test tintdouble global a 1 0 6 2 (1, 1) (2, 3) 0", // (2, 3) is popped into it + "test tintdouble global a 1 1 11 2 (13, 1) (13, 1) 0", // (13, 1) is popped into it + "test tintdouble p0 a 1 0 3 1 (1, 1) (2, 1) 0", + "test tintdouble p0 a 1 1 4 1 (2, 2) (2, 2) 0", + "test tintdouble p1 a 1 0 2 1 (11, 1) (12, 1) 0", + "test tintdouble p1 a 1 1 3 1 (12, 2) (12, 2) 0")) rs = tk.MustQuery("show stats_histograms where table_name='tintdouble' and is_index=1").Rows() c.Assert(rs[0][6].(string), Equals, "11") // g.ndv = p0.ndv + p1.ndv @@ -1508,12 +1517,12 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { "test tdoubledecimal p1 a 1 (13, 2.00) 3")) tk.MustQuery("show stats_buckets where table_name='tdoubledecimal' and is_index=1").Check(testkit.Rows( - "test tdoubledecimal global a 1 0 6 2 (1, 1.00) (2, 3.00) 0", // (2, 3) is popped into it - "test tdoubledecimal global a 1 1 11 2 (2, 3.00) (13, 1.00) 0", // (13, 1) is popped into it - "test tdoubledecimal p0 a 1 0 4 1 (1, 1.00) (2, 2.00) 4", - "test tdoubledecimal p0 a 1 1 4 0 (2, 3.00) (3, 1.00) 0", - "test tdoubledecimal p1 a 1 0 3 0 (11, 1.00) (13, 1.00) 3", - "test tdoubledecimal p1 a 1 1 3 0 (13, 2.00) (13, 2.00) 0")) + "test tdoubledecimal global a 1 0 6 2 (1, 1.00) (2, 3.00) 0", // (2, 3) is popped into it + "test tdoubledecimal global a 1 1 11 2 (13, 1.00) (13, 1.00) 0", // (13, 1) is popped into it + "test tdoubledecimal p0 a 1 0 3 1 (1, 1.00) (2, 1.00) 0", + "test tdoubledecimal p0 a 1 1 4 1 (2, 2.00) (2, 2.00) 0", + "test tdoubledecimal p1 a 1 0 2 1 (11, 1.00) (12, 1.00) 0", + "test tdoubledecimal p1 a 1 1 3 1 (12, 2.00) (12, 2.00) 0")) rs = tk.MustQuery("show stats_histograms where table_name='tdoubledecimal' and is_index=1").Rows() c.Assert(rs[0][6].(string), Equals, "11") // g.ndv = p0.ndv + p1.ndv @@ -1542,12 +1551,12 @@ func (s *testStatsSuite) TestGlobalStatsData3(c *C) { "test tstrdt p1 a 1 (13, 2000-01-02 00:00:00) 3")) tk.MustQuery("show stats_buckets where table_name='tstrdt' and is_index=1").Check(testkit.Rows( - "test tstrdt global a 1 0 6 2 (1, 2000-01-01 00:00:00) (2, 2000-01-03 00:00:00) 0", // (2, 3) is popped into it - "test tstrdt global a 1 1 11 2 (2, 2000-01-03 00:00:00) (13, 2000-01-01 00:00:00) 0", // (13, 1) is popped into it - "test tstrdt p0 a 1 0 4 1 (1, 2000-01-01 00:00:00) (2, 2000-01-02 00:00:00) 4", - "test tstrdt p0 a 1 1 4 0 (2, 2000-01-03 00:00:00) (3, 2000-01-01 00:00:00) 0", - "test tstrdt p1 a 1 0 3 0 (11, 2000-01-01 00:00:00) (13, 2000-01-01 00:00:00) 3", - "test tstrdt p1 a 1 1 3 0 (13, 2000-01-02 00:00:00) (13, 2000-01-02 00:00:00) 0")) + "test tstrdt global a 1 0 6 2 (1, 2000-01-01 00:00:00) (2, 2000-01-03 00:00:00) 0", // (2, 3) is popped into it + "test tstrdt global a 1 1 11 2 (13, 2000-01-01 00:00:00) (13, 2000-01-01 00:00:00) 0", // (13, 1) is popped into it + "test tstrdt p0 a 1 0 3 1 (1, 2000-01-01 00:00:00) (2, 2000-01-01 00:00:00) 0", + "test tstrdt p0 a 1 1 4 1 (2, 2000-01-02 00:00:00) (2, 2000-01-02 00:00:00) 0", + "test tstrdt p1 a 1 0 2 1 (11, 2000-01-01 00:00:00) (12, 2000-01-01 00:00:00) 0", + "test tstrdt p1 a 1 1 3 1 (12, 2000-01-02 00:00:00) (12, 2000-01-02 00:00:00) 0")) rs = tk.MustQuery("show stats_histograms where table_name='tstrdt' and is_index=1").Rows() c.Assert(rs[0][6].(string), Equals, "11") // g.ndv = p0.ndv + p1.ndv @@ -1899,7 +1908,7 @@ func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { "2 [1,2] 1.000000 1", "2 [1,3] -1.000000 1", )) - tk.MustExec("set @@session.tidb_analyze_version=3") + tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("analyze table t") tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( "2 [1,2] 1.000000 1", @@ -1935,7 +1944,7 @@ func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { "2 [1,2] 0.000000 1", "2 [1,3] 1.000000 1", )) - tk.MustExec("set @@session.tidb_analyze_version=3") + tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("analyze table t") tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( "2 [1,2] 0.000000 1", @@ -1948,7 +1957,7 @@ func (s *testStatsSuite) TestCorrelationStatsCompute(c *C) { "2 [1,2] 1.000000 1", "2 [1,3] 1.000000 1", )) - tk.MustExec("set @@session.tidb_analyze_version=3") + tk.MustExec("set @@session.tidb_analyze_version=2") tk.MustExec("analyze table t") tk.MustQuery("select type, column_ids, stats, status from mysql.stats_extended").Sort().Check(testkit.Rows( "2 [1,2] 1.000000 1", @@ -2053,8 +2062,8 @@ func (s *testStatsSuite) TestAnalyzeWithDynamicPartitionPruneMode(c *C) { tk.MustExec("insert into t values (3)") tk.MustExec("analyze table t partition p0 index a with 1 topn, 2 buckets") rows = tk.MustQuery("show stats_buckets where partition_name = 'global' and is_index=1").Rows() - c.Assert(len(rows), Equals, 2) - c.Assert(rows[1][6], Equals, "6") + c.Assert(len(rows), Equals, 1) + c.Assert(rows[0][6], Equals, "6") } func (s *testStatsSuite) TestPartitionPruneModeSessionVariable(c *C) { diff --git a/statistics/handle/update.go b/statistics/handle/update.go index 2251fd8a9bb32..03df857ec4a5d 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -1067,7 +1067,6 @@ var execOptionForAnalyze = map[int]sqlexec.OptionFuncAlias{ statistics.Version0: sqlexec.ExecOptionAnalyzeVer1, statistics.Version1: sqlexec.ExecOptionAnalyzeVer1, statistics.Version2: sqlexec.ExecOptionAnalyzeVer2, - statistics.Version3: sqlexec.ExecOptionAnalyzeVer3, } func (h *Handle) execAutoAnalyze(statsVer int, sql string, params ...interface{}) { diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index a8924021c5846..fe8107fd8679a 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -504,7 +504,8 @@ func (s *testStatsSuite) TestAutoUpdate(c *C) { hg, ok := stats.Indices[tableInfo.Indices[0].ID] c.Assert(ok, IsTrue) c.Assert(hg.NDV, Equals, int64(3)) - c.Assert(hg.Len(), Equals, 3) + c.Assert(hg.Len(), Equals, 0) + c.Assert(hg.TopN.Num(), Equals, 3) }) } @@ -1758,6 +1759,7 @@ func (s *testStatsSuite) TestAbnormalIndexFeedback(c *C) { for i := 0; i < 20; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i/5, i)) } + testKit.MustExec("set @@session.tidb_analyze_version = 1") testKit.MustExec("analyze table t with 3 buckets, 0 topn") testKit.MustExec("delete from t where a = 1") testKit.MustExec("delete from t where b > 10") @@ -1836,6 +1838,7 @@ func (s *testStatsSuite) TestFeedbackRanges(c *C) { err := h.HandleDDLEvent(<-h.DDLEventCh()) c.Assert(err, IsNil) c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) + testKit.MustExec("set @@session.tidb_analyze_version=1") testKit.MustExec("analyze table t with 3 buckets") for i := 30; i < 40; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) diff --git a/statistics/histogram.go b/statistics/histogram.go index f377b14c0c067..80d61b0c3bdf5 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -295,13 +295,10 @@ const ( Version1 = 1 // Version2 maintains the statistics in the following way. // Column stats: CM Sketch is not used. TopN and Histogram are built from samples. TopN + Histogram represent all data. - // Index stats: CM SKetch is not used. TopN and Histograms are built in TiKV using full data. NDV is also collected for each bucket in histogram. + // Index stats: CM SKetch is not used. TopN and Histograms are built from samples. // Then values covered by TopN is removed from Histogram. TopN + Histogram represent all data. + // Both Column and Index's NDVs are collected by full scan. Version2 = 2 - // Version3 is used for testing now. Once it finished, we will fallback the Version3 to Version2. - // The difference between Version2 and Version3 is that we construct the index's statistics based on sampling also. - // The data structure between them are then same. - Version3 = 3 ) // AnalyzeFlag is set when the statistics comes from analyze and has not been modified by feedback. diff --git a/statistics/integration_test.go b/statistics/integration_test.go index 9edef42f17698..20aea4b621e95 100644 --- a/statistics/integration_test.go +++ b/statistics/integration_test.go @@ -94,14 +94,16 @@ func (s *testIntegrationSuite) TestChangeVerTo2Behavior(c *C) { } tk.MustExec("set @@session.tidb_analyze_version = 1") tk.MustExec("analyze table t index idx") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead", + "Warning 1105 The version 2 would collect all statistics not only the selected indexes")) c.Assert(h.Update(is), IsNil) statsTblT = h.GetTableStats(tblT.Meta()) for _, idx := range statsTblT.Indices { c.Assert(idx.StatsVer, Equals, int64(2)) } tk.MustExec("analyze table t index") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 The analyze version from the session is not compatible with the existing statistics of the table. Use the existing version instead", + "Warning 1105 The version 2 would collect all statistics not only the selected indexes")) c.Assert(h.Update(is), IsNil) statsTblT = h.GetTableStats(tblT.Meta()) for _, idx := range statsTblT.Indices { @@ -165,13 +167,6 @@ func (s *testIntegrationSuite) TestFastAnalyzeOnVer2(c *C) { } } -func (s *testIntegrationSuite) TestHideAnalyzeVerOnShow(c *C) { - defer cleanEnv(c, s.store, s.do) - tk := testkit.NewTestKit(c, s.store) - // TODO: remove this test when the version2 is GA. - c.Assert(len(tk.MustQuery("show variables like '%analyze_version%'").Rows()), Equals, 0) -} - func (s *testIntegrationSuite) TestIncAnalyzeOnVer2(c *C) { defer cleanEnv(c, s.store, s.do) tk := testkit.NewTestKit(c, s.store) @@ -188,8 +183,8 @@ func (s *testIntegrationSuite) TestIncAnalyzeOnVer2(c *C) { tk.MustExec("analyze incremental table t index idx with 2 topn") // After analyze, there's two val in hist. tk.MustQuery("show stats_buckets where table_name = 't' and column_name = 'idx'").Check(testkit.Rows( - "test t idx 1 0 2 2 1 1 1", - "test t idx 1 1 3 0 2 4 1", + "test t idx 1 0 2 2 1 1 0", + "test t idx 1 1 3 1 3 3 0", )) // Two val in topn. tk.MustQuery("show stats_topn where table_name = 't' and column_name = 'idx'").Check(testkit.Rows( @@ -332,7 +327,7 @@ func (s *testIntegrationSuite) TestNULLOnFullSampling(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") - tk.MustExec("set @@session.tidb_analyze_version = 3;") + tk.MustExec("set @@session.tidb_analyze_version = 2;") tk.MustExec("create table t(a int, index idx(a))") tk.MustExec("insert into t values(1), (1), (1), (2), (2), (3), (4), (null), (null), (null)") var ( diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index ac12be22442b9..4b2b38ceed61d 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -406,6 +406,7 @@ func (s *testStatsSuite) TestEstimationForUnknownValues(c *C) { testKit.MustExec("use test") testKit.MustExec("drop table if exists t") testKit.MustExec("create table t(a int, b int, key idx(a, b))") + testKit.MustExec("set @@tidb_analyze_version=1") testKit.MustExec("analyze table t") for i := 0; i < 10; i++ { testKit.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) @@ -645,7 +646,7 @@ func (s *testStatsSuite) TestTopNOutOfHist(c *C) { defer cleanEnv(c, s.store, s.do) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") - testKit.MustExec("set tidb_analyze_version=3") + testKit.MustExec("set tidb_analyze_version=2") testKit.MustExec("drop table if exists topn_before_hist") testKit.MustExec("create table topn_before_hist(a int, index idx(a))") @@ -760,6 +761,7 @@ func (s *testStatsSuite) TestCollationColumnEstimate(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a varchar(20) collate utf8mb4_general_ci)") tk.MustExec("insert into t values('aaa'), ('bbb'), ('AAA'), ('BBB')") + tk.MustExec("set @@session.tidb_analyze_version=1") h := s.do.StatsHandle() c.Assert(h.DumpStatsDeltaToKV(handle.DumpAll), IsNil) tk.MustExec("analyze table t") diff --git a/statistics/testdata/stats_suite_out.json b/statistics/testdata/stats_suite_out.json index b60a351b6cead..33babd9401b4d 100644 --- a/statistics/testdata/stats_suite_out.json +++ b/statistics/testdata/stats_suite_out.json @@ -89,71 +89,70 @@ "test tint b 0 1 6 1 6 8 0", "test tint c 0 0 3 1 3 5 0", "test tint c 0 1 6 1 6 8 0", - "test tint singular 1 0 2 1 1 4 2", - "test tint singular 1 1 6 1 5 8 4", - "test tint multi 1 0 2 1 (1, 1) (4, 4) 2", - "test tint multi 1 1 6 1 (5, 5) (8, 8) 4", + "test tint singular 1 0 3 1 3 5 0", + "test tint singular 1 1 6 1 6 8 0", + "test tint multi 1 0 3 1 (3, 3) (5, 5) 0", + "test tint multi 1 1 6 1 (6, 6) (8, 8) 0", "test tdouble a 0 0 3 1 3 5 0", "test tdouble a 0 1 6 1 6 8 0", "test tdouble b 0 0 3 1 3 5 0", "test tdouble b 0 1 6 1 6 8 0", "test tdouble c 0 0 3 1 3 5 0", "test tdouble c 0 1 6 1 6 8 0", - "test tdouble singular 1 0 2 1 1 4 2", - "test tdouble singular 1 1 6 1 5 8 4", - "test tdouble multi 1 0 2 1 (1, 1) (4, 4) 2", - "test tdouble multi 1 1 6 1 (5, 5) (8, 8) 4", + "test tdouble singular 1 0 3 1 3 5 0", + "test tdouble singular 1 1 6 1 6 8 0", + "test tdouble multi 1 0 3 1 (3, 3) (5, 5) 0", + "test tdouble multi 1 1 6 1 (6, 6) (8, 8) 0", "test tdecimal a 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0", "test tdecimal a 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0", "test tdecimal b 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0", "test tdecimal b 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0", "test tdecimal c 0 0 3 1 3.00000000000000000000 5.00000000000000000000 0", "test tdecimal c 0 1 6 1 6.00000000000000000000 8.00000000000000000000 0", - "test tdecimal singular 1 0 2 1 1.00000000000000000000 4.00000000000000000000 2", - "test tdecimal singular 1 1 6 1 5.00000000000000000000 8.00000000000000000000 4", - "test tdecimal multi 1 0 2 1 (1.00000000000000000000, 1.00000000000000000000) (4.00000000000000000000, 4.00000000000000000000) 2", - "test tdecimal multi 1 1 6 1 (5.00000000000000000000, 5.00000000000000000000) (8.00000000000000000000, 8.00000000000000000000) 4", + "test tdecimal singular 1 0 3 1 3.00000000000000000000 5.00000000000000000000 0", + "test tdecimal singular 1 1 6 1 6.00000000000000000000 8.00000000000000000000 0", + "test tdecimal multi 1 0 3 1 (3.00000000000000000000, 3.00000000000000000000) (5.00000000000000000000, 5.00000000000000000000) 0", + "test tdecimal multi 1 1 6 1 (6.00000000000000000000, 6.00000000000000000000) (8.00000000000000000000, 8.00000000000000000000) 0", "test tstring a 0 0 3 1 3 5 0", "test tstring a 0 1 6 1 6 8 0", "test tstring b 0 0 3 1 3 5 0", "test tstring b 0 1 6 1 6 8 0", "test tstring c 0 0 3 1 3 5 0", "test tstring c 0 1 6 1 6 8 0", - "test tstring singular 1 0 2 1 1 4 2", - "test tstring singular 1 1 6 1 5 8 4", - "test tstring multi 1 0 2 1 (1, 1) (4, 4) 2", - "test tstring multi 1 1 6 1 (5, 5) (8, 8) 4", + "test tstring singular 1 0 3 1 3 5 0", + "test tstring singular 1 1 6 1 6 8 0", + "test tstring multi 1 0 3 1 (3, 3) (5, 5) 0", + "test tstring multi 1 1 6 1 (6, 6) (8, 8) 0", "test tdatetime a 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", "test tdatetime a 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", "test tdatetime b 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", "test tdatetime b 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", "test tdatetime c 0 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", "test tdatetime c 0 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", - "test tdatetime singular 1 0 0 0 2001-01-01 00:00:00 2001-01-02 00:00:00 0", - "test tdatetime singular 1 1 2 1 2001-01-03 00:00:00 2001-01-04 00:00:00 2", - "test tdatetime multi 1 0 0 0 (2001-01-01 00:00:00, 2001-01-01 00:00:00) (2001-01-02 00:00:00, 2001-01-02 00:00:00) 0", - "test tdatetime multi 1 1 2 1 (2001-01-03 00:00:00, 2001-01-03 00:00:00) (2001-01-04 00:00:00, 2001-01-04 00:00:00) 2", + "test tdatetime singular 1 0 1 1 2001-01-03 00:00:00 2001-01-03 00:00:00 0", + "test tdatetime singular 1 1 2 1 2001-01-04 00:00:00 2001-01-04 00:00:00 0", + "test tdatetime multi 1 0 1 1 (2001-01-03 00:00:00, 2001-01-03 00:00:00) (2001-01-03 00:00:00, 2001-01-03 00:00:00) 0", + "test tdatetime multi 1 1 2 1 (2001-01-04 00:00:00, 2001-01-04 00:00:00) (2001-01-04 00:00:00, 2001-01-04 00:00:00) 0", "test tprefix a 0 0 2 1 333 444 0", "test tprefix a 0 1 4 1 555 666 0", "test tprefix b 0 0 2 1 333 444 0", "test tprefix b 0 1 4 1 555 666 0", - "test tprefix prefixa 1 0 0 0 11 22 0", - "test tprefix prefixa 1 1 2 1 33 44 2", - "test tprefix prefixa 1 2 4 1 55 66 2", + "test tprefix prefixa 1 0 2 1 33 44 0", + "test tprefix prefixa 1 1 4 1 55 66 0", "test ct1 a 0 0 3 1 3 5 0", "test ct1 a 0 1 6 1 6 8 0", "test ct1 pk 0 0 3 1 3 5 0", "test ct1 pk 0 1 6 1 6 8 0", - "test ct1 PRIMARY 1 0 2 1 1 4 2", - "test ct1 PRIMARY 1 1 6 1 5 8 4", + "test ct1 PRIMARY 1 0 3 1 3 5 0", + "test ct1 PRIMARY 1 1 6 1 6 8 0", "test ct2 a 0 0 3 1 3 5 0", "test ct2 a 0 1 6 1 6 8 0", "test ct2 b 0 0 3 1 3 5 0", "test ct2 b 0 1 6 1 6 8 0", "test ct2 c 0 0 3 1 3 5 0", "test ct2 c 0 1 6 1 6 8 0", - "test ct2 PRIMARY 1 0 2 1 (1, 1) (4, 4) 2", - "test ct2 PRIMARY 1 1 6 1 (5, 5) (8, 8) 4" + "test ct2 PRIMARY 1 0 3 1 (3, 3) (5, 5) 0", + "test ct2 PRIMARY 1 1 6 1 (6, 6) (8, 8) 0" ], [ "TableReader_7 1.00 root data:Selection_6", @@ -161,8 +160,8 @@ " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" ], [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.a, 4)", + "TableReader_7 0.75 root data:Selection_6", + "└─Selection_6 0.75 cop[tikv] eq(test.tint.a, 4)", " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" ], [ @@ -176,9 +175,9 @@ "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:singular(a) range:[4,4], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "IndexLookUp_10 0.75 root ", + "├─IndexRangeScan_8(Build) 0.75 cop[tikv] table:tdouble, index:singular(a) range:[4,4], keep order:false", + "└─TableRowIDScan_9(Probe) 0.75 cop[tikv] table:tdouble keep order:false" ], [ "IndexLookUp_10 1.00 root ", @@ -191,9 +190,9 @@ "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:singular(a) range:[4.00000000000000000000,4.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "IndexLookUp_10 0.75 root ", + "├─IndexRangeScan_8(Build) 0.75 cop[tikv] table:tdecimal, index:singular(a) range:[4.00000000000000000000,4.00000000000000000000], keep order:false", + "└─TableRowIDScan_9(Probe) 0.75 cop[tikv] table:tdecimal keep order:false" ], [ "IndexLookUp_10 1.00 root ", @@ -206,8 +205,8 @@ " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" ], [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.a, \"4\")", + "TableReader_7 0.75 root data:Selection_6", + "└─Selection_6 0.75 cop[tikv] eq(test.tstring.a, \"4\")", " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" ], [ @@ -252,8 +251,8 @@ " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" ], [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tint.b, 4), eq(test.tint.c, 4)", + "TableReader_7 0.75 root data:Selection_6", + "└─Selection_6 0.75 cop[tikv] eq(test.tint.b, 4), eq(test.tint.c, 4)", " └─TableFullScan_5 8.00 cop[tikv] table:tint keep order:false" ], [ @@ -267,9 +266,9 @@ "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdouble, index:multi(b, c) range:[4 4,4 4], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdouble keep order:false" + "IndexLookUp_10 0.75 root ", + "├─IndexRangeScan_8(Build) 0.75 cop[tikv] table:tdouble, index:multi(b, c) range:[4 4,4 4], keep order:false", + "└─TableRowIDScan_9(Probe) 0.75 cop[tikv] table:tdouble keep order:false" ], [ "IndexLookUp_10 1.00 root ", @@ -282,9 +281,9 @@ "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" ], [ - "IndexLookUp_10 1.00 root ", - "├─IndexRangeScan_8(Build) 1.00 cop[tikv] table:tdecimal, index:multi(b, c) range:[4.00000000000000000000 4.00000000000000000000,4.00000000000000000000 4.00000000000000000000], keep order:false", - "└─TableRowIDScan_9(Probe) 1.00 cop[tikv] table:tdecimal keep order:false" + "IndexLookUp_10 0.75 root ", + "├─IndexRangeScan_8(Build) 0.75 cop[tikv] table:tdecimal, index:multi(b, c) range:[4.00000000000000000000 4.00000000000000000000,4.00000000000000000000 4.00000000000000000000], keep order:false", + "└─TableRowIDScan_9(Probe) 0.75 cop[tikv] table:tdecimal keep order:false" ], [ "IndexLookUp_10 1.00 root ", @@ -297,8 +296,8 @@ " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" ], [ - "TableReader_7 1.00 root data:Selection_6", - "└─Selection_6 1.00 cop[tikv] eq(test.tstring.b, \"4\"), eq(test.tstring.c, \"4\")", + "TableReader_7 0.75 root data:Selection_6", + "└─Selection_6 0.75 cop[tikv] eq(test.tstring.b, \"4\"), eq(test.tstring.c, \"4\")", " └─TableFullScan_5 8.00 cop[tikv] table:tstring keep order:false" ], [ @@ -397,8 +396,8 @@ " └─TableFullScan_5 4.00 cop[tikv] table:tdatetime keep order:false" ], [ - "TableReader_6 4.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 4.00 cop[tikv] table:ct1 range:[\"1\",\"4\"], keep order:false" + "TableReader_6 5.00 root data:TableRangeScan_5", + "└─TableRangeScan_5 5.00 cop[tikv] table:ct1 range:[\"1\",\"4\"], keep order:false" ], [ "TableReader_6 3.75 root data:TableRangeScan_5", @@ -608,7 +607,7 @@ }, { "SQL": "select * from t where a < 8 and (b > 10 or c < 3 or b > 4) and a > 2", - "Selectivity": 0.3125 + "Selectivity": 0 } ] } diff --git a/util/ranger/testdata/ranger_suite_out.json b/util/ranger/testdata/ranger_suite_out.json index f56a1567e6043..9def6a398c661 100644 --- a/util/ranger/testdata/ranger_suite_out.json +++ b/util/ranger/testdata/ranger_suite_out.json @@ -265,8 +265,8 @@ { "SQL": "select * from t where a = 1 and ((b = 1) or (b = 2 and c = 3));", "Plan": [ - "TableReader_6 2.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 2.00 cop[tikv] table:t range:[1 1,1 1], [1 2 3,1 2 3], keep order:false" + "TableReader_6 1.71 root data:TableRangeScan_5", + "└─TableRangeScan_5 1.71 cop[tikv] table:t range:[1 1,1 1], [1 2 3,1 2 3], keep order:false" ], "Result": [ "1 1 1" @@ -293,8 +293,8 @@ { "SQL": "select * from t use index(primary) where ((a = 1) or (a = 2 and b = 2)) and c = 3;", "Plan": [ - "TableReader_7 0.75 root data:Selection_6", - "└─Selection_6 0.75 cop[tikv] eq(test.t.c, 3), or(eq(test.t.a, 1), and(eq(test.t.a, 2), eq(test.t.b, 2)))", + "TableReader_7 0.68 root data:Selection_6", + "└─Selection_6 0.68 cop[tikv] eq(test.t.c, 3), or(eq(test.t.a, 1), and(eq(test.t.a, 2), eq(test.t.b, 2)))", " └─TableRangeScan_5 2.00 cop[tikv] table:t range:[1,1], [2,2], keep order:false" ], "Result": [ @@ -304,7 +304,7 @@ { "SQL": "select * from t where (a,b) in ((1,1),(2,2)) and c > 2 and (a,b,c) in ((1,1,1),(2,2,3));", "Plan": [ - "Selection_6 0.56 root gt(test.t.c, 2), or(and(eq(test.t.a, 1), eq(test.t.b, 1)), and(eq(test.t.a, 2), eq(test.t.b, 2)))", + "Selection_6 0.44 root gt(test.t.c, 2), or(and(eq(test.t.a, 1), eq(test.t.b, 1)), and(eq(test.t.a, 2), eq(test.t.b, 2)))", "└─Batch_Point_Get_5 2.00 root table:t, clustered index:PRIMARY(a, b, c) keep order:false, desc:false" ], "Result": [ @@ -314,8 +314,8 @@ { "SQL": "select * from t where (a,b) in ((1,1),(2,2)) and c > 2;", "Plan": [ - "TableReader_6 1.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 1.00 cop[tikv] table:t range:(1 1 2,1 1 +inf], (2 2 2,2 2 +inf], keep order:false" + "TableReader_6 1.19 root data:TableRangeScan_5", + "└─TableRangeScan_5 1.19 cop[tikv] table:t range:(1 1 2,1 1 +inf], (2 2 2,2 2 +inf], keep order:false" ], "Result": [ "2 2 3" @@ -324,8 +324,8 @@ { "SQL": "select * from t where ((a = 1 and b = 1) or (a = 2 and b = 2)) and c > 2;", "Plan": [ - "TableReader_6 1.00 root data:TableRangeScan_5", - "└─TableRangeScan_5 1.00 cop[tikv] table:t range:(1 1 2,1 1 +inf], (2 2 2,2 2 +inf], keep order:false" + "TableReader_6 1.19 root data:TableRangeScan_5", + "└─TableRangeScan_5 1.19 cop[tikv] table:t range:(1 1 2,1 1 +inf], (2 2 2,2 2 +inf], keep order:false" ], "Result": [ "2 2 3" @@ -364,8 +364,8 @@ { "SQL": "select * from t2 where t='aaaa';", "Plan": [ - "TableReader_7 0.00 root data:Selection_6", - "└─Selection_6 0.00 cop[tikv] eq(test.t2.t, \"aaaa\")", + "TableReader_7 1.00 root data:Selection_6", + "└─Selection_6 1.00 cop[tikv] eq(test.t2.t, \"aaaa\")", " └─TableRangeScan_5 2.00 cop[tikv] table:t2 range:[0,+inf], keep order:false" ], "Result": [ @@ -375,8 +375,8 @@ { "SQL": "select * from t2 where t='aaaa' or t = 'a';", "Plan": [ - "TableReader_7 0.80 root data:Selection_6", - "└─Selection_6 0.80 cop[tikv] or(eq(test.t2.t, \"aaaa\"), eq(test.t2.t, \"a\"))", + "TableReader_7 1.60 root data:Selection_6", + "└─Selection_6 1.60 cop[tikv] or(eq(test.t2.t, \"aaaa\"), eq(test.t2.t, \"a\"))", " └─TableRangeScan_5 2.00 cop[tikv] table:t2 range:[0,+inf], keep order:false" ], "Result": [ @@ -392,11 +392,11 @@ { "SQL": "select * from t;", "Plan": [ - "PartitionUnion_8 4.00 root ", - "├─TableReader_10 1.00 root data:TableFullScan_9", - "│ └─TableFullScan_9 1.00 cop[tikv] table:t, partition:p0 keep order:false", - "└─TableReader_12 3.00 root data:TableFullScan_11", - " └─TableFullScan_11 3.00 cop[tikv] table:t, partition:p1 keep order:false" + "PartitionUnion_8 4.00 root ", + "├─TableReader_10 1.00 root data:TableFullScan_9", + "│ └─TableFullScan_9 1.00 cop[tikv] table:t, partition:p0 keep order:false", + "└─TableReader_12 3.00 root data:TableFullScan_11", + " └─TableFullScan_11 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0000 0", @@ -463,7 +463,7 @@ "└─PartitionUnion_10 4.00 root ", " ├─TableReader_12 1.00 root data:TableFullScan_11", " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", - " └─TableReader_14 3.00 root data:TableFullScan_13", + " └─TableReader_14 3.00 root data:TableFullScan_13", " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ @@ -478,7 +478,7 @@ " ├─TableReader_12 1.00 root data:TableFullScan_11", " │ └─TableFullScan_11 1.00 cop[tikv] table:t, partition:p0 keep order:false", " └─TableReader_14 3.00 root data:TableFullScan_13", - " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" + " └─TableFullScan_13 3.00 cop[tikv] table:t, partition:p1 keep order:false" ], "Result": [ "\u0000 0", diff --git a/util/sqlexec/restricted_sql_executor.go b/util/sqlexec/restricted_sql_executor.go index 2a196262a9dec..4d714a474806e 100644 --- a/util/sqlexec/restricted_sql_executor.go +++ b/util/sqlexec/restricted_sql_executor.go @@ -72,11 +72,6 @@ var ExecOptionAnalyzeVer2 OptionFuncAlias = func(option *ExecOption) { option.AnalyzeVer = 2 } -// ExecOptionAnalyzeVer3 tells ExecRestrictedStmt to collect statistics with version3. -var ExecOptionAnalyzeVer3 OptionFuncAlias = func(option *ExecOption) { - option.AnalyzeVer = 3 -} - // ExecOptionWithSnapshot tells ExecRestrictedStmt to use a snapshot. func ExecOptionWithSnapshot(snapshot uint64) OptionFuncAlias { return func(option *ExecOption) { From 7cc1ebc6b02079c9508a802616ce5074c07256a6 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Sun, 6 Jun 2021 15:26:27 +0800 Subject: [PATCH 310/343] store/tikv: increase max backoff time of read requests (#25153) --- store/tikv/2pc.go | 47 ++++++++++++++++++++++++++------------- store/tikv/pessimistic.go | 14 +++++++++++- store/tikv/scan.go | 13 +++++++---- store/tikv/snapshot.go | 42 +++++++++++++++++++++++++++++----- store/tikv/test_probe.go | 3 ++- 5 files changed, 92 insertions(+), 27 deletions(-) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 049b944799aa6..2f8a94422f2ef 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -730,12 +730,15 @@ func (tm *ttlManager) close() { close(tm.ch) } -const pessimisticLockMaxBackoff = 20000 +const keepAliveMaxBackoff = 20000 // 20 seconds +const pessimisticLockMaxBackoff = 600000 // 10 minutes +const maxConsecutiveFailure = 10 func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { // Ticker is set to 1/2 of the ManagedLockTTL. ticker := time.NewTicker(time.Duration(atomic.LoadUint64(&ManagedLockTTL)) * time.Millisecond / 2) defer ticker.Stop() + keepFail := 0 for { select { case <-tm.ch: @@ -745,7 +748,7 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { if tm.lockCtx != nil && tm.lockCtx.Killed != nil && atomic.LoadUint32(tm.lockCtx.Killed) != 0 { return } - bo := retry.NewBackofferWithVars(context.Background(), pessimisticLockMaxBackoff, c.txn.vars) + bo := retry.NewBackofferWithVars(context.Background(), keepAliveMaxBackoff, c.txn.vars) now, err := c.store.getTimestampWithRetry(bo, c.txn.GetScope()) if err != nil { logutil.Logger(bo.GetCtx()).Warn("keepAlive get tso fail", @@ -774,20 +777,29 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { logutil.Logger(bo.GetCtx()).Info("send TxnHeartBeat", zap.Uint64("startTS", c.startTS), zap.Uint64("newTTL", newTTL)) startTime := time.Now() - _, err = sendTxnHeartBeat(bo, c.store, c.primary(), c.startTS, newTTL) + _, stopHeartBeat, err := sendTxnHeartBeat(bo, c.store, c.primary(), c.startTS, newTTL) if err != nil { + keepFail++ metrics.TxnHeartBeatHistogramError.Observe(time.Since(startTime).Seconds()) - logutil.Logger(bo.GetCtx()).Warn("send TxnHeartBeat failed", + logutil.Logger(bo.GetCtx()).Debug("send TxnHeartBeat failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) - return + if stopHeartBeat || keepFail > maxConsecutiveFailure { + logutil.Logger(bo.GetCtx()).Warn("stop TxnHeartBeat", + zap.Error(err), + zap.Int("consecutiveFailure", keepFail), + zap.Uint64("txnStartTS", c.startTS)) + return + } + continue } + keepFail = 0 metrics.TxnHeartBeatHistogramOK.Observe(time.Since(startTime).Seconds()) } } } -func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, ttl uint64) (uint64, error) { +func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, ttl uint64) (newTTL uint64, stopHeartBeat bool, err error) { req := tikvrpc.NewRequest(tikvrpc.CmdTxnHeartBeat, &pb.TxnHeartBeatRequest{ PrimaryLock: primary, StartVersion: startTS, @@ -796,31 +808,36 @@ func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, tt for { loc, err := store.GetRegionCache().LocateKey(bo, primary) if err != nil { - return 0, errors.Trace(err) + return 0, false, errors.Trace(err) } resp, err := store.SendReq(bo, req, loc.Region, client.ReadTimeoutShort) if err != nil { - return 0, errors.Trace(err) + return 0, false, errors.Trace(err) } regionErr, err := resp.GetRegionError() if err != nil { - return 0, errors.Trace(err) + return 0, false, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return 0, errors.Trace(err) + // For other region error and the fake region error, backoff because + // there's something wrong. + // For the real EpochNotMatch error, don't backoff. + if regionErr.GetEpochNotMatch() == nil || isFakeRegionError(regionErr) { + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + if err != nil { + return 0, false, errors.Trace(err) + } } continue } if resp.Resp == nil { - return 0, errors.Trace(tikverr.ErrBodyMissing) + return 0, false, errors.Trace(tikverr.ErrBodyMissing) } cmdResp := resp.Resp.(*pb.TxnHeartBeatResponse) if keyErr := cmdResp.GetError(); keyErr != nil { - return 0, errors.Errorf("txn %d heartbeat fail, primary key = %v, err = %s", startTS, hex.EncodeToString(primary), extractKeyErr(keyErr)) + return 0, true, errors.Errorf("txn %d heartbeat fail, primary key = %v, err = %s", startTS, hex.EncodeToString(primary), extractKeyErr(keyErr)) } - return cmdResp.GetLockTtl(), nil + return cmdResp.GetLockTtl(), false, nil } } diff --git a/store/tikv/pessimistic.go b/store/tikv/pessimistic.go index 37152652b1f0b..fe886e0369bdf 100644 --- a/store/tikv/pessimistic.go +++ b/store/tikv/pessimistic.go @@ -132,10 +132,22 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + // For other region error and the fake region error, backoff because + // there's something wrong. + // For the real EpochNotMatch error, don't backoff. + if regionErr.GetEpochNotMatch() == nil || isFakeRegionError(regionErr) { + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + if err != nil { + return errors.Trace(err) + } + } + same, err := batch.relocate(bo, c.store.regionCache) if err != nil { return errors.Trace(err) } + if same { + continue + } err = c.pessimisticLockMutations(bo, action.LockCtx, batch.mutations) return errors.Trace(err) } diff --git a/store/tikv/scan.go b/store/tikv/scan.go index eb94c77094472..c4d41764b24ef 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -87,7 +87,7 @@ func (s *Scanner) Value() []byte { return nil } -const scannerNextMaxBackoff = 20000 +const scannerNextMaxBackoff = 600000 // 10 minutes // Next return next element. func (s *Scanner) Next() error { @@ -229,9 +229,14 @@ func (s *Scanner) getData(bo *Backoffer) error { if regionErr != nil { logutil.BgLogger().Debug("scanner getData failed", zap.Stringer("regionErr", regionErr)) - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return errors.Trace(err) + // For other region error and the fake region error, backoff because + // there's something wrong. + // For the real EpochNotMatch error, don't backoff. + if regionErr.GetEpochNotMatch() == nil || isFakeRegionError(regionErr) { + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + if err != nil { + return errors.Trace(err) + } } continue } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index e9398e336ce86..740a7500bf587 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -131,7 +131,7 @@ func newTiKVSnapshot(store *KVStore, ts uint64, replicaReadSeed uint32) *KVSnaps } } -const batchGetMaxBackoff = 20000 +const batchGetMaxBackoff = 600000 // 10 minutes // SetSnapshotTS resets the timestamp for reads. func (s *KVSnapshot) SetSnapshotTS(ts uint64) { @@ -235,6 +235,19 @@ type batchKeys struct { keys [][]byte } +func (b *batchKeys) relocate(bo *Backoffer, c *RegionCache) (bool, error) { + begin, end := b.keys[0], b.keys[len(b.keys)-1] + loc, err := c.LocateKey(bo, begin) + if err != nil { + return false, errors.Trace(err) + } + if !loc.Contains(end) { + return false, nil + } + b.region = loc.Region + return true, nil +} + // appendBatchKeysBySize appends keys to b. It may split the keys to make // sure each batch's size does not exceed the limit. func appendBatchKeysBySize(b []batchKeys, region RegionVerID, keys [][]byte, sizeFn func([]byte) int, limit int) []batchKeys { @@ -339,10 +352,22 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec return errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + // For other region error and the fake region error, backoff because + // there's something wrong. + // For the real EpochNotMatch error, don't backoff. + if regionErr.GetEpochNotMatch() == nil || isFakeRegionError(regionErr) { + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + if err != nil { + return errors.Trace(err) + } + } + same, err := batch.relocate(bo, cli.regionCache) if err != nil { return errors.Trace(err) } + if same { + continue + } err = s.batchGetKeysByRegions(bo, pending, collectF) return errors.Trace(err) } @@ -402,7 +427,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec } } -const getMaxBackoff = 20000 +const getMaxBackoff = 600000 // 10 minutes // Get gets the value for key k from snapshot. func (s *KVSnapshot) Get(ctx context.Context, k []byte) ([]byte, error) { @@ -497,9 +522,14 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, return nil, errors.Trace(err) } if regionErr != nil { - err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) - if err != nil { - return nil, errors.Trace(err) + // For other region error and the fake region error, backoff because + // there's something wrong. + // For the real EpochNotMatch error, don't backoff. + if regionErr.GetEpochNotMatch() == nil || isFakeRegionError(regionErr) { + err = bo.Backoff(retry.BoRegionMiss, errors.New(regionErr.String())) + if err != nil { + return nil, errors.Trace(err) + } } continue } diff --git a/store/tikv/test_probe.go b/store/tikv/test_probe.go index 3e80e6310fe4b..d7eb5836d70ad 100644 --- a/store/tikv/test_probe.go +++ b/store/tikv/test_probe.go @@ -69,7 +69,8 @@ func (s StoreProbe) ClearTxnLatches() { // SendTxnHeartbeat renews a txn's ttl. func (s StoreProbe) SendTxnHeartbeat(ctx context.Context, key []byte, startTS uint64, ttl uint64) (uint64, error) { bo := retry.NewBackofferWithVars(ctx, PrewriteMaxBackoff, nil) - return sendTxnHeartBeat(bo, s.KVStore, key, startTS, ttl) + newTTL, _, err := sendTxnHeartBeat(bo, s.KVStore, key, startTS, ttl) + return newTTL, err } // LoadSafePoint from safepoint kv. From 01dc05f87386f48d935f01dd68a364adc50407fb Mon Sep 17 00:00:00 2001 From: YinWeiling <30956512+YinWeiling@users.noreply.github.com> Date: Sun, 6 Jun 2021 16:04:27 +0800 Subject: [PATCH 311/343] telemetry: add SQL statistics to telemetry report data (#24990) --- telemetry/data_slow_query.go | 46 ++------------ telemetry/data_window.go | 113 +++++++++++++++++++++++++++++++++++ 2 files changed, 119 insertions(+), 40 deletions(-) diff --git a/telemetry/data_slow_query.go b/telemetry/data_slow_query.go index c3da41c57efbf..b2408b3223a05 100644 --- a/telemetry/data_slow_query.go +++ b/telemetry/data_slow_query.go @@ -25,8 +25,6 @@ import ( "github.com/pingcap/tidb/domain/infosync" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/logutil" - "github.com/prometheus/client_golang/api" - promv1 "github.com/prometheus/client_golang/api/prometheus/v1" pmodel "github.com/prometheus/common/model" "go.uber.org/zap" ) @@ -90,7 +88,12 @@ func updateCurrentSQB(ctx sessionctx.Context) (err error) { } }() - value, err := querySlowQueryMetric(ctx) //TODO: judge error here + pQueryCtx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + pQueryTs := time.Now().Add(-time.Minute) + promQL := "tidb_server_slow_query_process_duration_seconds_bucket{sql_type=\"general\"}" + value, err := querySQLMetric(pQueryCtx, pQueryTs, promQL) + if err != nil && err != infosync.ErrPrometheusAddrIsNotSet { logutil.BgLogger().Info("querySlowQueryMetric got error") return err @@ -112,43 +115,6 @@ func updateCurrentSQB(ctx sessionctx.Context) (err error) { return nil } -func querySlowQueryMetric(sctx sessionctx.Context) (result pmodel.Value, err error) { - // Add retry to avoid network error. - var prometheusAddr string - for i := 0; i < 5; i++ { - //TODO: the prometheus will be Integrated into the PD, then we need to query the prometheus in PD directly, which need change the query API - prometheusAddr, err = infosync.GetPrometheusAddr() - if err == nil || err == infosync.ErrPrometheusAddrIsNotSet { - break - } - time.Sleep(100 * time.Millisecond) - } - if err != nil { - return nil, err - } - promClient, err := api.NewClient(api.Config{ - Address: prometheusAddr, - }) - if err != nil { - return nil, err - } - promQLAPI := promv1.NewAPI(promClient) - promQL := "tidb_server_slow_query_process_duration_seconds_bucket{sql_type=\"general\"}" - - ts := time.Now().Add(-time.Minute) - // Add retry to avoid network error. - ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) - defer cancel() - for i := 0; i < 5; i++ { - result, _, err = promQLAPI.Query(ctx, promQL, ts) - if err == nil { - break - } - time.Sleep(100 * time.Millisecond) - } - return result, err -} - // calculateDeltaSQB calculate the delta between current slow query bucket and last slow query bucket func calculateDeltaSQB() *SlowQueryBucket { deltaMap := make(SlowQueryBucket) diff --git a/telemetry/data_window.go b/telemetry/data_window.go index 5dd1f6e750bba..cb86f23bc09a2 100644 --- a/telemetry/data_window.go +++ b/telemetry/data_window.go @@ -14,10 +14,18 @@ package telemetry import ( + "context" "sync" "time" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/domain/infosync" + "github.com/pingcap/tidb/store/tikv/logutil" + "github.com/prometheus/client_golang/api" + promv1 "github.com/prometheus/client_golang/api/prometheus/v1" + pmodel "github.com/prometheus/common/model" "go.uber.org/atomic" + "go.uber.org/zap" ) var ( @@ -51,6 +59,7 @@ const ( maxSubWindowLength = int(ReportInterval / SubWindowSize) // TODO: Ceiling? maxSubWindowLengthInWindow = int(WindowSize / SubWindowSize) // TODO: Ceiling? + promReadTimeout = time.Second * 30 ) type windowData struct { @@ -58,6 +67,14 @@ type windowData struct { ExecuteCount uint64 `json:"executeCount"` TiFlashUsage tiFlashUsageData `json:"tiFlashUsage"` CoprCacheUsage coprCacheUsageData `json:"coprCacheUsage"` + SQLUsage sqlUsageData `json:"SQLUsage"` +} + +type sqlType map[string]int64 + +type sqlUsageData struct { + SQLTotal int64 `json:"total"` + SQLType sqlType `json:"type"` } type coprCacheUsageData struct { @@ -80,6 +97,75 @@ var ( subWindowsLock = sync.RWMutex{} ) +func getSQLSum(sqlTypeData *sqlType) int64 { + result := int64(0) + for _, v := range *sqlTypeData { + result += v + } + return result +} + +func readSQLMetric(timepoint time.Time, SQLResult *sqlUsageData) error { + ctx := context.TODO() + promQL := "sum(tidb_executor_statement_total{}) by (instance,type)" + result, err := querySQLMetric(ctx, timepoint, promQL) + if err != nil { + if err1, ok := err.(*promv1.Error); ok { + return errors.Errorf("query metric error, msg: %v, detail: %v", err1.Msg, err1.Detail) + } + return errors.Errorf("query metric error: %v", err.Error()) + } + + anylisSQLUsage(result, SQLResult) + return nil +} + +func querySQLMetric(ctx context.Context, queryTime time.Time, promQL string) (result pmodel.Value, err error) { + // Add retry to avoid network error. + var prometheusAddr string + for i := 0; i < 5; i++ { + //TODO: the prometheus will be Integrated into the PD, then we need to query the prometheus in PD directly, which need change the quire API + prometheusAddr, err = infosync.GetPrometheusAddr() + if err == nil || err == infosync.ErrPrometheusAddrIsNotSet { + break + } + time.Sleep(100 * time.Millisecond) + } + if err != nil { + return nil, err + } + promClient, err := api.NewClient(api.Config{ + Address: prometheusAddr, + }) + if err != nil { + return nil, err + } + promQLAPI := promv1.NewAPI(promClient) + ctx, cancel := context.WithTimeout(ctx, promReadTimeout) + defer cancel() + // Add retry to avoid network error. + for i := 0; i < 5; i++ { + result, _, err = promQLAPI.Query(ctx, promQL, queryTime) + if err == nil { + break + } + time.Sleep(100 * time.Millisecond) + } + return result, err +} + +func anylisSQLUsage(promResult pmodel.Value, SQLResult *sqlUsageData) { + switch promResult.Type() { + case pmodel.ValVector: + matrix := promResult.(pmodel.Vector) + for _, m := range matrix { + v := m.Value + promLable := string(m.Metric[pmodel.LabelName("type")]) + SQLResult.SQLType[promLable] = int64(float64(v)) + } + } +} + // RotateSubWindow rotates the telemetry sub window. func RotateSubWindow() { thisSubWindow := windowData{ @@ -98,7 +184,18 @@ func RotateSubWindow() { GTE80: CurrentCoprCacheHitRatioGTE80Count.Swap(0), GTE100: CurrentCoprCacheHitRatioGTE100Count.Swap(0), }, + SQLUsage: sqlUsageData{ + SQLTotal: 0, + SQLType: make(sqlType), + }, } + + if err := readSQLMetric(time.Now(), &thisSubWindow.SQLUsage); err != nil { + logutil.BgLogger().Error("Error exists when calling prometheus", zap.Error(err)) + + } + thisSubWindow.SQLUsage.SQLTotal = getSQLSum(&thisSubWindow.SQLUsage.SQLType) + subWindowsLock.Lock() rotatedSubWindows = append(rotatedSubWindows, &thisSubWindow) if len(rotatedSubWindows) > maxSubWindowLength { @@ -108,6 +205,14 @@ func RotateSubWindow() { subWindowsLock.Unlock() } +func calDeltaSQLTypeMap(cur sqlType, last sqlType) sqlType { + deltaMap := make(sqlType) + for key, value := range cur { + deltaMap[key] = value - (last)[key] + } + return deltaMap +} + // getWindowData returns data aggregated by window size. func getWindowData() []*windowData { results := make([]*windowData, 0) @@ -117,6 +222,12 @@ func getWindowData() []*windowData { i := 0 for i < len(rotatedSubWindows) { thisWindow := *rotatedSubWindows[i] + var startWindow windowData + if i == 0 { + startWindow = thisWindow + } else { + startWindow = *rotatedSubWindows[i-1] + } aggregatedSubWindows := 1 // Aggregate later sub windows i++ @@ -131,6 +242,8 @@ func getWindowData() []*windowData { thisWindow.CoprCacheUsage.GTE40 += rotatedSubWindows[i].CoprCacheUsage.GTE40 thisWindow.CoprCacheUsage.GTE80 += rotatedSubWindows[i].CoprCacheUsage.GTE80 thisWindow.CoprCacheUsage.GTE100 += rotatedSubWindows[i].CoprCacheUsage.GTE100 + thisWindow.SQLUsage.SQLTotal = rotatedSubWindows[i].SQLUsage.SQLTotal - startWindow.SQLUsage.SQLTotal + thisWindow.SQLUsage.SQLType = calDeltaSQLTypeMap(rotatedSubWindows[i].SQLUsage.SQLType, startWindow.SQLUsage.SQLType) aggregatedSubWindows++ i++ } From 47f0f15b14ed54fc2222f3e304e29df7b05e6805 Mon Sep 17 00:00:00 2001 From: Lynn Date: Sun, 6 Jun 2021 16:56:27 +0800 Subject: [PATCH 312/343] ddl, executor: handle original default value after changing column (#25184) --- ddl/column.go | 18 +++-- ddl/column_type_change_test.go | 121 +++++++++++++++++++++++++++++++-- 2 files changed, 128 insertions(+), 11 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 757f914d62ec6..9f981200f987e 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -861,12 +861,15 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in // Since column type change is implemented as adding a new column then substituting the old one. // Case exists when update-where statement fetch a NULL for not-null column without any default data, // it will errors. - // So we set zero original default value here to prevent this error. besides, in insert & update records, + // So we set zero original default value here to prevent this error. Besides, in insert & update records, // we have already implement using the casted value of relative column to insert rather than the origin // default value. - originDefVal, err := generateOriginDefaultValue(jobParam.newCol) - if err != nil { - return ver, errors.Trace(err) + originDefVal := oldCol.GetOriginDefaultValue() + if originDefVal == nil { + originDefVal, err = generateOriginDefaultValue(jobParam.newCol) + if err != nil { + return ver, errors.Trace(err) + } } if err = jobParam.changingCol.SetOriginDefaultValue(originDefVal); err != nil { return ver, errors.Trace(err) @@ -1076,6 +1079,13 @@ func (w *worker) doModifyColumnTypeWithData( changingColumnUniqueName := changingCol.Name changingCol.Name = colName changingCol.ChangeStateInfo = nil + // After changing the column, the column's type is change, so it needs to set OriginDefaultValue back + // so that there is no error in getting the default value from OriginDefaultValue. + // Besides, nil data that was not backfilled in the "add column" is backfilled after the column is changed. + // So it can set OriginDefaultValue to nil. + if err = changingCol.SetOriginDefaultValue(nil); err != nil { + return ver, errors.Trace(err) + } tblInfo.Indices = tblInfo.Indices[:len(tblInfo.Indices)-len(changingIdxs)] // Adjust table column offset. if err = adjustColumnInfoInModifyColumn(job, tblInfo, changingCol, oldCol, pos, changingColumnUniqueName.L); err != nil { diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 625917cb7e693..8e3f7d6d30e8a 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -16,6 +16,7 @@ package ddl_test import ( "context" "errors" + "fmt" "strconv" "strings" "sync" @@ -1599,6 +1600,28 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromJsonToOthers(c *C) { tk.MustQuery("select * from t").Check(testkit.Rows("{\"obj\": 100} [-1, 0, 1] 0 2001 0 2020 1991 2009 2020")) } +func (s *testColumnTypeChangeSuite) TestUpdateDataAfterChangeTimestampToDate(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + // Enable column change variable. + tk.Se.GetSessionVars().EnableChangeColumnType = true + tk.MustExec("drop table if exists t, t1") + tk.MustExec("create table t (col timestamp default '1971-06-09' not null, col1 int default 1, unique key(col1));") + tk.MustExec("alter table t modify column col date not null;") + tk.MustExec("update t set col = '2002-12-31';") + // point get + tk.MustExec("update t set col = '2002-12-31' where col1 = 1;") + + // Make sure the original default value isn't rewritten. + tk.MustExec("create table t1 (col timestamp default '1971-06-09' not null, col1 int default 1, unique key(col1));") + tk.MustExec("insert into t1 value('2001-01-01', 1);") + tk.MustExec("alter table t1 add column col2 timestamp default '2020-06-02' not null;") + tk.MustExec("alter table t1 modify column col2 date not null;") + tk.MustExec("update t1 set col = '2002-11-22';") + // point get + tk.MustExec("update t1 set col = '2002-12-31' where col1 = 1;") +} + // TestRowFormat is used to close issue #21391, the encoded row in column type change should be aware of the new row format. func (s *testColumnTypeChangeSuite) TestRowFormat(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -1668,15 +1691,12 @@ func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValue(c *C) { tk.MustExec("use test") // Enable column change variable. tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false - }() tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("use test") tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int, b int)") + tk.MustExec("create table t(a int, b int not null, unique key(a))") tk.MustExec("insert into t values(1, 1)") tk.MustExec("insert into t values(2, 2)") @@ -1687,7 +1707,8 @@ func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValue(c *C) { once bool checkErr error ) - hook.OnJobRunBeforeExported = func(job *model.Job) { + i := 0 + hook.OnJobUpdatedExported = func(job *model.Job) { if checkErr != nil { return } @@ -1709,7 +1730,8 @@ func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValue(c *C) { } // For writable column: // Insert/ Update should set the column with the casted-related column value. - _, err := tk1.Exec("insert into t values(3, 3)") + sql := fmt.Sprintf("insert into t values(%d, %d)", i+3, i+3) + _, err := tk1.Exec(sql) if err != nil { checkErr = err return @@ -1729,13 +1751,98 @@ func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValue(c *C) { return } } + i++ } } s.dom.DDL().(ddl.DDLForTest).SetHook(hook) tk.MustExec("alter table t modify column b tinyint NOT NULL") s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook) + c.Assert(checkErr, IsNil) + // Since getReorgInfo will stagnate StateWriteReorganization for a ddl round, so insert should exec 3 times. + tk.MustQuery("select * from t order by a").Check(testkit.Rows("1 -1", "2 -2", "3 3", "4 4", "5 5")) + tk.MustExec("drop table if exists t") +} + +func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValueAfterAddCol(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + // Enable column change variable. + tk.Se.GetSessionVars().EnableChangeColumnType = true + + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec("use test") + + tk.MustExec(fmt.Sprintf("set time_zone = 'UTC'")) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int not null, unique key(a))") + tk.MustExec("insert into t values(1, 1)") + tk.MustExec("insert into t values(2, 2)") + tk.MustExec("alter table t add column c timestamp default '1971-06-09' not null") + + tbl := testGetTableByName(c, tk.Se, "test", "t") + originalHook := s.dom.DDL().GetHook() + hook := &ddl.TestDDLCallback{Do: s.dom} + var ( + once bool + checkErr error + ) + i := 0 + hook.OnJobRunBeforeExported = func(job *model.Job) { + if checkErr != nil { + return + } + if tbl.Meta().ID != job.TableID { + return + } + if job.SchemaState == model.StateWriteOnly || job.SchemaState == model.StateWriteReorganization { + if !once { + once = true + tbl := testGetTableByName(c, tk1.Se, "test", "t") + if len(tbl.WritableCols()) != 4 { + checkErr = errors.New("assert the writable column number error") + return + } + if tbl.WritableCols()[3].OriginDefaultValue.(string) != "1971-06-09 00:00:00" { + checkErr = errors.New("assert the write only column origin default value error") + return + } + } + // For writable column: + // Insert / Update should set the column with the casted-related column value. + sql := fmt.Sprintf("insert into t values(%d, %d, '2021-06-06 12:13:14')", i+3, i+3) + _, err := tk1.Exec(sql) + if err != nil { + checkErr = err + return + } + if job.SchemaState == model.StateWriteOnly { + // The casted value will be inserted into changing column too. + // for point get + _, err := tk1.Exec("update t set b = -1 where a = 1") + if err != nil { + checkErr = err + return + } + } else { + // The casted value will be inserted into changing column too. + // for point get + _, err := tk1.Exec("update t set b = -2 where a = 2") + if err != nil { + checkErr = err + return + } + } + } + i++ + } + + s.dom.DDL().(ddl.DDLForTest).SetHook(hook) + tk.MustExec("alter table t modify column c date NOT NULL") + s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook) + c.Assert(checkErr, IsNil) // Since getReorgInfo will stagnate StateWriteReorganization for a ddl round, so insert should exec 3 times. - tk.MustQuery("select * from t order by a").Check(testkit.Rows("1 -1", "2 -2", "3 3", "3 3", "3 3")) + tk.MustQuery("select * from t order by a").Check( + testkit.Rows("1 -1 1971-06-09", "2 -2 1971-06-09", "5 5 2021-06-06", "6 6 2021-06-06", "7 7 2021-06-06")) tk.MustExec("drop table if exists t") } From 9659929b1cce4a6fecf39df48915de363605ce4b Mon Sep 17 00:00:00 2001 From: zhuo1ang Date: Mon, 7 Jun 2021 11:40:28 +0800 Subject: [PATCH 313/343] tikv: fix a typo (#25185) --- store/tikv/lock_resolver.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index 7ff963113bb3b..a41cf522224ef 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -533,7 +533,7 @@ func (lr *LockResolver) getTxnStatusFromLock(bo *Backoffer, l *Lock, callerStart errors.New("error txn not found and lock expired")) }) } - // For pessimistic lock resolving, if the primary lock dose not exist and rollbackIfNotExist is true, + // For pessimistic lock resolving, if the primary lock does not exist and rollbackIfNotExist is true, // The Action_LockNotExistDoNothing will be returned as the status. rollbackIfNotExist = true } else { From 5c95062cc34d6d37e2e921f9bddba6205b43ee3a Mon Sep 17 00:00:00 2001 From: Shirly Date: Mon, 7 Jun 2021 14:36:28 +0800 Subject: [PATCH 314/343] store/tikv: remove unused function and private the variable shuttingDown (#25194) --- store/tikv/backoff.go | 1 - store/tikv/region_request.go | 9 ++++---- store/tikv/retry/backoff.go | 45 ------------------------------------ 3 files changed, 4 insertions(+), 51 deletions(-) diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index 9dd4c3d8b0e04..34285a8a8d204 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -29,7 +29,6 @@ type BackoffConfig = retry.Config // Maximum total sleep time(in ms) for kv/cop commands. const ( gcResolveLockMaxBackoff = 100000 - pdRPCMaxBackoff = 20000 // CommitSecondaryMaxBackoff is max sleep time of the 'commit' command CommitSecondaryMaxBackoff = 41000 ) diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index f823c11a71408..26a8f5380faf0 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -43,20 +43,19 @@ import ( "github.com/pingcap/tidb/store/tikv/util" ) -// ShuttingDown is a flag to indicate tidb-server is exiting (Ctrl+C signal +// shuttingDown is a flag to indicate tidb-server is exiting (Ctrl+C signal // receved for example). If this flag is set, tikv client should not retry on // network error because tidb-server expect tikv client to exit as soon as possible. -// TODO: make it private when br is ready. -var ShuttingDown uint32 +var shuttingDown uint32 // StoreShuttingDown atomically stores ShuttingDown into v. func StoreShuttingDown(v uint32) { - atomic.StoreUint32(&ShuttingDown, v) + atomic.StoreUint32(&shuttingDown, v) } // LoadShuttingDown atomically loads ShuttingDown. func LoadShuttingDown() uint32 { - return atomic.LoadUint32(&ShuttingDown) + return atomic.LoadUint32(&shuttingDown) } // RegionRequestSender sends KV/Cop requests to tikv server. It handles network diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go index 716011f8836ab..ecb7608275de2 100644 --- a/store/tikv/retry/backoff.go +++ b/store/tikv/retry/backoff.go @@ -27,7 +27,6 @@ import ( tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" - "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/util" "go.uber.org/zap" "go.uber.org/zap/zapcore" @@ -105,50 +104,6 @@ func (b *Backoffer) BackoffWithMaxSleepTxnLockFast(maxSleepMs int, err error) er return b.BackoffWithCfgAndMaxSleep(cfg, maxSleepMs, err) } -// BackoffWithMaxSleep is deprecated, please use BackoffWithCfgAndMaxSleep instead. TODO: remove it when br is ready. -func (b *Backoffer) BackoffWithMaxSleep(typ int, maxSleepMs int, err error) error { - // Back off types. - const ( - boTiKVRPC int = iota - boTiFlashRPC - boTxnLock - boTxnLockFast - boPDRPC - boRegionMiss - boTiKVServerBusy - boTiFlashServerBusy - boTxnNotFound - boStaleCmd - boMaxTsNotSynced - ) - switch typ { - case boTiKVRPC: - return b.BackoffWithCfgAndMaxSleep(BoTiKVRPC, maxSleepMs, err) - case boTiFlashRPC: - return b.BackoffWithCfgAndMaxSleep(BoTiFlashRPC, maxSleepMs, err) - case boTxnLock: - return b.BackoffWithCfgAndMaxSleep(BoTxnLock, maxSleepMs, err) - case boTxnLockFast: - return b.BackoffWithCfgAndMaxSleep(BoTxnLockFast, maxSleepMs, err) - case boPDRPC: - return b.BackoffWithCfgAndMaxSleep(BoPDRPC, maxSleepMs, err) - case boRegionMiss: - return b.BackoffWithCfgAndMaxSleep(BoRegionMiss, maxSleepMs, err) - case boTiKVServerBusy: - return b.BackoffWithCfgAndMaxSleep(BoTiKVServerBusy, maxSleepMs, err) - case boTiFlashServerBusy: - return b.BackoffWithCfgAndMaxSleep(BoTiFlashServerBusy, maxSleepMs, err) - case boTxnNotFound: - return b.BackoffWithCfgAndMaxSleep(BoTxnNotFound, maxSleepMs, err) - case boStaleCmd: - return b.BackoffWithCfgAndMaxSleep(BoStaleCmd, maxSleepMs, err) - case boMaxTsNotSynced: - return b.BackoffWithCfgAndMaxSleep(BoMaxTsNotSynced, maxSleepMs, err) - } - cfg := NewConfig("", &metrics.BackoffHistogramEmpty, nil, tikverr.ErrUnknown) - return b.BackoffWithCfgAndMaxSleep(cfg, maxSleepMs, err) -} - // BackoffWithCfgAndMaxSleep sleeps a while base on the Config and records the error message // and never sleep more than maxSleepMs for each sleep. func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err error) error { From 3947c5c6e07638e3cb64218622f4cd19d166a253 Mon Sep 17 00:00:00 2001 From: mmyj Date: Mon, 7 Jun 2021 15:22:28 +0800 Subject: [PATCH 315/343] typo: GetlogicalTS4TaskMap -> GetLogicalTS4TaskMap (#25135) --- planner/core/find_best_task.go | 2 +- planner/core/plan.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 55610cdf293e2..3b73e4033ccac 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -212,7 +212,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl childTasks = childTasks[:0] // The curCntPlan records the number of possible plans for pp curCntPlan = 1 - TimeStampNow := p.GetlogicalTS4TaskMap() + TimeStampNow := p.GetLogicalTS4TaskMap() savedPlanID := p.ctx.GetSessionVars().PlanID for j, child := range p.children { childTask, cnt, err := child.findBestTask(pp.GetChildReqProps(j), &PlanCounterDisabled) diff --git a/planner/core/plan.go b/planner/core/plan.go index 2a62afa960b63..3c99884a80cf1 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -440,8 +440,8 @@ func (p *basePhysicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColum return nil } -// GetlogicalTS4TaskMap get the logical TimeStamp now to help rollback the TaskMap changes after that. -func (p *baseLogicalPlan) GetlogicalTS4TaskMap() uint64 { +// GetLogicalTS4TaskMap get the logical TimeStamp now to help rollback the TaskMap changes after that. +func (p *baseLogicalPlan) GetLogicalTS4TaskMap() uint64 { p.ctx.GetSessionVars().StmtCtx.TaskMapBakTS += 1 return p.ctx.GetSessionVars().StmtCtx.TaskMapBakTS } @@ -483,7 +483,7 @@ func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task task) key := prop.HashCode() if p.ctx.GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { // Empty string for useless change. - TS := p.GetlogicalTS4TaskMap() + TS := p.GetLogicalTS4TaskMap() p.taskMapBakTS = append(p.taskMapBakTS, TS) p.taskMapBak = append(p.taskMapBak, string(key)) } From 2017d2fd6868536200bf6ecfa7138cbd39cac89d Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Mon, 7 Jun 2021 15:40:29 +0800 Subject: [PATCH 316/343] types: fix wrong truncated val for bit type (#25198) --- expression/integration_test.go | 9 +++++++++ types/datum.go | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/expression/integration_test.go b/expression/integration_test.go index 6a2de6f4526b8..1cf17e1d36bfa 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -397,6 +397,15 @@ func (s *testIntegrationSuite) TestConvertToBit(c *C) { "Warning 1690 constant 599999999 overflows tinyint", "Warning 1406 Data Too Long, field len 63")) tk.MustQuery("select * from t;").Check(testkit.Rows("127 \u007f\xff\xff\xff\xff\xff\xff\xff")) + + // For issue 24900 + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(b bit(16));") + tk.MustExec("insert ignore into t values(0x3635313836),(0x333830);") + tk.MustQuery("show warnings;").Check(testkit.Rows( + "Warning 1406 Data Too Long, field len 16", + "Warning 1406 Data Too Long, field len 16")) + tk.MustQuery("select * from t;").Check(testkit.Rows("\xff\xff", "\xff\xff")) } func (s *testIntegrationSuite2) TestMathBuiltin(c *C) { diff --git a/types/datum.go b/types/datum.go index c744f514e0a9b..95e20e1d1b2f3 100644 --- a/types/datum.go +++ b/types/datum.go @@ -1478,7 +1478,7 @@ func (d *Datum) convertToMysqlBit(sc *stmtctx.StatementContext, target *FieldTyp return Datum{}, errors.Trace(ErrDataTooLong.GenWithStack("Data Too Long, field len %d", target.Flen)) } if target.Flen < 64 && uintValue >= 1<<(uint64(target.Flen)) { - uintValue &= (1 << (uint64(target.Flen))) - 1 + uintValue = (1 << (uint64(target.Flen))) - 1 err = ErrDataTooLong.GenWithStack("Data Too Long, field len %d", target.Flen) } byteSize := (target.Flen + 7) >> 3 From 840494dc1380bab667c0df72de7283c07c7c531e Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Mon, 7 Jun 2021 16:38:28 +0800 Subject: [PATCH 317/343] store/tikv: add and fix some metrics for the new retry logic (#25123) --- metrics/grafana/tidb.json | 236 ++++++++++++++++++++- store/tikv/2pc.go | 27 ++- store/tikv/metrics/metrics.go | 39 ++++ store/tikv/region_request.go | 10 +- store/tikv/retry/backoff.go | 12 +- store/tikv/txn.go | 8 +- store/tikv/util/execdetails.go | 8 +- util/execdetails/execdetails.go | 14 +- util/execdetails/execdetails_test.go | 39 ++-- util/stmtsummary/statement_summary.go | 10 +- util/stmtsummary/statement_summary_test.go | 65 +++--- 11 files changed, 385 insertions(+), 83 deletions(-) diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 4053986976fcd..5c78605bb62c1 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -5063,7 +5063,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "This metric shows OPS of async commit transactions.", + "description": "This metric shows the OPS of different types of transactions.", "editable": true, "error": false, "fill": 1, @@ -5079,13 +5079,13 @@ "legend": { "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -5103,6 +5103,13 @@ "stack": false, "steppedLine": false, "targets": [ + { + "expr": "sum(rate(tidb_tikvclient_commit_txn_counter{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)", "format": "time_series", @@ -5111,18 +5118,18 @@ "refId": "A" }, { - "refId": "B", "expr": "sum(rate(tidb_tikvclient_one_pc_txn_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", - "intervalFactor": 1, "format": "time_series", - "legendFormat": "1PC-{{type}}" + "intervalFactor": 1, + "legendFormat": "1PC-{{type}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Async Commit Transaction Counter", + "title": "Transaction Types Per Second", "tooltip": { "msResolution": false, "shared": true, @@ -5159,6 +5166,123 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "99th percentile of backoff count and duration in a transaction commit", + "editable": true, + "error": false, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 46 + }, + "id": 224, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sort": "avg", + "sortDesc": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "null as zero", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/count.*/", + "yaxis": 1 + }, + { + "alias": "/duration.*/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, rate(tidb_tikvclient_txn_commit_backoff_count_bucket{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "count - {{instance}}", + "refId": "A", + "step": 40 + }, + { + "expr": "histogram_quantile(0.99, rate(tidb_tikvclient_txn_commit_backoff_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[1m]))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "duration - {{instance}}", + "refId": "B", + "step": 40 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Transaction Commit .99 Backoff", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 2, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": "count", + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "s", + "label": "duration", + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, @@ -7003,6 +7127,102 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "This metric shows the reasons of replica selector failure (which needs a backoff).", + "editable": true, + "error": false, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 21 + }, + "id": 223, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tidb_tikvclient_replica_selector_failure_counter{tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Replica Selector Failure Per Second", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 2f8a94422f2ef..49096254743dc 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -1079,11 +1079,13 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) (err error) { commitDetail := c.getDetail() commitDetail.PrewriteTime = time.Since(start) - // TODO(youjiali1995): Record the backoff time of the last finished batch. It doesn't make sense to aggregate all batches'. if bo.GetTotalSleep() > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(bo.GetTotalSleep())*int64(time.Millisecond)) + boSleep := int64(bo.GetTotalSleep()) * int64(time.Millisecond) commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, bo.GetTypes()...) + if boSleep > commitDetail.Mu.CommitBackoffTime { + commitDetail.Mu.CommitBackoffTime = boSleep + commitDetail.Mu.BackoffTypes = bo.GetTypes() + } commitDetail.Mu.Unlock() } @@ -1242,8 +1244,8 @@ func (c *twoPhaseCommitter) commitTxn(ctx context.Context, commitDetail *util.Co err := c.commitMutations(commitBo, c.mutations) commitDetail.CommitTime = time.Since(start) if commitBo.GetTotalSleep() > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(commitBo.GetTotalSleep())*int64(time.Millisecond)) commitDetail.Mu.Lock() + commitDetail.Mu.CommitBackoffTime += int64(commitBo.GetTotalSleep()) * int64(time.Millisecond) commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, commitBo.GetTypes()...) commitDetail.Mu.Unlock() } @@ -1636,17 +1638,20 @@ func (batchExe *batchExecutor) startWorker(exitCh chan struct{}, ch chan error, singleBatchBackoffer, singleBatchCancel = batchExe.backoffer.Fork() defer singleBatchCancel() } - beforeSleep := singleBatchBackoffer.GetTotalSleep() ch <- batchExe.action.handleSingleBatch(batchExe.committer, singleBatchBackoffer, batch) commitDetail := batchExe.committer.getDetail() - if commitDetail != nil { // lock operations of pessimistic-txn will let commitDetail be nil - if delta := singleBatchBackoffer.GetTotalSleep() - beforeSleep; delta > 0 { - atomic.AddInt64(&commitDetail.CommitBackoffTime, int64(singleBatchBackoffer.GetTotalSleep()-beforeSleep)*int64(time.Millisecond)) - commitDetail.Mu.Lock() - commitDetail.Mu.BackoffTypes = append(commitDetail.Mu.BackoffTypes, singleBatchBackoffer.GetTypes()...) - commitDetail.Mu.Unlock() + // For prewrite, we record the max backoff time + if _, ok := batchExe.action.(actionPrewrite); ok { + commitDetail.Mu.Lock() + boSleep := int64(singleBatchBackoffer.GetTotalSleep()) * int64(time.Millisecond) + if boSleep > commitDetail.Mu.CommitBackoffTime { + commitDetail.Mu.CommitBackoffTime = boSleep + commitDetail.Mu.BackoffTypes = singleBatchBackoffer.GetTypes() } + commitDetail.Mu.Unlock() } + // Backoff time in the 2nd phase of a non-async-commit txn is added + // in the commitTxn method, so we don't add it here. }() } else { logutil.Logger(batchExe.backoffer.GetCtx()).Info("break startWorker", diff --git a/store/tikv/metrics/metrics.go b/store/tikv/metrics/metrics.go index 6b8ea32d456f7..7f6cb97b9dde8 100644 --- a/store/tikv/metrics/metrics.go +++ b/store/tikv/metrics/metrics.go @@ -61,6 +61,10 @@ var ( TiKVTSFutureWaitDuration prometheus.Histogram TiKVSafeTSUpdateCounter *prometheus.CounterVec TiKVSafeTSUpdateStats *prometheus.GaugeVec + TiKVReplicaSelectorFailureCounter *prometheus.CounterVec + TiKVRequestRetryTimesHistogram prometheus.Histogram + TiKVTxnCommitBackoffSeconds prometheus.Histogram + TiKVTxnCommitBackoffCount prometheus.Histogram ) // Label constants. @@ -431,6 +435,37 @@ func initMetrics(namespace, subsystem string) { Name: "safets_update_stats", Help: "stat of tikv updating safe_ts stats", }, []string{LblStore}) + TiKVReplicaSelectorFailureCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "replica_selector_failure_counter", + Help: "Counter of the reason why the replica selector cannot yield a potential leader.", + }, []string{LblType}) + TiKVRequestRetryTimesHistogram = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "request_retry_times", + Help: "Bucketed histogram of how many times a region request retries.", + Buckets: []float64{1, 2, 3, 4, 8, 16, 32, 64, 128, 256}, + }) + TiKVTxnCommitBackoffSeconds = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "txn_commit_backoff_seconds", + Help: "Bucketed histogram of the total backoff duration in committing a transaction.", + Buckets: prometheus.ExponentialBuckets(0.001, 2, 22), // 1ms ~ 2097s + }) + TiKVTxnCommitBackoffCount = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: namespace, + Subsystem: subsystem, + Name: "txn_commit_backoff_count", + Help: "Bucketed histogram of the backoff count in committing a transaction.", + Buckets: prometheus.ExponentialBuckets(1, 2, 12), // 1 ~ 2048 + }) initShortcuts() } @@ -488,6 +523,10 @@ func RegisterMetrics() { prometheus.MustRegister(TiKVTSFutureWaitDuration) prometheus.MustRegister(TiKVSafeTSUpdateCounter) prometheus.MustRegister(TiKVSafeTSUpdateStats) + prometheus.MustRegister(TiKVReplicaSelectorFailureCounter) + prometheus.MustRegister(TiKVRequestRetryTimesHistogram) + prometheus.MustRegister(TiKVTxnCommitBackoffSeconds) + prometheus.MustRegister(TiKVTxnCommitBackoffCount) } // readCounter reads the value of a prometheus.Counter. diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 26a8f5380faf0..61c82feb58f94 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -262,13 +262,15 @@ func (s *replicaSelector) nextReplica() *replica { const maxReplicaAttempt = 10 // next creates the RPCContext of the current candidate replica. -// It returns a SendError if runs out of all replicas of the cached region is invalidated. +// It returns a SendError if runs out of all replicas or the cached region is invalidated. func (s *replicaSelector) next(bo *Backoffer) (*RPCContext, error) { for { if !s.region.isValid() { + metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("invalid").Inc() return nil, nil } if s.isExhausted() { + metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("exhausted").Inc() s.invalidateRegion() return nil, nil } @@ -284,6 +286,7 @@ func (s *replicaSelector) next(bo *Backoffer) (*RPCContext, error) { storeFailEpoch := atomic.LoadUint32(&replica.store.epoch) if storeFailEpoch != replica.epoch { // TODO(youjiali1995): Is it necessary to invalidate the region? + metrics.TiKVReplicaSelectorFailureCounter.WithLabelValues("stale_store").Inc() s.invalidateRegion() return nil, nil } @@ -481,6 +484,11 @@ func (s *RegionRequestSender) SendReqCtx( s.reset() tryTimes := 0 + defer func() { + if tryTimes > 0 { + metrics.TiKVRequestRetryTimesHistogram.Observe(float64(tryTimes)) + } + }() for { if (tryTimes > 0) && (tryTimes%100 == 0) { logutil.Logger(bo.GetCtx()).Warn("retry", zap.Uint64("region", regionID.GetID()), zap.Int("times", tryTimes)) diff --git a/store/tikv/retry/backoff.go b/store/tikv/retry/backoff.go index ecb7608275de2..485a179ced6c9 100644 --- a/store/tikv/retry/backoff.go +++ b/store/tikv/retry/backoff.go @@ -47,6 +47,7 @@ type Backoffer struct { configs []*Config backoffSleepMS map[string]int backoffTimes map[string]int + parent *Backoffer } type txnStartCtxKeyType struct{} @@ -196,6 +197,7 @@ func (b *Backoffer) Clone() *Backoffer { totalSleep: b.totalSleep, errors: b.errors, vars: b.vars, + parent: b.parent, } } @@ -209,6 +211,7 @@ func (b *Backoffer) Fork() (*Backoffer, context.CancelFunc) { totalSleep: b.totalSleep, errors: b.errors, vars: b.vars, + parent: b, }, cancel } @@ -222,11 +225,14 @@ func (b *Backoffer) GetTotalSleep() int { return b.totalSleep } -// GetTypes returns type list. +// GetTypes returns type list of this backoff and all its ancestors. func (b *Backoffer) GetTypes() []string { typs := make([]string, 0, len(b.configs)) - for _, cfg := range b.configs { - typs = append(typs, cfg.String()) + for b != nil { + for _, cfg := range b.configs { + typs = append(typs, cfg.String()) + } + b = b.parent } return typs } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index f2c5fc24449c3..3cde92561d8ee 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -346,13 +346,19 @@ func (txn *KVTxn) Commit(ctx context.Context) error { } defer func() { + detail := committer.getDetail() + detail.Mu.Lock() + metrics.TiKVTxnCommitBackoffSeconds.Observe(float64(detail.Mu.CommitBackoffTime) / float64(time.Second)) + metrics.TiKVTxnCommitBackoffCount.Observe(float64(len(detail.Mu.BackoffTypes))) + detail.Mu.Unlock() + ctxValue := ctx.Value(util.CommitDetailCtxKey) if ctxValue != nil { commitDetail := ctxValue.(**util.CommitDetails) if *commitDetail != nil { (*commitDetail).TxnRetry++ } else { - *commitDetail = committer.getDetail() + *commitDetail = detail } } }() diff --git a/store/tikv/util/execdetails.go b/store/tikv/util/execdetails.go index 97c88a3687319..f8a8364fbb645 100644 --- a/store/tikv/util/execdetails.go +++ b/store/tikv/util/execdetails.go @@ -46,10 +46,10 @@ type CommitDetails struct { WaitPrewriteBinlogTime time.Duration CommitTime time.Duration LocalLatchTime time.Duration - CommitBackoffTime int64 Mu struct { sync.Mutex - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string } ResolveLockTime int64 WriteKeys int @@ -65,12 +65,12 @@ func (cd *CommitDetails) Merge(other *CommitDetails) { cd.WaitPrewriteBinlogTime += other.WaitPrewriteBinlogTime cd.CommitTime += other.CommitTime cd.LocalLatchTime += other.LocalLatchTime - cd.CommitBackoffTime += other.CommitBackoffTime cd.ResolveLockTime += other.ResolveLockTime cd.WriteKeys += other.WriteKeys cd.WriteSize += other.WriteSize cd.PrewriteRegionNum += other.PrewriteRegionNum cd.TxnRetry += other.TxnRetry + cd.Mu.CommitBackoffTime += other.Mu.CommitBackoffTime cd.Mu.BackoffTypes = append(cd.Mu.BackoffTypes, other.Mu.BackoffTypes...) } @@ -82,7 +82,6 @@ func (cd *CommitDetails) Clone() *CommitDetails { WaitPrewriteBinlogTime: cd.WaitPrewriteBinlogTime, CommitTime: cd.CommitTime, LocalLatchTime: cd.LocalLatchTime, - CommitBackoffTime: cd.CommitBackoffTime, ResolveLockTime: cd.ResolveLockTime, WriteKeys: cd.WriteKeys, WriteSize: cd.WriteSize, @@ -90,6 +89,7 @@ func (cd *CommitDetails) Clone() *CommitDetails { TxnRetry: cd.TxnRetry, } commit.Mu.BackoffTypes = append([]string{}, cd.Mu.BackoffTypes...) + commit.Mu.CommitBackoffTime = cd.Mu.CommitBackoffTime return commit } diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index faed7f8c9eef7..ac16e1fa13f09 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -142,11 +142,11 @@ func (d ExecDetails) String() string { if commitDetails.GetCommitTsTime > 0 { parts = append(parts, GetCommitTSTimeStr+": "+strconv.FormatFloat(commitDetails.GetCommitTsTime.Seconds(), 'f', -1, 64)) } - commitBackoffTime := atomic.LoadInt64(&commitDetails.CommitBackoffTime) + commitDetails.Mu.Lock() + commitBackoffTime := commitDetails.Mu.CommitBackoffTime if commitBackoffTime > 0 { parts = append(parts, CommitBackoffTimeStr+": "+strconv.FormatFloat(time.Duration(commitBackoffTime).Seconds(), 'f', -1, 64)) } - commitDetails.Mu.Lock() if len(commitDetails.Mu.BackoffTypes) > 0 { parts = append(parts, BackoffTypesStr+": "+fmt.Sprintf("%v", commitDetails.Mu.BackoffTypes)) } @@ -234,11 +234,11 @@ func (d ExecDetails) ToZapFields() (fields []zap.Field) { if commitDetails.GetCommitTsTime > 0 { fields = append(fields, zap.String("get_commit_ts_time", fmt.Sprintf("%v", strconv.FormatFloat(commitDetails.GetCommitTsTime.Seconds(), 'f', -1, 64)+"s"))) } - commitBackoffTime := atomic.LoadInt64(&commitDetails.CommitBackoffTime) + commitDetails.Mu.Lock() + commitBackoffTime := commitDetails.Mu.CommitBackoffTime if commitBackoffTime > 0 { fields = append(fields, zap.String("commit_backoff_time", fmt.Sprintf("%v", strconv.FormatFloat(time.Duration(commitBackoffTime).Seconds(), 'f', -1, 64)+"s"))) } - commitDetails.Mu.Lock() if len(commitDetails.Mu.BackoffTypes) > 0 { fields = append(fields, zap.String("backoff_types", fmt.Sprintf("%v", commitDetails.Mu.BackoffTypes))) } @@ -803,18 +803,18 @@ func (e *RuntimeStatsWithCommit) String() string { buf.WriteString(", commit:") buf.WriteString(FormatDuration(e.Commit.CommitTime)) } - commitBackoffTime := atomic.LoadInt64(&e.Commit.CommitBackoffTime) + e.Commit.Mu.Lock() + commitBackoffTime := e.Commit.Mu.CommitBackoffTime if commitBackoffTime > 0 { buf.WriteString(", backoff: {time: ") buf.WriteString(FormatDuration(time.Duration(commitBackoffTime))) - e.Commit.Mu.Lock() if len(e.Commit.Mu.BackoffTypes) > 0 { buf.WriteString(", type: ") buf.WriteString(e.formatBackoff(e.Commit.Mu.BackoffTypes)) } - e.Commit.Mu.Unlock() buf.WriteString("}") } + e.Commit.Mu.Unlock() if e.Commit.ResolveLockTime > 0 { buf.WriteString(", resolve_lock: ") buf.WriteString(FormatDuration(time.Duration(e.Commit.ResolveLockTime))) diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index 3d3a70959c180..461814e1b7d3e 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -34,18 +34,22 @@ func TestString(t *testing.T) { BackoffTime: time.Second, RequestCount: 1, CommitDetail: &util.CommitDetails{ - GetCommitTsTime: time.Second, - PrewriteTime: time.Second, - CommitTime: time.Second, - LocalLatchTime: time.Second, - CommitBackoffTime: int64(time.Second), + GetCommitTsTime: time.Second, + PrewriteTime: time.Second, + CommitTime: time.Second, + LocalLatchTime: time.Second, + Mu: struct { sync.Mutex - BackoffTypes []string - }{BackoffTypes: []string{ - "backoff1", - "backoff2", - }}, + CommitBackoffTime int64 + BackoffTypes []string + }{ + CommitBackoffTime: int64(time.Second), + BackoffTypes: []string{ + "backoff1", + "backoff2", + }, + }, ResolveLockTime: 1000000000, // 10^9 ns = 1s WriteKeys: 1, WriteSize: 1, @@ -200,14 +204,17 @@ func TestCopRuntimeStatsForTiFlash(t *testing.T) { } func TestRuntimeStatsWithCommit(t *testing.T) { commitDetail := &util.CommitDetails{ - GetCommitTsTime: time.Second, - PrewriteTime: time.Second, - CommitTime: time.Second, - CommitBackoffTime: int64(time.Second), + GetCommitTsTime: time.Second, + PrewriteTime: time.Second, + CommitTime: time.Second, Mu: struct { sync.Mutex - BackoffTypes []string - }{BackoffTypes: []string{"backoff1", "backoff2", "backoff1"}}, + CommitBackoffTime int64 + BackoffTypes []string + }{ + CommitBackoffTime: int64(time.Second), + BackoffTypes: []string{"backoff1", "backoff2", "backoff1"}, + }, ResolveLockTime: int64(time.Second), WriteKeys: 3, WriteSize: 66, diff --git a/util/stmtsummary/statement_summary.go b/util/stmtsummary/statement_summary.go index acfdaf405cd12..149196e0a4c31 100644 --- a/util/stmtsummary/statement_summary.go +++ b/util/stmtsummary/statement_summary.go @@ -783,11 +783,6 @@ func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeco if commitDetails.GetCommitTsTime > ssElement.maxGetCommitTsTime { ssElement.maxGetCommitTsTime = commitDetails.GetCommitTsTime } - commitBackoffTime := atomic.LoadInt64(&commitDetails.CommitBackoffTime) - ssElement.sumCommitBackoffTime += commitBackoffTime - if commitBackoffTime > ssElement.maxCommitBackoffTime { - ssElement.maxCommitBackoffTime = commitBackoffTime - } resolveLockTime := atomic.LoadInt64(&commitDetails.ResolveLockTime) ssElement.sumResolveLockTime += resolveLockTime if resolveLockTime > ssElement.maxResolveLockTime { @@ -815,6 +810,11 @@ func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeco ssElement.maxTxnRetry = commitDetails.TxnRetry } commitDetails.Mu.Lock() + commitBackoffTime := commitDetails.Mu.CommitBackoffTime + ssElement.sumCommitBackoffTime += commitBackoffTime + if commitBackoffTime > ssElement.maxCommitBackoffTime { + ssElement.maxCommitBackoffTime = commitBackoffTime + } ssElement.sumBackoffTimes += int64(len(commitDetails.Mu.BackoffTypes)) for _, backoffType := range commitDetails.Mu.BackoffTypes { ssElement.backoffTypes[backoffType] += 1 diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 751a8e501f715..f09398df68423 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -83,6 +83,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { planDigest: stmtExecInfo1.PlanDigest, } samplePlan, _ := stmtExecInfo1.PlanGenerator() + stmtExecInfo1.ExecDetail.CommitDetail.Mu.Lock() expectedSummaryElement := stmtSummaryByDigestElement{ beginTime: now + 60, endTime: now + 1860, @@ -120,8 +121,8 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { maxCommitTime: stmtExecInfo1.ExecDetail.CommitDetail.CommitTime, sumLocalLatchTime: stmtExecInfo1.ExecDetail.CommitDetail.LocalLatchTime, maxLocalLatchTime: stmtExecInfo1.ExecDetail.CommitDetail.LocalLatchTime, - sumCommitBackoffTime: stmtExecInfo1.ExecDetail.CommitDetail.CommitBackoffTime, - maxCommitBackoffTime: stmtExecInfo1.ExecDetail.CommitDetail.CommitBackoffTime, + sumCommitBackoffTime: stmtExecInfo1.ExecDetail.CommitDetail.Mu.CommitBackoffTime, + maxCommitBackoffTime: stmtExecInfo1.ExecDetail.CommitDetail.Mu.CommitBackoffTime, sumResolveLockTime: stmtExecInfo1.ExecDetail.CommitDetail.ResolveLockTime, maxResolveLockTime: stmtExecInfo1.ExecDetail.CommitDetail.ResolveLockTime, sumWriteKeys: int64(stmtExecInfo1.ExecDetail.CommitDetail.WriteKeys), @@ -141,6 +142,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { firstSeen: stmtExecInfo1.StartTime, lastSeen: stmtExecInfo1.StartTime, } + stmtExecInfo1.ExecDetail.CommitDetail.Mu.Unlock() history := list.New() history.PushBack(&expectedSummaryElement) expectedSummary := stmtSummaryByDigest{ @@ -186,16 +188,17 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { BackoffTime: 180, RequestCount: 20, CommitDetail: &util.CommitDetails{ - GetCommitTsTime: 500, - PrewriteTime: 50000, - CommitTime: 5000, - LocalLatchTime: 50, - CommitBackoffTime: 1000, + GetCommitTsTime: 500, + PrewriteTime: 50000, + CommitTime: 5000, + LocalLatchTime: 50, Mu: struct { sync.Mutex - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string }{ - BackoffTypes: []string{boTxnLockName}, + CommitBackoffTime: 1000, + BackoffTypes: []string{boTxnLockName}, }, ResolveLockTime: 10000, WriteKeys: 100000, @@ -258,8 +261,10 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { expectedSummaryElement.maxCommitTime = stmtExecInfo2.ExecDetail.CommitDetail.CommitTime expectedSummaryElement.sumLocalLatchTime += stmtExecInfo2.ExecDetail.CommitDetail.LocalLatchTime expectedSummaryElement.maxLocalLatchTime = stmtExecInfo2.ExecDetail.CommitDetail.LocalLatchTime - expectedSummaryElement.sumCommitBackoffTime += stmtExecInfo2.ExecDetail.CommitDetail.CommitBackoffTime - expectedSummaryElement.maxCommitBackoffTime = stmtExecInfo2.ExecDetail.CommitDetail.CommitBackoffTime + stmtExecInfo2.ExecDetail.CommitDetail.Mu.Lock() + expectedSummaryElement.sumCommitBackoffTime += stmtExecInfo2.ExecDetail.CommitDetail.Mu.CommitBackoffTime + expectedSummaryElement.maxCommitBackoffTime = stmtExecInfo2.ExecDetail.CommitDetail.Mu.CommitBackoffTime + stmtExecInfo2.ExecDetail.CommitDetail.Mu.Unlock() expectedSummaryElement.sumResolveLockTime += stmtExecInfo2.ExecDetail.CommitDetail.ResolveLockTime expectedSummaryElement.maxResolveLockTime = stmtExecInfo2.ExecDetail.CommitDetail.ResolveLockTime expectedSummaryElement.sumWriteKeys += int64(stmtExecInfo2.ExecDetail.CommitDetail.WriteKeys) @@ -313,16 +318,17 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { BackoffTime: 18, RequestCount: 2, CommitDetail: &util.CommitDetails{ - GetCommitTsTime: 50, - PrewriteTime: 5000, - CommitTime: 500, - LocalLatchTime: 5, - CommitBackoffTime: 100, + GetCommitTsTime: 50, + PrewriteTime: 5000, + CommitTime: 500, + LocalLatchTime: 5, Mu: struct { sync.Mutex - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string }{ - BackoffTypes: []string{boTxnLockName}, + CommitBackoffTime: 100, + BackoffTypes: []string{boTxnLockName}, }, ResolveLockTime: 1000, WriteKeys: 10000, @@ -370,7 +376,9 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) { expectedSummaryElement.sumPrewriteTime += stmtExecInfo3.ExecDetail.CommitDetail.PrewriteTime expectedSummaryElement.sumCommitTime += stmtExecInfo3.ExecDetail.CommitDetail.CommitTime expectedSummaryElement.sumLocalLatchTime += stmtExecInfo3.ExecDetail.CommitDetail.LocalLatchTime - expectedSummaryElement.sumCommitBackoffTime += stmtExecInfo3.ExecDetail.CommitDetail.CommitBackoffTime + stmtExecInfo3.ExecDetail.CommitDetail.Mu.Lock() + expectedSummaryElement.sumCommitBackoffTime += stmtExecInfo3.ExecDetail.CommitDetail.Mu.CommitBackoffTime + stmtExecInfo3.ExecDetail.CommitDetail.Mu.Unlock() expectedSummaryElement.sumResolveLockTime += stmtExecInfo3.ExecDetail.CommitDetail.ResolveLockTime expectedSummaryElement.sumWriteKeys += int64(stmtExecInfo3.ExecDetail.CommitDetail.WriteKeys) expectedSummaryElement.sumWriteSize += int64(stmtExecInfo3.ExecDetail.CommitDetail.WriteSize) @@ -569,16 +577,17 @@ func generateAnyExecInfo() *StmtExecInfo { BackoffTime: 80, RequestCount: 10, CommitDetail: &util.CommitDetails{ - GetCommitTsTime: 100, - PrewriteTime: 10000, - CommitTime: 1000, - LocalLatchTime: 10, - CommitBackoffTime: 200, + GetCommitTsTime: 100, + PrewriteTime: 10000, + CommitTime: 1000, + LocalLatchTime: 10, Mu: struct { sync.Mutex - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string }{ - BackoffTypes: []string{boTxnLockName}, + CommitBackoffTime: 200, + BackoffTypes: []string{boTxnLockName}, }, ResolveLockTime: 2000, WriteKeys: 20000, @@ -628,6 +637,7 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) { n := types.NewTime(types.FromGoTime(time.Unix(s.ssMap.beginTimeForCurInterval, 0)), mysql.TypeTimestamp, types.DefaultFsp) e := types.NewTime(types.FromGoTime(time.Unix(s.ssMap.beginTimeForCurInterval+1800, 0)), mysql.TypeTimestamp, types.DefaultFsp) t := types.NewTime(types.FromGoTime(stmtExecInfo1.StartTime), mysql.TypeTimestamp, types.DefaultFsp) + stmtExecInfo1.ExecDetail.CommitDetail.Mu.Lock() expectedDatum := []interface{}{n, e, "Select", stmtExecInfo1.SchemaName, stmtExecInfo1.Digest, stmtExecInfo1.NormalizedSQL, "db1.tb1,db2.tb2", "a", stmtExecInfo1.User, 1, 0, 0, int64(stmtExecInfo1.TotalLatency), int64(stmtExecInfo1.TotalLatency), int64(stmtExecInfo1.TotalLatency), int64(stmtExecInfo1.TotalLatency), @@ -646,7 +656,7 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) { int64(stmtExecInfo1.ExecDetail.CommitDetail.PrewriteTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.PrewriteTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.CommitTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.CommitTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.GetCommitTsTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.GetCommitTsTime), - stmtExecInfo1.ExecDetail.CommitDetail.CommitBackoffTime, stmtExecInfo1.ExecDetail.CommitDetail.CommitBackoffTime, + stmtExecInfo1.ExecDetail.CommitDetail.Mu.CommitBackoffTime, stmtExecInfo1.ExecDetail.CommitDetail.Mu.CommitBackoffTime, stmtExecInfo1.ExecDetail.CommitDetail.ResolveLockTime, stmtExecInfo1.ExecDetail.CommitDetail.ResolveLockTime, int64(stmtExecInfo1.ExecDetail.CommitDetail.LocalLatchTime), int64(stmtExecInfo1.ExecDetail.CommitDetail.LocalLatchTime), stmtExecInfo1.ExecDetail.CommitDetail.WriteKeys, stmtExecInfo1.ExecDetail.CommitDetail.WriteKeys, @@ -656,6 +666,7 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) { fmt.Sprintf("%s:1", boTxnLockName), stmtExecInfo1.MemMax, stmtExecInfo1.MemMax, stmtExecInfo1.DiskMax, stmtExecInfo1.DiskMax, 0, 0, 0, 0, 0, stmtExecInfo1.StmtCtx.AffectedRows(), t, t, 0, 0, 0, stmtExecInfo1.OriginalSQL, stmtExecInfo1.PrevSQL, "plan_digest", ""} + stmtExecInfo1.ExecDetail.CommitDetail.Mu.Unlock() match(c, datums[0], expectedDatum...) datums = s.ssMap.ToHistoryDatum(nil, true) c.Assert(len(datums), Equals, 1) From f3e2b798221bfc721f02d92449ec24788202746c Mon Sep 17 00:00:00 2001 From: Shirly Date: Mon, 7 Jun 2021 18:08:28 +0800 Subject: [PATCH 318/343] store/tikv: share type OneByOneSuite for tests related to tikv (#25098) --- session/pessimistic_test.go | 8 +-- session/session_test.go | 10 ++-- store/driver/sql_fail_test.go | 3 +- store/driver/util_test.go | 26 ++-------- store/tikv/client/client_fail_test.go | 4 +- store/tikv/client/client_test.go | 25 +--------- store/tikv/mockstore/test_suite.go | 58 ++++++++++++++++++++++ store/tikv/tests/1pc_test.go | 5 +- store/tikv/tests/async_commit_fail_test.go | 11 ++-- store/tikv/tests/async_commit_test.go | 7 +-- store/tikv/tests/snapshot_fail_test.go | 5 +- store/tikv/tests/ticlient_slow_test.go | 3 +- store/tikv/tests/util_test.go | 26 ++-------- store/tikv/tikv_test.go | 28 +---------- 14 files changed, 102 insertions(+), 117 deletions(-) create mode 100644 store/tikv/mockstore/test_suite.go diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index 6add080091fbf..c0d82b8006b84 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -35,7 +35,9 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" storeerr "github.com/pingcap/tidb/store/driver/error" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/store/tikv/mockstore" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" @@ -231,7 +233,7 @@ func (s *testPessimisticSuite) TestDeadlock(c *C) { } func (s *testPessimisticSuite) TestSingleStatementRollback(c *C) { - if *withTiKV { + if *mockstore.WithTiKV { c.Skip("skip with tikv because cluster manipulate is not available") } tk := testkit.NewTestKitWithInit(c, s.store) @@ -2077,7 +2079,7 @@ func (s *testPessimisticSuite) TestSelectForUpdateConflictRetry(c *C) { func (s *testPessimisticSuite) TestAsyncCommitWithSchemaChange(c *C) { // TODO: implement commit_ts calculation in unistore - if !*withTiKV { + if !*mockstore.WithTiKV { return } @@ -2151,7 +2153,7 @@ func (s *testPessimisticSuite) TestAsyncCommitWithSchemaChange(c *C) { func (s *testPessimisticSuite) Test1PCWithSchemaChange(c *C) { // TODO: implement commit_ts calculation in unistore - if !*withTiKV { + if !*mockstore.WithTiKV { return } diff --git a/session/session_test.go b/session/session_test.go index 0e9364018c574..1328fdbce2c0d 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -52,6 +52,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/tikv" + tikvmockstore "github.com/pingcap/tidb/store/tikv/mockstore" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" tikvutil "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/table/tables" @@ -67,7 +68,6 @@ import ( ) var ( - withTiKV = flag.Bool("with-tikv", false, "run tests with TiKV cluster started. (not use the mock server)") pdAddrs = flag.String("pd-addrs", "127.0.0.1:2379", "pd addrs") pdAddrChan chan string initPdAddrsOnce sync.Once @@ -183,7 +183,7 @@ func initPdAddrs() { func (s *testSessionSuiteBase) SetUpSuite(c *C) { testleak.BeforeTest() - if *withTiKV { + if *tikvmockstore.WithTiKV { initPdAddrs() s.pdAddr = <-pdAddrChan var d driver.TiKVDriver @@ -219,7 +219,7 @@ func (s *testSessionSuiteBase) TearDownSuite(c *C) { s.dom.Close() s.store.Close() testleak.AfterTest(c)() - if *withTiKV { + if *tikvmockstore.WithTiKV { pdAddrChan <- s.pdAddr } } @@ -3370,7 +3370,7 @@ func (s *testSessionSerialSuite) TestSetTxnScope(c *C) { func (s *testSessionSerialSuite) TestGlobalAndLocalTxn(c *C) { // Because the PD config of check_dev_2 test is not compatible with local/global txn yet, // so we will skip this test for now. - if *withTiKV { + if *tikvmockstore.WithTiKV { return } tk := testkit.NewTestKitWithInit(c, s.store) @@ -3766,7 +3766,7 @@ func (s *testSessionSerialSuite) TestDoDDLJobQuit(c *C) { func (s *testBackupRestoreSuite) TestBackupAndRestore(c *C) { // only run BR SQL integration test with tikv store. - if *withTiKV { + if *tikvmockstore.WithTiKV { cfg := config.GetGlobalConfig() cfg.Store = "tikv" cfg.Path = s.pdAddr diff --git a/store/driver/sql_fail_test.go b/store/driver/sql_fail_test.go index dd0dcc6538c44..3618c5ec59b15 100644 --- a/store/driver/sql_fail_test.go +++ b/store/driver/sql_fail_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/store/tikv/mockstore" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" ) @@ -54,7 +55,7 @@ func (s *testSQLSuiteBase) SetUpSuite(c *C) { var err error s.store = NewTestStore(c) // actual this is better done in `OneByOneSuite.SetUpSuite`, but this would cause circle dependency - if *WithTiKV { + if *mockstore.WithTiKV { session.ResetStoreForWithTiKVTest(s.store) } diff --git a/store/driver/util_test.go b/store/driver/util_test.go index f46a482bd823e..a740b055b052b 100644 --- a/store/driver/util_test.go +++ b/store/driver/util_test.go @@ -17,7 +17,6 @@ import ( "context" "flag" "fmt" - "sync" "testing" . "github.com/pingcap/check" @@ -26,6 +25,7 @@ import ( "github.com/pingcap/tidb/store/copr" "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/store/tikv/mockstore" ) func TestT(t *testing.T) { @@ -34,9 +34,7 @@ func TestT(t *testing.T) { } var ( - withTiKVGlobalLock sync.RWMutex - WithTiKV = flag.Bool("with-tikv", false, "run tests with TiKV cluster started. (not use the mock server)") - pdAddrs = flag.String("pd-addrs", "127.0.0.1:2379", "pd addrs") + pdAddrs = flag.String("pd-addrs", "127.0.0.1:2379", "pd addrs") ) // NewTestStore creates a kv.Storage for testing purpose. @@ -45,7 +43,7 @@ func NewTestStore(c *C) kv.Storage { flag.Parse() } - if *WithTiKV { + if *mockstore.WithTiKV { var d TiKVDriver store, err := d.Open(fmt.Sprintf("tikv://%s", *pdAddrs)) c.Assert(err, IsNil) @@ -82,20 +80,4 @@ func clearStorage(store kv.Storage) error { } // OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one. -type OneByOneSuite struct{} - -func (s *OneByOneSuite) SetUpSuite(c *C) { - if *WithTiKV { - withTiKVGlobalLock.Lock() - } else { - withTiKVGlobalLock.RLock() - } -} - -func (s *OneByOneSuite) TearDownSuite(c *C) { - if *WithTiKV { - withTiKVGlobalLock.Unlock() - } else { - withTiKVGlobalLock.RUnlock() - } -} +type OneByOneSuite = mockstore.OneByOneSuite diff --git a/store/tikv/client/client_fail_test.go b/store/tikv/client/client_fail_test.go index e1279886162a0..e443d46d9184e 100644 --- a/store/tikv/client/client_fail_test.go +++ b/store/tikv/client/client_fail_test.go @@ -33,11 +33,11 @@ type testClientFailSuite struct { func (s *testClientFailSuite) SetUpSuite(_ *C) { // This lock make testClientFailSuite runs exclusively. - withTiKVGlobalLock.Lock() + s.LockGlobalTiKV() } func (s testClientFailSuite) TearDownSuite(_ *C) { - withTiKVGlobalLock.Unlock() + s.UnLockGlobalTiKV() } func (s *testClientFailSuite) TestPanicInRecvLoop(c *C) { diff --git a/store/tikv/client/client_test.go b/store/tikv/client/client_test.go index 7b4b652f55c85..b3645d990723a 100644 --- a/store/tikv/client/client_test.go +++ b/store/tikv/client/client_test.go @@ -15,7 +15,6 @@ package client import ( "context" - "flag" "fmt" "sync" "sync/atomic" @@ -29,33 +28,13 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/tikvpb" "github.com/pingcap/tidb/store/tikv/config" + "github.com/pingcap/tidb/store/tikv/mockstore" "github.com/pingcap/tidb/store/tikv/tikvrpc" "google.golang.org/grpc/metadata" ) -var ( - withTiKVGlobalLock sync.RWMutex - WithTiKV = flag.Bool("with-tikv", false, "run tests with TiKV cluster started. (not use the mock server)") -) - // OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one. -type OneByOneSuite struct{} - -func (s *OneByOneSuite) SetUpSuite(c *C) { - if *WithTiKV { - withTiKVGlobalLock.Lock() - } else { - withTiKVGlobalLock.RLock() - } -} - -func (s *OneByOneSuite) TearDownSuite(c *C) { - if *WithTiKV { - withTiKVGlobalLock.Unlock() - } else { - withTiKVGlobalLock.RUnlock() - } -} +type OneByOneSuite = mockstore.OneByOneSuite func TestT(t *testing.T) { CustomVerboseFlag = true diff --git a/store/tikv/mockstore/test_suite.go b/store/tikv/mockstore/test_suite.go new file mode 100644 index 0000000000000..9ec7a0fa8aeef --- /dev/null +++ b/store/tikv/mockstore/test_suite.go @@ -0,0 +1,58 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package mockstore + +import ( + "flag" + "sync" + + "github.com/pingcap/check" +) + +var ( + withTiKVGlobalLock sync.RWMutex + // WithTiKV is the flag which indicates whether it runs with tikv. + WithTiKV = flag.Bool("with-tikv", false, "run tests with TiKV cluster started. (not use the mock server)") +) + +// OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one. +type OneByOneSuite struct{} + +// SetUpSuite implements the interface check.Suite. +func (s *OneByOneSuite) SetUpSuite(c *check.C) { + if *WithTiKV { + withTiKVGlobalLock.Lock() + } else { + withTiKVGlobalLock.RLock() + } +} + +// TearDownSuite implements the interface check.Suite. +func (s *OneByOneSuite) TearDownSuite(c *check.C) { + if *WithTiKV { + withTiKVGlobalLock.Unlock() + } else { + withTiKVGlobalLock.RUnlock() + } +} + +// LockGlobalTiKV locks withTiKVGlobalLock. +func (s *OneByOneSuite) LockGlobalTiKV() { + withTiKVGlobalLock.Lock() +} + +// UnLockGlobalTiKV unlocks withTiKVGlobalLock +func (s *OneByOneSuite) UnLockGlobalTiKV() { + withTiKVGlobalLock.Unlock() +} diff --git a/store/tikv/tests/1pc_test.go b/store/tikv/tests/1pc_test.go index 6ff57700f4e3c..2b7e5501f1600 100644 --- a/store/tikv/tests/1pc_test.go +++ b/store/tikv/tests/1pc_test.go @@ -19,6 +19,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/metrics" + "github.com/pingcap/tidb/store/tikv/mockstore" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/util" ) @@ -179,7 +180,7 @@ func (s *testOnePCSuite) Test1PCIsolation(c *C) { func (s *testOnePCSuite) Test1PCDisallowMultiRegion(c *C) { // This test doesn't support tikv mode. - if *WithTiKV { + if *mockstore.WithTiKV { return } @@ -247,7 +248,7 @@ func (s *testOnePCSuite) Test1PCLinearizability(c *C) { func (s *testOnePCSuite) Test1PCWithMultiDC(c *C) { // It requires setting placement rules to run with TiKV - if *WithTiKV { + if *mockstore.WithTiKV { return } diff --git a/store/tikv/tests/async_commit_fail_test.go b/store/tikv/tests/async_commit_fail_test.go index cfe1ad0595eb1..e6d15f847bbe7 100644 --- a/store/tikv/tests/async_commit_fail_test.go +++ b/store/tikv/tests/async_commit_fail_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/mockstore" "github.com/pingcap/tidb/store/tikv/util" ) @@ -43,7 +44,7 @@ func (s *testAsyncCommitFailSuite) SetUpTest(c *C) { // committing primary region task. func (s *testAsyncCommitFailSuite) TestFailAsyncCommitPrewriteRpcErrors(c *C) { // This test doesn't support tikv mode because it needs setting failpoint in unistore. - if *WithTiKV { + if *mockstore.WithTiKV { return } @@ -75,7 +76,7 @@ func (s *testAsyncCommitFailSuite) TestFailAsyncCommitPrewriteRpcErrors(c *C) { func (s *testAsyncCommitFailSuite) TestAsyncCommitPrewriteCancelled(c *C) { // This test doesn't support tikv mode because it needs setting failpoint in unistore. - if *WithTiKV { + if *mockstore.WithTiKV { return } @@ -135,7 +136,7 @@ func (s *testAsyncCommitFailSuite) TestPointGetWithAsyncCommit(c *C) { func (s *testAsyncCommitFailSuite) TestSecondaryListInPrimaryLock(c *C) { // This test doesn't support tikv mode. - if *WithTiKV { + if *mockstore.WithTiKV { return } @@ -237,7 +238,7 @@ func (s *testAsyncCommitFailSuite) TestAsyncCommitContextCancelCausingUndetermin // TestAsyncCommitRPCErrorThenWriteConflict verifies that the determined failure error overwrites undetermined error. func (s *testAsyncCommitFailSuite) TestAsyncCommitRPCErrorThenWriteConflict(c *C) { // This test doesn't support tikv mode because it needs setting failpoint in unistore. - if *WithTiKV { + if *mockstore.WithTiKV { return } @@ -260,7 +261,7 @@ func (s *testAsyncCommitFailSuite) TestAsyncCommitRPCErrorThenWriteConflict(c *C // overwrites the undetermined error in the parent. func (s *testAsyncCommitFailSuite) TestAsyncCommitRPCErrorThenWriteConflictInChild(c *C) { // This test doesn't support tikv mode because it needs setting failpoint in unistore. - if *WithTiKV { + if *mockstore.WithTiKV { return } diff --git a/store/tikv/tests/async_commit_test.go b/store/tikv/tests/async_commit_test.go index f67482e69a44e..4afb548204c13 100644 --- a/store/tikv/tests/async_commit_test.go +++ b/store/tikv/tests/async_commit_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/mockstore" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -47,7 +48,7 @@ type testAsyncCommitCommon struct { } func (s *testAsyncCommitCommon) setUpTest(c *C) { - if *WithTiKV { + if *mockstore.WithTiKV { s.store = NewTestStore(c) return } @@ -195,7 +196,7 @@ func (s *testAsyncCommitSuite) lockKeysWithAsyncCommit(c *C, keys, values [][]by func (s *testAsyncCommitSuite) TestCheckSecondaries(c *C) { // This test doesn't support tikv mode. - if *WithTiKV { + if *mockstore.WithTiKV { return } @@ -402,7 +403,7 @@ func (s *testAsyncCommitSuite) TestAsyncCommitLinearizability(c *C) { // TestAsyncCommitWithMultiDC tests that async commit can only be enabled in global transactions func (s *testAsyncCommitSuite) TestAsyncCommitWithMultiDC(c *C) { // It requires setting placement rules to run with TiKV - if *WithTiKV { + if *mockstore.WithTiKV { return } diff --git a/store/tikv/tests/snapshot_fail_test.go b/store/tikv/tests/snapshot_fail_test.go index ed812b4f46e00..6740dafdc523b 100644 --- a/store/tikv/tests/snapshot_fail_test.go +++ b/store/tikv/tests/snapshot_fail_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" + "github.com/pingcap/tidb/store/tikv/mockstore" ) type testSnapshotFailSuite struct { @@ -62,7 +63,7 @@ func (s *testSnapshotFailSuite) cleanup(c *C) { func (s *testSnapshotFailSuite) TestBatchGetResponseKeyError(c *C) { // Meaningless to test with tikv because it has a mock key error - if *WithTiKV { + if *mockstore.WithTiKV { return } defer s.cleanup(c) @@ -91,7 +92,7 @@ func (s *testSnapshotFailSuite) TestBatchGetResponseKeyError(c *C) { func (s *testSnapshotFailSuite) TestScanResponseKeyError(c *C) { // Meaningless to test with tikv because it has a mock key error - if *WithTiKV { + if *mockstore.WithTiKV { return } defer s.cleanup(c) diff --git a/store/tikv/tests/ticlient_slow_test.go b/store/tikv/tests/ticlient_slow_test.go index 61f6748874d14..97f0edcfda397 100644 --- a/store/tikv/tests/ticlient_slow_test.go +++ b/store/tikv/tests/ticlient_slow_test.go @@ -21,10 +21,11 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/store/tikv/mockstore" ) func (s *testTiclientSuite) TestSplitRegionIn2PC(c *C) { - if *WithTiKV { + if *mockstore.WithTiKV { c.Skip("scatter will timeout with single node TiKV") } config := tikv.ConfigProbe{} diff --git a/store/tikv/tests/util_test.go b/store/tikv/tests/util_test.go index b39584393f12b..2fb841b059716 100644 --- a/store/tikv/tests/util_test.go +++ b/store/tikv/tests/util_test.go @@ -18,7 +18,6 @@ import ( "flag" "fmt" "strings" - "sync" "unsafe" . "github.com/pingcap/check" @@ -28,14 +27,13 @@ import ( "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" + "github.com/pingcap/tidb/store/tikv/mockstore" "github.com/pingcap/tidb/store/tikv/util/codec" pd "github.com/tikv/pd/client" ) var ( - withTiKVGlobalLock sync.RWMutex - WithTiKV = flag.Bool("with-tikv", false, "run tests with TiKV cluster started. (not use the mock server)") - pdAddrs = flag.String("pd-addrs", "127.0.0.1:2379", "pd addrs") + pdAddrs = flag.String("pd-addrs", "127.0.0.1:2379", "pd addrs") ) // NewTestStore creates a KVStore for testing purpose. @@ -44,7 +42,7 @@ func NewTestStore(c *C) *tikv.KVStore { flag.Parse() } - if *WithTiKV { + if *mockstore.WithTiKV { addrs := strings.Split(*pdAddrs, ",") pdClient, err := pd.NewClient(addrs, pd.SecurityOption{}) c.Assert(err, IsNil) @@ -86,23 +84,7 @@ func clearStorage(store *tikv.KVStore) error { } // OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one. -type OneByOneSuite struct{} - -func (s *OneByOneSuite) SetUpSuite(c *C) { - if *WithTiKV { - withTiKVGlobalLock.Lock() - } else { - withTiKVGlobalLock.RLock() - } -} - -func (s *OneByOneSuite) TearDownSuite(c *C) { - if *WithTiKV { - withTiKVGlobalLock.Unlock() - } else { - withTiKVGlobalLock.RUnlock() - } -} +type OneByOneSuite = mockstore.OneByOneSuite func encodeKey(prefix, s string) []byte { return codec.EncodeBytes(nil, []byte(fmt.Sprintf("%s_%s", prefix, s))) diff --git a/store/tikv/tikv_test.go b/store/tikv/tikv_test.go index 391af8e2663f9..1a4fc28fed7b9 100644 --- a/store/tikv/tikv_test.go +++ b/store/tikv/tikv_test.go @@ -14,39 +14,15 @@ package tikv import ( - "flag" "os" - "sync" "testing" . "github.com/pingcap/check" + "github.com/pingcap/tidb/store/tikv/mockstore" "github.com/pingcap/tidb/util/logutil" ) -var ( - withTiKVGlobalLock sync.RWMutex - WithTiKV = flag.Bool("with-tikv", false, "run tests with TiKV cluster started. (not use the mock server)") -) - -// OneByOneSuite is a suite, When with-tikv flag is true, there is only one storage, so the test suite have to run one by one. -type OneByOneSuite struct{} - -func (s *OneByOneSuite) SetUpSuite(c *C) { - if *WithTiKV { - withTiKVGlobalLock.Lock() - } else { - withTiKVGlobalLock.RLock() - } -} - -func (s *OneByOneSuite) TearDownSuite(c *C) { - if *WithTiKV { - withTiKVGlobalLock.Unlock() - } else { - withTiKVGlobalLock.RUnlock() - } -} - +type OneByOneSuite = mockstore.OneByOneSuite type testTiKVSuite struct { OneByOneSuite } From ae20fa83535ad3c96bb08e7d1da100859c0a3a27 Mon Sep 17 00:00:00 2001 From: bb7133 Date: Mon, 7 Jun 2021 20:14:28 +0800 Subject: [PATCH 319/343] executor: skip unstable test case TestApplyGoroutinePanic (#25180) --- executor/parallel_apply_test.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/executor/parallel_apply_test.go b/executor/parallel_apply_test.go index a72944ebda507..93448f0e3f92f 100644 --- a/executor/parallel_apply_test.go +++ b/executor/parallel_apply_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/testkit" ) @@ -569,6 +570,10 @@ func (s *testSuite) TestApplyCacheRatio(c *C) { } func (s *testSuite) TestApplyGoroutinePanic(c *C) { + if israce.RaceEnabled { + c.Skip("race detected, skip it temporarily and fix it before 20210619") + } + tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("set tidb_enable_parallel_apply=true") tk.MustExec("drop table if exists t1, t2") From 5005bc08cb33a6fc54613c0fb73ffdb46b6934c9 Mon Sep 17 00:00:00 2001 From: Howie Date: Mon, 7 Jun 2021 20:28:29 +0800 Subject: [PATCH 320/343] ddl: tidb panic while query hash partition table with is null condition (#23849) --- planner/core/integration_test.go | 15 +++++++++++++++ planner/core/rule_partition_processor.go | 11 +++++++---- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index b9329eb741bf9..fa936be24e4fc 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3649,6 +3649,21 @@ func (s *testIntegrationSuite) TestIssue23736(c *C) { c.Assert(tk.MustUseIndex("select /*+ stream_agg() */ count(1) from t0 where c > 10 and b < 2", "c"), IsFalse) } +// https://github.com/pingcap/tidb/issues/23802 +func (s *testIntegrationSuite) TestPanicWhileQueryTableWithIsNull(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("drop table if exists NT_HP27193") + tk.MustExec("CREATE TABLE `NT_HP27193` ( `COL1` int(20) DEFAULT NULL, `COL2` varchar(20) DEFAULT NULL, `COL4` datetime DEFAULT NULL, `COL3` bigint(20) DEFAULT NULL, `COL5` float DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY HASH ( `COL1`%`COL3` ) PARTITIONS 10;") + _, err := tk.Exec("select col1 from NT_HP27193 where col1 is null;") + c.Assert(err, IsNil) + tk.MustExec("INSERT INTO NT_HP27193 (COL2, COL4, COL3, COL5) VALUES ('m', '2020-05-04 13:15:27', 8, 2602)") + _, err = tk.Exec("select col1 from NT_HP27193 where col1 is null;") + c.Assert(err, IsNil) + tk.MustExec("drop table if exists NT_HP27193") +} + func (s *testIntegrationSuite) TestIssue23846(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 17f8c5b2a88b1..90a03991864ab 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -129,11 +129,11 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl partIdx[i].Index = i colLen = append(colLen, types.UnspecifiedLength) } - datchedResult, err := ranger.DetachCondAndBuildRangeForPartition(ctx, conds, partIdx, colLen) + detachedResult, err := ranger.DetachCondAndBuildRangeForPartition(ctx, conds, partIdx, colLen) if err != nil { return nil, nil, err } - ranges := datchedResult.Ranges + ranges := detachedResult.Ranges used := make([]int, 0, len(ranges)) for _, r := range ranges { if r.IsPointNullable(ctx.GetSessionVars().StmtCtx) { @@ -143,7 +143,10 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl break } } - pos, isNull, err := pe.EvalInt(ctx, chunk.MutRowFromDatums(r.HighVal).ToRow()) + highLowVals := make([]types.Datum, 0, len(r.HighVal)+len(r.LowVal)) + highLowVals = append(highLowVals, r.HighVal...) + highLowVals = append(highLowVals, r.LowVal...) + pos, isNull, err := pe.EvalInt(ctx, chunk.MutRowFromDatums(highLowVals).ToRow()) if err != nil { return nil, nil, err } @@ -225,7 +228,7 @@ func (s *partitionProcessor) findUsedPartitions(ctx sessionctx.Context, tbl tabl ret = append(ret, used[i]) } } - return ret, datchedResult.RemainedConds, nil + return ret, detachedResult.RemainedConds, nil } func (s *partitionProcessor) convertToIntSlice(or partitionRangeOR, pi *model.PartitionInfo, partitionNames []model.CIStr) []int { From d1613086e151aa11cac5d85930e7994f133cbd30 Mon Sep 17 00:00:00 2001 From: hykych <455905706@qq.com> Date: Mon, 7 Jun 2021 20:40:28 +0800 Subject: [PATCH 321/343] go.mod, go.sum: upgrade automaxprocs version from v1.2.0 to v1.4.0 (#24903) --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 7004ac7b657a2..daf00d34eb2fc 100644 --- a/go.mod +++ b/go.mod @@ -66,7 +66,7 @@ require ( github.com/xitongsys/parquet-go v1.5.5-0.20201110004701-b09c49d6d457 // indirect go.etcd.io/etcd v0.5.0-alpha.5.0.20200824191128-ae9734ed278b go.uber.org/atomic v1.7.0 - go.uber.org/automaxprocs v1.2.0 + go.uber.org/automaxprocs v1.4.0 go.uber.org/multierr v1.7.0 // indirect go.uber.org/zap v1.16.0 golang.org/x/crypto v0.0.0-20200820211705-5c72a883971a // indirect diff --git a/go.sum b/go.sum index be3ebf435d139..bab7be1bc60be 100644 --- a/go.sum +++ b/go.sum @@ -616,8 +616,8 @@ go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/automaxprocs v1.2.0 h1:+RUihKM+nmYUoB9w0D0Ov5TJ2PpFO2FgenTxMJiZBZA= -go.uber.org/automaxprocs v1.2.0/go.mod h1:YfO3fm683kQpzETxlTGZhGIVmXAhaw3gxeBADbpZtnU= +go.uber.org/automaxprocs v1.4.0 h1:CpDZl6aOlLhReez+8S3eEotD7Jx0Os++lemPlMULQP0= +go.uber.org/automaxprocs v1.4.0/go.mod h1:/mTEdr7LvHhs0v7mjdxDreTz1OG5zdZGqgOnhWiR/+Q= go.uber.org/dig v1.8.0/go.mod h1:X34SnWGr8Fyla9zQNO2GSO2D+TIuqB14OS8JhYocIyw= go.uber.org/fx v1.10.0/go.mod h1:vLRicqpG/qQEzno4SYU86iCwfT95EZza+Eba0ItuxqY= go.uber.org/goleak v0.10.0 h1:G3eWbSNIskeRqtsN/1uI5B+eP73y3JUuBsv9AZjehb4= From 811253785c0ed3ac84904517a73a2667f70f19f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Mon, 7 Jun 2021 21:06:28 +0800 Subject: [PATCH 322/343] planner, executor: add stale read compatibility for temporary table (#25206) --- executor/adapter.go | 4 ++ executor/builder.go | 47 ++++++++++++++++++-- executor/compiler.go | 21 ++++----- executor/point_get.go | 5 +++ executor/stale_txn_test.go | 90 ++++++++++++++++++++++++++++++++++++++ planner/core/preprocess.go | 6 ++- 6 files changed, 158 insertions(+), 15 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 308676dc704a7..b324fdb331a69 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -189,6 +189,8 @@ type ExecStmt struct { // SnapshotTS stores the timestamp for stale read. // It is not equivalent to session variables's snapshot ts, it only use to build the executor. SnapshotTS uint64 + // ExplicitStaleness means whether the 'SELECT' clause are using 'AS OF TIMESTAMP' to perform stale read explicitly. + ExplicitStaleness bool // InfoSchema stores a reference to the schema information. InfoSchema infoschema.InfoSchema // Plan stores a reference to the final physical plan. @@ -288,6 +290,7 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) { } a.InfoSchema = ret.InfoSchema a.SnapshotTS = ret.SnapshotTS + a.ExplicitStaleness = ret.ExplicitStaleness p, names, err := planner.Optimize(ctx, a.Ctx, a.StmtNode, a.InfoSchema) if err != nil { return 0, err @@ -791,6 +794,7 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { b := newExecutorBuilder(ctx, a.InfoSchema, a.Ti) b.snapshotTS = a.SnapshotTS + b.explicitStaleness = a.ExplicitStaleness e := b.build(a.Plan) if b.err != nil { return nil, errors.Trace(b.err) diff --git a/executor/builder.go b/executor/builder.go index 4a1b45965a531..0662b963c57d2 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -83,6 +83,8 @@ type executorBuilder struct { err error // err is set when there is error happened during Executor building process. hasLock bool Ti *TelemetryInfo + // ExplicitStaleness means whether the 'SELECT' clause are using 'AS OF TIMESTAMP' to perform stale read explicitly. + explicitStaleness bool } // CTEStorages stores resTbl and iterInTbl for CTEExec. @@ -1371,6 +1373,11 @@ func (b *executorBuilder) buildTableDual(v *plannercore.PhysicalTableDual) Execu return e } +// IsStaleness returns if the query is staleness +func (b *executorBuilder) IsStaleness() bool { + return b.ctx.GetSessionVars().TxnCtx.IsStaleness || b.explicitStaleness +} + // `getSnapshotTS` returns the timestamp of the snapshot that a reader should read. func (b *executorBuilder) getSnapshotTS() (uint64, error) { // `refreshForUpdateTSForRC` should always be invoked before returning the cached value to @@ -2642,6 +2649,10 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea return nil, err } ts := v.GetTableScan() + if ts.Table.TempTableType != model.TempTableNone && b.IsStaleness() { + return nil, errors.New("can not stale read temporary table") + } + tbl, _ := b.is.TableByID(ts.Table.ID) isPartition, physicalTableID := ts.IsPartition() if isPartition { @@ -2736,6 +2747,11 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E } ts := v.GetTableScan() + if ts.Table.TempTableType != model.TempTableNone && b.IsStaleness() { + b.err = errors.New("can not stale read temporary table") + return nil + } + ret.ranges = ts.Ranges sctx := b.ctx.GetSessionVars().StmtCtx sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) @@ -2944,13 +2960,18 @@ func buildNoRangeIndexReader(b *executorBuilder, v *plannercore.PhysicalIndexRea } func (b *executorBuilder) buildIndexReader(v *plannercore.PhysicalIndexReader) Executor { + is := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) + if is.Table.TempTableType != model.TempTableNone && b.IsStaleness() { + b.err = errors.New("can not stale read temporary table") + return nil + } + ret, err := buildNoRangeIndexReader(b, v) if err != nil { b.err = err return nil } - is := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) ret.ranges = is.Ranges sctx := b.ctx.GetSessionVars().StmtCtx sctx.IndexNames = append(sctx.IndexNames, is.Table.Name.O+":"+is.Index.Name.O) @@ -3091,13 +3112,18 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn } func (b *executorBuilder) buildIndexLookUpReader(v *plannercore.PhysicalIndexLookUpReader) Executor { + is := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) + if is.Table.TempTableType != model.TempTableNone && b.IsStaleness() { + b.err = errors.New("can not stale read temporary table") + return nil + } + ret, err := buildNoRangeIndexLookUpReader(b, v) if err != nil { b.err = err return nil } - is := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) ret.ranges = is.Ranges @@ -3197,6 +3223,12 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd } func (b *executorBuilder) buildIndexMergeReader(v *plannercore.PhysicalIndexMergeReader) Executor { + ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) + if ts.Table.TempTableType != model.TempTableNone && b.IsStaleness() { + b.err = errors.New("can not stale read temporary table") + return nil + } + ret, err := buildNoRangeIndexMergeReader(b, v) if err != nil { b.err = err @@ -3216,7 +3248,6 @@ func (b *executorBuilder) buildIndexMergeReader(v *plannercore.PhysicalIndexMerg } } } - ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) executorCounterIndexMergeReaderExecutor.Inc() @@ -3972,6 +4003,11 @@ func NewRowDecoder(ctx sessionctx.Context, schema *expression.Schema, tbl *model } func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan) Executor { + if plan.TblInfo.TempTableType != model.TempTableNone && b.IsStaleness() { + b.err = errors.New("can not stale read temporary table") + return nil + } + startTS, err := b.getSnapshotTS() if err != nil { b.err = err @@ -4104,6 +4140,11 @@ func fullRangePartition(idxArr []int) bool { } func (b *executorBuilder) buildTableSample(v *plannercore.PhysicalTableSample) *TableSampleExecutor { + if v.TableInfo.Meta().TempTableType != model.TempTableNone && b.IsStaleness() { + b.err = errors.New("can not stale read temporary table") + return nil + } + startTS, err := b.getSnapshotTS() if err != nil { b.err = err diff --git a/executor/compiler.go b/executor/compiler.go index 5f0454c66390a..c763c43067047 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -69,16 +69,17 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm lowerPriority = needLowerPriority(finalPlan) } return &ExecStmt{ - GoCtx: ctx, - SnapshotTS: ret.SnapshotTS, - InfoSchema: ret.InfoSchema, - Plan: finalPlan, - LowerPriority: lowerPriority, - Text: stmtNode.Text(), - StmtNode: stmtNode, - Ctx: c.Ctx, - OutputNames: names, - Ti: &TelemetryInfo{}, + GoCtx: ctx, + SnapshotTS: ret.SnapshotTS, + ExplicitStaleness: ret.ExplicitStaleness, + InfoSchema: ret.InfoSchema, + Plan: finalPlan, + LowerPriority: lowerPriority, + Text: stmtNode.Text(), + StmtNode: stmtNode, + Ctx: c.Ctx, + OutputNames: names, + Ti: &TelemetryInfo{}, }, nil } diff --git a/executor/point_get.go b/executor/point_get.go index 9d09a6acf6fb0..3d1908d0fe394 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -42,6 +42,11 @@ import ( ) func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { + if p.TblInfo.TempTableType != model.TempTableNone && b.IsStaleness() { + b.err = errors.New("can not stale read temporary table") + return nil + } + startTS, err := b.getSnapshotTS() if err != nil { b.err = err diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index 44def0981bed0..d7fa8a8bdec27 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -15,6 +15,7 @@ package executor_test import ( "fmt" + "strings" "time" . "github.com/pingcap/check" @@ -772,3 +773,92 @@ func (s *testStaleTxnSuite) TestSetTransactionInfoSchema(c *C) { tk.MustExec("commit") c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer3) } + +func (s *testStaleTxnSuite) TestStaleReadTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. + safePointName := "tikv_gc_safe_point" + safePointValue := "20160102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) + + tk.MustExec("set @@tidb_enable_global_temporary_table=1") + tk.MustExec("use test") + tk.MustExec("drop table if exists tmp1") + tk.MustExec("create global temporary table tmp1 " + + "(id int not null primary key, code int not null, value int default null, unique key code(code))" + + "on commit delete rows") + time.Sleep(time.Second) + tk.MustGetErrMsg("select * from tmp1 as of timestamp NOW() where id=1", "can not stale read temporary table") + + queries := []struct { + sql string + }{ + { + sql: "select * from tmp1 where id=1", + }, + { + sql: "select * from tmp1 where code=1", + }, + { + sql: "select * from tmp1 where id in (1, 2, 3)", + }, + { + sql: "select * from tmp1 where code in (1, 2, 3)", + }, + { + sql: "select * from tmp1 where id > 1", + }, + { + sql: "select /*+use_index(tmp1, code)*/ * from tmp1 where code > 1", + }, + { + sql: "select /*+use_index(tmp1, code)*/ code from tmp1 where code > 1", + }, + { + sql: "select * from tmp1 tablesample regions()", + }, + { + sql: "select /*+ use_index_merge(tmp1, primary, code) */ * from tmp1 where id > 1 or code > 2", + }, + } + + addStaleReadToSQL := func(sql string) string { + idx := strings.Index(sql, " where ") + if idx < 0 { + return "" + } + return sql[0:idx] + " as of timestamp NOW()" + sql[idx:] + } + + for _, query := range queries { + sql := addStaleReadToSQL(query.sql) + if sql != "" { + tk.MustGetErrMsg(sql, "can not stale read temporary table") + } + } + + tk.MustExec("start transaction read only as of timestamp NOW()") + for _, query := range queries { + tk.MustGetErrMsg(query.sql, "can not stale read temporary table") + } + tk.MustExec("commit") + + for _, query := range queries { + tk.MustExec(query.sql) + } + + tk.MustExec("set transaction read only as of timestamp NOW()") + tk.MustExec("start transaction") + for _, query := range queries { + tk.MustGetErrMsg(query.sql, "can not stale read temporary table") + } + tk.MustExec("commit") + + for _, query := range queries { + tk.MustExec(query.sql) + } +} diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 5e6f41b7b4fb8..70e1391d3361c 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -132,8 +132,9 @@ const ( // PreprocessorReturn is used to retain information obtained in the preprocessor. type PreprocessorReturn struct { - SnapshotTS uint64 - InfoSchema infoschema.InfoSchema + SnapshotTS uint64 + ExplicitStaleness bool + InfoSchema infoschema.InfoSchema } // preprocessor is an ast.Visitor that preprocess @@ -1420,6 +1421,7 @@ func (p *preprocessor) handleAsOf(node *ast.AsOfClause) { return } p.SnapshotTS = ts + p.ExplicitStaleness = true p.InfoSchema = is } if p.SnapshotTS != ts { From 9de175386600e07424b8668a9a11bf97e9db461a Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Mon, 7 Jun 2021 20:44:28 -0600 Subject: [PATCH 323/343] *: fix permissions of brie RESTORE to be RESTORE_ADMIN (#25039) --- executor/brie.go | 9 +++++++-- executor/executor_test.go | 1 + planner/core/logical_plan_test.go | 4 ++-- planner/core/planbuilder.go | 14 +++++++++++--- privilege/privileges/privileges.go | 1 + 5 files changed, 22 insertions(+), 7 deletions(-) diff --git a/executor/brie.go b/executor/brie.go index e26da72d3647b..a4438afd49575 100644 --- a/executor/brie.go +++ b/executor/brie.go @@ -403,9 +403,14 @@ func (gs *tidbGlueSession) CreateSession(store kv.Storage) (glue.Session, error) } // Execute implements glue.Session +// These queries execute without privilege checking, since the calling statements +// such as BACKUP and RESTORE have already been privilege checked. func (gs *tidbGlueSession) Execute(ctx context.Context, sql string) error { - // FIXME: br relies on a deprecated API, it may be unsafe - _, err := gs.se.(sqlexec.SQLExecutor).Execute(ctx, sql) + stmt, err := gs.se.(sqlexec.RestrictedSQLExecutor).ParseWithParams(ctx, sql) + if err != nil { + return err + } + _, _, err = gs.se.(sqlexec.RestrictedSQLExecutor).ExecRestrictedStmt(ctx, stmt) return err } diff --git a/executor/executor_test.go b/executor/executor_test.go index 744f4f29c7b95..439a6362c90be 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -358,6 +358,7 @@ func (s *testSuiteP1) TestShow(c *C) { "Update Tables To update existing rows", "Usage Server Admin No privileges - allow connect only", "BACKUP_ADMIN Server Admin ", + "RESTORE_ADMIN Server Admin ", "SYSTEM_VARIABLES_ADMIN Server Admin ", "ROLE_ADMIN Server Admin ", "CONNECTION_ADMIN Server Admin ", diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index f59aceb40ee86..1fdf5f3ab83ca 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -1257,7 +1257,7 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { { sql: "RESTORE DATABASE test FROM 'local:///tmp/a'", ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", false}, + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "RESTORE_ADMIN", false}, }, }, { @@ -1269,7 +1269,7 @@ func (s *testPlanSuite) TestVisitInfo(c *C) { { sql: "SHOW RESTORES", ans: []visitInfo{ - {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "BACKUP_ADMIN", false}, + {mysql.ExtendedPriv, "", "", "", ErrSpecificAccessDenied, false, "RESTORE_ADMIN", false}, }, }, { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 6b4d4118379f8..6f36d6d0a9ad8 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2245,9 +2245,12 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, case ast.ShowCreateView: err := ErrSpecificAccessDenied.GenWithStackByArgs("SHOW VIEW") b.visitInfo = appendVisitInfo(b.visitInfo, mysql.ShowViewPriv, show.Table.Schema.L, show.Table.Name.L, "", err) - case ast.ShowBackups, ast.ShowRestores: + case ast.ShowBackups: err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or BACKUP_ADMIN") b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "BACKUP_ADMIN", false, err) + case ast.ShowRestores: + err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or RESTORE_ADMIN") + b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "RESTORE_ADMIN", false, err) case ast.ShowTableNextRowId: p := &ShowNextRowID{TableName: show.Table} p.setSchemaAndNames(buildShowNextRowID()) @@ -2343,8 +2346,13 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, b.visitInfo = collectVisitInfoFromGrantStmt(b.ctx, b.visitInfo, raw) case *ast.BRIEStmt: p.setSchemaAndNames(buildBRIESchema()) - err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or BACKUP_ADMIN") - b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "BACKUP_ADMIN", false, err) + if raw.Kind == ast.BRIEKindRestore { + err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or RESTORE_ADMIN") + b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "RESTORE_ADMIN", false, err) + } else { + err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or BACKUP_ADMIN") + b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "BACKUP_ADMIN", false, err) + } case *ast.GrantRoleStmt: err := ErrSpecificAccessDenied.GenWithStackByArgs("SUPER or ROLE_ADMIN") b.visitInfo = appendDynamicVisitInfo(b.visitInfo, "ROLE_ADMIN", false, err) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index cb68345738396..29bf61d9a25bf 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -40,6 +40,7 @@ var SkipWithGrant = false var _ privilege.Manager = (*UserPrivileges)(nil) var dynamicPrivs = []string{ "BACKUP_ADMIN", + "RESTORE_ADMIN", "SYSTEM_VARIABLES_ADMIN", "ROLE_ADMIN", "CONNECTION_ADMIN", From e2f09ca2e166e65398e6b78371883004e12ea587 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 8 Jun 2021 14:26:28 +0800 Subject: [PATCH 324/343] store/tikv: make RegionStore private (#25231) --- store/tikv/region_cache.go | 50 +++++++++++++++++++------------------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 12295947b2448..a4ed1b6d23bd2 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -102,9 +102,9 @@ type Region struct { // AccessIndex represent the index for accessIndex array type AccessIndex int -// RegionStore represents region stores info +// regionStore represents region stores info // it will be store as unsafe.Pointer and be load at once -type RegionStore struct { +type regionStore struct { workTiKVIdx AccessIndex // point to current work peer in meta.Peers and work store in stores(same idx) for tikv peer proxyTiKVIdx AccessIndex // point to the tikv peer that can forward requests to the leader. -1 means not using proxy workTiFlashIdx int32 // point to current work peer in meta.Peers and work store in stores(same idx) for tiflash peer @@ -113,12 +113,12 @@ type RegionStore struct { accessIndex [NumAccessMode][]int // AccessMode => idx in stores } -func (r *RegionStore) accessStore(mode AccessMode, idx AccessIndex) (int, *Store) { +func (r *regionStore) accessStore(mode AccessMode, idx AccessIndex) (int, *Store) { sidx := r.accessIndex[mode][idx] return sidx, r.stores[sidx] } -func (r *RegionStore) getAccessIndex(mode AccessMode, store *Store) AccessIndex { +func (r *regionStore) getAccessIndex(mode AccessMode, store *Store) AccessIndex { for index, sidx := range r.accessIndex[mode] { if r.stores[sidx].storeID == store.storeID { return AccessIndex(index) @@ -127,14 +127,14 @@ func (r *RegionStore) getAccessIndex(mode AccessMode, store *Store) AccessIndex return -1 } -func (r *RegionStore) accessStoreNum(mode AccessMode) int { +func (r *regionStore) accessStoreNum(mode AccessMode) int { return len(r.accessIndex[mode]) } // clone clones region store struct. -func (r *RegionStore) clone() *RegionStore { +func (r *regionStore) clone() *regionStore { storeEpochs := make([]uint32, len(r.stores)) - rs := &RegionStore{ + rs := ®ionStore{ workTiFlashIdx: r.workTiFlashIdx, proxyTiKVIdx: r.proxyTiKVIdx, workTiKVIdx: r.workTiKVIdx, @@ -150,7 +150,7 @@ func (r *RegionStore) clone() *RegionStore { } // return next follower store's index -func (r *RegionStore) follower(seed uint32, op *storeSelectorOp) AccessIndex { +func (r *regionStore) follower(seed uint32, op *storeSelectorOp) AccessIndex { l := uint32(r.accessStoreNum(TiKVOnly)) if l <= 1 { return r.workTiKVIdx @@ -171,7 +171,7 @@ func (r *RegionStore) follower(seed uint32, op *storeSelectorOp) AccessIndex { } // return next leader or follower store's index -func (r *RegionStore) kvPeer(seed uint32, op *storeSelectorOp) AccessIndex { +func (r *regionStore) kvPeer(seed uint32, op *storeSelectorOp) AccessIndex { if op.leaderOnly { return r.workTiKVIdx } @@ -191,7 +191,7 @@ func (r *RegionStore) kvPeer(seed uint32, op *storeSelectorOp) AccessIndex { return candidates[seed%uint32(len(candidates))] } -func (r *RegionStore) filterStoreCandidate(aidx AccessIndex, op *storeSelectorOp) bool { +func (r *regionStore) filterStoreCandidate(aidx AccessIndex, op *storeSelectorOp) bool { _, s := r.accessStore(TiKVOnly, aidx) // filter label unmatched store return s.IsLabelsMatch(op.labels) @@ -201,7 +201,7 @@ func (r *RegionStore) filterStoreCandidate(aidx AccessIndex, op *storeSelectorOp func (r *Region) init(bo *Backoffer, c *RegionCache) error { // region store pull used store from global store map // to avoid acquire storeMu in later access. - rs := &RegionStore{ + rs := ®ionStore{ workTiKVIdx: 0, proxyTiKVIdx: -1, workTiFlashIdx: 0, @@ -248,12 +248,12 @@ func (r *Region) init(bo *Backoffer, c *RegionCache) error { return nil } -func (r *Region) getStore() (store *RegionStore) { - store = (*RegionStore)(atomic.LoadPointer(&r.store)) +func (r *Region) getStore() (store *regionStore) { + store = (*regionStore)(atomic.LoadPointer(&r.store)) return } -func (r *Region) compareAndSwapStore(oldStore, newStore *RegionStore) bool { +func (r *Region) compareAndSwapStore(oldStore, newStore *regionStore) bool { return atomic.CompareAndSwapPointer(&r.store, unsafe.Pointer(oldStore), unsafe.Pointer(newStore)) } @@ -798,7 +798,7 @@ func (c *RegionCache) OnSendFailForTiFlash(bo *Backoffer, store *Store, region R } } -func (c *RegionCache) markRegionNeedBeRefill(s *Store, storeIdx int, rs *RegionStore) int { +func (c *RegionCache) markRegionNeedBeRefill(s *Store, storeIdx int, rs *regionStore) int { incEpochStoreIdx := -1 // invalidate regions in store. epoch := rs.storeEpochs[storeIdx] @@ -1456,7 +1456,7 @@ func (c *RegionCache) getStoreAddr(bo *Backoffer, region *Region, store *Store) } } -func (c *RegionCache) getProxyStore(region *Region, store *Store, rs *RegionStore, workStoreIdx AccessIndex) (proxyStore *Store, proxyAccessIdx AccessIndex, proxyStoreIdx int) { +func (c *RegionCache) getProxyStore(region *Region, store *Store, rs *regionStore, workStoreIdx AccessIndex) (proxyStore *Store, proxyAccessIdx AccessIndex, proxyStoreIdx int) { if !c.enableForwarding || store.storeType != tikvrpc.TiKV || atomic.LoadInt32(&store.needForwarding) == 0 { return } @@ -1680,7 +1680,7 @@ func (r *Region) GetLeaderStoreID() uint64 { return r.meta.Peers[storeIdx].StoreId } -func (r *Region) getKvStorePeer(rs *RegionStore, aidx AccessIndex) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { +func (r *Region) getKvStorePeer(rs *regionStore, aidx AccessIndex) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { storeIdx, store = rs.accessStore(TiKVOnly, aidx) peer = r.meta.Peers[storeIdx] accessIdx = aidx @@ -1688,17 +1688,17 @@ func (r *Region) getKvStorePeer(rs *RegionStore, aidx AccessIndex) (store *Store } // WorkStorePeer returns current work store with work peer. -func (r *Region) WorkStorePeer(rs *RegionStore) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { +func (r *Region) WorkStorePeer(rs *regionStore) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { return r.getKvStorePeer(rs, rs.workTiKVIdx) } // FollowerStorePeer returns a follower store with follower peer. -func (r *Region) FollowerStorePeer(rs *RegionStore, followerStoreSeed uint32, op *storeSelectorOp) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { +func (r *Region) FollowerStorePeer(rs *regionStore, followerStoreSeed uint32, op *storeSelectorOp) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { return r.getKvStorePeer(rs, rs.follower(followerStoreSeed, op)) } // AnyStorePeer returns a leader or follower store with the associated peer. -func (r *Region) AnyStorePeer(rs *RegionStore, followerStoreSeed uint32, op *storeSelectorOp) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { +func (r *Region) AnyStorePeer(rs *regionStore, followerStoreSeed uint32, op *storeSelectorOp) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { return r.getKvStorePeer(rs, rs.kvPeer(followerStoreSeed, op)) } @@ -1785,14 +1785,14 @@ retry: return } -func (r *RegionStore) switchNextFlashPeer(rr *Region, currentPeerIdx AccessIndex) { +func (r *regionStore) switchNextFlashPeer(rr *Region, currentPeerIdx AccessIndex) { nextIdx := (currentPeerIdx + 1) % AccessIndex(r.accessStoreNum(TiFlashOnly)) newRegionStore := r.clone() newRegionStore.workTiFlashIdx = int32(nextIdx) rr.compareAndSwapStore(r, newRegionStore) } -func (r *RegionStore) switchNextTiKVPeer(rr *Region, currentPeerIdx AccessIndex) { +func (r *regionStore) switchNextTiKVPeer(rr *Region, currentPeerIdx AccessIndex) { if r.workTiKVIdx != currentPeerIdx { return } @@ -1805,7 +1805,7 @@ func (r *RegionStore) switchNextTiKVPeer(rr *Region, currentPeerIdx AccessIndex) // switchNextProxyStore switches the index of the peer that will forward requests to the leader to the next peer. // If proxy is currently not used on this region, the value of `currentProxyIdx` should be -1, and a random peer will // be select in this case. -func (r *RegionStore) switchNextProxyStore(rr *Region, currentProxyIdx AccessIndex, incEpochStoreIdx int) { +func (r *regionStore) switchNextProxyStore(rr *Region, currentProxyIdx AccessIndex, incEpochStoreIdx int) { if r.proxyTiKVIdx != currentProxyIdx { return } @@ -1837,7 +1837,7 @@ func (r *RegionStore) switchNextProxyStore(rr *Region, currentProxyIdx AccessInd rr.compareAndSwapStore(r, newRegionStore) } -func (r *RegionStore) setProxyStoreIdx(rr *Region, idx AccessIndex) { +func (r *regionStore) setProxyStoreIdx(rr *Region, idx AccessIndex) { if r.proxyTiKVIdx == idx { return } @@ -1851,7 +1851,7 @@ func (r *RegionStore) setProxyStoreIdx(rr *Region, idx AccessIndex) { zap.Bool("success", success)) } -func (r *RegionStore) unsetProxyStoreIfNeeded(rr *Region) { +func (r *regionStore) unsetProxyStoreIfNeeded(rr *Region) { r.setProxyStoreIdx(rr, -1) } From d03bd1834afe683f42ab2dc9171c36595601d970 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 8 Jun 2021 14:48:29 +0800 Subject: [PATCH 325/343] store/tikv: move function GetStoreTypeByMeta to sub package tikvrpc (#25233) --- store/tikv/region_cache.go | 4 ++-- store/tikv/store_type.go | 14 +------------- store/tikv/tikvrpc/endpoint.go | 19 +++++++++++++++++++ 3 files changed, 22 insertions(+), 15 deletions(-) diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index a4ed1b6d23bd2..5bcf9d0c3b522 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -1988,7 +1988,7 @@ func (s *Store) initResolve(bo *Backoffer, c *RegionCache) (addr string, err err } s.addr = addr s.saddr = store.GetStatusAddress() - s.storeType = GetStoreTypeByMeta(store) + s.storeType = tikvrpc.GetStoreTypeByMeta(store) s.labels = store.GetLabels() // Shouldn't have other one changing its state concurrently, but we still use changeResolveStateTo for safety. s.changeResolveStateTo(unresolved, resolved) @@ -2030,7 +2030,7 @@ func (s *Store) reResolve(c *RegionCache) (bool, error) { return false, nil } - storeType := GetStoreTypeByMeta(store) + storeType := tikvrpc.GetStoreTypeByMeta(store) addr = store.GetAddress() if s.addr != addr || !s.IsSameLabels(store.GetLabels()) { newStore := &Store{storeID: s.storeID, addr: addr, saddr: store.GetStatusAddress(), storeType: storeType, labels: store.GetLabels(), state: uint64(resolved)} diff --git a/store/tikv/store_type.go b/store/tikv/store_type.go index e77a5580d6447..a5add49b9f9f4 100644 --- a/store/tikv/store_type.go +++ b/store/tikv/store_type.go @@ -43,19 +43,7 @@ func (a AccessMode) String() string { } } -// Constants to determine engine type. -// They should be synced with PD. -const ( - engineLabelKey = "engine" - engineLabelTiFlash = "tiflash" -) - // GetStoreTypeByMeta gets store type by store meta pb. func GetStoreTypeByMeta(store *metapb.Store) tikvrpc.EndpointType { - for _, label := range store.Labels { - if label.Key == engineLabelKey && label.Value == engineLabelTiFlash { - return tikvrpc.TiFlash - } - } - return tikvrpc.TiKV + return tikvrpc.GetStoreTypeByMeta(store) } diff --git a/store/tikv/tikvrpc/endpoint.go b/store/tikv/tikvrpc/endpoint.go index 8eb881c81efb2..3cd6fabbea794 100644 --- a/store/tikv/tikvrpc/endpoint.go +++ b/store/tikv/tikvrpc/endpoint.go @@ -13,6 +13,8 @@ package tikvrpc +import "github.com/pingcap/kvproto/pkg/metapb" + // EndpointType represents the type of a remote endpoint.. type EndpointType uint8 @@ -35,3 +37,20 @@ func (t EndpointType) Name() string { } return "unspecified" } + +// Constants to determine engine type. +// They should be synced with PD. +const ( + engineLabelKey = "engine" + engineLabelTiFlash = "tiflash" +) + +// GetStoreTypeByMeta gets store type by store meta pb. +func GetStoreTypeByMeta(store *metapb.Store) EndpointType { + for _, label := range store.Labels { + if label.Key == engineLabelKey && label.Value == engineLabelTiFlash { + return TiFlash + } + } + return TiKV +} From 1aea274cca93437f2e73e3d412951dd6a9892f32 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 8 Jun 2021 14:58:28 +0800 Subject: [PATCH 326/343] store/tikv: make AccessMode private (#25232) --- store/tikv/region_cache.go | 80 +++++++++++++++---------------- store/tikv/region_cache_test.go | 2 +- store/tikv/region_request.go | 10 ++-- store/tikv/region_request_test.go | 8 ++-- store/tikv/store_type.go | 22 ++++----- 5 files changed, 61 insertions(+), 61 deletions(-) diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 5bcf9d0c3b522..76bae651c8093 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -110,15 +110,15 @@ type regionStore struct { workTiFlashIdx int32 // point to current work peer in meta.Peers and work store in stores(same idx) for tiflash peer stores []*Store // stores in this region storeEpochs []uint32 // snapshots of store's epoch, need reload when `storeEpochs[curr] != stores[cur].fail` - accessIndex [NumAccessMode][]int // AccessMode => idx in stores + accessIndex [numAccessMode][]int // AccessMode => idx in stores } -func (r *regionStore) accessStore(mode AccessMode, idx AccessIndex) (int, *Store) { +func (r *regionStore) accessStore(mode accessMode, idx AccessIndex) (int, *Store) { sidx := r.accessIndex[mode][idx] return sidx, r.stores[sidx] } -func (r *regionStore) getAccessIndex(mode AccessMode, store *Store) AccessIndex { +func (r *regionStore) getAccessIndex(mode accessMode, store *Store) AccessIndex { for index, sidx := range r.accessIndex[mode] { if r.stores[sidx].storeID == store.storeID { return AccessIndex(index) @@ -127,7 +127,7 @@ func (r *regionStore) getAccessIndex(mode AccessMode, store *Store) AccessIndex return -1 } -func (r *regionStore) accessStoreNum(mode AccessMode) int { +func (r *regionStore) accessStoreNum(mode accessMode) int { return len(r.accessIndex[mode]) } @@ -142,7 +142,7 @@ func (r *regionStore) clone() *regionStore { storeEpochs: storeEpochs, } copy(storeEpochs, r.storeEpochs) - for i := 0; i < int(NumAccessMode); i++ { + for i := 0; i < int(numAccessMode); i++ { rs.accessIndex[i] = make([]int, len(r.accessIndex[i])) copy(rs.accessIndex[i], r.accessIndex[i]) } @@ -151,7 +151,7 @@ func (r *regionStore) clone() *regionStore { // return next follower store's index func (r *regionStore) follower(seed uint32, op *storeSelectorOp) AccessIndex { - l := uint32(r.accessStoreNum(TiKVOnly)) + l := uint32(r.accessStoreNum(tiKVOnly)) if l <= 1 { return r.workTiKVIdx } @@ -161,7 +161,7 @@ func (r *regionStore) follower(seed uint32, op *storeSelectorOp) AccessIndex { if followerIdx >= r.workTiKVIdx { followerIdx++ } - storeIdx, s := r.accessStore(TiKVOnly, followerIdx) + storeIdx, s := r.accessStore(tiKVOnly, followerIdx) if r.storeEpochs[storeIdx] == atomic.LoadUint32(&s.epoch) && r.filterStoreCandidate(followerIdx, op) { return followerIdx } @@ -175,10 +175,10 @@ func (r *regionStore) kvPeer(seed uint32, op *storeSelectorOp) AccessIndex { if op.leaderOnly { return r.workTiKVIdx } - candidates := make([]AccessIndex, 0, r.accessStoreNum(TiKVOnly)) - for i := 0; i < r.accessStoreNum(TiKVOnly); i++ { + candidates := make([]AccessIndex, 0, r.accessStoreNum(tiKVOnly)) + for i := 0; i < r.accessStoreNum(tiKVOnly); i++ { accessIdx := AccessIndex(i) - storeIdx, s := r.accessStore(TiKVOnly, accessIdx) + storeIdx, s := r.accessStore(tiKVOnly, accessIdx) if r.storeEpochs[storeIdx] != atomic.LoadUint32(&s.epoch) || !r.filterStoreCandidate(accessIdx, op) { continue } @@ -192,7 +192,7 @@ func (r *regionStore) kvPeer(seed uint32, op *storeSelectorOp) AccessIndex { } func (r *regionStore) filterStoreCandidate(aidx AccessIndex, op *storeSelectorOp) bool { - _, s := r.accessStore(TiKVOnly, aidx) + _, s := r.accessStore(tiKVOnly, aidx) // filter label unmatched store return s.IsLabelsMatch(op.labels) } @@ -227,9 +227,9 @@ func (r *Region) init(bo *Backoffer, c *RegionCache) error { availablePeers = append(availablePeers, p) switch store.storeType { case tikvrpc.TiKV: - rs.accessIndex[TiKVOnly] = append(rs.accessIndex[TiKVOnly], len(rs.stores)) + rs.accessIndex[tiKVOnly] = append(rs.accessIndex[tiKVOnly], len(rs.stores)) case tikvrpc.TiFlash: - rs.accessIndex[TiFlashOnly] = append(rs.accessIndex[TiFlashOnly], len(rs.stores)) + rs.accessIndex[tiFlashOnly] = append(rs.accessIndex[tiFlashOnly], len(rs.stores)) } rs.stores = append(rs.stores, store) rs.storeEpochs = append(rs.storeEpochs, atomic.LoadUint32(&store.epoch)) @@ -426,7 +426,7 @@ type RPCContext struct { AccessIdx AccessIndex Store *Store Addr string - AccessMode AccessMode + AccessMode accessMode ProxyStore *Store // nil means proxy is not used ProxyAccessIdx AccessIndex // valid when ProxyStore is not nil ProxyAddr string // valid when ProxyStore is not nil @@ -570,11 +570,11 @@ func (c *RegionCache) GetTiKVRPCContext(bo *Backoffer, id RegionVerID, replicaRe AccessIdx: accessIdx, Store: store, Addr: addr, - AccessMode: TiKVOnly, + AccessMode: tiKVOnly, ProxyStore: proxyStore, ProxyAccessIdx: proxyAccessIdx, ProxyAddr: proxyAddr, - TiKVNum: regionStore.accessStoreNum(TiKVOnly), + TiKVNum: regionStore.accessStoreNum(tiKVOnly), }, nil } @@ -593,13 +593,13 @@ func (c *RegionCache) GetAllValidTiFlashStores(id RegionVerID, currentStore *Sto return allStores } regionStore := cachedRegion.getStore() - currentIndex := regionStore.getAccessIndex(TiFlashOnly, currentStore) + currentIndex := regionStore.getAccessIndex(tiFlashOnly, currentStore) if currentIndex == -1 { return allStores } - for startOffset := 1; startOffset < regionStore.accessStoreNum(TiFlashOnly); startOffset++ { - accessIdx := AccessIndex((int(currentIndex) + startOffset) % regionStore.accessStoreNum(TiFlashOnly)) - storeIdx, store := regionStore.accessStore(TiFlashOnly, accessIdx) + for startOffset := 1; startOffset < regionStore.accessStoreNum(tiFlashOnly); startOffset++ { + accessIdx := AccessIndex((int(currentIndex) + startOffset) % regionStore.accessStoreNum(tiFlashOnly)) + storeIdx, store := regionStore.accessStore(tiFlashOnly, accessIdx) if store.getResolveState() == needCheck { continue } @@ -635,9 +635,9 @@ func (c *RegionCache) GetTiFlashRPCContext(bo *Backoffer, id RegionVerID, loadBa } else { sIdx = int(atomic.LoadInt32(®ionStore.workTiFlashIdx)) } - for i := 0; i < regionStore.accessStoreNum(TiFlashOnly); i++ { - accessIdx := AccessIndex((sIdx + i) % regionStore.accessStoreNum(TiFlashOnly)) - storeIdx, store := regionStore.accessStore(TiFlashOnly, accessIdx) + for i := 0; i < regionStore.accessStoreNum(tiFlashOnly); i++ { + accessIdx := AccessIndex((sIdx + i) % regionStore.accessStoreNum(tiFlashOnly)) + storeIdx, store := regionStore.accessStore(tiFlashOnly, accessIdx) addr, err := c.getStoreAddr(bo, cachedRegion, store) if err != nil { return nil, err @@ -668,8 +668,8 @@ func (c *RegionCache) GetTiFlashRPCContext(bo *Backoffer, id RegionVerID, loadBa AccessIdx: accessIdx, Store: store, Addr: addr, - AccessMode: TiFlashOnly, - TiKVNum: regionStore.accessStoreNum(TiKVOnly), + AccessMode: tiFlashOnly, + TiKVNum: regionStore.accessStoreNum(tiKVOnly), }, nil } @@ -774,7 +774,7 @@ func (c *RegionCache) OnSendFailForTiFlash(bo *Backoffer, store *Store, region R return } - accessMode := TiFlashOnly + accessMode := tiFlashOnly accessIdx := rs.getAccessIndex(accessMode, store) if accessIdx == -1 { logutil.Logger(bo.GetCtx()).Warn("can not get access index for region " + region.String()) @@ -858,7 +858,7 @@ func (c *RegionCache) OnSendFail(bo *Backoffer, ctx *RPCContext, scheduleReload } // try next peer to found new leader. - if ctx.AccessMode == TiKVOnly { + if ctx.AccessMode == tiKVOnly { if startForwarding || ctx.ProxyStore != nil { var currentProxyIdx AccessIndex = -1 if ctx.ProxyStore != nil { @@ -1142,7 +1142,7 @@ func (c *RegionCache) insertRegionToCache(cachedRegion *Region) { // To solve it, one solution is always to try a different peer if the invalid reason of the old cached region is no-leader. // There is a small probability that the current peer who reports no-leader becomes a leader and TiDB has to retry once in this case. if InvalidReason(atomic.LoadInt32((*int32)(&oldRegion.invalidReason))) == NoLeader { - store.workTiKVIdx = (oldRegionStore.workTiKVIdx + 1) % AccessIndex(store.accessStoreNum(TiKVOnly)) + store.workTiKVIdx = (oldRegionStore.workTiKVIdx + 1) % AccessIndex(store.accessStoreNum(tiKVOnly)) } // Invalidate the old region in case it's not invalidated and some requests try with the stale region information. oldRegion.invalidate(Other) @@ -1462,11 +1462,11 @@ func (c *RegionCache) getProxyStore(region *Region, store *Store, rs *regionStor } if rs.proxyTiKVIdx >= 0 { - storeIdx, proxyStore := rs.accessStore(TiKVOnly, rs.proxyTiKVIdx) + storeIdx, proxyStore := rs.accessStore(tiKVOnly, rs.proxyTiKVIdx) return proxyStore, rs.proxyTiKVIdx, storeIdx } - tikvNum := rs.accessStoreNum(TiKVOnly) + tikvNum := rs.accessStoreNum(tiKVOnly) if tikvNum <= 1 { return } @@ -1485,7 +1485,7 @@ func (c *RegionCache) getProxyStore(region *Region, store *Store, rs *regionStor if index == int(workStoreIdx) { continue } - storeIdx, store := rs.accessStore(TiKVOnly, AccessIndex(index)) + storeIdx, store := rs.accessStore(tiKVOnly, AccessIndex(index)) // Skip unreachable stores. if atomic.LoadInt32(&store.needForwarding) != 0 { continue @@ -1663,25 +1663,25 @@ func (r *Region) GetMeta() *metapb.Region { // GetLeaderPeerID returns leader peer ID. func (r *Region) GetLeaderPeerID() uint64 { store := r.getStore() - if int(store.workTiKVIdx) >= store.accessStoreNum(TiKVOnly) { + if int(store.workTiKVIdx) >= store.accessStoreNum(tiKVOnly) { return 0 } - storeIdx, _ := store.accessStore(TiKVOnly, store.workTiKVIdx) + storeIdx, _ := store.accessStore(tiKVOnly, store.workTiKVIdx) return r.meta.Peers[storeIdx].Id } // GetLeaderStoreID returns the store ID of the leader region. func (r *Region) GetLeaderStoreID() uint64 { store := r.getStore() - if int(store.workTiKVIdx) >= store.accessStoreNum(TiKVOnly) { + if int(store.workTiKVIdx) >= store.accessStoreNum(tiKVOnly) { return 0 } - storeIdx, _ := store.accessStore(TiKVOnly, store.workTiKVIdx) + storeIdx, _ := store.accessStore(tiKVOnly, store.workTiKVIdx) return r.meta.Peers[storeIdx].StoreId } func (r *Region) getKvStorePeer(rs *regionStore, aidx AccessIndex) (store *Store, peer *metapb.Peer, accessIdx AccessIndex, storeIdx int) { - storeIdx, store = rs.accessStore(TiKVOnly, aidx) + storeIdx, store = rs.accessStore(tiKVOnly, aidx) peer = r.meta.Peers[storeIdx] accessIdx = aidx return @@ -1769,7 +1769,7 @@ retry: // switch to new leader. oldRegionStore := r.getStore() var leaderIdx AccessIndex - for i, gIdx := range oldRegionStore.accessIndex[TiKVOnly] { + for i, gIdx := range oldRegionStore.accessIndex[tiKVOnly] { if gIdx == globalStoreIdx { leaderIdx = AccessIndex(i) } @@ -1786,7 +1786,7 @@ retry: } func (r *regionStore) switchNextFlashPeer(rr *Region, currentPeerIdx AccessIndex) { - nextIdx := (currentPeerIdx + 1) % AccessIndex(r.accessStoreNum(TiFlashOnly)) + nextIdx := (currentPeerIdx + 1) % AccessIndex(r.accessStoreNum(tiFlashOnly)) newRegionStore := r.clone() newRegionStore.workTiFlashIdx = int32(nextIdx) rr.compareAndSwapStore(r, newRegionStore) @@ -1796,7 +1796,7 @@ func (r *regionStore) switchNextTiKVPeer(rr *Region, currentPeerIdx AccessIndex) if r.workTiKVIdx != currentPeerIdx { return } - nextIdx := (currentPeerIdx + 1) % AccessIndex(r.accessStoreNum(TiKVOnly)) + nextIdx := (currentPeerIdx + 1) % AccessIndex(r.accessStoreNum(tiKVOnly)) newRegionStore := r.clone() newRegionStore.workTiKVIdx = nextIdx rr.compareAndSwapStore(r, newRegionStore) @@ -1810,7 +1810,7 @@ func (r *regionStore) switchNextProxyStore(rr *Region, currentProxyIdx AccessInd return } - tikvNum := r.accessStoreNum(TiKVOnly) + tikvNum := r.accessStoreNum(tiKVOnly) var nextIdx AccessIndex // If the region is not using proxy before, randomly select a non-leader peer for the first try. diff --git a/store/tikv/region_cache_test.go b/store/tikv/region_cache_test.go index ccb685a680004..ea42dbd79cda8 100644 --- a/store/tikv/region_cache_test.go +++ b/store/tikv/region_cache_test.go @@ -1445,7 +1445,7 @@ func BenchmarkOnRequestFail(b *testing.B) { AccessIdx: accessIdx, Peer: peer, Store: store, - AccessMode: TiKVOnly, + AccessMode: tiKVOnly, } r := cache.GetCachedRegionWithRLock(rpcCtx.Region) if r != nil { diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 61c82feb58f94..247282476a717 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -228,8 +228,8 @@ func newReplicaSelector(regionCache *RegionCache, regionID RegionVerID) (*replic return nil, nil } regionStore := cachedRegion.getStore() - replicas := make([]*replica, 0, regionStore.accessStoreNum(TiKVOnly)) - for _, storeIdx := range regionStore.accessIndex[TiKVOnly] { + replicas := make([]*replica, 0, regionStore.accessStoreNum(tiKVOnly)) + for _, storeIdx := range regionStore.accessIndex[tiKVOnly] { replicas = append(replicas, &replica{ store: regionStore.stores[storeIdx], peer: cachedRegion.meta.Peers[storeIdx], @@ -298,7 +298,7 @@ func (s *replicaSelector) next(bo *Backoffer) (*RPCContext, error) { Peer: replica.peer, Store: replica.store, Addr: addr, - AccessMode: TiKVOnly, + AccessMode: tiKVOnly, TiKVNum: len(s.replicas), }, nil } @@ -840,9 +840,9 @@ func (s *RegionRequestSender) NeedReloadRegion(ctx *RPCContext) (need bool) { s.failProxyStoreIDs[ctx.ProxyStore.storeID] = struct{}{} } - if ctx.AccessMode == TiKVOnly && len(s.failStoreIDs)+len(s.failProxyStoreIDs) >= ctx.TiKVNum { + if ctx.AccessMode == tiKVOnly && len(s.failStoreIDs)+len(s.failProxyStoreIDs) >= ctx.TiKVNum { need = true - } else if ctx.AccessMode == TiFlashOnly && len(s.failStoreIDs) >= len(ctx.Meta.Peers)-ctx.TiKVNum { + } else if ctx.AccessMode == tiFlashOnly && len(s.failStoreIDs) >= len(ctx.Meta.Peers)-ctx.TiKVNum { need = true } else if len(s.failStoreIDs)+len(s.failProxyStoreIDs) >= len(ctx.Meta.Peers) { need = true diff --git a/store/tikv/region_request_test.go b/store/tikv/region_request_test.go index 2c0c39cc6f5bd..7492d43caf89b 100644 --- a/store/tikv/region_request_test.go +++ b/store/tikv/region_request_test.go @@ -873,12 +873,12 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector(c *C) { // Create a fake region and change its leader to the last peer. regionStore = regionStore.clone() regionStore.workTiKVIdx = AccessIndex(len(regionStore.stores) - 1) - sidx, _ := regionStore.accessStore(TiKVOnly, regionStore.workTiKVIdx) + sidx, _ := regionStore.accessStore(tiKVOnly, regionStore.workTiKVIdx) regionStore.stores[sidx].epoch++ regionStore.storeEpochs[sidx]++ // Add a TiFlash peer to the region. peer := &metapb.Peer{Id: s.cluster.AllocID(), StoreId: s.cluster.AllocID()} - regionStore.accessIndex[TiFlashOnly] = append(regionStore.accessIndex[TiFlashOnly], len(regionStore.stores)) + regionStore.accessIndex[tiFlashOnly] = append(regionStore.accessIndex[tiFlashOnly], len(regionStore.stores)) regionStore.stores = append(regionStore.stores, &Store{storeID: peer.StoreId, storeType: tikvrpc.TiFlash}) regionStore.storeEpochs = append(regionStore.storeEpochs, 0) @@ -899,7 +899,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector(c *C) { c.Assert(err, IsNil) c.Assert(replicaSelector.region, Equals, region) // Should only contains TiKV stores. - c.Assert(len(replicaSelector.replicas), Equals, regionStore.accessStoreNum(TiKVOnly)) + c.Assert(len(replicaSelector.replicas), Equals, regionStore.accessStoreNum(tiKVOnly)) c.Assert(len(replicaSelector.replicas), Equals, len(regionStore.stores)-1) c.Assert(replicaSelector.nextReplicaIdx == 0, IsTrue) c.Assert(replicaSelector.isExhausted(), IsFalse) @@ -926,7 +926,7 @@ func (s *testRegionRequestToThreeStoresSuite) TestReplicaSelector(c *C) { c.Assert(rpcCtx.Store, Equals, replicaSelector.replicas[replicaSelector.nextReplicaIdx-1].store) c.Assert(rpcCtx.Peer, Equals, replicaSelector.replicas[replicaSelector.nextReplicaIdx-1].peer) c.Assert(rpcCtx.Addr, Equals, replicaSelector.replicas[replicaSelector.nextReplicaIdx-1].store.addr) - c.Assert(rpcCtx.AccessMode, Equals, TiKVOnly) + c.Assert(rpcCtx.AccessMode, Equals, tiKVOnly) } // Verify the correctness of next() diff --git a/store/tikv/store_type.go b/store/tikv/store_type.go index a5add49b9f9f4..893885d929955 100644 --- a/store/tikv/store_type.go +++ b/store/tikv/store_type.go @@ -20,23 +20,23 @@ import ( "github.com/pingcap/tidb/store/tikv/tikvrpc" ) -// AccessMode uses to index stores for different region cache access requirements. -type AccessMode int +// accessMode uses to index stores for different region cache access requirements. +type accessMode int const ( - // TiKVOnly indicates stores list that use for TiKv access(include both leader request and follower read). - TiKVOnly AccessMode = iota - // TiFlashOnly indicates stores list that use for TiFlash request. - TiFlashOnly - // NumAccessMode reserved to keep max access mode value. - NumAccessMode + // tiKVOnly indicates stores list that use for TiKv access(include both leader request and follower read). + tiKVOnly accessMode = iota + // tiFlashOnly indicates stores list that use for TiFlash request. + tiFlashOnly + // numAccessMode reserved to keep max access mode value. + numAccessMode ) -func (a AccessMode) String() string { +func (a accessMode) String() string { switch a { - case TiKVOnly: + case tiKVOnly: return "TiKvOnly" - case TiFlashOnly: + case tiFlashOnly: return "TiFlashOnly" default: return fmt.Sprintf("%d", a) From bce55b716d016fd182f40e84698eb80764733f54 Mon Sep 17 00:00:00 2001 From: Shirly Date: Tue, 8 Jun 2021 15:10:28 +0800 Subject: [PATCH 327/343] store/tikv: make groupSortedMutationByRegion as a normal function in 2pc (#25209) --- store/tikv/2pc.go | 41 ++++++++++++++++++++++++++++++++++++-- store/tikv/region_cache.go | 37 ---------------------------------- 2 files changed, 39 insertions(+), 39 deletions(-) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 49096254743dc..ee714d61c93c4 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -508,9 +508,46 @@ func (c *twoPhaseCommitter) doActionOnMutations(bo *Backoffer, action twoPhaseCo return c.doActionOnGroupMutations(bo, action, groups) } +type groupedMutations struct { + region RegionVerID + mutations CommitterMutations +} + +// groupSortedMutationsByRegion separates keys into groups by their belonging Regions. +func groupSortedMutationsByRegion(c *RegionCache, bo *retry.Backoffer, m CommitterMutations) ([]groupedMutations, error) { + var ( + groups []groupedMutations + lastLoc *KeyLocation + ) + lastUpperBound := 0 + for i := 0; i < m.Len(); i++ { + if lastLoc == nil || !lastLoc.Contains(m.GetKey(i)) { + if lastLoc != nil { + groups = append(groups, groupedMutations{ + region: lastLoc.Region, + mutations: m.Slice(lastUpperBound, i), + }) + lastUpperBound = i + } + var err error + lastLoc, err = c.LocateKey(bo, m.GetKey(i)) + if err != nil { + return nil, errors.Trace(err) + } + } + } + if lastLoc != nil { + groups = append(groups, groupedMutations{ + region: lastLoc.Region, + mutations: m.Slice(lastUpperBound, m.Len()), + }) + } + return groups, nil +} + // groupMutations groups mutations by region, then checks for any large groups and in that case pre-splits the region. func (c *twoPhaseCommitter) groupMutations(bo *Backoffer, mutations CommitterMutations) ([]groupedMutations, error) { - groups, err := c.store.regionCache.groupSortedMutationsByRegion(bo, mutations) + groups, err := groupSortedMutationsByRegion(c.store.regionCache, bo, mutations) if err != nil { return nil, errors.Trace(err) } @@ -531,7 +568,7 @@ func (c *twoPhaseCommitter) groupMutations(bo *Backoffer, mutations CommitterMut } // Reload region cache again. if didPreSplit { - groups, err = c.store.regionCache.groupSortedMutationsByRegion(bo, mutations) + groups, err = groupSortedMutationsByRegion(c.store.regionCache, bo, mutations) if err != nil { return nil, errors.Trace(err) } diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index 76bae651c8093..f2c8a1b16f419 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -963,43 +963,6 @@ func (c *RegionCache) GroupKeysByRegion(bo *Backoffer, keys [][]byte, filter fun return groups, first, nil } -type groupedMutations struct { - region RegionVerID - mutations CommitterMutations -} - -// groupSortedMutationsByRegion separates keys into groups by their belonging Regions. -func (c *RegionCache) groupSortedMutationsByRegion(bo *Backoffer, m CommitterMutations) ([]groupedMutations, error) { - var ( - groups []groupedMutations - lastLoc *KeyLocation - ) - lastUpperBound := 0 - for i := 0; i < m.Len(); i++ { - if lastLoc == nil || !lastLoc.Contains(m.GetKey(i)) { - if lastLoc != nil { - groups = append(groups, groupedMutations{ - region: lastLoc.Region, - mutations: m.Slice(lastUpperBound, i), - }) - lastUpperBound = i - } - var err error - lastLoc, err = c.LocateKey(bo, m.GetKey(i)) - if err != nil { - return nil, errors.Trace(err) - } - } - } - if lastLoc != nil { - groups = append(groups, groupedMutations{ - region: lastLoc.Region, - mutations: m.Slice(lastUpperBound, m.Len()), - }) - } - return groups, nil -} - // ListRegionIDsInKeyRange lists ids of regions in [start_key,end_key]. func (c *RegionCache) ListRegionIDsInKeyRange(bo *Backoffer, startKey, endKey []byte) (regionIDs []uint64, err error) { for { From eb91585d6d01138911e5d2aa598cfe42724e9405 Mon Sep 17 00:00:00 2001 From: bb7133 Date: Tue, 8 Jun 2021 15:35:54 +0800 Subject: [PATCH 328/343] executor: temporarily skip some unstable test cases. (#25224) - testResourceTagSuite.TestResourceGroupTag (#25189) - testSuiteJoin1.TestIndexLookupJoin (#25191) - testSuite1.TestAnalyzeIndex (#25188) - testSuite1.TestAnalyzeIncremental (#25187) - testVectorizeSuite2.TestVectorizedBuiltinTimeEvalOneVec (#25169) --- executor/analyze_test.go | 2 ++ executor/executor_test.go | 4 ++++ executor/join_test.go | 1 + 3 files changed, 7 insertions(+) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 53e7ab8bd2908..f9765aa08a096 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -559,6 +559,7 @@ func (s *testSuite1) TestIssue15752(c *C) { } func (s *testSuite1) TestAnalyzeIndex(c *C) { + c.Skip("unstable, skip it and fix it before 20210622") tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -582,6 +583,7 @@ func (s *testSuite1) TestAnalyzeIndex(c *C) { } func (s *testSuite1) TestAnalyzeIncremental(c *C) { + c.Skip("unstable, skip it and fix it before 20210622") tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("set @@tidb_analyze_version = 1") diff --git a/executor/executor_test.go b/executor/executor_test.go index 439a6362c90be..7cb72b266e263 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -73,6 +73,7 @@ import ( "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/gcutil" + "github.com/pingcap/tidb/util/israce" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mock" @@ -8414,6 +8415,9 @@ func (s testSerialSuite) TestTemporaryTableNoNetwork(c *C) { } func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { + if israce.RaceEnabled { + c.Skip("unstable, skip it and fix it before 20210622") + } tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t;") diff --git a/executor/join_test.go b/executor/join_test.go index bfd0048a63b3d..3519b14893491 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -1182,6 +1182,7 @@ func (s *testSuiteJoin1) TestIssue15850JoinNullValue(c *C) { } func (s *testSuiteJoin1) TestIndexLookupJoin(c *C) { + c.Skip("unstable, skip it and fix it before 20210622") tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("set @@tidb_init_chunk_size=2") From a7f3c4d8bd8922fb916491e534a7df22d7e9e7f1 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 8 Jun 2021 15:44:28 +0800 Subject: [PATCH 329/343] planner/core: change agg cost factor (#25210) --- planner/core/physical_plans.go | 12 +- planner/core/task.go | 18 +- .../testdata/integration_serial_suite_in.json | 24 +- .../integration_serial_suite_out.json | 291 +++--------------- planner/implementation/simple_plans.go | 4 +- 5 files changed, 71 insertions(+), 278 deletions(-) diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index a80dd96a28259..d3169c865e07b 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1048,7 +1048,7 @@ func (p *basePhysicalAgg) numDistinctFunc() (num int) { return } -func (p *basePhysicalAgg) getAggFuncCostFactor() (factor float64) { +func (p *basePhysicalAgg) getAggFuncCostFactor(isMPP bool) (factor float64) { factor = 0.0 for _, agg := range p.AggFuncs { if fac, ok := aggFuncFactor[agg.Name]; ok { @@ -1058,7 +1058,15 @@ func (p *basePhysicalAgg) getAggFuncCostFactor() (factor float64) { } } if factor == 0 { - factor = 1.0 + if isMPP { + // The default factor 1.0 will lead to 1-phase agg in pseudo stats settings. + // But in mpp cases, 2-phase is more usual. So we change this factor. + // TODO: This is still a little tricky and might cause regression. We should + // calibrate these factors and polish our cost model in the future. + factor = aggFuncFactor[ast.AggFuncFirstRow] + } else { + factor = 1.0 + } } return } diff --git a/planner/core/task.go b/planner/core/task.go index 8559afa8ab6a3..fcd8fd0ebe270 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1829,7 +1829,7 @@ func (p *PhysicalStreamAgg) attach2Task(tasks ...task) task { // GetCost computes cost of stream aggregation considering CPU/memory. func (p *PhysicalStreamAgg) GetCost(inputRows float64, isRoot bool) float64 { - aggFuncFactor := p.getAggFuncCostFactor() + aggFuncFactor := p.getAggFuncCostFactor(false) var cpuCost float64 sessVars := p.ctx.GetSessionVars() if isRoot { @@ -1876,7 +1876,7 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { if proj != nil { attachPlan2Task(proj, mpp) } - mpp.addCost(p.GetCost(inputRows, false)) + mpp.addCost(p.GetCost(inputRows, false, true)) p.cost = mpp.cost() return mpp case Mpp2Phase: @@ -1909,7 +1909,7 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { attachPlan2Task(proj, newMpp) } // TODO: how to set 2-phase cost? - newMpp.addCost(p.GetCost(inputRows, false)) + newMpp.addCost(p.GetCost(inputRows, false, true)) finalAgg.SetCost(mpp.cost()) if proj != nil { proj.SetCost(mpp.cost()) @@ -1920,14 +1920,14 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { if partialAgg != nil { attachPlan2Task(partialAgg, mpp) } - mpp.addCost(p.GetCost(inputRows, false)) + mpp.addCost(p.GetCost(inputRows, false, true)) if partialAgg != nil { partialAgg.SetCost(mpp.cost()) } t = mpp.convertToRootTask(p.ctx) inputRows = t.count() attachPlan2Task(finalAgg, t) - t.addCost(p.GetCost(inputRows, true)) + t.addCost(p.GetCost(inputRows, true, false)) finalAgg.SetCost(t.cost()) return t default: @@ -1958,7 +1958,7 @@ func (p *PhysicalHashAgg) attach2Task(tasks ...task) task { partialAgg.SetChildren(cop.indexPlan) cop.indexPlan = partialAgg } - cop.addCost(p.GetCost(inputRows, false)) + cop.addCost(p.GetCost(inputRows, false, false)) } // In `newPartialAggregate`, we are using stats of final aggregation as stats // of `partialAgg`, so the network cost of transferring result rows of `partialAgg` @@ -1991,16 +1991,16 @@ func (p *PhysicalHashAgg) attach2Task(tasks ...task) task { // hash aggregation, it would cause under-estimation as the reason mentioned in comment above. // To make it simple, we also treat 2-phase parallel hash aggregation in TiDB layer as // 1-phase when computing cost. - t.addCost(p.GetCost(inputRows, true)) + t.addCost(p.GetCost(inputRows, true, false)) p.cost = t.cost() return t } // GetCost computes the cost of hash aggregation considering CPU/memory. -func (p *PhysicalHashAgg) GetCost(inputRows float64, isRoot bool) float64 { +func (p *PhysicalHashAgg) GetCost(inputRows float64, isRoot bool, isMPP bool) float64 { cardinality := p.statsInfo().RowCount numDistinctFunc := p.numDistinctFunc() - aggFuncFactor := p.getAggFuncCostFactor() + aggFuncFactor := p.getAggFuncCostFactor(isMPP) var cpuCost float64 sessVars := p.ctx.GetSessionVars() if isRoot { diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 57d2ffa9068f1..785a3730a13c8 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -62,8 +62,8 @@ { "name": "TestMPPOuterJoinBuildSideForBroadcastJoin", "cases": [ - "explain format = 'brief' select count(*) from a left join b on a.id = b.id", - "explain format = 'brief' select count(*) from b right join a on a.id = b.id" + "explain format = 'brief' select count(*) from a left join b on a.id = b.id", + "explain format = 'brief' select count(*) from b right join a on a.id = b.id" ] }, { @@ -101,26 +101,6 @@ "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)" ] }, - { - "name": "TestBroadcastJoin", - "cases": [ - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t), broadcast_join_local(d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t), broadcast_join_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)" - ] - }, { "name": "TestJoinNotSupportedByTiFlash", "cases": [ diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 373a09470a69c..5ac2f53b78117 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -944,208 +944,6 @@ } ] }, - { - "Name": "TestBroadcastJoin", - "Cases": [ - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d3_t keep order:false, global read", - " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d2_t keep order:false, global read", - " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t), broadcast_join_local(d1_t) */ count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false, global read" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t,d2_t,d3_t), broadcast_join_local(d2_t) */ count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d3_t keep order:false, global read", - " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d2_t keep order:false", - " └─HashJoin(Probe) 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false, global read" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─TableFullScan(Build) 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false, global read" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col1, 10)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col2, 10)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", - " ├─TableFullScan(Build) 2.00 batchCop[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false, global read" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 8.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10), other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─Selection(Build) 8.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false, global read", - " └─TableFullScan(Probe) 2.00 batchCop[tiflash] table:d1_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 batchCop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 batchCop[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─Selection(Build) 2.00 batchCop[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─Selection(Probe) 8.00 batchCop[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 batchCop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─TableFullScan(Build) 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - }, - { - "SQL": "explain format = 'brief' select /*+ broadcast_join(fact_t,d1_t) */ count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", - "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:HashAgg", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 6.40 batchCop[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─TableFullScan(Build) 2.00 batchCop[tiflash] table:d1_t keep order:false, global read", - " └─TableFullScan(Probe) 8.00 batchCop[tiflash] table:fact_t keep order:false" - ] - } - ] - }, { "Name": "TestJoinNotSupportedByTiFlash", "Cases": [ @@ -2123,23 +1921,24 @@ "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null", " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, funcs:firstrow(test.tt.col_varchar_64)->test.tt.col_varchar_64, funcs:firstrow(test.tt.col_char_64_not_null)->test.tt.col_char_64_not_null", - " └─ExchangeReceiver 15609.38 batchCop[tiflash] ", - " └─ExchangeSender 15609.38 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64, test.tt.col_char_64_not_null", - " └─HashJoin 15609.38 batchCop[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", - " ├─ExchangeReceiver(Build) 10000.00 batchCop[tiflash] ", - " │ └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#29", - " │ └─Projection 10000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null, cast(test.tt.col_char_64_not_null, varchar(64) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#29", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─HashJoin(Probe) 12487.50 batchCop[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", - " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_key", - " │ └─Projection 9990.00 batchCop[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_key))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t2 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", - " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64", - " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_64))", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t3 keep order:false, stats:pseudo" + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64, test.tt.col_char_64_not_null", + " └─HashAgg 8000.00 batchCop[tiflash] group by:test.tt.col_char_64_not_null, test.tt.col_varchar_64, ", + " └─HashJoin 15609.38 batchCop[tiflash] inner join, equal:[eq(test.tt.col_char_64_not_null, test.tt.col_varchar_64)]", + " ├─ExchangeReceiver(Build) 10000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#30", + " │ └─Projection 10000.00 batchCop[tiflash] test.tt.col_varchar_64, test.tt.col_char_64_not_null, cast(test.tt.col_char_64_not_null, varchar(64) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin)->Column#30", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t1 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12487.50 batchCop[tiflash] inner join, equal:[eq(test.tt.col_varchar_key, test.tt.col_varchar_64) eq(Column#19, test.tt.col_decimal_30_10_key)]", + " ├─ExchangeReceiver(Build) 9990.00 batchCop[tiflash] ", + " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_key", + " │ └─Projection 9990.00 batchCop[tiflash] test.tt.col_varchar_key, cast(test.tt.col_tinyint, decimal(20,0) BINARY)->Column#19", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_key))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.tt.col_varchar_64", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.tt.col_varchar_64))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t3 keep order:false, stats:pseudo" ] } ] @@ -2348,20 +2147,22 @@ " └─HashJoin 7992.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", " └─HashAgg 7992.00 batchCop[tiflash] group by:Column#32, funcs:sum(Column#30)->Column#11, funcs:firstrow(Column#31)->test.t.id", " └─Projection 9990.00 batchCop[tiflash] cast(test.t.id, decimal(32,0) BINARY)->Column#30, test.t.id, test.t.id", " └─HashJoin 9990.00 batchCop[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 batchCop[tiflash] test.t.id", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", @@ -2438,10 +2239,11 @@ " └─Projection 6400.00 batchCop[tiflash] cast(test.t.id, decimal(32,0) BINARY)->Column#21, test.t.value", " └─Projection 6400.00 batchCop[tiflash] test.t.id, test.t.value", " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", - " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.value", - " └─Selection 8000.00 batchCop[tiflash] gt(cast(test.t.id, decimal(20,0) BINARY), test.t.value)", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeReceiver 6400.00 batchCop[tiflash] ", + " └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.value", + " └─HashAgg 6400.00 batchCop[tiflash] group by:test.t.id, test.t.value, ", + " └─Selection 8000.00 batchCop[tiflash] gt(cast(test.t.id, decimal(20,0) BINARY), test.t.value)", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -2575,10 +2377,11 @@ " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#7, test.t.id", " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.t.id)->test.t.id", " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", @@ -2598,10 +2401,11 @@ " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 batchCop[tiflash] Column#11, test.t.id", " └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:sum(Column#14)->Column#11, funcs:firstrow(test.t.id)->test.t.id", " └─ExchangeReceiver 7992.00 batchCop[tiflash] ", @@ -2613,10 +2417,11 @@ " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 batchCop[tiflash] test.t.id", " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 batchCop[tiflash] ", - " │ └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", - " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 batchCop[tiflash] ", + " │ └─ExchangeSender 7992.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.id", + " │ └─HashAgg 7992.00 batchCop[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 batchCop[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] diff --git a/planner/implementation/simple_plans.go b/planner/implementation/simple_plans.go index cb49fd0e10225..0a727576b3950 100644 --- a/planner/implementation/simple_plans.go +++ b/planner/implementation/simple_plans.go @@ -85,7 +85,7 @@ type TiDBHashAggImpl struct { // CalcCost implements Implementation CalcCost interface. func (agg *TiDBHashAggImpl) CalcCost(outCount float64, children ...memo.Implementation) float64 { hashAgg := agg.plan.(*plannercore.PhysicalHashAgg) - selfCost := hashAgg.GetCost(children[0].GetPlan().Stats().RowCount, true) + selfCost := hashAgg.GetCost(children[0].GetPlan().Stats().RowCount, true, false) agg.cost = selfCost + children[0].GetCost() return agg.cost } @@ -110,7 +110,7 @@ type TiKVHashAggImpl struct { // CalcCost implements Implementation CalcCost interface. func (agg *TiKVHashAggImpl) CalcCost(outCount float64, children ...memo.Implementation) float64 { hashAgg := agg.plan.(*plannercore.PhysicalHashAgg) - selfCost := hashAgg.GetCost(children[0].GetPlan().Stats().RowCount, false) + selfCost := hashAgg.GetCost(children[0].GetPlan().Stats().RowCount, false, false) agg.cost = selfCost + children[0].GetCost() return agg.cost } From 75388187f3865db18bc70ca1ef30260d85dba2e2 Mon Sep 17 00:00:00 2001 From: Arenatlx <314806019@qq.com> Date: Tue, 8 Jun 2021 17:16:28 +0800 Subject: [PATCH 330/343] ddl: make the column type change switch transparent (#25213) --- ddl/column_test.go | 2 ++ ddl/column_type_change_test.go | 1 + ddl/db_change_test.go | 5 +++++ ddl/db_integration_test.go | 4 ++++ ddl/db_test.go | 4 ++++ executor/executor_test.go | 7 +++++++ expression/integration_test.go | 7 +++++++ sessionctx/variable/tidb_vars.go | 2 +- 8 files changed, 31 insertions(+), 1 deletion(-) diff --git a/ddl/column_test.go b/ddl/column_test.go index 10b883a76935a..289f026bf1d0a 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -1159,6 +1159,8 @@ func (s *testColumnSuite) TestModifyColumn(c *C) { WithLease(testLease), ) ctx := testNewContext(d) + ctx.GetSessionVars().EnableChangeColumnType = false + defer func() { err := d.Stop() c.Assert(err, IsNil) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 8e3f7d6d30e8a..5764875a75e7f 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -1651,6 +1651,7 @@ func (s *testColumnTypeChangeSuite) TestRowFormat(c *C) { func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFlenErrorMsg(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") + tk.Se.GetSessionVars().EnableChangeColumnType = false tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int4)") diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 041f35c7734a8..befcd8d31c8c3 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -82,6 +82,11 @@ func (s *testStateChangeSuiteBase) SetUpSuite(c *C) { c.Assert(err, IsNil) _, err = s.se.Execute(context.Background(), "use test_db_state") c.Assert(err, IsNil) + // Set the variable to default 0 as it was before in case of modifying the test. + _, err = s.se.Execute(context.Background(), "set @@global.tidb_enable_change_column_type=0") + c.Assert(err, IsNil) + _, err = s.se.Execute(context.Background(), "set @@tidb_enable_change_column_type=0") + c.Assert(err, IsNil) s.p = parser.New() } diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index c65fb27de399e..e37585ce384e8 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -88,6 +88,10 @@ func setupIntegrationSuite(s *testIntegrationSuite, c *C) { s.ctx = se.(sessionctx.Context) _, err = se.Execute(context.Background(), "create database test_db") c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "set @@global.tidb_enable_change_column_type=0") + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "set @@tidb_enable_change_column_type=0") + c.Assert(err, IsNil) } func tearDownIntegrationSuiteTest(s *testIntegrationSuite, c *C) { diff --git a/ddl/db_test.go b/ddl/db_test.go index 2a114b49d205f..e67b0a9c1795f 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -123,6 +123,10 @@ func setUpSuite(s *testDBSuite, c *C) { c.Assert(err, IsNil) _, err = s.s.Execute(context.Background(), "set @@global.tidb_max_delta_schema_count= 4096") c.Assert(err, IsNil) + _, err = s.s.Execute(context.Background(), "set @@global.tidb_enable_change_column_type=0") + c.Assert(err, IsNil) + _, err = s.s.Execute(context.Background(), "set @@tidb_enable_change_column_type=0") + c.Assert(err, IsNil) } func tearDownSuite(s *testDBSuite, c *C) { diff --git a/executor/executor_test.go b/executor/executor_test.go index 7cb72b266e263..eef914d9de15d 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -202,6 +202,13 @@ func (s *baseTestSuite) SetUpSuite(c *C) { } d, err := session.BootstrapSession(s.store) c.Assert(err, IsNil) + se, err := session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + // Set the variable to default 0 as it was before in case of modifying the test. + _, err = se.Execute(context.Background(), "set @@global.tidb_enable_change_column_type=0") + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "set @@tidb_enable_change_column_type=0") + c.Assert(err, IsNil) d.SetStatsUpdating(true) s.domain = d config.UpdateGlobal(func(conf *config.Config) { diff --git a/expression/integration_test.go b/expression/integration_test.go index 1cf17e1d36bfa..cc1b160c405e3 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -94,6 +94,13 @@ func (s *testIntegrationSuiteBase) SetUpSuite(c *C) { s.store, s.dom, err = newStoreWithBootstrap() c.Assert(err, IsNil) s.ctx = mock.NewContext() + se, err := session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + // Set the variable to default 0 as it was before in case of modifying the test. + _, err = se.Execute(context.Background(), "set @@global.tidb_enable_change_column_type=0") + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "set @@tidb_enable_change_column_type=0") + c.Assert(err, IsNil) } func (s *testIntegrationSuiteBase) TearDownSuite(c *C) { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index e93414bc5e26e..d9c23a0d49b48 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -650,7 +650,7 @@ const ( DefTiDBDDLReorgBatchSize = 256 DefTiDBDDLErrorCountLimit = 512 DefTiDBMaxDeltaSchemaCount = 1024 - DefTiDBChangeColumnType = false + DefTiDBChangeColumnType = true DefTiDBChangeMultiSchema = false DefTiDBPointGetCache = false DefTiDBEnableAlterPlacement = false From cad482c036c0356eb50ca9a75c44e85d4011839e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 8 Jun 2021 19:46:28 +0800 Subject: [PATCH 331/343] planner/core: remove the union branch with dual table. (#25218) --- planner/core/exhaust_physical_plans.go | 22 +++++- planner/core/find_best_task.go | 4 +- planner/core/integration_test.go | 39 ++++++++++ planner/core/task.go | 20 ++++-- .../testdata/integration_serial_suite_in.json | 10 +++ .../integration_serial_suite_out.json | 72 +++++++++++++++++++ 6 files changed, 159 insertions(+), 8 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 51fb32086a3aa..d7a17d3017221 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2170,6 +2170,10 @@ func (p *baseLogicalPlan) exhaustPhysicalPlans(_ *property.PhysicalProperty) ([] // canPushToCop checks if it can be pushed to some stores. For TiKV, it only checks datasource. // For TiFlash, it will check whether the operator is supported, but note that the check might be inaccrute. func (p *baseLogicalPlan) canPushToCop(storeTp kv.StoreType) bool { + return p.canPushToCopImpl(storeTp, false) +} + +func (p *baseLogicalPlan) canPushToCopImpl(storeTp kv.StoreType, considerDual bool) bool { ret := true for _, ch := range p.children { switch c := ch.(type) { @@ -2181,7 +2185,21 @@ func (p *baseLogicalPlan) canPushToCop(storeTp kv.StoreType) bool { } } ret = ret && validDs - case *LogicalAggregation, *LogicalProjection, *LogicalSelection, *LogicalJoin, *LogicalUnionAll: + case *LogicalUnionAll: + if storeTp == kv.TiFlash { + ret = ret && c.canPushToCopImpl(storeTp, true) + } else { + return false + } + case *LogicalProjection: + if storeTp == kv.TiFlash { + ret = ret && c.canPushToCopImpl(storeTp, considerDual) + } else { + return false + } + case *LogicalTableDual: + return storeTp == kv.TiFlash && considerDual + case *LogicalAggregation, *LogicalSelection, *LogicalJoin: if storeTp == kv.TiFlash { ret = ret && c.canPushToCop(storeTp) } else { @@ -2559,7 +2577,7 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) if prop.TaskTp == property.MppTaskType && prop.PartitionTp != property.AnyType { return nil, true, nil } - canUseMpp := p.ctx.GetSessionVars().IsMPPAllowed() && p.canPushToCop(kv.TiFlash) + canUseMpp := p.ctx.GetSessionVars().IsMPPAllowed() && p.canPushToCopImpl(kv.TiFlash, true) chReqProps := make([]*property.PhysicalProperty, 0, len(p.children)) for range p.children { if canUseMpp && prop.TaskTp == property.MppTaskType { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 3b73e4033ccac..71d85a7d8b503 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -146,7 +146,7 @@ func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, planCou }.Init(p.ctx, p.stats, p.blockOffset) dual.SetSchema(p.schema) planCounter.Dec(1) - return &rootTask{p: dual}, 1, nil + return &rootTask{p: dual, isEmpty: p.RowCount == 0}, 1, nil } func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp) (task, int64, error) { @@ -1964,7 +1964,7 @@ func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, planCounter * pcte := PhysicalCTE{SeedPlan: sp, RecurPlan: rp, CTE: p.cte, cteAsName: p.cteAsName}.Init(p.ctx, p.stats) pcte.SetSchema(p.schema) - t = &rootTask{pcte, sp.statsInfo().RowCount} + t = &rootTask{pcte, sp.statsInfo().RowCount, false} p.cte.cteTask = t return t, 1, nil } diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index fa936be24e4fc..9c95d5026f79b 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -3347,6 +3347,45 @@ func (s *testIntegrationSerialSuite) TestPushDownAggForMPP(c *C) { } } +func (s *testIntegrationSerialSuite) TestMppUnionAll(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t (a int not null, b int, c varchar(20))") + tk.MustExec("create table t1 (a int, b int not null, c double)") + + // 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 == "t" || tblInfo.Name.L == "t1" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + } + +} + func (s *testIntegrationSerialSuite) TestMppJoinDecimal(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/task.go b/planner/core/task.go index fcd8fd0ebe270..eacc5dbf73e6d 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1058,8 +1058,10 @@ func setTableScanToTableRowIDScan(p PhysicalPlan) { // rootTask is the final sink node of a plan graph. It should be a single goroutine on tidb. type rootTask struct { - p PhysicalPlan - cst float64 + p PhysicalPlan + cst float64 + isEmpty bool // isEmpty indicates if this task contains a dual table and returns empty data. + // TODO: The flag 'isEmpty' is only checked by Projection and UnionAll. We should support more cases in the future. } func (t *rootTask) copy() task { @@ -1322,6 +1324,9 @@ func (p *PhysicalProjection) attach2Task(tasks ...task) task { t = attachPlan2Task(p, t) t.addCost(p.GetCost(t.count())) p.cost = t.cost() + if root, ok := tasks[0].(*rootTask); ok && root.isEmpty { + t.(*rootTask).isEmpty = true + } return t } @@ -1336,10 +1341,15 @@ func (p *PhysicalUnionAll) attach2MppTasks(tasks ...task) task { childMaxCost = childCost } childPlans = append(childPlans, mpp.plan()) + } else if root, ok := tk.(*rootTask); ok && root.isEmpty { + continue } else { return invalidTask } } + if len(childPlans) == 0 { + return invalidTask + } p.SetChildren(childPlans...) t.cst = childMaxCost p.cost = t.cost() @@ -1347,8 +1357,10 @@ func (p *PhysicalUnionAll) attach2MppTasks(tasks ...task) task { } func (p *PhysicalUnionAll) attach2Task(tasks ...task) task { - if _, ok := tasks[0].(*mppTask); ok { - return p.attach2MppTasks(tasks...) + for _, t := range tasks { + if _, ok := t.(*mppTask); ok { + return p.attach2MppTasks(tasks...) + } } t := &rootTask{p: p} childPlans := make([]PhysicalPlan, 0, len(tasks)) diff --git a/planner/core/testdata/integration_serial_suite_in.json b/planner/core/testdata/integration_serial_suite_in.json index 785a3730a13c8..267f5e532ddc6 100644 --- a/planner/core/testdata/integration_serial_suite_in.json +++ b/planner/core/testdata/integration_serial_suite_in.json @@ -230,6 +230,16 @@ "desc format = 'brief' select id from t as A where not exists (select 1 from t where t.id=A.id)" ] }, + { + "name": "TestMppUnionAll", + "cases": [ + "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1) tt", + "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1 union all select a, b from t where false) tt", + "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1) tt", + "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1 where false) tt", + "explain format = 'brief' select count(*) from (select a , b from t where false union all select a , c from t1 where false) tt" + ] + }, { "name": "TestMppJoinDecimal", "cases": [ diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 5ac2f53b78117..c22d300c268e1 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -1761,6 +1761,78 @@ } ] }, + { + "Name": "TestMppUnionAll", + "Cases": [ + { + "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1) tt", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#12)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 cop[tiflash] funcs:count(1)->Column#12", + " └─Union 20000.00 cop[tiflash] ", + " ├─Projection 10000.00 cop[tiflash] cast(test.t.a, int(11) BINARY)->Column#9, test.t.b", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 cop[tiflash] test.t1.a, cast(test.t1.b, int(11) BINARY)->Column#10", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , b from t1 union all select a, b from t where false) tt", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#16)->Column#15", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 cop[tiflash] funcs:count(1)->Column#16", + " └─Union 20000.00 cop[tiflash] ", + " ├─Projection 10000.00 cop[tiflash] cast(test.t.a, int(11) BINARY)->Column#13, test.t.b", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 cop[tiflash] test.t1.a, cast(test.t1.b, int(11) BINARY)->Column#14", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1) tt", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#14)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 cop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 cop[tiflash] funcs:count(1)->Column#14", + " └─Union 20000.00 cop[tiflash] ", + " ├─Projection 10000.00 cop[tiflash] cast(Column#9, int(11) BINARY)->Column#9, Column#10", + " │ └─Projection 10000.00 cop[tiflash] test.t.a, cast(test.t.b, double BINARY)->Column#10", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 10000.00 cop[tiflash] test.t1.a, cast(test.t1.c, double BINARY)->Column#10", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from (select a , b from t union all select a , c from t1 where false) tt", + "Plan": [ + "HashAgg 1.00 root funcs:count(Column#14)->Column#11", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#14", + " └─Union 10000.00 batchCop[tiflash] ", + " └─Projection 10000.00 batchCop[tiflash] cast(Column#9, int(11) BINARY)->Column#9, Column#10", + " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.b, double BINARY)->Column#10", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from (select a , b from t where false union all select a , c from t1 where false) tt", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─Union 0.00 root ", + " ├─Projection 0.00 root test.t.a, cast(test.t.b, double BINARY)->Column#10", + " │ └─TableDual 0.00 root rows:0", + " └─Projection 0.00 root test.t1.a, cast(test.t1.c, double BINARY)->Column#10", + " └─TableDual 0.00 root rows:0" + ] + } + ] + }, { "Name": "TestMppJoinDecimal", "Cases": [ From f81ef5579551a0523d18b049eb25ab3375bcfb48 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 8 Jun 2021 20:08:28 +0800 Subject: [PATCH 332/343] planner: solve an incompatible issue about dynamic-mode and TiFlash and enable dynamic mode by default (#25255) --- distsql/select_result.go | 51 ++++++++++++++++++++++++++++++++ executor/builder.go | 16 ++++++++++ executor/partition_table_test.go | 20 +++++++++++++ executor/table_reader.go | 50 +++++++++++++++++++++++++++++++ session/bootstrap.go | 2 +- sessionctx/variable/sysvar.go | 2 +- 6 files changed, 139 insertions(+), 2 deletions(-) diff --git a/distsql/select_result.go b/distsql/select_result.go index fa4315b55a041..d003343027dea 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -57,6 +57,7 @@ var ( var ( _ SelectResult = (*selectResult)(nil) _ SelectResult = (*streamResult)(nil) + _ SelectResult = (*serialSelectResults)(nil) ) // SelectResult is an iterator of coprocessor partial results. @@ -69,6 +70,56 @@ type SelectResult interface { Close() error } +// NewSerialSelectResults create a SelectResult which will read each SelectResult serially. +func NewSerialSelectResults(selectResults []SelectResult) SelectResult { + return &serialSelectResults{ + selectResults: selectResults, + cur: 0, + } +} + +// serialSelectResults reads each SelectResult serially +type serialSelectResults struct { + selectResults []SelectResult + cur int +} + +func (ssr *serialSelectResults) NextRaw(ctx context.Context) ([]byte, error) { + for ssr.cur < len(ssr.selectResults) { + resultSubset, err := ssr.selectResults[ssr.cur].NextRaw(ctx) + if err != nil { + return nil, err + } + if len(resultSubset) > 0 { + return resultSubset, nil + } + ssr.cur++ // move to the next SelectResult + } + return nil, nil +} + +func (ssr *serialSelectResults) Next(ctx context.Context, chk *chunk.Chunk) error { + for ssr.cur < len(ssr.selectResults) { + if err := ssr.selectResults[ssr.cur].Next(ctx, chk); err != nil { + return err + } + if chk.NumRows() > 0 { + return nil + } + ssr.cur++ // move to the next SelectResult + } + return nil +} + +func (ssr *serialSelectResults) Close() (err error) { + for _, r := range ssr.selectResults { + if rerr := r.Close(); rerr != nil { + err = rerr + } + } + return +} + type selectResult struct { label string resp kv.Response diff --git a/executor/builder.go b/executor/builder.go index 0662b963c57d2..15ea2be70deb9 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3483,6 +3483,22 @@ type kvRangeBuilderFromRangeAndPartition struct { partitions []table.PhysicalTable } +func (h kvRangeBuilderFromRangeAndPartition) buildKeyRangeSeparately(ranges []*ranger.Range) ([]int64, [][]kv.KeyRange, error) { + var ret [][]kv.KeyRange + var pids []int64 + for _, p := range h.partitions { + pid := p.GetPhysicalID() + meta := p.Meta() + kvRange, err := distsql.TableHandleRangesToKVRanges(h.sctx.GetSessionVars().StmtCtx, []int64{pid}, meta != nil && meta.IsCommonHandle, ranges, nil) + if err != nil { + return nil, nil, err + } + pids = append(pids, pid) + ret = append(ret, kvRange) + } + return pids, ret, nil +} + func (h kvRangeBuilderFromRangeAndPartition) buildKeyRange(_ int64, ranges []*ranger.Range) ([]kv.KeyRange, error) { var ret []kv.KeyRange for _, p := range h.partitions { diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index 84bf7513a6514..11ec1b2a097e3 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -2403,6 +2403,26 @@ func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) { } } +func (s *partitionTableSuite) TestDynamicModeByDefault(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create database test_dynamic_by_default") + + tk.MustExec(`create table trange(a int, b int, primary key(a) clustered, index idx_b(b)) partition by range(a) ( + partition p0 values less than(300), + partition p1 values less than(500), + partition p2 values less than(1100));`) + tk.MustExec(`create table thash(a int, b int, primary key(a) clustered, index idx_b(b)) partition by hash(a) partitions 4;`) + + for _, q := range []string{ + "explain select * from trange where a>400", + "explain select * from thash where a>=100", + } { + for _, r := range tk.MustQuery(q).Rows() { + c.Assert(strings.Contains(strings.ToLower(r[0].(string)), "partitionunion"), IsFalse) + } + } +} + func (s *partitionTableSuite) TestIssue24636(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create database test_issue_24636") diff --git a/executor/table_reader.go b/executor/table_reader.go index fc5e5a2c11096..88531e9c86363 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -54,6 +54,7 @@ func (sr selectResultHook) SelectResult(ctx context.Context, sctx sessionctx.Con type kvRangeBuilder interface { buildKeyRange(pid int64, ranges []*ranger.Range) ([]kv.KeyRange, error) + buildKeyRangeSeparately(ranges []*ranger.Range) ([]int64, [][]kv.KeyRange, error) } // TableReaderExecutor sends DAG request and reads table data from kv layer. @@ -238,6 +239,23 @@ func (e *TableReaderExecutor) Close() error { // buildResp first builds request and sends it to tikv using distsql.Select. It uses SelectResult returned by the callee // to fetch all results. func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (distsql.SelectResult, error) { + if e.storeType == kv.TiFlash && e.kvRangeBuilder != nil { + // TiFlash cannot support to access multiple tables/partitions within one KVReq, so we have to build KVReq for each partition separately. + kvReqs, err := e.buildKVReqSeparately(ctx, ranges) + if err != nil { + return nil, err + } + var results []distsql.SelectResult + for _, kvReq := range kvReqs { + result, err := e.SelectResult(ctx, e.ctx, kvReq, retTypes(e), e.feedback, getPhysicalPlanIDs(e.plans), e.id) + if err != nil { + return nil, err + } + results = append(results, result) + } + return distsql.NewSerialSelectResults(results), nil + } + kvReq, err := e.buildKVReq(ctx, ranges) if err != nil { return nil, err @@ -251,6 +269,38 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra return result, nil } +func (e *TableReaderExecutor) buildKVReqSeparately(ctx context.Context, ranges []*ranger.Range) ([]*kv.Request, error) { + pids, kvRanges, err := e.kvRangeBuilder.buildKeyRangeSeparately(ranges) + if err != nil { + return nil, err + } + var kvReqs []*kv.Request + for i, kvRange := range kvRanges { + e.kvRanges = append(e.kvRanges, kvRange...) + if err := updateExecutorTableID(ctx, e.dagPB.RootExecutor, pids[i], true); err != nil { + return nil, err + } + var builder distsql.RequestBuilder + reqBuilder := builder.SetKeyRanges(kvRange) + kvReq, err := reqBuilder. + SetDAGRequest(e.dagPB). + SetStartTS(e.startTS). + SetDesc(e.desc). + SetKeepOrder(e.keepOrder). + SetStreaming(e.streaming). + SetFromSessionVars(e.ctx.GetSessionVars()). + SetFromInfoSchema(e.ctx.GetInfoSchema()). + SetMemTracker(e.memTracker). + SetStoreType(e.storeType). + SetAllowBatchCop(e.batchCop).Build() + if err != nil { + return nil, err + } + kvReqs = append(kvReqs, kvReq) + } + return kvReqs, nil +} + func (e *TableReaderExecutor) buildKVReq(ctx context.Context, ranges []*ranger.Range) (*kv.Request, error) { var builder distsql.RequestBuilder var reqBuilder *distsql.RequestBuilder diff --git a/session/bootstrap.go b/session/bootstrap.go index 2a310e162ab7e..fd448bd0eeef8 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -1610,7 +1610,7 @@ func doDMLWorks(s Session) { vVal = strconv.Itoa(variable.DefTiDBRowFormatV2) } if v.Name == variable.TiDBPartitionPruneMode { - vVal = string(variable.Static) + vVal = variable.DefTiDBPartitionPruneMode if flag.Lookup("test.v") != nil || flag.Lookup("check.v") != nil || config.CheckTableBeforeDrop { // enable Dynamic Prune by default in test case. vVal = string(variable.Dynamic) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 4c4bd38b14d5b..9ca8072cee286 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1583,7 +1583,7 @@ var defaultSysVars = []*SysVar{ s.EnableClusteredIndex = TiDBOptEnableClustered(val) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBPartitionPruneMode, Value: string(Static), Hidden: true, Type: TypeStr, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBPartitionPruneMode, Value: DefTiDBPartitionPruneMode, Hidden: true, Type: TypeStr, Validation: func(vars *SessionVars, normalizedValue string, originalValue string, scope ScopeFlag) (string, error) { mode := PartitionPruneMode(normalizedValue).Update() if !mode.Valid() { return normalizedValue, ErrWrongTypeForVar.GenWithStackByArgs(TiDBPartitionPruneMode) From dd3edf5994b6dc971c8f4e6065582a135e50bd22 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Wed, 9 Jun 2021 10:48:29 +0800 Subject: [PATCH 333/343] executor: fix bug when use limit in CTE (#25261) --- executor/cte.go | 2 +- executor/cte_test.go | 71 ++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 72 insertions(+), 1 deletion(-) diff --git a/executor/cte.go b/executor/cte.go index fb0aaccb3d512..6ac26a50541b0 100644 --- a/executor/cte.go +++ b/executor/cte.go @@ -328,7 +328,7 @@ func (e *CTEExec) nextChunkLimit(req *chunk.Chunk) error { numRows := uint64(res.NumRows()) if e.cursor+numRows > e.limitEnd { numRows = e.limitEnd - e.cursor - req.Append(res.CopyConstructSel(), 0, int(numRows)+1) + req.Append(res.CopyConstructSel(), 0, int(numRows)) } else { req.SwapColumns(res.CopyConstructSel()) } diff --git a/executor/cte_test.go b/executor/cte_test.go index aa7c2804425c8..4c6e1cf993975 100644 --- a/executor/cte_test.go +++ b/executor/cte_test.go @@ -373,4 +373,75 @@ func (test *CTETestSuite) TestCTEWithLimit(c *check.C) { err = tk.QueryToErr("with recursive cte1 as (select 1/c1 c1 from t1 union select c1 + 1 c1 from cte1 where c1 < 2 limit 1) select * from cte1;") c.Assert(err, check.NotNil) c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + + tk.MustExec("set cte_max_recursion_depth = 1000;") + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int);") + tk.MustExec("insert into t1 values(1), (2), (3);") + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 0 offset 2) select * from cte1;") + rows.Check(testkit.Rows()) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 1 offset 2) select * from cte1;") + rows.Check(testkit.Rows("3")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 2 offset 2) select * from cte1;") + rows.Check(testkit.Rows("3", "4")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 3 offset 2) select * from cte1;") + rows.Check(testkit.Rows("3", "4", "5")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 4 offset 2) select * from cte1;") + rows.Check(testkit.Rows("3", "4", "5", "6")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 0 offset 3) select * from cte1;") + rows.Check(testkit.Rows()) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 1 offset 3) select * from cte1;") + rows.Check(testkit.Rows("4")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 2 offset 3) select * from cte1;") + rows.Check(testkit.Rows("4", "5")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 3 offset 3) select * from cte1;") + rows.Check(testkit.Rows("4", "5", "6")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 4 offset 3) select * from cte1;") + rows.Check(testkit.Rows("4", "5", "6", "7")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 0 offset 4) select * from cte1;") + rows.Check(testkit.Rows()) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 1 offset 4) select * from cte1;") + rows.Check(testkit.Rows("5")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 2 offset 4) select * from cte1;") + rows.Check(testkit.Rows("5", "6")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 3 offset 4) select * from cte1;") + rows.Check(testkit.Rows("5", "6", "7")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union select c1 + 1 from cte1 limit 4 offset 4) select * from cte1;") + rows.Check(testkit.Rows("5", "6", "7", "8")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 0 offset 2) select * from cte1;") + rows.Check(testkit.Rows()) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 1 offset 2) select * from cte1;") + rows.Check(testkit.Rows("3")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 2 offset 2) select * from cte1;") + rows.Check(testkit.Rows("3", "2")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 3 offset 2) select * from cte1;") + rows.Check(testkit.Rows("3", "2", "3")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 4 offset 2) select * from cte1;") + rows.Check(testkit.Rows("3", "2", "3", "4")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 0 offset 3) select * from cte1;") + rows.Check(testkit.Rows()) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 1 offset 3) select * from cte1;") + rows.Check(testkit.Rows("2")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 2 offset 3) select * from cte1;") + rows.Check(testkit.Rows("2", "3")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 3 offset 3) select * from cte1;") + rows.Check(testkit.Rows("2", "3", "4")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 4 offset 3) select * from cte1;") + rows.Check(testkit.Rows("2", "3", "4", "3")) + + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 0 offset 4) select * from cte1;") + rows.Check(testkit.Rows()) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 1 offset 4) select * from cte1;") + rows.Check(testkit.Rows("3")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 2 offset 4) select * from cte1;") + rows.Check(testkit.Rows("3", "4")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 3 offset 4) select * from cte1;") + rows.Check(testkit.Rows("3", "4", "3")) + rows = tk.MustQuery("with recursive cte1(c1) as (select c1 from t1 union all select c1 + 1 from cte1 limit 4 offset 4) select * from cte1;") + rows.Check(testkit.Rows("3", "4", "3", "4")) } From ac06d78691bee952c9ac3f7022b837b653b6920a Mon Sep 17 00:00:00 2001 From: wjHuang Date: Wed, 9 Jun 2021 10:56:29 +0800 Subject: [PATCH 334/343] *: enable column-type-change totally (#25263) --- ddl/column.go | 2 +- ddl/column_test.go | 15 ++-- ddl/column_type_change_test.go | 108 --------------------------- ddl/db_change_test.go | 44 +---------- ddl/db_integration_test.go | 14 ++-- ddl/db_test.go | 38 +--------- ddl/ddl_api.go | 37 ++++----- ddl/failtest/fail_db_test.go | 18 ++--- ddl/generated_column.go | 4 +- executor/admin_test.go | 3 +- executor/ddl_test.go | 2 +- executor/executor_test.go | 17 ----- executor/set_test.go | 12 --- expression/integration_test.go | 9 +-- session/pessimistic_test.go | 4 - session/schema_amender.go | 2 +- sessionctx/variable/session.go | 4 - sessionctx/variable/sysvar.go | 4 - sessionctx/variable/tidb_vars.go | 4 - sessionctx/variable/varsutil_test.go | 8 -- 20 files changed, 46 insertions(+), 303 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 9f981200f987e..ff3fb5c0a39ce 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -850,7 +850,7 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in newColName := model.NewCIStr(genChangingColumnUniqueName(tblInfo, oldCol)) if mysql.HasPriKeyFlag(oldCol.Flag) { job.State = model.JobStateCancelled - msg := "tidb_enable_change_column_type is true and this column has primary key flag" + msg := "this column has primary key flag" return ver, errUnsupportedModifyColumn.GenWithStackByArgs(msg) } diff --git a/ddl/column_test.go b/ddl/column_test.go index 289f026bf1d0a..793c7bec45b2f 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -1159,7 +1159,6 @@ func (s *testColumnSuite) TestModifyColumn(c *C) { WithLease(testLease), ) ctx := testNewContext(d) - ctx.GetSessionVars().EnableChangeColumnType = false defer func() { err := d.Stop() @@ -1171,19 +1170,19 @@ func (s *testColumnSuite) TestModifyColumn(c *C) { err error }{ {"int", "bigint", nil}, - {"int", "int unsigned", errUnsupportedModifyColumn.GenWithStackByArgs("can't change unsigned integer to signed or vice versa, and tidb_enable_change_column_type is false")}, + {"int", "int unsigned", nil}, {"varchar(10)", "text", nil}, {"varbinary(10)", "blob", nil}, {"text", "blob", errUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8mb4 to binary")}, - {"varchar(10)", "varchar(8)", errUnsupportedModifyColumn.GenWithStackByArgs("length 8 is less than origin 10, and tidb_enable_change_column_type is false")}, + {"varchar(10)", "varchar(8)", nil}, {"varchar(10)", "varchar(11)", nil}, {"varchar(10) character set utf8 collate utf8_bin", "varchar(10) character set utf8", nil}, - {"decimal(2,1)", "decimal(3,2)", errUnsupportedModifyColumn.GenWithStackByArgs("decimal change from decimal(2, 1) to decimal(3, 2), and tidb_enable_change_column_type is false")}, - {"decimal(2,1)", "decimal(2,2)", errUnsupportedModifyColumn.GenWithStackByArgs("decimal change from decimal(2, 1) to decimal(2, 2), and tidb_enable_change_column_type is false")}, + {"decimal(2,1)", "decimal(3,2)", nil}, + {"decimal(2,1)", "decimal(2,2)", nil}, {"decimal(2,1)", "decimal(2,1)", nil}, - {"decimal(2,1)", "int", errUnsupportedModifyColumn.GenWithStackByArgs("type int(11) not match origin decimal(2,1), and tidb_enable_change_column_type is false")}, - {"decimal", "int", errUnsupportedModifyColumn.GenWithStackByArgs("type int(11) not match origin decimal(10,0), and tidb_enable_change_column_type is false")}, - {"decimal(2,1)", "bigint", errUnsupportedModifyColumn.GenWithStackByArgs("type bigint(20) not match origin decimal(2,1), and tidb_enable_change_column_type is false")}, + {"decimal(2,1)", "int", nil}, + {"decimal", "int", nil}, + {"decimal(2,1)", "bigint", nil}, } for _, tt := range tests { ftA := s.colDefStrToFieldType(c, tt.origin) diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 5764875a75e7f..d0ca4f10c35a6 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -74,11 +74,6 @@ func (s *testColumnTypeChangeSuite) TearDownSuite(c *C) { func (s *testColumnTypeChangeSuite) TestColumnTypeChangeBetweenInteger(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false - }() // Modify column from null to not null. tk.MustExec("drop table if exists t") @@ -146,11 +141,6 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeStateBetweenInteger(c *C tk.MustExec("drop table if exists t") tk.MustExec("create table t (c1 int, c2 int)") tk.MustExec("insert into t(c1, c2) values (1, 1)") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false - }() // use new session to check meta in callback function. internalTK := testkit.NewTestKit(c, s.store) @@ -221,11 +211,6 @@ func (s *testColumnTypeChangeSuite) TestRollbackColumnTypeChangeBetweenInteger(c tk.MustExec("drop table if exists t") tk.MustExec("create table t (c1 bigint, c2 bigint)") tk.MustExec("insert into t(c1, c2) values (1, 1)") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false - }() tbl := testGetTableByName(c, tk.Se, "test", "t") c.Assert(tbl, NotNil) @@ -308,11 +293,6 @@ func init() { func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromIntegerToOthers(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false - }() prepare := func(tk *testkit.TestKit) { tk.MustExec("drop table if exists t") @@ -463,26 +443,12 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromIntegerToOthers(c *C func (s *testColumnTypeChangeSuite) TestColumnTypeChangeBetweenVarcharAndNonVarchar(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableChangeColumnType = false collate.SetNewCollationEnabledForTest(true) defer collate.SetNewCollationEnabledForTest(false) tk.MustExec("drop database if exists col_type_change_char;") tk.MustExec("create database col_type_change_char;") tk.MustExec("use col_type_change_char;") - tk.MustExec("create table t(a char(10), b varchar(10));") - tk.MustExec("insert into t values ('aaa ', 'bbb ');") - tk.MustExec("alter table t change column a a char(10);") - tk.MustExec("alter table t change column b b varchar(10);") - tk.MustGetErrCode("alter table t change column a a varchar(10);", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t change column b b char(10);", mysql.ErrUnsupportedDDLOperation) - - tk.MustExec("alter table t add index idx_a(a);") - tk.MustExec("alter table t add index idx_b(b);") - tk.MustGetErrCode("alter table t change column a a varchar(10);", mysql.ErrUnsupportedDDLOperation) - tk.MustGetErrCode("alter table t change column b b char(10);", mysql.ErrUnsupportedDDLOperation) - tk.MustExec("admin check table t;") - tk.Se.GetSessionVars().EnableChangeColumnType = true // https://github.com/pingcap/tidb/issues/23624 tk.MustExec("drop table if exists t;") tk.MustExec("create table t(b varchar(10));") @@ -506,14 +472,11 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeBetweenVarcharAndNonVarc func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromStringToOthers(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true // Set time zone to UTC. originalTz := tk.Se.GetSessionVars().TimeZone tk.Se.GetSessionVars().TimeZone = time.UTC defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false tk.Se.GetSessionVars().TimeZone = originalTz }() @@ -736,14 +699,11 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromStringToOthers(c *C) func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromNumericToOthers(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true // Set time zone to UTC. originalTz := tk.Se.GetSessionVars().TimeZone tk.Se.GetSessionVars().TimeZone = time.UTC defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false tk.Se.GetSessionVars().TimeZone = originalTz }() @@ -1002,10 +962,6 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromNumericToOthers(c *C func (s *testColumnTypeChangeSuite) TestColumnTypeChangeIgnoreDisplayLength(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false - }() originalHook := s.dom.DDL().GetHook() defer s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook) @@ -1049,14 +1005,11 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeIgnoreDisplayLength(c *C func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromDateTimeTypeToOthers(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true // Set time zone to UTC. originalTz := tk.Se.GetSessionVars().TimeZone tk.Se.GetSessionVars().TimeZone = time.UTC defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false tk.Se.GetSessionVars().TimeZone = originalTz }() @@ -1229,14 +1182,11 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromDateTimeTypeToOthers func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromJsonToOthers(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true // Set time zone to UTC. originalTz := tk.Se.GetSessionVars().TimeZone tk.Se.GetSessionVars().TimeZone = time.UTC defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false tk.Se.GetSessionVars().TimeZone = originalTz }() @@ -1603,8 +1553,6 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFromJsonToOthers(c *C) { func (s *testColumnTypeChangeSuite) TestUpdateDataAfterChangeTimestampToDate(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true tk.MustExec("drop table if exists t, t1") tk.MustExec("create table t (col timestamp default '1971-06-09' not null, col1 int default 1, unique key(col1));") tk.MustExec("alter table t modify column col date not null;") @@ -1626,11 +1574,6 @@ func (s *testColumnTypeChangeSuite) TestUpdateDataAfterChangeTimestampToDate(c * func (s *testColumnTypeChangeSuite) TestRowFormat(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = false - }() tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key, v varchar(10))") tk.MustExec("insert into t values (1, \"123\");") @@ -1647,40 +1590,6 @@ func (s *testColumnTypeChangeSuite) TestRowFormat(c *C) { tk.MustExec("drop table if exists t") } -// Close issue #17530 -func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFlenErrorMsg(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.Se.GetSessionVars().EnableChangeColumnType = false - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int4)") - _, err := tk.Exec("alter table t MODIFY COLUMN a tinyint(11)") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: length 4 is less than origin 11, and tidb_enable_change_column_type is false") - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (a decimal(20))") - tk.MustExec("insert into t values (12345678901234567890)") - _, err = tk.Exec("alter table t modify column a bigint") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: type bigint(20) not match origin decimal(20,0), and tidb_enable_change_column_type is false") - - tk.MustExec("drop table if exists t") - tk.MustExec("create table a (a bigint(2))") - tk.MustExec("insert into a values(123456),(789123)") - _, err = tk.Exec("alter table a modify column a tinyint") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: length 4 is less than origin 20, and tidb_enable_change_column_type is false") - - tk.MustExec("drop table if exists t") - tk.MustExec("CREATE TABLE t ( id int not null primary key auto_increment, token varchar(512) NOT NULL DEFAULT '', index (token))") - tk.MustExec("INSERT INTO t VALUES (NULL, 'aa')") - _, err = tk.Exec("ALTER TABLE t CHANGE COLUMN token token varchar(255) DEFAULT '' NOT NULL") - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: length 255 is less than origin 512, and tidb_enable_change_column_type is false") -} - // Close issue #22395 // Background: // Since the changing column is implemented as adding a new column and substitute the old one when it finished. @@ -1690,8 +1599,6 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeFlenErrorMsg(c *C) { func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValue(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("use test") @@ -1767,8 +1674,6 @@ func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValue(c *C) { func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValueAfterAddCol(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("use test") @@ -1886,7 +1791,6 @@ func (s *testColumnTypeChangeSuite) TestChangingAttributeOfColumnWithFK(c *C) { func (s *testColumnTypeChangeSuite) TestAlterPrimaryKeyToNull(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - tk.Se.GetSessionVars().EnableChangeColumnType = true tk.MustExec("drop table if exists t, t1") tk.MustExec("create table t(a int not null, b int not null, primary key(a, b));") @@ -1901,7 +1805,6 @@ func (s *testColumnTypeChangeSuite) TestAlterPrimaryKeyToNull(c *C) { func (s testColumnTypeChangeSuite) TestChangeUnsignedIntToDatetime(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") - tk.Se.GetSessionVars().EnableChangeColumnType = true tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int(10) unsigned default null, b bigint unsigned, c tinyint unsigned);") @@ -1973,8 +1876,6 @@ func (s *testColumnTypeChangeSuite) TestDDLExitWhenCancelMeetPanic(c *C) { func (s *testColumnTypeChangeSuite) TestChangeIntToBitWillPanicInBackfillIndexes(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE `t` (" + @@ -2004,8 +1905,6 @@ func (s *testColumnTypeChangeSuite) TestChangeIntToBitWillPanicInBackfillIndexes func (s *testColumnTypeChangeSuite) TestCancelCTCInReorgStateWillCauseGoroutineLeak(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true failpoint.Enable("github.com/pingcap/tidb/ddl/mockInfiniteReorgLogic", `return(true)`) defer func() { @@ -2038,8 +1937,6 @@ func (s *testColumnTypeChangeSuite) TestCancelCTCInReorgStateWillCauseGoroutineL tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("use test") - // Enable column change variable. - tk1.Se.GetSessionVars().EnableChangeColumnType = true var ( wg = sync.WaitGroup{} alterErr error @@ -2060,8 +1957,6 @@ func (s *testColumnTypeChangeSuite) TestCancelCTCInReorgStateWillCauseGoroutineL func (s *testColumnTypeChangeSuite) TestCTCShouldCastTheDefaultValue(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") @@ -2092,8 +1987,6 @@ func (s *testColumnTypeChangeSuite) TestCTCShouldCastTheDefaultValue(c *C) { func (s *testColumnTypeChangeSuite) TestCTCCastBitToBinary(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true // For point 1: tk.MustExec("drop table if exists t") @@ -2120,7 +2013,6 @@ func (s *testColumnTypeChangeSuite) TestCTCCastBitToBinary(c *C) { func (s *testColumnTypeChangeSuite) TestChangePrefixedIndexColumnToNonPrefixOne(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") - tk.Se.GetSessionVars().EnableChangeColumnType = true tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a text, unique index idx(a(2)));") diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index befcd8d31c8c3..dcc5cb23418fb 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -82,11 +82,6 @@ func (s *testStateChangeSuiteBase) SetUpSuite(c *C) { c.Assert(err, IsNil) _, err = s.se.Execute(context.Background(), "use test_db_state") c.Assert(err, IsNil) - // Set the variable to default 0 as it was before in case of modifying the test. - _, err = s.se.Execute(context.Background(), "set @@global.tidb_enable_change_column_type=0") - c.Assert(err, IsNil) - _, err = s.se.Execute(context.Background(), "set @@tidb_enable_change_column_type=0") - c.Assert(err, IsNil) s.p = parser.New() } @@ -499,10 +494,10 @@ func (s *testStateChangeSuite) TestAppendEnum(c *C) { c.Assert(err.Error(), Equals, "[types:1265]Data truncated for column 'c2' at row 1") failAlterTableSQL1 := "alter table t change c2 c2 enum('N') DEFAULT 'N'" _, err = s.se.Execute(context.Background(), failAlterTableSQL1) - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: the number of enum column's elements is less than the original: 2, and tidb_enable_change_column_type is false") + c.Assert(err, IsNil) failAlterTableSQL2 := "alter table t change c2 c2 int default 0" _, err = s.se.Execute(context.Background(), failAlterTableSQL2) - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: type int(11) not match origin enum('N','Y'), and tidb_enable_change_column_type is false") + c.Assert(err, IsNil) alterTableSQL := "alter table t change c2 c2 enum('N','Y','A') DEFAULT 'A'" _, err = s.se.Execute(context.Background(), alterTableSQL) c.Assert(err, IsNil) @@ -581,11 +576,6 @@ func (s *serialTestStateChangeSuite) TestWriteReorgForModifyColumnWithUniqIdx(c // TestWriteReorgForModifyColumnWithPKIsHandle tests whether the correct columns is used in PhysicalIndexScan's ToPB function. func (s *serialTestStateChangeSuite) TestWriteReorgForModifyColumnWithPKIsHandle(c *C) { modifyColumnSQL := "alter table tt change column c cc tinyint unsigned not null default 1 first" - enableChangeColumnType := s.se.GetSessionVars().EnableChangeColumnType - s.se.GetSessionVars().EnableChangeColumnType = true - defer func() { - s.se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType - }() _, err := s.se.Execute(context.Background(), "use test_db_state") c.Assert(err, IsNil) @@ -643,12 +633,6 @@ func (s *serialTestStateChangeSuite) TestDeleteOnlyForModifyColumnWithoutDefault } func (s *serialTestStateChangeSuite) testModifyColumn(c *C, state model.SchemaState, modifyColumnSQL string, idx idxType) { - enableChangeColumnType := s.se.GetSessionVars().EnableChangeColumnType - s.se.GetSessionVars().EnableChangeColumnType = true - defer func() { - s.se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType - }() - _, err := s.se.Execute(context.Background(), "use test_db_state") c.Assert(err, IsNil) switch idx { @@ -1062,18 +1046,11 @@ func (s *testStateChangeSuite) TestParallelAlterModifyColumn(c *C) { } func (s *testStateChangeSuite) TestParallelAddGeneratedColumnAndAlterModifyColumn(c *C) { - _, err := s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 1") - c.Assert(err, IsNil) - defer func() { - _, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0") - c.Assert(err, IsNil) - }() - sql1 := "ALTER TABLE t ADD COLUMN f INT GENERATED ALWAYS AS(a+1);" sql2 := "ALTER TABLE t MODIFY COLUMN a tinyint;" f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, oldCol is a dependent column 'a' for generated column") + c.Assert(err2.Error(), Equals, "[ddl:8200]Unsupported modify column: oldCol is a dependent column 'a' for generated column") _, err := s.se.Execute(context.Background(), "select * from t") c.Assert(err, IsNil) } @@ -1081,18 +1058,11 @@ func (s *testStateChangeSuite) TestParallelAddGeneratedColumnAndAlterModifyColum } func (s *testStateChangeSuite) TestParallelAlterModifyColumnAndAddPK(c *C) { - _, err := s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 1") - c.Assert(err, IsNil) - defer func() { - _, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0") - c.Assert(err, IsNil) - }() - sql1 := "ALTER TABLE t ADD PRIMARY KEY (b) NONCLUSTERED;" sql2 := "ALTER TABLE t MODIFY COLUMN b tinyint;" f := func(c *C, err1, err2 error) { c.Assert(err1, IsNil) - c.Assert(err2.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true and this column has primary key flag") + c.Assert(err2.Error(), Equals, "[ddl:8200]Unsupported modify column: this column has primary key flag") _, err := s.se.Execute(context.Background(), "select * from t") c.Assert(err, IsNil) } @@ -1745,12 +1715,6 @@ func (s *serialTestStateChangeSuite) TestModifyColumnTypeArgs(c *C) { tk.MustExec("drop table if exists t_modify_column_args") tk.MustExec("create table t_modify_column_args(a int, unique(a))") - enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType - }() - _, err := tk.Exec("alter table t_modify_column_args modify column a tinyint") c.Assert(err, NotNil) // error goes like `mock update version and tableInfo error,jobID=xx` diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index e37585ce384e8..d91158f991be2 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -88,10 +88,6 @@ func setupIntegrationSuite(s *testIntegrationSuite, c *C) { s.ctx = se.(sessionctx.Context) _, err = se.Execute(context.Background(), "create database test_db") c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "set @@global.tidb_enable_change_column_type=0") - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "set @@tidb_enable_change_column_type=0") - c.Assert(err, IsNil) } func tearDownIntegrationSuiteTest(s *testIntegrationSuite, c *C) { @@ -959,15 +955,15 @@ func (s *testIntegrationSuite5) TestModifyColumnOption(c *C) { tk.MustExec("create table t2 (b char, c int)") assertErrCode("alter table t2 modify column c int references t1(a)", errMsg) _, err := tk.Exec("alter table t1 change a a varchar(16)") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: type varchar(16) not match origin int(11), and tidb_enable_change_column_type is false") + c.Assert(err, IsNil) _, err = tk.Exec("alter table t1 change a a varchar(10)") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: type varchar(10) not match origin int(11), and tidb_enable_change_column_type is false") + c.Assert(err, IsNil) _, err = tk.Exec("alter table t1 change a a datetime") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: type datetime not match origin int(11), and tidb_enable_change_column_type is false") + c.Assert(err, IsNil) _, err = tk.Exec("alter table t1 change a a int(11) unsigned") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: can't change unsigned integer to signed or vice versa, and tidb_enable_change_column_type is false") + c.Assert(err, IsNil) _, err = tk.Exec("alter table t2 change b b int(11) unsigned") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: type int(11) not match origin char(1), and tidb_enable_change_column_type is false") + c.Assert(err, IsNil) } func (s *testIntegrationSuite4) TestIndexOnMultipleGeneratedColumn(c *C) { diff --git a/ddl/db_test.go b/ddl/db_test.go index e67b0a9c1795f..0082322b53680 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -123,10 +123,6 @@ func setUpSuite(s *testDBSuite, c *C) { c.Assert(err, IsNil) _, err = s.s.Execute(context.Background(), "set @@global.tidb_max_delta_schema_count= 4096") c.Assert(err, IsNil) - _, err = s.s.Execute(context.Background(), "set @@global.tidb_enable_change_column_type=0") - c.Assert(err, IsNil) - _, err = s.s.Execute(context.Background(), "set @@tidb_enable_change_column_type=0") - c.Assert(err, IsNil) } func tearDownSuite(s *testDBSuite, c *C) { @@ -2372,7 +2368,7 @@ func (s *testDBSuite4) TestChangeColumn(c *C) { sql = "alter table t4 change c2 a bigint not null;" tk.MustGetErrCode(sql, mysql.WarnDataTruncated) sql = "alter table t3 modify en enum('a', 'z', 'b', 'c') not null default 'a'" - tk.MustGetErrCode(sql, errno.ErrUnsupportedDDLOperation) + tk.MustExec(sql) // Rename to an existing column. s.mustExec(tk, c, "alter table t3 add column a bigint") sql = "alter table t3 change aa a bigint" @@ -4005,12 +4001,6 @@ func (s *testSerialDBSuite) TestModifyColumnnReorgInfo(c *C) { // Make sure the count of regions more than backfill workers. tk.MustQuery("split table t1 between (0) and (8192) regions 8;").Check(testkit.Rows("8 1")) - enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType - }() - tbl := s.testGetTable(c, "t1") originalHook := s.dom.DDL().GetHook() defer s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook) @@ -4111,11 +4101,8 @@ func (s *testSerialDBSuite) TestModifyColumnnReorgInfo(c *C) { func (s *testSerialDBSuite) TestModifyColumnNullToNotNullWithChangingVal2(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType - tk.Se.GetSessionVars().EnableChangeColumnType = true c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockInsertValueAfterCheckNull", `return("insert into test.tt values (NULL, NULL)")`), IsNil) defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType err := failpoint.Disable("github.com/pingcap/tidb/ddl/mockInsertValueAfterCheckNull") c.Assert(err, IsNil) }() @@ -4144,7 +4131,6 @@ func (s *testSerialDBSuite) TestModifyColumnNullToNotNullWithChangingVal(c *C) { func (s *testSerialDBSuite) TestModifyColumnBetweenStringTypes(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.Se.GetSessionVars().EnableChangeColumnType = true // varchar to varchar tk.MustExec("drop table if exists tt;") @@ -4226,14 +4212,6 @@ func testModifyColumnNullToNotNull(c *C, s *testDBSuite, enableChangeColumnType s.mustExec(tk, c, "drop table if exists t1") s.mustExec(tk, c, "create table t1 (c1 int, c2 int);") - if enableChangeColumnType { - enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType - }() - } - tbl := s.testGetTable(c, "t1") getModifyColumn(c, s.s.(sessionctx.Context), s.schemaName, "t1", "c2", false) @@ -5222,15 +5200,12 @@ func testModifyColumnTime(c *C, store kv.Storage, tests []testModifyColumnTimeCa tk := testkit.NewTestKit(c, store) tk.MustExec("use test_db") - enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType - tk.Se.GetSessionVars().EnableChangeColumnType = true // Set time zone to UTC. originalTz := tk.Se.GetSessionVars().TimeZone tk.Se.GetSessionVars().TimeZone = time.UTC defer func() { variable.SetDDLErrorCountLimit(limit) - tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType tk.Se.GetSessionVars().TimeZone = originalTz }() @@ -6523,13 +6498,6 @@ func (s *testSerialDBSuite) TestColumnTypeChangeGenUniqueChangingName(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t") - enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType - config.RestoreFunc()() - }() - hook := &ddl.TestDDLCallback{} var checkErr error assertChangingColName := "_col$_c2_0" @@ -6647,8 +6615,6 @@ func (s *testSerialDBSuite) TestColumnTypeChangeGenUniqueChangingName(c *C) { func (s *testSerialDBSuite) TestModifyColumnTypeWithWarnings(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true // Test normal warnings. tk.MustExec("drop table if exists t") @@ -6684,8 +6650,6 @@ func (s *testSerialDBSuite) TestModifyColumnTypeWithWarnings(c *C) { func (s *testSerialDBSuite) TestModifyColumnTypeWhenInterception(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") - // Enable column change variable. - tk.Se.GetSessionVars().EnableChangeColumnType = true // Test normal warnings. tk.MustExec("drop table if exists t") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 11c5141572075..9b670911c4b71 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3487,7 +3487,7 @@ func checkModifyCharsetAndCollation(toCharset, toCollate, origCharset, origColla // There are two cases when types incompatible: // 1. returned canReorg == true: types can be changed by reorg // 2. returned canReorg == false: type change not supported yet -func CheckModifyTypeCompatible(origin *types.FieldType, to *types.FieldType) (canReorg bool, errMsg string, err error) { +func CheckModifyTypeCompatible(origin *types.FieldType, to *types.FieldType) (canReorg bool, err error) { // Deal with the same type. if origin.Tp == to.Tp { if origin.Tp == mysql.TypeEnum || origin.Tp == mysql.TypeSet { @@ -3497,13 +3497,13 @@ func CheckModifyTypeCompatible(origin *types.FieldType, to *types.FieldType) (ca } if len(to.Elems) < len(origin.Elems) { msg := fmt.Sprintf("the number of %s column's elements is less than the original: %d", typeVar, len(origin.Elems)) - return true, msg, errUnsupportedModifyColumn.GenWithStackByArgs(msg) + return true, errUnsupportedModifyColumn.GenWithStackByArgs(msg) } for index, originElem := range origin.Elems { toElem := to.Elems[index] if originElem != toElem { msg := fmt.Sprintf("cannot modify %s column value %s to %s", typeVar, originElem, toElem) - return true, msg, errUnsupportedModifyColumn.GenWithStackByArgs(msg) + return true, errUnsupportedModifyColumn.GenWithStackByArgs(msg) } } } @@ -3514,21 +3514,21 @@ func CheckModifyTypeCompatible(origin *types.FieldType, to *types.FieldType) (ca // remains the same. if to.Flen != origin.Flen || to.Decimal != origin.Decimal || mysql.HasUnsignedFlag(to.Flag) != mysql.HasUnsignedFlag(origin.Flag) { msg := fmt.Sprintf("decimal change from decimal(%d, %d) to decimal(%d, %d)", origin.Flen, origin.Decimal, to.Flen, to.Decimal) - return true, msg, errUnsupportedModifyColumn.GenWithStackByArgs(msg) + return true, errUnsupportedModifyColumn.GenWithStackByArgs(msg) } } needReorg, reason := needReorgToChange(origin, to) if !needReorg { - return false, "", nil + return false, nil } - return true, reason, errUnsupportedModifyColumn.GenWithStackByArgs(reason) + return true, errUnsupportedModifyColumn.GenWithStackByArgs(reason) } // Deal with the different type. if !checkTypeChangeSupported(origin, to) { unsupportedMsg := fmt.Sprintf("change from original type %v to %v is currently unsupported yet", origin.CompactStr(), to.CompactStr()) - return false, unsupportedMsg, errUnsupportedModifyColumn.GenWithStackByArgs(unsupportedMsg) + return false, errUnsupportedModifyColumn.GenWithStackByArgs(unsupportedMsg) } // Check if different type can directly convert and no need to reorg. @@ -3537,13 +3537,13 @@ func CheckModifyTypeCompatible(origin *types.FieldType, to *types.FieldType) (ca if stringToString || integerToInteger { needReorg, reason := needReorgToChange(origin, to) if !needReorg { - return false, "", nil + return false, nil } - return true, reason, errUnsupportedModifyColumn.GenWithStackByArgs(reason) + return true, errUnsupportedModifyColumn.GenWithStackByArgs(reason) } notCompatibleMsg := fmt.Sprintf("type %v not match origin %v", to.CompactStr(), origin.CompactStr()) - return true, notCompatibleMsg, errUnsupportedModifyColumn.GenWithStackByArgs(notCompatibleMsg) + return true, errUnsupportedModifyColumn.GenWithStackByArgs(notCompatibleMsg) } func needReorgToChange(origin *types.FieldType, to *types.FieldType) (needReorg bool, reasonMsg string) { @@ -3605,20 +3605,15 @@ func checkTypeChangeSupported(origin *types.FieldType, to *types.FieldType) bool // checkModifyTypes checks if the 'origin' type can be modified to 'to' type no matter directly change // or change by reorg. It returns error if the two types are incompatible and correlated change are not -// supported. However, even the two types can be change, if the flag "tidb_enable_change_column_type" not -// set, or the "origin" type contains primary key, error will be returned. +// supported. However, even the two types can be change, if the "origin" type contains primary key, error will be returned. func checkModifyTypes(ctx sessionctx.Context, origin *types.FieldType, to *types.FieldType, needRewriteCollationData bool) error { - canReorg, changeColumnErrMsg, err := CheckModifyTypeCompatible(origin, to) + canReorg, err := CheckModifyTypeCompatible(origin, to) if err != nil { if !canReorg { return errors.Trace(err) } - enableChangeColumnType := ctx.GetSessionVars().EnableChangeColumnType - if !enableChangeColumnType { - msg := fmt.Sprintf("%s, and tidb_enable_change_column_type is false", changeColumnErrMsg) - return errUnsupportedModifyColumn.GenWithStackByArgs(msg) - } else if mysql.HasPriKeyFlag(origin.Flag) { - msg := "tidb_enable_change_column_type is true and this column has primary key flag" + if mysql.HasPriKeyFlag(origin.Flag) { + msg := "this column has primary key flag" return errUnsupportedModifyColumn.GenWithStackByArgs(msg) } } @@ -3896,12 +3891,12 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or } return nil, errors.Trace(err) } - if ctx.GetSessionVars().EnableChangeColumnType && needChangeColumnData(col.ColumnInfo, newCol.ColumnInfo) { + if needChangeColumnData(col.ColumnInfo, newCol.ColumnInfo) { if err = isGeneratedRelatedColumn(t.Meta(), newCol.ColumnInfo, col.ColumnInfo); err != nil { return nil, errors.Trace(err) } if t.Meta().Partition != nil { - return nil, errUnsupportedModifyColumn.GenWithStackByArgs("tidb_enable_change_column_type is true, table is partition table") + return nil, errUnsupportedModifyColumn.GenWithStackByArgs("table is partition table") } } diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index 8ccb7a5adb594..d70d32699fd84 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -460,16 +460,10 @@ func (s *testFailDBSuite) TestModifyColumn(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t;") - enableChangeColumnType := tk.Se.GetSessionVars().EnableChangeColumnType - tk.Se.GetSessionVars().EnableChangeColumnType = true - defer func() { - tk.Se.GetSessionVars().EnableChangeColumnType = enableChangeColumnType - }() - tk.MustExec("create table t (a int not null default 1, b int default 2, c int not null default 0, primary key(c), index idx(b), index idx1(a), index idx2(b, c))") tk.MustExec("insert into t values(1, 2, 3), (11, 22, 33)") _, err := tk.Exec("alter table t change column c cc mediumint") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true and this column has primary key flag") + c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: this column has primary key flag") tk.MustExec("alter table t change column b bb mediumint first") dom := domain.GetDomain(tk.Se) is := dom.InfoSchema() @@ -516,16 +510,16 @@ func (s *testFailDBSuite) TestModifyColumn(c *C) { // Test unsupport statements. tk.MustExec("create table t1(a int) partition by hash (a) partitions 2") _, err = tk.Exec("alter table t1 modify column a mediumint") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, table is partition table") + c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: table is partition table") tk.MustExec("create table t2(id int, a int, b int generated always as (abs(a)) virtual, c int generated always as (a+1) stored)") _, err = tk.Exec("alter table t2 modify column b mediumint") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, newCol IsGenerated false, oldCol IsGenerated true") + c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: newCol IsGenerated false, oldCol IsGenerated true") _, err = tk.Exec("alter table t2 modify column c mediumint") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, newCol IsGenerated false, oldCol IsGenerated true") + c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: newCol IsGenerated false, oldCol IsGenerated true") _, err = tk.Exec("alter table t2 modify column a mediumint generated always as(id+1) stored") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, newCol IsGenerated true, oldCol IsGenerated false") + c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: newCol IsGenerated true, oldCol IsGenerated false") _, err = tk.Exec("alter table t2 modify column a mediumint") - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: tidb_enable_change_column_type is true, oldCol is a dependent column 'a' for generated column") + c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: oldCol is a dependent column 'a' for generated column") // Test multiple rows of data. tk.MustExec("create table t3(a int not null default 1, b int default 2, c int not null default 0, primary key(c), index idx(b), index idx1(a), index idx2(b, c))") diff --git a/ddl/generated_column.go b/ddl/generated_column.go index 657a27ec3db4f..21bd3478e8e0f 100644 --- a/ddl/generated_column.go +++ b/ddl/generated_column.go @@ -154,11 +154,11 @@ func hasDependentByGeneratedColumn(tblInfo *model.TableInfo, colName model.CIStr func isGeneratedRelatedColumn(tblInfo *model.TableInfo, newCol, col *model.ColumnInfo) error { if newCol.IsGenerated() || col.IsGenerated() { // TODO: Make it compatible with MySQL error. - msg := fmt.Sprintf("tidb_enable_change_column_type is true, newCol IsGenerated %v, oldCol IsGenerated %v", newCol.IsGenerated(), col.IsGenerated()) + msg := fmt.Sprintf("newCol IsGenerated %v, oldCol IsGenerated %v", newCol.IsGenerated(), col.IsGenerated()) return errUnsupportedModifyColumn.GenWithStackByArgs(msg) } if ok, dep := hasDependentByGeneratedColumn(tblInfo, col.Name); ok { - msg := fmt.Sprintf("tidb_enable_change_column_type is true, oldCol is a dependent column '%s' for generated column", dep) + msg := fmt.Sprintf("oldCol is a dependent column '%s' for generated column", dep) return errUnsupportedModifyColumn.GenWithStackByArgs(msg) } return nil diff --git a/executor/admin_test.go b/executor/admin_test.go index 34d22ae58b2f4..da555ef999578 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -1076,8 +1076,7 @@ func (s *testSuite8) TestAdminCheckTable(c *C) { tk.MustExec(`create table t1 (a decimal(2,1), index(a))`) tk.MustExec(`insert into t1 set a='1.9'`) err = tk.ExecToErr(`alter table t1 modify column a decimal(3,2);`) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:8200]Unsupported modify column: decimal change from decimal(2, 1) to decimal(3, 2), and tidb_enable_change_column_type is false") + c.Assert(err, IsNil) tk.MustExec(`delete from t1;`) tk.MustExec(`admin check table t1;`) } diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 67c8b8978caa8..922dd04ebdf2e 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -544,7 +544,7 @@ func (s *testSuite6) TestAlterTableModifyColumn(c *C) { c.Assert(err, NotNil) _, err = tk.Exec("alter table mc modify column c2 varchar(8)") - c.Assert(err, NotNil) + c.Assert(err, IsNil) tk.MustExec("alter table mc modify column c2 varchar(11)") tk.MustExec("alter table mc modify column c2 text(13)") tk.MustExec("alter table mc modify column c2 text") diff --git a/executor/executor_test.go b/executor/executor_test.go index eef914d9de15d..f8f41506ccc30 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -202,13 +202,6 @@ func (s *baseTestSuite) SetUpSuite(c *C) { } d, err := session.BootstrapSession(s.store) c.Assert(err, IsNil) - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - // Set the variable to default 0 as it was before in case of modifying the test. - _, err = se.Execute(context.Background(), "set @@global.tidb_enable_change_column_type=0") - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "set @@tidb_enable_change_column_type=0") - c.Assert(err, IsNil) d.SetStatsUpdating(true) s.domain = d config.UpdateGlobal(func(conf *config.Config) { @@ -288,16 +281,6 @@ func (s *testSuiteP1) TestBind(c *C) { tk.MustExec("drop session binding for select * from testbind") } -func (s *testSuiteP1) TestChange(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(a int)") - tk.MustExec("alter table t change a b int") - tk.MustExec("alter table t change b c bigint") - c.Assert(tk.ExecToErr("alter table t change c d varchar(100)"), NotNil) -} - func (s *testSuiteP1) TestChangePumpAndDrainer(c *C) { tk := testkit.NewTestKit(c, s.store) // change pump or drainer's state need connect to etcd diff --git a/executor/set_test.go b/executor/set_test.go index 9684c51012bd8..b258cec1ca6a9 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -374,18 +374,6 @@ func (s *testSerialSuite1) TestSetVar(c *C) { tk.MustQuery("select @@session.tidb_store_limit;").Check(testkit.Rows("0")) tk.MustQuery("select @@global.tidb_store_limit;").Check(testkit.Rows("100")) - tk.MustQuery("select @@global.tidb_enable_change_column_type;").Check(testkit.Rows("0")) - tk.MustExec("set global tidb_enable_change_column_type = 1") - tk.MustQuery("select @@global.tidb_enable_change_column_type;").Check(testkit.Rows("1")) - tk.MustExec("set global tidb_enable_change_column_type = off") - tk.MustQuery("select @@global.tidb_enable_change_column_type;").Check(testkit.Rows("0")) - // test tidb_enable_change_column_type in session scope. - tk.MustQuery("select @@session.tidb_enable_change_column_type;").Check(testkit.Rows("0")) - tk.MustExec("set @@session.tidb_enable_change_column_type = 1") - tk.MustQuery("select @@session.tidb_enable_change_column_type;").Check(testkit.Rows("1")) - tk.MustExec("set @@session.tidb_enable_change_column_type = off") - tk.MustQuery("select @@session.tidb_enable_change_column_type;").Check(testkit.Rows("0")) - tk.MustQuery("select @@session.tidb_metric_query_step;").Check(testkit.Rows("60")) tk.MustExec("set @@session.tidb_metric_query_step = 120") _, err = tk.Exec("set @@session.tidb_metric_query_step = 9") diff --git a/expression/integration_test.go b/expression/integration_test.go index cc1b160c405e3..7b3078e89f4ac 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -94,13 +94,6 @@ func (s *testIntegrationSuiteBase) SetUpSuite(c *C) { s.store, s.dom, err = newStoreWithBootstrap() c.Assert(err, IsNil) s.ctx = mock.NewContext() - se, err := session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - // Set the variable to default 0 as it was before in case of modifying the test. - _, err = se.Execute(context.Background(), "set @@global.tidb_enable_change_column_type=0") - c.Assert(err, IsNil) - _, err = se.Execute(context.Background(), "set @@tidb_enable_change_column_type=0") - c.Assert(err, IsNil) } func (s *testIntegrationSuiteBase) TearDownSuite(c *C) { @@ -8084,7 +8077,7 @@ func (s *testIntegrationSerialSuite) TestIssue19804(c *C) { tk.MustExec(`create table t(a set('a', 'b', 'c'));`) tk.MustExec(`alter table t change a a set('a', 'b', 'c', 'd');`) tk.MustExec(`insert into t values('d');`) - tk.MustGetErrMsg(`alter table t change a a set('a', 'b', 'c', 'e', 'f');`, "[ddl:8200]Unsupported modify column: cannot modify set column value d to e, and tidb_enable_change_column_type is false") + tk.MustGetErrMsg(`alter table t change a a set('a', 'b', 'c', 'e', 'f');`, "[types:1265]Data truncated for column 'a', value is 'KindMysqlSet d'") } func (s *testIntegrationSerialSuite) TestIssue20209(c *C) { diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index c0d82b8006b84..775fa89cb3f28 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -2334,10 +2334,6 @@ func (s *testPessimisticSuite) TestAmendForUniqueIndex(c *C) { func (s *testPessimisticSuite) TestAmendWithColumnTypeChange(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("set global tidb_enable_change_column_type = 1;") - defer func() { - tk.MustExec("set global tidb_enable_change_column_type = 0;") - }() tk2 := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop database if exists test_db") tk.MustExec("create database test_db") diff --git a/session/schema_amender.go b/session/schema_amender.go index 19c7ac5a9667f..4d00b38873b37 100644 --- a/session/schema_amender.go +++ b/session/schema_amender.go @@ -170,7 +170,7 @@ func colChangeAmendable(colAtStart *model.ColumnInfo, colAtCommit *model.ColumnI if colAtStart.Charset != colAtCommit.Charset || colAtStart.Collate != colAtCommit.Collate { return errors.Trace(errors.Errorf("charset or collate is not matched for column=%v", colAtCommit.Name.String())) } - _, _, err := ddl.CheckModifyTypeCompatible(&colAtStart.FieldType, &colAtCommit.FieldType) + _, err := ddl.CheckModifyTypeCompatible(&colAtStart.FieldType, &colAtCommit.FieldType) if err != nil { return errors.Trace(err) } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 93db60de55e50..cd076bc50f4b5 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -617,9 +617,6 @@ type SessionVars struct { // DDLReorgPriority is the operation priority of adding indices. DDLReorgPriority int - // EnableChangeColumnType is used to control whether to enable the change column type. - EnableChangeColumnType bool - // EnableChangeMultiSchema is used to control whether to enable the multi schema change. EnableChangeMultiSchema bool @@ -1030,7 +1027,6 @@ func NewSessionVars() *SessionVars { EnableClusteredIndex: DefTiDBEnableClusteredIndex, EnableParallelApply: DefTiDBEnableParallelApply, ShardAllocateStep: DefTiDBShardAllocateStep, - EnableChangeColumnType: DefTiDBChangeColumnType, EnableChangeMultiSchema: DefTiDBChangeMultiSchema, EnablePointGetCache: DefTiDBPointGetCache, EnableAlterPlacement: DefTiDBEnableAlterPlacement, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 9ca8072cee286..9a51c17e2c6f6 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1317,10 +1317,6 @@ var defaultSysVars = []*SysVar{ SetMaxDeltaSchemaCount(tidbOptInt64(val, DefTiDBMaxDeltaSchemaCount)) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableChangeColumnType, Value: BoolToOnOff(DefTiDBChangeColumnType), Hidden: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { - s.EnableChangeColumnType = TiDBOptOn(val) - return nil - }}, {Scope: ScopeGlobal, Name: TiDBEnableChangeMultiSchema, Value: BoolToOnOff(DefTiDBChangeMultiSchema), Hidden: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableChangeMultiSchema = TiDBOptOn(val) return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index d9c23a0d49b48..2c2e232efbb4c 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -374,9 +374,6 @@ const ( // It can be: PRIORITY_LOW, PRIORITY_NORMAL, PRIORITY_HIGH TiDBDDLReorgPriority = "tidb_ddl_reorg_priority" - // TiDBEnableChangeColumnType is used to control whether to enable the change column type. - TiDBEnableChangeColumnType = "tidb_enable_change_column_type" - // TiDBEnableChangeMultiSchema is used to control whether to enable the change multi schema. TiDBEnableChangeMultiSchema = "tidb_enable_change_multi_schema" @@ -650,7 +647,6 @@ const ( DefTiDBDDLReorgBatchSize = 256 DefTiDBDDLErrorCountLimit = 512 DefTiDBMaxDeltaSchemaCount = 1024 - DefTiDBChangeColumnType = true DefTiDBChangeMultiSchema = false DefTiDBPointGetCache = false DefTiDBEnableAlterPlacement = false diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index fa3fe12144efa..78a5b5c492378 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -102,7 +102,6 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.FoundInBinding, Equals, DefTiDBFoundInBinding) c.Assert(vars.AllowAutoRandExplicitInsert, Equals, DefTiDBAllowAutoRandExplicitInsert) c.Assert(vars.ShardAllocateStep, Equals, int64(DefTiDBShardAllocateStep)) - c.Assert(vars.EnableChangeColumnType, Equals, DefTiDBChangeColumnType) c.Assert(vars.AnalyzeVersion, Equals, DefTiDBAnalyzeVersion) c.Assert(vars.CTEMaxRecursionDepth, Equals, DefCTEMaxRecursionDepth) @@ -490,13 +489,6 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { err = SetSessionSystemVar(v, TiDBFoundInBinding, "1") c.Assert(err, ErrorMatches, ".*]Variable 'last_plan_from_binding' is a read only variable") - err = SetSessionSystemVar(v, TiDBEnableChangeColumnType, "ON") - c.Assert(err, IsNil) - val, err = GetSessionOrGlobalSystemVar(v, TiDBEnableChangeColumnType) - c.Assert(err, IsNil) - c.Assert(val, Equals, "ON") - c.Assert(v.systems[TiDBEnableChangeColumnType], Equals, "ON") - err = SetSessionSystemVar(v, "UnknownVariable", "on") c.Assert(err, ErrorMatches, ".*]Unknown system variable 'UnknownVariable'") From 8bf35daf1a76ad2ecbfa5d8dc7d95cac87fc035c Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 9 Jun 2021 12:44:28 +0800 Subject: [PATCH 335/343] planner: disable mpp_outer_join_fixed_build_side by default (#25240) --- executor/tiflash_test.go | 42 +++++++++++ .../integration_serial_suite_out.json | 70 +++++++++---------- sessionctx/variable/tidb_vars.go | 2 +- 3 files changed, 78 insertions(+), 36 deletions(-) diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index f6d478649887c..d4a4f873e6db5 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -87,6 +87,9 @@ func (s *tiflashTestSuite) TestReadPartitionTable(c *C) { tk.MustExec("insert into t values(2,0)") tk.MustExec("insert into t values(3,0)") tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") tk.MustQuery("select /*+ STREAM_AGG() */ count(*) from t").Check(testkit.Rows("3")) tk.MustQuery("select * from t order by a").Check(testkit.Rows("1 0", "2 0", "3 0")) @@ -131,6 +134,9 @@ func (s *tiflashTestSuite) TestReadUnsigedPK(c *C) { tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") tk.MustExec("set @@session.tidb_opt_broadcast_join=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") tk.MustQuery("select count(*) from t1 , t where t1.a = t.a").Check(testkit.Rows("5")) tk.MustQuery("select count(*) from t1 , t where t1.a = t.a and ((t1.a < 9223372036854775800 and t1.a > 2) or (t1.a <= 1 and t1.a > -1))").Check(testkit.Rows("3")) @@ -163,6 +169,9 @@ func (s *tiflashTestSuite) TestMppExecution(c *C) { tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") for i := 0; i < 20; i++ { // test if it is stable. tk.MustQuery("select count(*) from t1 , t where t1.a = t.a").Check(testkit.Rows("3")) @@ -296,6 +305,9 @@ func (s *tiflashTestSuite) TestTiFlashPartitionTableShuffledHashJoin(c *C) { tk.MustExec("SET tidb_broadcast_join_threshold_count=0") tk.MustExec("SET tidb_broadcast_join_threshold_size=0") tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") lr := func() (int, int) { l, r := rand.Intn(400), rand.Intn(400) @@ -354,6 +366,9 @@ func (s *tiflashTestSuite) TestTiFlashPartitionTableReader(c *C) { err := domain.GetDomain(tk.Se).DDL().UpdateTableReplicaInfo(tk.Se, tb.Meta().ID, true) c.Assert(err, IsNil) } + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") vals := make([]string, 0, 500) for i := 0; i < 500; i++ { @@ -404,6 +419,9 @@ func (s *tiflashTestSuite) TestPartitionTable(c *C) { failpoint.Enable("github.com/pingcap/tidb/executor/checkUseMPP", `return(true)`) tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") failpoint.Enable("github.com/pingcap/tidb/executor/checkTotalMPPTasks", `return(4)`) tk.MustQuery("select count(*) from t").Check(testkit.Rows("4")) failpoint.Disable("github.com/pingcap/tidb/executor/checkTotalMPPTasks") @@ -488,6 +506,9 @@ func (s *tiflashTestSuite) TestMppEnum(c *C) { tk.MustExec("insert into t values(3,'zca')") tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") tk.MustQuery("select t1.b from t t1 join t t2 on t1.a = t2.a order by t1.b").Check(testkit.Rows("aca", "bca", "zca")) } @@ -509,6 +530,9 @@ func (s *tiflashTestSuite) TestCancelMppTasks(c *C) { c.Assert(err, IsNil) tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") atomic.StoreUint32(&tk.Se.GetSessionVars().Killed, 0) c.Assert(failpoint.Enable(hang, `return(true)`), IsNil) wg := &sync.WaitGroup{} @@ -553,6 +577,9 @@ func (s *tiflashTestSuite) TestMppGoroutinesExitFromErrors(c *C) { tk.MustExec("insert into t1 values(3,0)") tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") c.Assert(failpoint.Enable(mppNonRootTaskError, `return(true)`), IsNil) c.Assert(failpoint.Enable(hang, `return(true)`), IsNil) @@ -581,6 +608,9 @@ func (s *tiflashTestSuite) TestMppUnionAll(c *C) { tk.MustExec("insert into x1 values (1, 1), (2, 2), (3, 3), (4, 4)") tk.MustExec("insert into x2 values (5, 1), (2, 2), (3, 3), (4, 4)") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") // test join + union (join + select) tk.MustQuery("select x1.a, x.a from x1 left join (select x2.b a, x1.b from x1 join x2 on x1.a = x2.b union all select * from x1 ) x on x1.a = x.a order by x1.a").Check(testkit.Rows("1 1", "1 1", "2 2", "2 2", "3 3", "3 3", "4 4", "4 4")) @@ -626,6 +656,9 @@ func (s *tiflashTestSuite) TestMppApply(c *C) { tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") // table full scan with correlated filter tk.MustQuery("select /*+ agg_to_cop(), hash_agg()*/ count(*) from x1 where a >= any (select a from x2 where x1.a = x2.a) order by 1;").Check(testkit.Rows("3")) // table range scan with correlated access conditions @@ -659,6 +692,9 @@ func (s *tiflashTestSuite) TestTiFlashVirtualColumn(c *C) { tk.MustExec("set @@session.tidb_isolation_read_engines=\"tiflash\"") tk.MustExec("set @@session.tidb_allow_mpp=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") tk.MustQuery("select /*+ hash_agg() */ count(*) from t1 where c > b'01'").Check(testkit.Rows("2")) tk.MustQuery("select /*+ hash_agg() */ count(*) from t2 where c > 1").Check(testkit.Rows("2")) @@ -707,6 +743,9 @@ func (s *tiflashTestSuite) TestTiFlashPartitionTableShuffledHashAggregation(c *C } tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") tk.MustExec("set @@session.tidb_allow_mpp=2") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") lr := func() (int, int) { l, r := rand.Intn(400), rand.Intn(400) @@ -777,6 +816,9 @@ func (s *tiflashTestSuite) TestTiFlashPartitionTableBroadcastJoin(c *C) { tk.MustExec("set @@session.tidb_isolation_read_engines='tiflash'") tk.MustExec("set @@session.tidb_allow_mpp=2") tk.MustExec("set @@session.tidb_opt_broadcast_join=ON") + // mock executor does not support use outer table as build side for outer join, so need to + // force the inner table as build side + tk.MustExec("set tidb_opt_mpp_outer_join_fixed_build_side=1") lr := func() (int, int) { l, r := rand.Intn(400), rand.Intn(400) diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index c22d300c268e1..48afc8171f401 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -754,13 +754,13 @@ " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", - " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false", - " └─ExchangeReceiver(Probe) 4.00 batchCop[tiflash] ", - " └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", - " └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false" + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { @@ -848,13 +848,13 @@ " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 batchCop[tiflash] funcs:count(1)->Column#12", " └─HashJoin 32.00 batchCop[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", - " ├─ExchangeReceiver(Build) 16.00 batchCop[tiflash] ", - " │ └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", - " │ └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false", - " └─ExchangeReceiver(Probe) 4.00 batchCop[tiflash] ", - " └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", - " └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false" + " ├─ExchangeReceiver(Build) 4.00 batchCop[tiflash] ", + " │ └─ExchangeSender 4.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.d1_t.d1_k", + " │ └─TableFullScan 4.00 batchCop[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 batchCop[tiflash] ", + " └─ExchangeSender 16.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: test.fact_t.d1_k", + " └─Selection 16.00 batchCop[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 batchCop[tiflash] table:fact_t keep order:false" ] }, { @@ -1963,27 +1963,27 @@ "└─TableReader 19492.21 root data:ExchangeSender", " └─ExchangeSender 19492.21 cop[tiflash] ExchangeType: PassThrough", " └─HashJoin 19492.21 cop[tiflash] right outer join, equal:[eq(test.t.c3, test.t.c4)]", - " ├─Projection(Build) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", - " │ └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, test.t.c3)]", - " │ ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", - " │ │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#25", - " │ │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", - " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", - " │ └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", - " │ └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c5", - " │ └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", - " │ ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", - " │ │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2", - " │ │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", - " │ └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", - " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c1", - " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", - " │ └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo", - " └─ExchangeReceiver(Probe) 10000.00 cop[tiflash] ", - " └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#27", - " └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c4, decimal(40,20))->Column#27", - " └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo" + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#27", + " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c4, decimal(40,20))->Column#27", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t4 keep order:false, stats:pseudo", + " └─Projection(Probe) 15593.77 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5", + " └─HashJoin 15593.77 cop[tiflash] inner join, equal:[eq(test.t.c5, test.t.c3)]", + " ├─ExchangeReceiver(Build) 10000.00 cop[tiflash] ", + " │ └─ExchangeSender 10000.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: Column#25", + " │ └─Projection 10000.00 cop[tiflash] test.t.c1, test.t.c2, test.t.c3, test.t.c4, test.t.c5, cast(test.t.c3, decimal(40,20))->Column#25", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t3 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 12475.01 cop[tiflash] ", + " └─ExchangeSender 12475.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c5", + " └─HashJoin 12475.01 cop[tiflash] inner join, equal:[eq(test.t.c2, test.t.c1)]", + " ├─ExchangeReceiver(Build) 9980.01 cop[tiflash] ", + " │ └─ExchangeSender 9980.01 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c2", + " │ └─Selection 9980.01 cop[tiflash] not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", + " └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: test.t.c1", + " └─Selection 9990.00 cop[tiflash] not(isnull(test.t.c1))", + " └─TableFullScan 10000.00 cop[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 2c2e232efbb4c..33400361d10d6 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -595,7 +595,7 @@ const ( DefOptAggPushDown = false DefOptBCJ = false DefOptCartesianBCJ = 1 - DefOptMPPOuterJoinFixedBuildSide = true + DefOptMPPOuterJoinFixedBuildSide = false DefOptWriteRowID = false DefOptCorrelationThreshold = 0.9 DefOptCorrelationExpFactor = 1 From d5a0d176db39bae07f4584df119e81170dade1eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Wed, 9 Jun 2021 13:18:29 +0800 Subject: [PATCH 336/343] executor: add history read compatibility for temporary table (#25237) --- executor/builder.go | 50 ++++++++++++-------- executor/executor_test.go | 95 ++++++++++++++++++++++++++++++++++++++ executor/point_get.go | 4 +- executor/stale_txn_test.go | 90 ------------------------------------ 4 files changed, 128 insertions(+), 111 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 15ea2be70deb9..64204b4b61211 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1373,11 +1373,6 @@ func (b *executorBuilder) buildTableDual(v *plannercore.PhysicalTableDual) Execu return e } -// IsStaleness returns if the query is staleness -func (b *executorBuilder) IsStaleness() bool { - return b.ctx.GetSessionVars().TxnCtx.IsStaleness || b.explicitStaleness -} - // `getSnapshotTS` returns the timestamp of the snapshot that a reader should read. func (b *executorBuilder) getSnapshotTS() (uint64, error) { // `refreshForUpdateTSForRC` should always be invoked before returning the cached value to @@ -2649,8 +2644,8 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea return nil, err } ts := v.GetTableScan() - if ts.Table.TempTableType != model.TempTableNone && b.IsStaleness() { - return nil, errors.New("can not stale read temporary table") + if err = b.validCanReadTemporaryTable(ts.Table); err != nil { + return nil, err } tbl, _ := b.is.TableByID(ts.Table.ID) @@ -2747,8 +2742,8 @@ func (b *executorBuilder) buildTableReader(v *plannercore.PhysicalTableReader) E } ts := v.GetTableScan() - if ts.Table.TempTableType != model.TempTableNone && b.IsStaleness() { - b.err = errors.New("can not stale read temporary table") + if err = b.validCanReadTemporaryTable(ts.Table); err != nil { + b.err = err return nil } @@ -2961,8 +2956,8 @@ func buildNoRangeIndexReader(b *executorBuilder, v *plannercore.PhysicalIndexRea func (b *executorBuilder) buildIndexReader(v *plannercore.PhysicalIndexReader) Executor { is := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) - if is.Table.TempTableType != model.TempTableNone && b.IsStaleness() { - b.err = errors.New("can not stale read temporary table") + if err := b.validCanReadTemporaryTable(is.Table); err != nil { + b.err = err return nil } @@ -3113,8 +3108,8 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn func (b *executorBuilder) buildIndexLookUpReader(v *plannercore.PhysicalIndexLookUpReader) Executor { is := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) - if is.Table.TempTableType != model.TempTableNone && b.IsStaleness() { - b.err = errors.New("can not stale read temporary table") + if err := b.validCanReadTemporaryTable(is.Table); err != nil { + b.err = err return nil } @@ -3224,8 +3219,8 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd func (b *executorBuilder) buildIndexMergeReader(v *plannercore.PhysicalIndexMergeReader) Executor { ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) - if ts.Table.TempTableType != model.TempTableNone && b.IsStaleness() { - b.err = errors.New("can not stale read temporary table") + if err := b.validCanReadTemporaryTable(ts.Table); err != nil { + b.err = err return nil } @@ -4019,8 +4014,8 @@ func NewRowDecoder(ctx sessionctx.Context, schema *expression.Schema, tbl *model } func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan) Executor { - if plan.TblInfo.TempTableType != model.TempTableNone && b.IsStaleness() { - b.err = errors.New("can not stale read temporary table") + if err := b.validCanReadTemporaryTable(plan.TblInfo); err != nil { + b.err = err return nil } @@ -4156,8 +4151,8 @@ func fullRangePartition(idxArr []int) bool { } func (b *executorBuilder) buildTableSample(v *plannercore.PhysicalTableSample) *TableSampleExecutor { - if v.TableInfo.Meta().TempTableType != model.TempTableNone && b.IsStaleness() { - b.err = errors.New("can not stale read temporary table") + if err := b.validCanReadTemporaryTable(v.TableInfo.Meta()); err != nil { + b.err = err return nil } @@ -4274,3 +4269,20 @@ func (b *executorBuilder) buildCTETableReader(v *plannercore.PhysicalCTETable) E chkIdx: 0, } } + +func (b *executorBuilder) validCanReadTemporaryTable(tbl *model.TableInfo) error { + if tbl.TempTableType == model.TempTableNone { + return nil + } + + sessionVars := b.ctx.GetSessionVars() + if sessionVars.SnapshotTS != 0 { + return errors.New("can not read temporary table when 'tidb_snapshot' is set") + } + + if sessionVars.TxnCtx.IsStaleness || b.explicitStaleness { + return errors.New("can not stale read temporary table") + } + + return nil +} diff --git a/executor/executor_test.go b/executor/executor_test.go index f8f41506ccc30..d34a421724ea7 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8521,3 +8521,98 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { c.Assert(checkCnt > 0, IsTrue, commentf) } } + +func (s *testStaleTxnSuite) TestStaleOrHistoryReadTemporaryTable(c *C) { + tk := testkit.NewTestKit(c, s.store) + // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. + safePointName := "tikv_gc_safe_point" + safePointValue := "20160102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) + + tk.MustExec("set @@tidb_enable_global_temporary_table=1") + tk.MustExec("use test") + tk.MustExec("drop table if exists tmp1") + tk.MustExec("create global temporary table tmp1 " + + "(id int not null primary key, code int not null, value int default null, unique key code(code))" + + "on commit delete rows") + + // sleep 1us to make test stale + time.Sleep(time.Microsecond) + + queries := []struct { + sql string + }{ + { + sql: "select * from tmp1 where id=1", + }, + { + sql: "select * from tmp1 where code=1", + }, + { + sql: "select * from tmp1 where id in (1, 2, 3)", + }, + { + sql: "select * from tmp1 where code in (1, 2, 3)", + }, + { + sql: "select * from tmp1 where id > 1", + }, + { + sql: "select /*+use_index(tmp1, code)*/ * from tmp1 where code > 1", + }, + { + sql: "select /*+use_index(tmp1, code)*/ code from tmp1 where code > 1", + }, + { + sql: "select * from tmp1 tablesample regions()", + }, + { + sql: "select /*+ use_index_merge(tmp1, primary, code) */ * from tmp1 where id > 1 or code > 2", + }, + } + + addStaleReadToSQL := func(sql string) string { + idx := strings.Index(sql, " where ") + if idx < 0 { + return "" + } + return sql[0:idx] + " as of timestamp NOW(6)" + sql[idx:] + } + + for _, query := range queries { + sql := addStaleReadToSQL(query.sql) + if sql != "" { + tk.MustGetErrMsg(sql, "can not stale read temporary table") + } + } + + tk.MustExec("start transaction read only as of timestamp NOW(6)") + for _, query := range queries { + tk.MustGetErrMsg(query.sql, "can not stale read temporary table") + } + tk.MustExec("commit") + + for _, query := range queries { + tk.MustExec(query.sql) + } + + tk.MustExec("set transaction read only as of timestamp NOW(6)") + tk.MustExec("start transaction") + for _, query := range queries { + tk.MustGetErrMsg(query.sql, "can not stale read temporary table") + } + tk.MustExec("commit") + + for _, query := range queries { + tk.MustExec(query.sql) + } + + tk.MustExec("set @@tidb_snapshot=NOW(6)") + for _, query := range queries { + tk.MustGetErrMsg(query.sql, "can not read temporary table when 'tidb_snapshot' is set") + } +} diff --git a/executor/point_get.go b/executor/point_get.go index 3d1908d0fe394..51c9bc5877273 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -42,8 +42,8 @@ import ( ) func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { - if p.TblInfo.TempTableType != model.TempTableNone && b.IsStaleness() { - b.err = errors.New("can not stale read temporary table") + if err := b.validCanReadTemporaryTable(p.TblInfo); err != nil { + b.err = err return nil } diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index d7fa8a8bdec27..44def0981bed0 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -15,7 +15,6 @@ package executor_test import ( "fmt" - "strings" "time" . "github.com/pingcap/check" @@ -773,92 +772,3 @@ func (s *testStaleTxnSuite) TestSetTransactionInfoSchema(c *C) { tk.MustExec("commit") c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer3) } - -func (s *testStaleTxnSuite) TestStaleReadTemporaryTable(c *C) { - tk := testkit.NewTestKit(c, s.store) - // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. - safePointName := "tikv_gc_safe_point" - safePointValue := "20160102-15:04:05 -0700" - safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" - updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') - ON DUPLICATE KEY - UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) - tk.MustExec(updateSafePoint) - - tk.MustExec("set @@tidb_enable_global_temporary_table=1") - tk.MustExec("use test") - tk.MustExec("drop table if exists tmp1") - tk.MustExec("create global temporary table tmp1 " + - "(id int not null primary key, code int not null, value int default null, unique key code(code))" + - "on commit delete rows") - time.Sleep(time.Second) - tk.MustGetErrMsg("select * from tmp1 as of timestamp NOW() where id=1", "can not stale read temporary table") - - queries := []struct { - sql string - }{ - { - sql: "select * from tmp1 where id=1", - }, - { - sql: "select * from tmp1 where code=1", - }, - { - sql: "select * from tmp1 where id in (1, 2, 3)", - }, - { - sql: "select * from tmp1 where code in (1, 2, 3)", - }, - { - sql: "select * from tmp1 where id > 1", - }, - { - sql: "select /*+use_index(tmp1, code)*/ * from tmp1 where code > 1", - }, - { - sql: "select /*+use_index(tmp1, code)*/ code from tmp1 where code > 1", - }, - { - sql: "select * from tmp1 tablesample regions()", - }, - { - sql: "select /*+ use_index_merge(tmp1, primary, code) */ * from tmp1 where id > 1 or code > 2", - }, - } - - addStaleReadToSQL := func(sql string) string { - idx := strings.Index(sql, " where ") - if idx < 0 { - return "" - } - return sql[0:idx] + " as of timestamp NOW()" + sql[idx:] - } - - for _, query := range queries { - sql := addStaleReadToSQL(query.sql) - if sql != "" { - tk.MustGetErrMsg(sql, "can not stale read temporary table") - } - } - - tk.MustExec("start transaction read only as of timestamp NOW()") - for _, query := range queries { - tk.MustGetErrMsg(query.sql, "can not stale read temporary table") - } - tk.MustExec("commit") - - for _, query := range queries { - tk.MustExec(query.sql) - } - - tk.MustExec("set transaction read only as of timestamp NOW()") - tk.MustExec("start transaction") - for _, query := range queries { - tk.MustGetErrMsg(query.sql, "can not stale read temporary table") - } - tk.MustExec("commit") - - for _, query := range queries { - tk.MustExec(query.sql) - } -} From 6026b6389195f522634068d513976172866fccd4 Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 9 Jun 2021 13:28:28 +0800 Subject: [PATCH 337/343] *: update kvrpcpb to not use abbreviations (#25265) --- executor/executor_test.go | 17 +++++----- executor/seqtest/seq_executor_test.go | 30 ++++++++--------- session/schema_amender.go | 28 ++++++++-------- store/tikv/2pc.go | 48 +++++++++++++-------------- store/tikv/cleanup.go | 8 ++--- store/tikv/commit.go | 8 ++--- store/tikv/pessimistic.go | 20 +++++------ store/tikv/prewrite.go | 12 +++---- store/tikv/scan.go | 14 ++++---- store/tikv/snapshot.go | 19 +++++------ store/tikv/test_probe.go | 5 ++- store/tikv/tests/2pc_test.go | 5 ++- store/tikv/tests/prewrite_test.go | 6 ++-- store/tikv/tests/snapshot_test.go | 8 ++--- store/tikv/tests/store_test.go | 16 ++++----- 15 files changed, 120 insertions(+), 124 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index d34a421724ea7..38dea78d1ac5b 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -32,7 +32,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" @@ -3277,7 +3276,7 @@ const ( type checkRequestClient struct { tikv.Client - priority pb.CommandPri + priority kvrpcpb.CommandPri lowPriorityCnt uint32 mu struct { sync.RWMutex @@ -3286,12 +3285,12 @@ type checkRequestClient struct { } } -func (c *checkRequestClient) setCheckPriority(priority pb.CommandPri) { +func (c *checkRequestClient) setCheckPriority(priority kvrpcpb.CommandPri) { atomic.StoreInt32((*int32)(&c.priority), int32(priority)) } -func (c *checkRequestClient) getCheckPriority() pb.CommandPri { - return (pb.CommandPri)(atomic.LoadInt32((*int32)(&c.priority))) +func (c *checkRequestClient) getCheckPriority() kvrpcpb.CommandPri { + return (kvrpcpb.CommandPri)(atomic.LoadInt32((*int32)(&c.priority))) } func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { @@ -3315,7 +3314,7 @@ func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req * return nil, errors.New("fail to set priority") } } else if req.Type == tikvrpc.CmdPrewrite { - if c.getCheckPriority() == pb.CommandPri_Low { + if c.getCheckPriority() == kvrpcpb.CommandPri_Low { atomic.AddUint32(&c.lowPriorityCnt, 1) } } @@ -3403,7 +3402,7 @@ func (s *testSuite2) TestAddIndexPriority(c *C) { cli.mu.checkFlags = checkDDLAddIndexPriority cli.mu.Unlock() - cli.setCheckPriority(pb.CommandPri_Low) + cli.setCheckPriority(kvrpcpb.CommandPri_Low) tk.MustExec("alter table t1 add index t1_index (id);") c.Assert(atomic.LoadUint32(&cli.lowPriorityCnt) > 0, IsTrue) @@ -3419,7 +3418,7 @@ func (s *testSuite2) TestAddIndexPriority(c *C) { cli.mu.checkFlags = checkDDLAddIndexPriority cli.mu.Unlock() - cli.setCheckPriority(pb.CommandPri_Normal) + cli.setCheckPriority(kvrpcpb.CommandPri_Normal) tk.MustExec("alter table t1 add index t1_index (id);") cli.mu.Lock() @@ -3433,7 +3432,7 @@ func (s *testSuite2) TestAddIndexPriority(c *C) { cli.mu.checkFlags = checkDDLAddIndexPriority cli.mu.Unlock() - cli.setCheckPriority(pb.CommandPri_High) + cli.setCheckPriority(kvrpcpb.CommandPri_High) tk.MustExec("alter table t1 add index t1_index (id);") cli.mu.Lock() diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index bcecfc8d52ad4..34199a149794f 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -33,7 +33,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -1095,19 +1095,19 @@ func (s *seqTestSuite) TestBatchInsertDelete(c *C) { type checkPrioClient struct { tikv.Client - priority pb.CommandPri + priority kvrpcpb.CommandPri mu struct { sync.RWMutex checkPrio bool } } -func (c *checkPrioClient) setCheckPriority(priority pb.CommandPri) { +func (c *checkPrioClient) setCheckPriority(priority kvrpcpb.CommandPri) { atomic.StoreInt32((*int32)(&c.priority), int32(priority)) } -func (c *checkPrioClient) getCheckPriority() pb.CommandPri { - return (pb.CommandPri)(atomic.LoadInt32((*int32)(&c.priority))) +func (c *checkPrioClient) getCheckPriority() kvrpcpb.CommandPri { + return (kvrpcpb.CommandPri)(atomic.LoadInt32((*int32)(&c.priority))) } func (c *checkPrioClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { @@ -1174,11 +1174,11 @@ func (s *seqTestSuite1) TestCoprocessorPriority(c *C) { cli.mu.checkPrio = true cli.mu.Unlock() - cli.setCheckPriority(pb.CommandPri_High) + cli.setCheckPriority(kvrpcpb.CommandPri_High) tk.MustQuery("select id from t where id = 1") tk.MustQuery("select * from t1 where id = 1") - cli.setCheckPriority(pb.CommandPri_Normal) + cli.setCheckPriority(kvrpcpb.CommandPri_Normal) tk.MustQuery("select count(*) from t") tk.MustExec("update t set id = 3") tk.MustExec("delete from t") @@ -1193,43 +1193,43 @@ func (s *seqTestSuite1) TestCoprocessorPriority(c *C) { conf.Log.ExpensiveThreshold = 0 }) - cli.setCheckPriority(pb.CommandPri_High) + cli.setCheckPriority(kvrpcpb.CommandPri_High) tk.MustQuery("select id from t where id = 1") tk.MustQuery("select * from t1 where id = 1") tk.MustExec("delete from t where id = 2") tk.MustExec("update t set id = 2 where id = 1") - cli.setCheckPriority(pb.CommandPri_Low) + cli.setCheckPriority(kvrpcpb.CommandPri_Low) tk.MustQuery("select count(*) from t") tk.MustExec("delete from t") tk.MustExec("insert into t values (3)") // Test priority specified by SQL statement. - cli.setCheckPriority(pb.CommandPri_High) + cli.setCheckPriority(kvrpcpb.CommandPri_High) tk.MustQuery("select HIGH_PRIORITY * from t") - cli.setCheckPriority(pb.CommandPri_Low) + cli.setCheckPriority(kvrpcpb.CommandPri_Low) tk.MustQuery("select LOW_PRIORITY id from t where id = 1") - cli.setCheckPriority(pb.CommandPri_High) + cli.setCheckPriority(kvrpcpb.CommandPri_High) tk.MustExec("set tidb_force_priority = 'HIGH_PRIORITY'") tk.MustQuery("select * from t").Check(testkit.Rows("3")) tk.MustExec("update t set id = id + 1") tk.MustQuery("select v from t1 where id = 0 or id = 1").Check(testkit.Rows("0", "1")) - cli.setCheckPriority(pb.CommandPri_Low) + cli.setCheckPriority(kvrpcpb.CommandPri_Low) tk.MustExec("set tidb_force_priority = 'LOW_PRIORITY'") tk.MustQuery("select * from t").Check(testkit.Rows("4")) tk.MustExec("update t set id = id + 1") tk.MustQuery("select v from t1 where id = 0 or id = 1").Check(testkit.Rows("0", "1")) - cli.setCheckPriority(pb.CommandPri_Normal) + cli.setCheckPriority(kvrpcpb.CommandPri_Normal) tk.MustExec("set tidb_force_priority = 'DELAYED'") tk.MustQuery("select * from t").Check(testkit.Rows("5")) tk.MustExec("update t set id = id + 1") tk.MustQuery("select v from t1 where id = 0 or id = 1").Check(testkit.Rows("0", "1")) - cli.setCheckPriority(pb.CommandPri_Low) + cli.setCheckPriority(kvrpcpb.CommandPri_Low) tk.MustExec("set tidb_force_priority = 'NO_PRIORITY'") tk.MustQuery("select * from t").Check(testkit.Rows("6")) tk.MustExec("update t set id = id + 1") diff --git a/session/schema_amender.go b/session/schema_amender.go index 4d00b38873b37..08216637e9f74 100644 --- a/session/schema_amender.go +++ b/session/schema_amender.go @@ -21,7 +21,7 @@ import ( "reflect" "github.com/pingcap/errors" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/ddl" @@ -283,13 +283,13 @@ func (a *amendCollector) collectTblAmendOps(sctx sessionctx.Context, phyTblID in } // mayGenDelIndexRowKeyOp returns if the row key op could generate Op_Del index key mutations. -func mayGenDelIndexRowKeyOp(keyOp pb.Op) bool { - return keyOp == pb.Op_Del || keyOp == pb.Op_Put +func mayGenDelIndexRowKeyOp(keyOp kvrpcpb.Op) bool { + return keyOp == kvrpcpb.Op_Del || keyOp == kvrpcpb.Op_Put } // mayGenPutIndexRowKeyOp returns if the row key op could generate Op_Put/Op_Insert index key mutations. -func mayGenPutIndexRowKeyOp(keyOp pb.Op) bool { - return keyOp == pb.Op_Put || keyOp == pb.Op_Insert +func mayGenPutIndexRowKeyOp(keyOp kvrpcpb.Op) bool { + return keyOp == kvrpcpb.Op_Put || keyOp == kvrpcpb.Op_Insert } // amendOp is an amend operation for a specific schema change, new mutations will be generated using input ones. @@ -394,9 +394,9 @@ func (a *amendOperationAddIndex) genMutations(ctx context.Context, sctx sessionc } for i := 0; i < len(insertedMutations.GetKeys()); i++ { key := insertedMutations.GetKeys()[i] - destKeyOp := pb.Op_Insert + destKeyOp := kvrpcpb.Op_Insert if _, ok := a.deletedOldIndexKeys[string(key)]; ok { - destKeyOp = pb.Op_Put + destKeyOp = kvrpcpb.Op_Put } resAddMutations.Push(destKeyOp, key, insertedMutations.GetValues()[i], insertedMutations.GetPessimisticFlags()[i]) } @@ -477,14 +477,14 @@ func (a *amendOperationAddIndex) genNewIdxKey(ctx context.Context, sctx sessionc if err != nil { return nil, errors.Trace(err) } - newIndexOp := pb.Op_Put + newIndexOp := kvrpcpb.Op_Put isPessimisticLock := false if _, ok := a.insertedNewIndexKeys[string(newIdxKey)]; ok { return nil, errors.Trace(errors.Errorf("amend process key same key=%v found for index=%v in table=%v", newIdxKey, a.info.indexInfoAtCommit.Meta().Name, a.info.tblInfoAtCommit.Meta().Name)) } if a.info.indexInfoAtCommit.Meta().Unique { - newIndexOp = pb.Op_Insert + newIndexOp = kvrpcpb.Op_Insert isPessimisticLock = true } a.insertedNewIndexKeys[string(newIdxKey)] = struct{}{} @@ -515,7 +515,7 @@ func (a *amendOperationAddIndex) genOldIdxKey(ctx context.Context, sctx sessionc isPessimisticLock = true } a.deletedOldIndexKeys[string(newIdxKey)] = struct{}{} - return &tikv.PlainMutation{KeyOp: pb.Op_Del, Key: newIdxKey, Value: emptyVal, IsPessimisticLock: isPessimisticLock}, nil + return &tikv.PlainMutation{KeyOp: kvrpcpb.Op_Del, Key: newIdxKey, Value: emptyVal, IsPessimisticLock: isPessimisticLock}, nil } return nil, nil } @@ -540,12 +540,12 @@ func (s *SchemaAmender) getAmendableKeys(commitMutations tikv.CommitterMutations } keyOp := commitMutations.GetOp(i) switch keyOp { - case pb.Op_Put: + case kvrpcpb.Op_Put: addKeys = append(addKeys, byteKey) removeKeys = append(removeKeys, byteKey) - case pb.Op_Insert: + case kvrpcpb.Op_Insert: addKeys = append(addKeys, byteKey) - case pb.Op_Del: + case kvrpcpb.Op_Del: removeKeys = append(removeKeys, byteKey) } } @@ -594,7 +594,7 @@ func (s *SchemaAmender) prepareKvMap(ctx context.Context, commitMutations tikv.C func (s *SchemaAmender) checkDupKeys(ctx context.Context, mutations tikv.CommitterMutations) error { // Check if there are duplicate key entries. - checkMap := make(map[string]pb.Op) + checkMap := make(map[string]kvrpcpb.Op) for i := 0; i < mutations.Len(); i++ { key := mutations.GetKey(i) keyOp := mutations.GetOp(i) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index ee714d61c93c4..235d60203d940 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -27,7 +27,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/config" @@ -66,7 +66,7 @@ type twoPhaseCommitter struct { mutations *memBufferMutations lockTTL uint64 commitTS uint64 - priority pb.CommandPri + priority kvrpcpb.CommandPri sessionID uint64 // sessionID is used for log. cleanWg sync.WaitGroup detail unsafe.Pointer @@ -149,8 +149,8 @@ func (m *memBufferMutations) GetValue(i int) []byte { return v } -func (m *memBufferMutations) GetOp(i int) pb.Op { - return pb.Op(m.handles[i].UserData >> 1) +func (m *memBufferMutations) GetOp(i int) kvrpcpb.Op { + return kvrpcpb.Op(m.handles[i].UserData >> 1) } func (m *memBufferMutations) IsPessimisticLock(i int) bool { @@ -164,7 +164,7 @@ func (m *memBufferMutations) Slice(from, to int) CommitterMutations { } } -func (m *memBufferMutations) Push(op pb.Op, isPessimisticLock bool, handle unionstore.MemKeyHandle) { +func (m *memBufferMutations) Push(op kvrpcpb.Op, isPessimisticLock bool, handle unionstore.MemKeyHandle) { aux := uint16(op) << 1 if isPessimisticLock { aux |= 1 @@ -178,7 +178,7 @@ type CommitterMutations interface { Len() int GetKey(i int) []byte GetKeys() [][]byte - GetOp(i int) pb.Op + GetOp(i int) kvrpcpb.Op GetValue(i int) []byte IsPessimisticLock(i int) bool Slice(from, to int) CommitterMutations @@ -186,7 +186,7 @@ type CommitterMutations interface { // PlainMutations contains transaction operations. type PlainMutations struct { - ops []pb.Op + ops []kvrpcpb.Op keys [][]byte values [][]byte isPessimisticLock []bool @@ -195,7 +195,7 @@ type PlainMutations struct { // NewPlainMutations creates a PlainMutations object with sizeHint reserved. func NewPlainMutations(sizeHint int) PlainMutations { return PlainMutations{ - ops: make([]pb.Op, 0, sizeHint), + ops: make([]kvrpcpb.Op, 0, sizeHint), keys: make([][]byte, 0, sizeHint), values: make([][]byte, 0, sizeHint), isPessimisticLock: make([]bool, 0, sizeHint), @@ -219,7 +219,7 @@ func (c *PlainMutations) Slice(from, to int) CommitterMutations { } // Push another mutation into mutations. -func (c *PlainMutations) Push(op pb.Op, key []byte, value []byte, isPessimisticLock bool) { +func (c *PlainMutations) Push(op kvrpcpb.Op, key []byte, value []byte, isPessimisticLock bool) { c.ops = append(c.ops, op) c.keys = append(c.keys, key) c.values = append(c.values, value) @@ -242,7 +242,7 @@ func (c *PlainMutations) GetKeys() [][]byte { } // GetOps returns the key ops. -func (c *PlainMutations) GetOps() []pb.Op { +func (c *PlainMutations) GetOps() []kvrpcpb.Op { return c.ops } @@ -257,7 +257,7 @@ func (c *PlainMutations) GetPessimisticFlags() []bool { } // GetOp returns the key op at index. -func (c *PlainMutations) GetOp(i int) pb.Op { +func (c *PlainMutations) GetOp(i int) kvrpcpb.Op { return c.ops[i] } @@ -276,7 +276,7 @@ func (c *PlainMutations) IsPessimisticLock(i int) bool { // PlainMutation represents a single transaction operation. type PlainMutation struct { - KeyOp pb.Op + KeyOp kvrpcpb.Op Key []byte Value []byte IsPessimisticLock bool @@ -343,13 +343,13 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error { key := it.Key() flags := it.Flags() var value []byte - var op pb.Op + var op kvrpcpb.Op if !it.HasValue() { if !flags.HasLocked() { continue } - op = pb.Op_Lock + op = kvrpcpb.Op_Lock lockCnt++ } else { value = it.Value() @@ -362,12 +362,12 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error { // If the key was locked before, we should prewrite the lock even if // the KV needn't be committed according to the filter. Otherwise, we // were forgetting removing pessimistic locks added before. - op = pb.Op_Lock + op = kvrpcpb.Op_Lock lockCnt++ } else { - op = pb.Op_Put + op = kvrpcpb.Op_Put if flags.HasPresumeKeyNotExists() { - op = pb.Op_Insert + op = kvrpcpb.Op_Insert } putCnt++ } @@ -375,13 +375,13 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error { if !txn.IsPessimistic() && flags.HasPresumeKeyNotExists() { // delete-your-writes keys in optimistic txn need check not exists in prewrite-phase // due to `Op_CheckNotExists` doesn't prewrite lock, so mark those keys should not be used in commit-phase. - op = pb.Op_CheckNotExists + op = kvrpcpb.Op_CheckNotExists checkCnt++ memBuf.UpdateFlags(key, kv.SetPrewriteOnly) } else { // normal delete keys in optimistic txn can be delete without not exists checking // delete-your-writes keys in pessimistic txn can ensure must be no exists so can directly delete them - op = pb.Op_Del + op = kvrpcpb.Op_Del delCnt++ } } @@ -394,7 +394,7 @@ func (c *twoPhaseCommitter) initKeysAndMutations() error { c.mutations.Push(op, isPessimistic, it.Handle()) size += len(key) + len(value) - if len(c.primaryKey) == 0 && op != pb.Op_CheckNotExists { + if len(c.primaryKey) == 0 && op != kvrpcpb.Op_CheckNotExists { c.primaryKey = key } } @@ -452,7 +452,7 @@ func (c *twoPhaseCommitter) asyncSecondaries() [][]byte { secondaries := make([][]byte, 0, c.mutations.Len()) for i := 0; i < c.mutations.Len(); i++ { k := c.mutations.GetKey(i) - if bytes.Equal(k, c.primary()) || c.mutations.GetOp(i) == pb.Op_CheckNotExists { + if bytes.Equal(k, c.primary()) || c.mutations.GetOp(i) == kvrpcpb.Op_CheckNotExists { continue } secondaries = append(secondaries, k) @@ -837,7 +837,7 @@ func (tm *ttlManager) keepAlive(c *twoPhaseCommitter) { } func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, ttl uint64) (newTTL uint64, stopHeartBeat bool, err error) { - req := tikvrpc.NewRequest(tikvrpc.CmdTxnHeartBeat, &pb.TxnHeartBeatRequest{ + req := tikvrpc.NewRequest(tikvrpc.CmdTxnHeartBeat, &kvrpcpb.TxnHeartBeatRequest{ PrimaryLock: primary, StartVersion: startTS, AdviseLockTtl: ttl, @@ -870,7 +870,7 @@ func sendTxnHeartBeat(bo *Backoffer, store *KVStore, primary []byte, startTS, tt if resp.Resp == nil { return 0, false, errors.Trace(tikverr.ErrBodyMissing) } - cmdResp := resp.Resp.(*pb.TxnHeartBeatResponse) + cmdResp := resp.Resp.(*kvrpcpb.TxnHeartBeatResponse) if keyErr := cmdResp.GetError(); keyErr != nil { return 0, true, errors.Errorf("txn %d heartbeat fail, primary key = %v, err = %s", startTS, hex.EncodeToString(primary), extractKeyErr(keyErr)) } @@ -1428,7 +1428,7 @@ func (c *twoPhaseCommitter) tryAmendTxn(ctx context.Context, startInfoSchema Sch key := addMutations.GetKey(i) op := addMutations.GetOp(i) var err error - if op == pb.Op_Del { + if op == kvrpcpb.Op_Del { err = memBuf.Delete(key) } else { err = memBuf.Set(key, addMutations.GetValue(i)) diff --git a/store/tikv/cleanup.go b/store/tikv/cleanup.go index 553e00ff360a7..08f65e4434236 100644 --- a/store/tikv/cleanup.go +++ b/store/tikv/cleanup.go @@ -15,7 +15,7 @@ package tikv import ( "github.com/pingcap/errors" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -38,10 +38,10 @@ func (actionCleanup) tiKVTxnRegionsNumHistogram() prometheus.Observer { } func (actionCleanup) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { - req := tikvrpc.NewRequest(tikvrpc.CmdBatchRollback, &pb.BatchRollbackRequest{ + req := tikvrpc.NewRequest(tikvrpc.CmdBatchRollback, &kvrpcpb.BatchRollbackRequest{ Keys: batch.mutations.GetKeys(), StartVersion: c.startTS, - }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) + }, kvrpcpb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) resp, err := c.store.SendReq(bo, req, batch.region, client.ReadTimeoutShort) if err != nil { return errors.Trace(err) @@ -58,7 +58,7 @@ func (actionCleanup) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batc err = c.cleanupMutations(bo, batch.mutations) return errors.Trace(err) } - if keyErr := resp.Resp.(*pb.BatchRollbackResponse).GetError(); keyErr != nil { + if keyErr := resp.Resp.(*kvrpcpb.BatchRollbackResponse).GetError(); keyErr != nil { err = errors.Errorf("session %d 2PC cleanup failed: %s", c.sessionID, keyErr) logutil.BgLogger().Debug("2PC failed cleanup key", zap.Error(err), diff --git a/store/tikv/commit.go b/store/tikv/commit.go index b6456bbe8d3cb..9eb60d41d474f 100644 --- a/store/tikv/commit.go +++ b/store/tikv/commit.go @@ -19,7 +19,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv/client" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" @@ -44,11 +44,11 @@ func (actionCommit) tiKVTxnRegionsNumHistogram() prometheus.Observer { func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { keys := batch.mutations.GetKeys() - req := tikvrpc.NewRequest(tikvrpc.CmdCommit, &pb.CommitRequest{ + req := tikvrpc.NewRequest(tikvrpc.CmdCommit, &kvrpcpb.CommitRequest{ StartVersion: c.startTS, Keys: keys, CommitVersion: c.commitTS, - }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) + }, kvrpcpb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) tBegin := time.Now() attempts := 0 @@ -104,7 +104,7 @@ func (actionCommit) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch if resp.Resp == nil { return errors.Trace(tikverr.ErrBodyMissing) } - commitResp := resp.Resp.(*pb.CommitResponse) + commitResp := resp.Resp.(*kvrpcpb.CommitResponse) // Here we can make sure tikv has processed the commit primary key request. So // we can clean undetermined error. if batch.isPrimary && !c.isAsyncCommit() { diff --git a/store/tikv/pessimistic.go b/store/tikv/pessimistic.go index fe886e0369bdf..235535be8379b 100644 --- a/store/tikv/pessimistic.go +++ b/store/tikv/pessimistic.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv/client" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" @@ -70,14 +70,14 @@ func (actionPessimisticRollback) tiKVTxnRegionsNumHistogram() prometheus.Observe func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { m := batch.mutations - mutations := make([]*pb.Mutation, m.Len()) + mutations := make([]*kvrpcpb.Mutation, m.Len()) for i := 0; i < m.Len(); i++ { - mut := &pb.Mutation{ - Op: pb.Op_PessimisticLock, + mut := &kvrpcpb.Mutation{ + Op: kvrpcpb.Op_PessimisticLock, Key: m.GetKey(i), } - if c.txn.us.HasPresumeKeyNotExists(m.GetKey(i)) || (c.doingAmend && m.GetOp(i) == pb.Op_Insert) { - mut.Assertion = pb.Assertion_NotExist + if c.txn.us.HasPresumeKeyNotExists(m.GetKey(i)) || (c.doingAmend && m.GetOp(i) == kvrpcpb.Op_Insert) { + mut.Assertion = kvrpcpb.Assertion_NotExist } mutations[i] = mut } @@ -92,7 +92,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * logutil.BgLogger().Info("[failpoint] injected lock ttl = 1 on pessimistic lock", zap.Uint64("txnStartTS", c.startTS), zap.Strings("keys", keys)) }) - req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticLock, &pb.PessimisticLockRequest{ + req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticLock, &kvrpcpb.PessimisticLockRequest{ Mutations: mutations, PrimaryLock: c.primary(), StartVersion: c.startTS, @@ -102,7 +102,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * WaitTimeout: action.LockWaitTime, ReturnValues: action.ReturnValues, MinCommitTs: c.forUpdateTS + 1, - }, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: action.LockCtx.ResourceGroupTag}) + }, kvrpcpb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: action.LockCtx.ResourceGroupTag}) lockWaitStartTime := action.WaitStartTime for { // if lockWaitTime set, refine the request `WaitTimeout` field based on timeout limit @@ -154,7 +154,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * if resp.Resp == nil { return errors.Trace(tikverr.ErrBodyMissing) } - lockResp := resp.Resp.(*pb.PessimisticLockResponse) + lockResp := resp.Resp.(*kvrpcpb.PessimisticLockResponse) keyErrs := lockResp.GetErrors() if len(keyErrs) == 0 { if action.ReturnValues { @@ -228,7 +228,7 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * } func (actionPessimisticRollback) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) error { - req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticRollback, &pb.PessimisticRollbackRequest{ + req := tikvrpc.NewRequest(tikvrpc.CmdPessimisticRollback, &kvrpcpb.PessimisticRollbackRequest{ StartVersion: c.startTS, ForUpdateTs: c.forUpdateTS, Keys: batch.mutations.GetKeys(), diff --git a/store/tikv/prewrite.go b/store/tikv/prewrite.go index 0e6afa96010b2..c221ad4b52c44 100644 --- a/store/tikv/prewrite.go +++ b/store/tikv/prewrite.go @@ -22,7 +22,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" "github.com/pingcap/failpoint" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv/client" "github.com/pingcap/tidb/store/tikv/config" tikverr "github.com/pingcap/tidb/store/tikv/error" @@ -48,10 +48,10 @@ func (actionPrewrite) tiKVTxnRegionsNumHistogram() prometheus.Observer { func (c *twoPhaseCommitter) buildPrewriteRequest(batch batchMutations, txnSize uint64) *tikvrpc.Request { m := batch.mutations - mutations := make([]*pb.Mutation, m.Len()) + mutations := make([]*kvrpcpb.Mutation, m.Len()) isPessimisticLock := make([]bool, m.Len()) for i := 0; i < m.Len(); i++ { - mutations[i] = &pb.Mutation{ + mutations[i] = &kvrpcpb.Mutation{ Op: m.GetOp(i), Key: m.GetKey(i), Value: m.GetValue(i), @@ -88,7 +88,7 @@ func (c *twoPhaseCommitter) buildPrewriteRequest(batch batchMutations, txnSize u }) } - req := &pb.PrewriteRequest{ + req := &kvrpcpb.PrewriteRequest{ Mutations: mutations, PrimaryLock: c.primary(), StartVersion: c.startTS, @@ -117,7 +117,7 @@ func (c *twoPhaseCommitter) buildPrewriteRequest(batch batchMutations, txnSize u req.TryOnePc = true } - return tikvrpc.NewRequest(tikvrpc.CmdPrewrite, req, pb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) + return tikvrpc.NewRequest(tikvrpc.CmdPrewrite, req, kvrpcpb.Context{Priority: c.priority, SyncLog: c.syncLog, ResourceGroupTag: c.resourceGroupTag}) } func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoffer, batch batchMutations) (err error) { @@ -214,7 +214,7 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff if resp.Resp == nil { return errors.Trace(tikverr.ErrBodyMissing) } - prewriteResp := resp.Resp.(*pb.PrewriteResponse) + prewriteResp := resp.Resp.(*kvrpcpb.PrewriteResponse) keyErrs := prewriteResp.GetErrors() if len(keyErrs) == 0 { if batch.isPrimary { diff --git a/store/tikv/scan.go b/store/tikv/scan.go index c4d41764b24ef..2f83c2db113be 100644 --- a/store/tikv/scan.go +++ b/store/tikv/scan.go @@ -18,7 +18,7 @@ import ( "context" "github.com/pingcap/errors" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv/client" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/kv" @@ -32,7 +32,7 @@ import ( type Scanner struct { snapshot *KVSnapshot batchSize int - cache []*pb.KvPair + cache []*kvrpcpb.KvPair idx int nextStartKey []byte endKey []byte @@ -148,7 +148,7 @@ func (s *Scanner) startTS() uint64 { return s.snapshot.version } -func (s *Scanner) resolveCurrentLock(bo *Backoffer, current *pb.KvPair) error { +func (s *Scanner) resolveCurrentLock(bo *Backoffer, current *kvrpcpb.KvPair) error { ctx := context.Background() val, err := s.snapshot.get(ctx, bo, current.Key) if err != nil { @@ -191,8 +191,8 @@ func (s *Scanner) getData(bo *Backoffer) error { reqStartKey = loc.StartKey } } - sreq := &pb.ScanRequest{ - Context: &pb.Context{ + sreq := &kvrpcpb.ScanRequest{ + Context: &kvrpcpb.Context{ Priority: s.snapshot.priority.ToPB(), NotFillCache: s.snapshot.notFillCache, IsolationLevel: s.snapshot.isolationLevel.ToPB(), @@ -211,7 +211,7 @@ func (s *Scanner) getData(bo *Backoffer) error { sreq.Reverse = true } s.snapshot.mu.RLock() - req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdScan, sreq, s.snapshot.mu.replicaRead, &s.snapshot.replicaReadSeed, pb.Context{ + req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdScan, sreq, s.snapshot.mu.replicaRead, &s.snapshot.replicaReadSeed, kvrpcpb.Context{ Priority: s.snapshot.priority.ToPB(), NotFillCache: s.snapshot.notFillCache, TaskId: s.snapshot.mu.taskID, @@ -243,7 +243,7 @@ func (s *Scanner) getData(bo *Backoffer) error { if resp.Resp == nil { return errors.Trace(tikverr.ErrBodyMissing) } - cmdScanResp := resp.Resp.(*pb.ScanResponse) + cmdScanResp := resp.Resp.(*kvrpcpb.ScanResponse) err = s.snapshot.store.CheckVisibility(s.startTS()) if err != nil { diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index 740a7500bf587..c49310477e62c 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -27,7 +27,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/store/tikv/client" tikverr "github.com/pingcap/tidb/store/tikv/error" @@ -321,10 +320,10 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec pending := batch.keys for { s.mu.RLock() - req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdBatchGet, &pb.BatchGetRequest{ + req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdBatchGet, &kvrpcpb.BatchGetRequest{ Keys: pending, Version: s.version, - }, s.mu.replicaRead, &s.replicaReadSeed, pb.Context{ + }, s.mu.replicaRead, &s.replicaReadSeed, kvrpcpb.Context{ Priority: s.priority.ToPB(), NotFillCache: s.notFillCache, TaskId: s.mu.taskID, @@ -374,7 +373,7 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec if resp.Resp == nil { return errors.Trace(tikverr.ErrBodyMissing) } - batchGetResp := resp.Resp.(*pb.BatchGetResponse) + batchGetResp := resp.Resp.(*kvrpcpb.BatchGetResponse) var ( lockedKeys [][]byte locks []*Lock @@ -486,10 +485,10 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, }() } req := tikvrpc.NewReplicaReadRequest(tikvrpc.CmdGet, - &pb.GetRequest{ + &kvrpcpb.GetRequest{ Key: k, Version: s.version, - }, s.mu.replicaRead, &s.replicaReadSeed, pb.Context{ + }, s.mu.replicaRead, &s.replicaReadSeed, kvrpcpb.Context{ Priority: s.priority.ToPB(), NotFillCache: s.notFillCache, TaskId: s.mu.taskID, @@ -536,7 +535,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, if resp.Resp == nil { return nil, errors.Trace(tikverr.ErrBodyMissing) } - cmdGetResp := resp.Resp.(*pb.GetResponse) + cmdGetResp := resp.Resp.(*kvrpcpb.GetResponse) if cmdGetResp.ExecDetailsV2 != nil { s.mergeExecDetail(cmdGetResp.ExecDetailsV2) } @@ -572,7 +571,7 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, } } -func (s *KVSnapshot) mergeExecDetail(detail *pb.ExecDetailsV2) { +func (s *KVSnapshot) mergeExecDetail(detail *kvrpcpb.ExecDetailsV2) { s.mu.Lock() defer s.mu.Unlock() if detail == nil || s.mu.stats == nil { @@ -687,14 +686,14 @@ func (s *KVSnapshot) SnapCacheSize() int { return len(s.mu.cached) } -func extractLockFromKeyErr(keyErr *pb.KeyError) (*Lock, error) { +func extractLockFromKeyErr(keyErr *kvrpcpb.KeyError) (*Lock, error) { if locked := keyErr.GetLocked(); locked != nil { return NewLock(locked), nil } return nil, extractKeyErr(keyErr) } -func extractKeyErr(keyErr *pb.KeyError) error { +func extractKeyErr(keyErr *kvrpcpb.KeyError) error { if val, err := util.MockRetryableErrorResp.Eval(); err == nil { if val.(bool) { keyErr.Conflict = nil diff --git a/store/tikv/test_probe.go b/store/tikv/test_probe.go index d7eb5836d70ad..eb813335a6dd7 100644 --- a/store/tikv/test_probe.go +++ b/store/tikv/test_probe.go @@ -21,7 +21,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/store/tikv/retry" "github.com/pingcap/tidb/store/tikv/tikvrpc" @@ -406,7 +405,7 @@ func (s SnapshotProbe) RecordBackoffInfo(bo *Backoffer) { } // MergeExecDetail merges exec stats into snapshot's stats. -func (s SnapshotProbe) MergeExecDetail(detail *pb.ExecDetailsV2) { +func (s SnapshotProbe) MergeExecDetail(detail *kvrpcpb.ExecDetailsV2) { s.mergeExecDetail(detail) } @@ -422,7 +421,7 @@ type LockProbe struct { } // ExtractLockFromKeyErr makes a Lock based on a key error. -func (l LockProbe) ExtractLockFromKeyErr(err *pb.KeyError) (*Lock, error) { +func (l LockProbe) ExtractLockFromKeyErr(err *kvrpcpb.KeyError) (*Lock, error) { return extractLockFromKeyErr(err) } diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index dc59e2b1bbcea..2ecb6267b0f50 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -28,7 +28,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" drivertxn "github.com/pingcap/tidb/store/driver/txn" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/config" @@ -1043,10 +1042,10 @@ func (s *testCommitterSuite) TestResolvePessimisticLock(c *C) { c.Assert(err, IsNil) mutation := commit.MutationsOfKeys([][]byte{untouchedIndexKey, noValueIndexKey}) c.Assert(mutation.Len(), Equals, 2) - c.Assert(mutation.GetOp(0), Equals, pb.Op_Lock) + c.Assert(mutation.GetOp(0), Equals, kvrpcpb.Op_Lock) c.Assert(mutation.GetKey(0), BytesEquals, untouchedIndexKey) c.Assert(mutation.GetValue(0), BytesEquals, untouchedIndexValue) - c.Assert(mutation.GetOp(1), Equals, pb.Op_Lock) + c.Assert(mutation.GetOp(1), Equals, kvrpcpb.Op_Lock) c.Assert(mutation.GetKey(1), BytesEquals, noValueIndexKey) c.Assert(mutation.GetValue(1), BytesEquals, []byte{}) } diff --git a/store/tikv/tests/prewrite_test.go b/store/tikv/tests/prewrite_test.go index 6f75959b4afe4..1b80d05abf373 100644 --- a/store/tikv/tests/prewrite_test.go +++ b/store/tikv/tests/prewrite_test.go @@ -15,7 +15,7 @@ package tikv_test import ( . "github.com/pingcap/check" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/store/tikv" ) @@ -45,9 +45,9 @@ func (s *testPrewriteSuite) TestSetMinCommitTSInAsyncCommit(c *C) { c.Assert(err, IsNil) committer.SetUseAsyncCommit() - buildRequest := func() *pb.PrewriteRequest { + buildRequest := func() *kvrpcpb.PrewriteRequest { req := committer.BuildPrewriteRequest(1, 1, 1, committer.GetMutations(), 1) - return req.Req.(*pb.PrewriteRequest) + return req.Req.(*kvrpcpb.PrewriteRequest) } // no forUpdateTS diff --git a/store/tikv/tests/snapshot_test.go b/store/tikv/tests/snapshot_test.go index ee2a71730b5bf..1ae3f303740ff 100644 --- a/store/tikv/tests/snapshot_test.go +++ b/store/tikv/tests/snapshot_test.go @@ -23,7 +23,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" "github.com/pingcap/tidb/store/tikv/logutil" @@ -279,12 +279,12 @@ func (s *testSnapshotSuite) TestSnapshotRuntimeStats(c *C) { snapshot.RecordBackoffInfo(bo) expect := "Get:{num_rpc:4, total_time:2s},txnLockFast_backoff:{num:2, total_time:60ms}" c.Assert(snapshot.FormatStats(), Equals, expect) - detail := &pb.ExecDetailsV2{ - TimeDetail: &pb.TimeDetail{ + detail := &kvrpcpb.ExecDetailsV2{ + TimeDetail: &kvrpcpb.TimeDetail{ WaitWallTimeMs: 100, ProcessWallTimeMs: 100, }, - ScanDetailV2: &pb.ScanDetailV2{ + ScanDetailV2: &kvrpcpb.ScanDetailV2{ ProcessedVersions: 10, TotalVersions: 15, RocksdbBlockReadCount: 20, diff --git a/store/tikv/tests/store_test.go b/store/tikv/tests/store_test.go index f8055a96e4fb9..6a989965c1dc8 100644 --- a/store/tikv/tests/store_test.go +++ b/store/tikv/tests/store_test.go @@ -19,7 +19,7 @@ import ( "time" . "github.com/pingcap/check" - pb "github.com/pingcap/kvproto/pkg/kvrpcpb" + "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/oracle/oracles" @@ -95,15 +95,15 @@ func (s *testStoreSuite) TestOracle(c *C) { type checkRequestClient struct { tikv.Client - priority pb.CommandPri + priority kvrpcpb.CommandPri } func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { resp, err := c.Client.SendRequest(ctx, addr, req, timeout) if c.priority != req.Priority { if resp.Resp != nil { - if getResp, ok := resp.Resp.(*pb.GetResponse); ok { - getResp.Error = &pb.KeyError{ + if getResp, ok := resp.Resp.(*kvrpcpb.GetResponse); ok { + getResp.Error = &kvrpcpb.KeyError{ Abort: "request check error", } } @@ -121,7 +121,7 @@ func (s *testStoreSuite) TestRequestPriority(c *C) { // Cover 2PC commit. txn, err := s.store.Begin() c.Assert(err, IsNil) - client.priority = pb.CommandPri_High + client.priority = kvrpcpb.CommandPri_High txn.SetPriority(tikv.PriorityHigh) err = txn.Set([]byte("key"), []byte("value")) c.Assert(err, IsNil) @@ -131,20 +131,20 @@ func (s *testStoreSuite) TestRequestPriority(c *C) { // Cover the basic Get request. txn, err = s.store.Begin() c.Assert(err, IsNil) - client.priority = pb.CommandPri_Low + client.priority = kvrpcpb.CommandPri_Low txn.SetPriority(tikv.PriorityLow) _, err = txn.Get(context.TODO(), []byte("key")) c.Assert(err, IsNil) // A counter example. - client.priority = pb.CommandPri_Low + client.priority = kvrpcpb.CommandPri_Low txn.SetPriority(tikv.PriorityNormal) _, err = txn.Get(context.TODO(), []byte("key")) // err is translated to "try again later" by backoffer, so doesn't check error value here. c.Assert(err, NotNil) // Cover Seek request. - client.priority = pb.CommandPri_High + client.priority = kvrpcpb.CommandPri_High txn.SetPriority(tikv.PriorityHigh) iter, err := txn.Iter([]byte("key"), nil) c.Assert(err, IsNil) From 6d69d13d10138b8f1eab1e36a669a43c390c714e Mon Sep 17 00:00:00 2001 From: disksing Date: Wed, 9 Jun 2021 14:02:28 +0800 Subject: [PATCH 338/343] store/tikv: cleanup dependency (#25283) --- store/tikv/tikv_test.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/store/tikv/tikv_test.go b/store/tikv/tikv_test.go index 1a4fc28fed7b9..8d25938cf0df2 100644 --- a/store/tikv/tikv_test.go +++ b/store/tikv/tikv_test.go @@ -14,12 +14,10 @@ package tikv import ( - "os" "testing" . "github.com/pingcap/check" "github.com/pingcap/tidb/store/tikv/mockstore" - "github.com/pingcap/tidb/util/logutil" ) type OneByOneSuite = mockstore.OneByOneSuite @@ -29,8 +27,6 @@ type testTiKVSuite struct { func TestT(t *testing.T) { CustomVerboseFlag = true - logLevel := os.Getenv("log_level") - logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false)) TestingT(t) } From cbb6f4e64cb9e46f3db3d36c8c1abbd80ebd5b73 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 9 Jun 2021 18:48:30 +0800 Subject: [PATCH 339/343] *: add global system variable `tmp_table_size` (#24827) --- errors.toml | 5 +++ executor/set_test.go | 4 +-- session/session_test.go | 31 +++++++++++++++++ sessionctx/variable/noop.go | 1 - sessionctx/variable/session.go | 7 ++++ sessionctx/variable/sysvar.go | 10 ++++-- sessionctx/variable/tidb_vars.go | 1 + sessionctx/variable/varsutil_test.go | 1 + table/table.go | 2 ++ table/tables/tables.go | 50 +++++++++++++++++++++++++--- util/tableutil/tableutil.go | 3 ++ 11 files changed, 106 insertions(+), 9 deletions(-) diff --git a/errors.toml b/errors.toml index ce6ce2e58206c..1c6903d1905e5 100644 --- a/errors.toml +++ b/errors.toml @@ -1381,6 +1381,11 @@ error = ''' Unknown column '%-.192s' in '%-.192s' ''' +["table:1114"] +error = ''' +The table '%-.192s' is full +''' + ["table:1192"] error = ''' Can't execute the given command because you have active locked tables or an active transaction diff --git a/executor/set_test.go b/executor/set_test.go index b258cec1ca6a9..c4dcc1b608e46 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -884,9 +884,9 @@ func (s *testSuite5) TestValidateSetVar(c *C) { result = tk.MustQuery("select @@tmp_table_size;") result.Check(testkit.Rows("167772161")) - tk.MustExec("set @@tmp_table_size=18446744073709551615") + tk.MustExec("set @@tmp_table_size=9223372036854775807") result = tk.MustQuery("select @@tmp_table_size;") - result.Check(testkit.Rows("18446744073709551615")) + result.Check(testkit.Rows("9223372036854775807")) _, err = tk.Exec("set @@tmp_table_size=18446744073709551616") c.Assert(terror.ErrorEqual(err, variable.ErrWrongTypeForVar), IsTrue) diff --git a/session/session_test.go b/session/session_test.go index 1328fdbce2c0d..61e90f6f4b73e 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" @@ -4532,6 +4533,36 @@ func (s *testSessionSuite) TestInTxnPSProtoPointGet(c *C) { tk.MustExec("commit") } +func (s *testSessionSuite) TestTMPTableSize(c *C) { + // Test the @@tmp_table_size system variable. + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("set tidb_enable_global_temporary_table=on") + tk.MustExec("create global temporary table t (c1 int, c2 varchar(512)) on commit delete rows") + + tk.MustQuery("select @@global.tmp_table_size").Check(testkit.Rows(strconv.Itoa(variable.DefTMPTableSize))) + c.Assert(tk.Se.GetSessionVars().TMPTableSize, Equals, int64(variable.DefTMPTableSize)) + + // Min value 1024, so the result is change to 1024, with a warning. + tk.MustExec("set @@global.tmp_table_size = 123") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tmp_table_size value: '123'")) + + // Change the session scope value. + tk.MustExec("set @@session.tmp_table_size = 2097152") + c.Assert(tk.Se.GetSessionVars().TMPTableSize, Equals, int64(2097152)) + + // Check in another sessin, change session scope value does not affect the global scope. + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustQuery("select @@global.tmp_table_size").Check(testkit.Rows("1024")) + + // The value is now 1024, check the error when table size exceed it. + tk.MustExec("set @@session.tmp_table_size = 1024") + tk.MustExec("begin") + tk.MustExec("insert into t values (1, repeat('x', 512))") + tk.MustExec("insert into t values (1, repeat('x', 512))") + tk.MustGetErrCode("insert into t values (1, repeat('x', 512))", errno.ErrRecordFileFull) +} + func (s *testSessionSuite) TestTiDBEnableGlobalTemporaryTable(c *C) { // Test the @@tidb_enable_global_temporary_table system variable. tk := testkit.NewTestKit(c, s.store) diff --git a/sessionctx/variable/noop.go b/sessionctx/variable/noop.go index 93b3c1c9ce084..fe3af2d33605d 100644 --- a/sessionctx/variable/noop.go +++ b/sessionctx/variable/noop.go @@ -83,7 +83,6 @@ var noopSysVars = []*SysVar{ {Scope: ScopeNone, Name: "performance_schema_max_statement_classes", Value: "168"}, {Scope: ScopeGlobal, Name: "server_id", Value: "0"}, {Scope: ScopeGlobal, Name: "innodb_flushing_avg_loops", Value: "30"}, - {Scope: ScopeGlobal | ScopeSession, Name: TmpTableSize, Value: "16777216", Type: TypeUnsigned, MinValue: 1024, MaxValue: math.MaxUint64, AutoConvertOutOfRange: true, IsHintUpdatable: true}, {Scope: ScopeGlobal, Name: "innodb_max_purge_lag", Value: "0"}, {Scope: ScopeGlobal | ScopeSession, Name: "preload_buffer_size", Value: "32768"}, {Scope: ScopeGlobal, Name: CheckProxyUsers, Value: Off, Type: TypeBool}, diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index cd076bc50f4b5..772882153a134 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -836,6 +836,12 @@ type SessionVars struct { // see https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_cte_max_recursion_depth CTEMaxRecursionDepth int + // The temporary table size threshold + // In MySQL, when a temporary table exceed this size, it spills to disk. + // In TiDB, as we do not support spill to disk for now, an error is reported. + // See https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_tmp_table_size + TMPTableSize int64 + // EnableGlobalTemporaryTable indicates whether to enable global temporary table EnableGlobalTemporaryTable bool } @@ -1041,6 +1047,7 @@ func NewSessionVars() *SessionVars { EnableIndexMergeJoin: DefTiDBEnableIndexMergeJoin, AllowFallbackToTiKV: make(map[kv.StoreType]struct{}), CTEMaxRecursionDepth: DefCTEMaxRecursionDepth, + TMPTableSize: DefTMPTableSize, EnableGlobalTemporaryTable: DefTiDBEnableGlobalTemporaryTable, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 9a51c17e2c6f6..ff8650317dfa8 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1670,8 +1670,14 @@ var defaultSysVars = []*SysVar{ {Scope: ScopeGlobal, Name: TiDBGCRunInterval, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64}, {Scope: ScopeGlobal, Name: TiDBGCLifetime, Value: "10m0s", Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: math.MaxInt64}, {Scope: ScopeGlobal, Name: TiDBGCConcurrency, Value: "-1", Type: TypeInt, MinValue: 1, MaxValue: 128, AllowAutoValue: true}, + {Scope: ScopeGlobal, Name: TiDBGCScanLockMode, Value: "PHYSICAL", Type: TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}}, {Scope: ScopeGlobal, Name: TiDBGCScanLockMode, Value: "LEGACY", Type: TypeEnum, PossibleValues: []string{"PHYSICAL", "LEGACY"}}, + // See https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_tmp_table_size + {Scope: ScopeGlobal | ScopeSession, Name: TMPTableSize, Value: strconv.Itoa(DefTMPTableSize), Type: TypeUnsigned, MinValue: 1024, MaxValue: math.MaxInt64, AutoConvertOutOfRange: true, IsHintUpdatable: true, AllowEmpty: true, SetSession: func(s *SessionVars, val string) error { + s.TMPTableSize = tidbOptInt64(val, DefTMPTableSize) + return nil + }}, // variable for top SQL feature. {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableTopSQL, Value: BoolToOnOff(DefTiDBTopSQLEnable), Type: TypeBool, AllowEmpty: true, GetSession: func(s *SessionVars) (string, error) { return BoolToOnOff(TopSQLVariable.Enable.Load()), nil @@ -1854,8 +1860,8 @@ const ( MaxConnectErrors = "max_connect_errors" // TableDefinitionCache is the name for 'table_definition_cache' system variable. TableDefinitionCache = "table_definition_cache" - // TmpTableSize is the name for 'tmp_table_size' system variable. - TmpTableSize = "tmp_table_size" + // TMPTableSize is the name for 'tmp_table_size' system variable. + TMPTableSize = "tmp_table_size" // Timestamp is the name for 'timestamp' system variable. Timestamp = "timestamp" // ConnectTimeout is the name for 'connect_timeout' system variable. diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 33400361d10d6..2a5fd6360cacc 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -705,6 +705,7 @@ const ( DefTiDBTopSQLMaxStatementCount = 200 DefTiDBTopSQLReportIntervalSeconds = 60 DefTiDBEnableGlobalTemporaryTable = false + DefTMPTableSize = 16777216 ) // Process global variables. diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 78a5b5c492378..41f379a9d6353 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -104,6 +104,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.ShardAllocateStep, Equals, int64(DefTiDBShardAllocateStep)) c.Assert(vars.AnalyzeVersion, Equals, DefTiDBAnalyzeVersion) c.Assert(vars.CTEMaxRecursionDepth, Equals, DefCTEMaxRecursionDepth) + c.Assert(vars.TMPTableSize, Equals, int64(DefTMPTableSize)) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.MemQuota)) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.BatchSize)) diff --git a/table/table.go b/table/table.go index b39304adcaaa6..f76210a9bdd42 100644 --- a/table/table.go +++ b/table/table.go @@ -97,6 +97,8 @@ var ( ErrSequenceHasRunOut = dbterror.ClassTable.NewStd(mysql.ErrSequenceRunOut) // ErrRowDoesNotMatchGivenPartitionSet returns when the destination partition conflict with the partition selection. ErrRowDoesNotMatchGivenPartitionSet = dbterror.ClassTable.NewStd(mysql.ErrRowDoesNotMatchGivenPartitionSet) + // ErrTempTableFull returns a table is full error, it's used by temporary table now. + ErrTempTableFull = dbterror.ClassTable.NewStd(mysql.ErrRecordFileFull) ) // RecordIterFunc is used for low-level record iteration. diff --git a/table/tables/tables.go b/table/tables/tables.go index 003e2aa4081a3..c78d1909cf135 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -324,7 +324,12 @@ func (t *TableCommon) UpdateRecord(ctx context.Context, sctx sessionctx.Context, defer memBuffer.Cleanup(sh) if m := t.Meta(); m.TempTableType == model.TempTableGlobal { - addTemporaryTable(sctx, m) + if tmpTable := addTemporaryTable(sctx, m); tmpTable != nil { + if tmpTable.GetSize() > sctx.GetSessionVars().TMPTableSize { + return table.ErrTempTableFull.GenWithStackByArgs(m.Name.O) + } + defer handleTempTableSize(tmpTable, txn.Size(), txn) + } } var colIDs, binlogColIDs []int64 @@ -589,9 +594,20 @@ func TryGetCommonPkColumns(tbl table.Table) []*table.Column { return pkCols } -func addTemporaryTable(sctx sessionctx.Context, tblInfo *model.TableInfo) { +func addTemporaryTable(sctx sessionctx.Context, tblInfo *model.TableInfo) tableutil.TempTable { tempTable := sctx.GetSessionVars().GetTemporaryTable(tblInfo) tempTable.SetModified(true) + return tempTable +} + +// The size of a temporary table is calculated by accumulating the transaction size delta. +func handleTempTableSize(t tableutil.TempTable, txnSizeBefore int, txn kv.Transaction) { + txnSizeNow := txn.Size() + delta := txnSizeNow - txnSizeBefore + + oldSize := t.GetSize() + newSize := oldSize + int64(delta) + t.SetSize(newSize) } // AddRecord implements table.Table AddRecord interface. @@ -607,7 +623,12 @@ func (t *TableCommon) AddRecord(sctx sessionctx.Context, r []types.Datum, opts . } if m := t.Meta(); m.TempTableType == model.TempTableGlobal { - addTemporaryTable(sctx, m) + if tmpTable := addTemporaryTable(sctx, m); tmpTable != nil { + if tmpTable.GetSize() > sctx.GetSessionVars().TMPTableSize { + return nil, table.ErrTempTableFull.GenWithStackByArgs(m.Name.O) + } + defer handleTempTableSize(tmpTable, txn.Size(), txn) + } } var ctx context.Context @@ -1008,8 +1029,17 @@ func (t *TableCommon) RemoveRecord(ctx sessionctx.Context, h kv.Handle, r []type return err } + txn, err := ctx.Txn(true) + if err != nil { + return err + } if m := t.Meta(); m.TempTableType == model.TempTableGlobal { - addTemporaryTable(ctx, m) + if tmpTable := addTemporaryTable(ctx, m); tmpTable != nil { + if tmpTable.GetSize() > ctx.GetSessionVars().TMPTableSize { + return table.ErrTempTableFull.GenWithStackByArgs(m.Name.O) + } + defer handleTempTableSize(tmpTable, txn.Size(), txn) + } } // The table has non-public column and this column is doing the operation of "modify/change column". @@ -1785,6 +1815,8 @@ type TemporaryTable struct { stats *statistics.Table // The autoID allocator of this table. autoIDAllocator autoid.Allocator + // Table size. + size int64 } // TempTableFromMeta builds a TempTable from model.TableInfo. @@ -1815,3 +1847,13 @@ func (t *TemporaryTable) GetModified() bool { func (t *TemporaryTable) GetStats() interface{} { return t.stats } + +// GetSize gets the table size. +func (t *TemporaryTable) GetSize() int64 { + return t.size +} + +// SetSize sets the table size. +func (t *TemporaryTable) SetSize(v int64) { + t.size = v +} diff --git a/util/tableutil/tableutil.go b/util/tableutil/tableutil.go index 11cbe626dcc56..bf5d7caac2732 100644 --- a/util/tableutil/tableutil.go +++ b/util/tableutil/tableutil.go @@ -33,6 +33,9 @@ type TempTable interface { // The stats of this table (*statistics.Table). // Define the return type as interface{} here to avoid cycle imports. GetStats() interface{} + + GetSize() int64 + SetSize(int64) } // TempTableFromMeta builds a TempTable from *model.TableInfo. From 9900f889cb89571a78b26ddf0f82b801b54518a1 Mon Sep 17 00:00:00 2001 From: Alkaid Jiang <38248129+jyz0309@users.noreply.github.com> Date: Wed, 9 Jun 2021 05:56:29 -0500 Subject: [PATCH 340/343] Metric: Collect TiKV Read Duration Metric for SLI/SLO (#23884) --- distsql/select_result.go | 8 ++++++++ metrics/metrics.go | 1 + metrics/sli.go | 10 ++++++++++ store/copr/coprocessor.go | 1 + store/tikv/snapshot.go | 7 +++++++ store/tikv/util/execdetails.go | 3 +++ util/sli/sli.go | 11 +++++++++-- 7 files changed, 39 insertions(+), 2 deletions(-) diff --git a/distsql/select_result.go b/distsql/select_result.go index d003343027dea..ef1a99e193e57 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -41,6 +41,7 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/sli" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) @@ -341,6 +342,13 @@ func (r *selectResult) updateCopRuntimeStats(ctx context.Context, copStats *copr if r.rootPlanID <= 0 || r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl == nil || callee == "" { return } + + if copStats.ScanDetail != nil { + readKeys := copStats.ScanDetail.ProcessedKeys + readTime := copStats.TimeDetail.KvReadWallTimeMs.Seconds() + sli.ObserveReadSLI(uint64(readKeys), readTime) + } + if r.stats == nil { id := r.rootPlanID r.stats = &selectResultRuntimeStats{ diff --git a/metrics/metrics.go b/metrics/metrics.go index 542398e7bbdee..8cbca511c078e 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -151,6 +151,7 @@ func RegisterMetrics() { prometheus.MustRegister(TiFlashQueryTotalCounter) prometheus.MustRegister(SmallTxnWriteDuration) prometheus.MustRegister(TxnWriteThroughput) + prometheus.MustRegister(TiKVSmallReadDuration) prometheus.MustRegister(LoadSysVarCacheCounter) tikvmetrics.InitMetrics(TiDB, TiKVClient) diff --git a/metrics/sli.go b/metrics/sli.go index 2e926de099997..c6d0239550851 100644 --- a/metrics/sli.go +++ b/metrics/sli.go @@ -37,4 +37,14 @@ var ( Help: "Bucketed histogram of transaction write throughput (bytes/second).", Buckets: prometheus.ExponentialBuckets(64, 1.3, 40), // 64 bytes/s ~ 2.3MB/s }) + + // TiKVSmallReadDuration uses to collect small request read duration. + TiKVSmallReadDuration = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "sli", + Name: "tikv_small_read_duration", + Help: "Read time of TiKV small read.", + Buckets: prometheus.ExponentialBuckets(0.0005, 2, 28), // 0.5ms ~ 74h + }) ) diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index d8ebf02d48136..49e6d1d281ced 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -785,6 +785,7 @@ func (worker *copIteratorWorker) logTimeCopTask(costTime time.Duration, task *co if timeDetail != nil { logStr += fmt.Sprintf(" kv_process_ms:%d", timeDetail.ProcessWallTimeMs) logStr += fmt.Sprintf(" kv_wait_ms:%d", timeDetail.WaitWallTimeMs) + logStr += fmt.Sprintf(" kv_read_ms:%d", timeDetail.KvReadWallTimeMs) if timeDetail.ProcessWallTimeMs <= minLogKVProcessTime { logStr = strings.Replace(logStr, "TIME_COP_PROCESS", "TIME_COP_WAIT", 1) } diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index c49310477e62c..f08b6f906f537 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -37,6 +37,7 @@ import ( "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" + "github.com/pingcap/tidb/util/sli" "go.uber.org/zap" ) @@ -402,6 +403,9 @@ func (s *KVSnapshot) batchGetSingleRegion(bo *Backoffer, batch batchKeys, collec } } if batchGetResp.ExecDetailsV2 != nil { + readKeys := len(batchGetResp.Pairs) + readTime := float64(batchGetResp.ExecDetailsV2.GetTimeDetail().GetKvReadWallTimeMs() / 1000) + sli.ObserveReadSLI(uint64(readKeys), readTime) s.mergeExecDetail(batchGetResp.ExecDetailsV2) } if len(lockedKeys) > 0 { @@ -537,6 +541,9 @@ func (s *KVSnapshot) get(ctx context.Context, bo *Backoffer, k []byte) ([]byte, } cmdGetResp := resp.Resp.(*kvrpcpb.GetResponse) if cmdGetResp.ExecDetailsV2 != nil { + readKeys := len(cmdGetResp.Value) + readTime := float64(cmdGetResp.ExecDetailsV2.GetTimeDetail().GetKvReadWallTimeMs() / 1000) + sli.ObserveReadSLI(uint64(readKeys), readTime) s.mergeExecDetail(cmdGetResp.ExecDetailsV2) } val := cmdGetResp.GetValue() diff --git a/store/tikv/util/execdetails.go b/store/tikv/util/execdetails.go index f8a8364fbb645..0f2dab372ff16 100644 --- a/store/tikv/util/execdetails.go +++ b/store/tikv/util/execdetails.go @@ -270,6 +270,8 @@ type TimeDetail struct { // cannot be excluded for now, like Mutex wait time, which is included in this field, so that // this field is called wall time instead of CPU time. WaitTime time.Duration + // KvReadWallTimeMs is the time used in KV Scan/Get. + KvReadWallTimeMs time.Duration } // String implements the fmt.Stringer interface. @@ -297,5 +299,6 @@ func (td *TimeDetail) MergeFromTimeDetail(timeDetail *kvrpcpb.TimeDetail) { if timeDetail != nil { td.WaitTime += time.Duration(timeDetail.WaitWallTimeMs) * time.Millisecond td.ProcessTime += time.Duration(timeDetail.ProcessWallTimeMs) * time.Millisecond + td.KvReadWallTimeMs += time.Duration(timeDetail.KvReadWallTimeMs) * time.Millisecond } } diff --git a/util/sli/sli.go b/util/sli/sli.go index 1c48f9a03a510..80d8cdc6f7868 100644 --- a/util/sli/sli.go +++ b/util/sli/sli.go @@ -94,12 +94,12 @@ func (t *TxnWriteThroughputSLI) IsInvalid() bool { const ( smallTxnAffectRow = 20 - smallTxnWriteSize = 1 * 1024 * 1024 // 1MB + smallTxnSize = 1 * 1024 * 1024 // 1MB ) // IsSmallTxn exports for testing. func (t *TxnWriteThroughputSLI) IsSmallTxn() bool { - return t.affectRow <= smallTxnAffectRow && t.writeSize <= smallTxnWriteSize + return t.affectRow <= smallTxnAffectRow && t.writeSize <= smallTxnSize } // Reset exports for testing. @@ -117,3 +117,10 @@ func (t *TxnWriteThroughputSLI) String() string { return fmt.Sprintf("invalid: %v, affectRow: %v, writeSize: %v, readKeys: %v, writeKeys: %v, writeTime: %v", t.invalid, t.affectRow, t.writeSize, t.readKeys, t.writeKeys, t.writeTime.String()) } + +// ObserveReadSLI observes the read SLI metric. +func ObserveReadSLI(readKeys uint64, readTime float64) { + if readKeys <= smallTxnAffectRow && readKeys != 0 && readTime != 0 { + metrics.TiKVSmallReadDuration.Observe(readTime) + } +} From 3aa13dd20fa29e4a3934c04445329fae30a11807 Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Wed, 9 Jun 2021 20:52:30 +0800 Subject: [PATCH 341/343] expresssion: fix unstable time_vec_test (#25238) --- expression/bench_test.go | 7 +++++++ expression/builtin_time_vec_test.go | 3 ++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/expression/bench_test.go b/expression/bench_test.go index 3a2b6559c29cb..51d001a8b07d8 100644 --- a/expression/bench_test.go +++ b/expression/bench_test.go @@ -263,6 +263,13 @@ func (g *defaultGener) gen() interface{} { case types.ETDatetime, types.ETTimestamp: gt := getRandomTime(g.randGen.Rand) t := types.NewTime(gt, convertETType(g.eType), 0) + // TiDB has DST time problem, and it causes ErrWrongValue. + // We should ignore ambiguous Time. See https://timezonedb.com/time-zones/Asia/Shanghai. + for _, err := t.GoTime(time.Local); err != nil; { + gt = getRandomTime(g.randGen.Rand) + t = types.NewTime(gt, convertETType(g.eType), 0) + _, err = t.GoTime(time.Local) + } return t case types.ETDuration: d := types.Duration{ diff --git a/expression/builtin_time_vec_test.go b/expression/builtin_time_vec_test.go index a757b867b783c..389c257460e1b 100644 --- a/expression/builtin_time_vec_test.go +++ b/expression/builtin_time_vec_test.go @@ -516,7 +516,8 @@ var vecBuiltinTimeCases = map[string][]vecExprBenchCase{ { retEvalType: types.ETDatetime, childrenTypes: []types.EvalType{types.ETInt}, - geners: []dataGenerator{newRangeInt64Gener(0, math.MaxInt64)}, + // TiDB has DST time problem. Change the random ranges to [2000-01-01 00:00:01, +inf] + geners: []dataGenerator{newRangeInt64Gener(248160190726144000, math.MaxInt64)}, }, }, // Todo: how to inject the safeTS for better testing. From e29496c649a72d8d2d7b6cd2bc61245bf0030a71 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Wed, 9 Jun 2021 07:06:29 -0600 Subject: [PATCH 342/343] server: fix test for clienterrors (#25278) --- server/server_test.go | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/server/server_test.go b/server/server_test.go index 4caa4f69563f7..461b59a070bb1 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -2057,6 +2057,7 @@ func (cli *testServerClient) runTestInfoschemaClientErrors(t *C) { if rows.Next() { rows.Scan(&errors, &warnings) } + rows.Close() if test.incrementErrors { errors++ @@ -2064,14 +2065,22 @@ func (cli *testServerClient) runTestInfoschemaClientErrors(t *C) { if test.incrementWarnings { warnings++ } - - dbt.db.Query(test.stmt) // ignore results and errors (query table) + var err error + rows, err = dbt.db.Query(test.stmt) + if err == nil { + // make sure to read the result since the error/warnings are populated in the network send code. + if rows.Next() { + var fake string + rows.Scan(&fake) + } + rows.Close() + } var newErrors, newWarnings int rows = dbt.mustQuery("SELECT SUM(error_count), SUM(warning_count) FROM information_schema."+tbl+" WHERE error_number = ? GROUP BY error_number", test.errCode) if rows.Next() { rows.Scan(&newErrors, &newWarnings) } - + rows.Close() dbt.Check(newErrors, Equals, errors) dbt.Check(newWarnings, Equals, warnings) } From 0ed8eafa9bf269eb2c3974ce67184aa05fbabc38 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Wed, 9 Jun 2021 21:14:29 +0800 Subject: [PATCH 343/343] executor: add tablesample compatibility for temporary table (#25288) --- executor/builder.go | 4 ++-- executor/executor_test.go | 7 +++---- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 64204b4b61211..70f76120d5dcf 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -4151,8 +4151,8 @@ func fullRangePartition(idxArr []int) bool { } func (b *executorBuilder) buildTableSample(v *plannercore.PhysicalTableSample) *TableSampleExecutor { - if err := b.validCanReadTemporaryTable(v.TableInfo.Meta()); err != nil { - b.err = err + if v.TableInfo.Meta().TempTableType != model.TempTableNone { + b.err = errors.New("TABLESAMPLE clause can not be applied to temporary tables") return nil } diff --git a/executor/executor_test.go b/executor/executor_test.go index 38dea78d1ac5b..5cdd7516b8005 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -8521,7 +8521,7 @@ func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { } } -func (s *testStaleTxnSuite) TestStaleOrHistoryReadTemporaryTable(c *C) { +func (s *testStaleTxnSuite) TestInvalidReadTemporaryTable(c *C) { tk := testkit.NewTestKit(c, s.store) // For mocktikv, safe point is not initialized, we manually insert it for snapshot to use. safePointName := "tikv_gc_safe_point" @@ -8542,6 +8542,8 @@ func (s *testStaleTxnSuite) TestStaleOrHistoryReadTemporaryTable(c *C) { // sleep 1us to make test stale time.Sleep(time.Microsecond) + tk.MustGetErrMsg("select * from tmp1 tablesample regions()", "TABLESAMPLE clause can not be applied to temporary tables") + queries := []struct { sql string }{ @@ -8566,9 +8568,6 @@ func (s *testStaleTxnSuite) TestStaleOrHistoryReadTemporaryTable(c *C) { { sql: "select /*+use_index(tmp1, code)*/ code from tmp1 where code > 1", }, - { - sql: "select * from tmp1 tablesample regions()", - }, { sql: "select /*+ use_index_merge(tmp1, primary, code) */ * from tmp1 where id > 1 or code > 2", },