From 05a4c65937a3d1a7ccd43b4779f4b8c546b7cb43 Mon Sep 17 00:00:00 2001 From: glorv Date: Fri, 21 Jan 2022 15:03:46 +0800 Subject: [PATCH 01/21] lightning: support escaped dump file name (#31513) close pingcap/tidb#31512 --- br/pkg/lightning/config/config.go | 7 +- br/pkg/lightning/mydump/loader_test.go | 109 +++++++++++++++++++++++++ br/pkg/lightning/mydump/router.go | 31 +++++-- 3 files changed, 138 insertions(+), 9 deletions(-) diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index 1819e6528c060..15c8d4dc80ffc 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -509,8 +509,10 @@ type FileRouteRule struct { Type string `json:"type" toml:"type" yaml:"type"` Key string `json:"key" toml:"key" yaml:"key"` Compression string `json:"compression" toml:"compression" yaml:"compression"` - // TODO: DataCharacterSet here can overide the same field in [mydumper.csv] with a higher level. - // This could provide users a more flexable usage to configure different files with + // unescape the schema/table name only used in lightning's internal logic now. + Unescape bool `json:"-" toml:"-" yaml:"-"` + // TODO: DataCharacterSet here can override the same field in [mydumper.csv] with a higher level. + // This could provide users a more flexible usage to configure different files with // different data charsets. // DataCharacterSet string `toml:"data-character-set" json:"data-character-set"` } @@ -1021,6 +1023,7 @@ func (cfg *Config) CheckAndAdjustTiDBPort(ctx context.Context, mustHaveInternalC if cfg.TiDB.Port <= 0 { return errors.New("invalid `tidb.port` setting") } + if mustHaveInternalConnections && len(cfg.TiDB.PdAddr) == 0 { return errors.New("invalid `tidb.pd-addr` setting") } diff --git a/br/pkg/lightning/mydump/loader_test.go b/br/pkg/lightning/mydump/loader_test.go index 76bc50eba2793..9217b01190001 100644 --- a/br/pkg/lightning/mydump/loader_test.go +++ b/br/pkg/lightning/mydump/loader_test.go @@ -622,3 +622,112 @@ func (s *testMydumpLoaderSuite) TestFileRouting(c *C) { }, }) } + +func (s *testMydumpLoaderSuite) TestInputWithSpecialChars(c *C) { + /* + Path/ + test-schema-create.sql + test.t%22-schema.sql + test.t%22.0.sql + test.t%2522-schema.sql + test.t%2522.0.csv + test.t%gg-schema.sql + test.t%gg.csv + test.t+gg-schema.sql + test.t+gg.csv + + db%22.t%2522-schema.sql + db%22.t%2522.0.csv + */ + + s.touch(c, "test-schema-create.sql") + s.touch(c, "test.t%22-schema.sql") + s.touch(c, "test.t%22.sql") + s.touch(c, "test.t%2522-schema.sql") + s.touch(c, "test.t%2522.csv") + s.touch(c, "test.t%gg-schema.sql") + s.touch(c, "test.t%gg.csv") + s.touch(c, "test.t+gg-schema.sql") + s.touch(c, "test.t+gg.csv") + + s.touch(c, "db%22-schema-create.sql") + s.touch(c, "db%22.t%2522-schema.sql") + s.touch(c, "db%22.t%2522.0.csv") + + mdl, err := md.NewMyDumpLoader(context.Background(), s.cfg) + c.Assert(err, IsNil) + c.Assert(mdl.GetDatabases(), DeepEquals, []*md.MDDatabaseMeta{ + { + Name: `db"`, + SchemaFile: filepath.FromSlash("db%22-schema-create.sql"), + Tables: []*md.MDTableMeta{ + { + DB: `db"`, + Name: "t%22", + SchemaFile: md.FileInfo{ + TableName: filter.Table{Schema: `db"`, Name: "t%22"}, + FileMeta: md.SourceFileMeta{Path: filepath.FromSlash("db%22.t%2522-schema.sql"), Type: md.SourceTypeTableSchema}, + }, + DataFiles: []md.FileInfo{ + { + TableName: filter.Table{Schema: `db"`, Name: "t%22"}, + FileMeta: md.SourceFileMeta{Path: filepath.FromSlash("db%22.t%2522.0.csv"), Type: md.SourceTypeCSV, SortKey: "0"}, + }, + }, + IndexRatio: 0, + IsRowOrdered: true, + }, + }, + }, + { + Name: "test", + SchemaFile: filepath.FromSlash("test-schema-create.sql"), + Tables: []*md.MDTableMeta{ + { + DB: "test", + Name: `t"`, + SchemaFile: md.FileInfo{ + TableName: filter.Table{Schema: "test", Name: `t"`}, + FileMeta: md.SourceFileMeta{Path: filepath.FromSlash("test.t%22-schema.sql"), Type: md.SourceTypeTableSchema}, + }, + DataFiles: []md.FileInfo{{TableName: filter.Table{Schema: "test", Name: `t"`}, FileMeta: md.SourceFileMeta{Path: "test.t%22.sql", Type: md.SourceTypeSQL}}}, + IndexRatio: 0, + IsRowOrdered: true, + }, + { + DB: "test", + Name: "t%22", + SchemaFile: md.FileInfo{ + TableName: filter.Table{Schema: "test", Name: "t%22"}, + FileMeta: md.SourceFileMeta{Path: filepath.FromSlash("test.t%2522-schema.sql"), Type: md.SourceTypeTableSchema}, + }, + DataFiles: []md.FileInfo{{TableName: filter.Table{Schema: "test", Name: "t%22"}, FileMeta: md.SourceFileMeta{Path: "test.t%2522.csv", Type: md.SourceTypeCSV}}}, + IndexRatio: 0, + IsRowOrdered: true, + }, + { + DB: "test", + Name: "t%gg", + SchemaFile: md.FileInfo{ + TableName: filter.Table{Schema: "test", Name: "t%gg"}, + FileMeta: md.SourceFileMeta{Path: filepath.FromSlash("test.t%gg-schema.sql"), Type: md.SourceTypeTableSchema}, + }, + DataFiles: []md.FileInfo{{TableName: filter.Table{Schema: "test", Name: "t%gg"}, FileMeta: md.SourceFileMeta{Path: "test.t%gg.csv", Type: md.SourceTypeCSV}}}, + IndexRatio: 0, + IsRowOrdered: true, + }, + { + DB: "test", + Name: "t+gg", + SchemaFile: md.FileInfo{ + TableName: filter.Table{Schema: "test", Name: "t+gg"}, + FileMeta: md.SourceFileMeta{Path: filepath.FromSlash("test.t+gg-schema.sql"), Type: md.SourceTypeTableSchema}, + }, + DataFiles: []md.FileInfo{{TableName: filter.Table{Schema: "test", Name: "t+gg"}, FileMeta: md.SourceFileMeta{Path: "test.t+gg.csv", Type: md.SourceTypeCSV}}}, + IndexRatio: 0, + IsRowOrdered: true, + }, + }, + }, + }) +} diff --git a/br/pkg/lightning/mydump/router.go b/br/pkg/lightning/mydump/router.go index dc9eba5b8454c..8c16039e63fbf 100644 --- a/br/pkg/lightning/mydump/router.go +++ b/br/pkg/lightning/mydump/router.go @@ -1,14 +1,18 @@ package mydump import ( + "net/url" "regexp" "strconv" "strings" "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/filter" - "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/util/slice" + "go.uber.org/zap" + + "github.com/pingcap/tidb/br/pkg/lightning/config" + "github.com/pingcap/tidb/br/pkg/lightning/log" ) type SourceType int @@ -106,13 +110,13 @@ var defaultFileRouteRules = []*config.FileRouteRule{ // ignore *-schema-trigger.sql, *-schema-post.sql files {Pattern: `(?i).*(-schema-trigger|-schema-post)\.sql$`, Type: "ignore"}, // db schema create file pattern, matches files like '{schema}-schema-create.sql' - {Pattern: `(?i)^(?:[^/]*/)*([^/.]+)-schema-create\.sql$`, Schema: "$1", Table: "", Type: SchemaSchema}, + {Pattern: `(?i)^(?:[^/]*/)*([^/.]+)-schema-create\.sql$`, Schema: "$1", Table: "", Type: SchemaSchema, Unescape: true}, // table schema create file pattern, matches files like '{schema}.{table}-schema.sql' - {Pattern: `(?i)^(?:[^/]*/)*([^/.]+)\.(.*?)-schema\.sql$`, Schema: "$1", Table: "$2", Type: TableSchema}, + {Pattern: `(?i)^(?:[^/]*/)*([^/.]+)\.(.*?)-schema\.sql$`, Schema: "$1", Table: "$2", Type: TableSchema, Unescape: true}, // view schema create file pattern, matches files like '{schema}.{table}-schema-view.sql' - {Pattern: `(?i)^(?:[^/]*/)*([^/.]+)\.(.*?)-schema-view\.sql$`, Schema: "$1", Table: "$2", Type: ViewSchema}, + {Pattern: `(?i)^(?:[^/]*/)*([^/.]+)\.(.*?)-schema-view\.sql$`, Schema: "$1", Table: "$2", Type: ViewSchema, Unescape: true}, // source file pattern, matches files like '{schema}.{table}.0001.{sql|csv}' - {Pattern: `(?i)^(?:[^/]*/)*([^/.]+)\.(.*?)(?:\.([0-9]+))?\.(sql|csv|parquet)$`, Schema: "$1", Table: "$2", Type: "$4", Key: "$3"}, + {Pattern: `(?i)^(?:[^/]*/)*([^/.]+)\.(.*?)(?:\.([0-9]+))?\.(sql|csv|parquet)$`, Schema: "$1", Table: "$2", Type: "$4", Key: "$3", Unescape: true}, } // // RouteRule is a rule to route file path to target schema/table @@ -217,8 +221,21 @@ func (p regexRouterParser) Parse(r *config.FileRouteRule) (*RegexRouter, error) return rule, nil } + setValue := func(target *string, value string, unescape bool) { + if unescape { + val, err := url.PathUnescape(value) + if err != nil { + log.L().Warn("unescape string failed, will be ignored", zap.String("value", value), + zap.Error(err)) + } else { + value = val + } + } + *target = value + } + err = p.parseFieldExtractor(rule, "schema", r.Schema, func(result *RouteResult, value string) error { - result.Schema = value + setValue(&result.Schema, value, r.Unescape) return nil }) if err != nil { @@ -228,7 +245,7 @@ func (p regexRouterParser) Parse(r *config.FileRouteRule) (*RegexRouter, error) // special case: when the pattern is for db schema, should not parse table name if r.Type != SchemaSchema { err = p.parseFieldExtractor(rule, "table", r.Table, func(result *RouteResult, value string) error { - result.Name = value + setValue(&result.Name, value, r.Unescape) return nil }) if err != nil { From e88f01a477310ca31cf556dcd464f2ad6d0b7b0d Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Fri, 21 Jan 2022 16:11:46 +0800 Subject: [PATCH 02/21] session: tiny fix for several session test cases (#31095) close pingcap/tidb#31094 --- session/pessimistic_test.go | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index aa7e58eb6c616..9cd82abe32151 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -205,6 +205,8 @@ func (s *testPessimisticSuite) TestDeadlock(c *C) { syncCh <- nil _, err := tk2.Exec("update deadlock set v = v + 1 where k = 1") syncCh <- err + + tk2.MustExec("rollback") }() <-syncCh _, err1 := tk1.Exec("update deadlock set v = v + 1 where k = 2") @@ -601,10 +603,17 @@ func (s *testPessimisticSuite) TestAsyncRollBackNoWait(c *C) { tk.MustExec("commit") // This statement success for now, but its lock will be rollbacked later by the // lingering rollback request, as forUpdateTS doesn't change. - tk2.MustQuery("select * from tk where c1 > 0 for update nowait") + tk2.MustQuery("select * from tk where c1 > 0 for update nowait").Check(testkit.Rows("1 1", "2 2", "3 3", "4 4", "5 17")) tk2.MustQuery("select * from tk where c1 = 5 for update nowait").Check(testkit.Rows("5 17")) tk3.MustExec("begin pessimistic") + // TODO: @coocood skip the following test in https://github.com/pingcap/tidb/pull/13553/ + // Remove this code block and figure out why it's skipped. + // ---------------------- + tk2.MustExec("rollback") + tk3.MustExec("rollback") + // ---------------------- + c.Skip("tk3 is blocking because tk2 didn't rollback itself") // tk3 succ because tk2 rollback itself. tk3.MustExec("update tk set c2 = 1 where c1 = 5") @@ -664,6 +673,8 @@ func (s *testPessimisticSuite) TestKillStopTTLManager(c *C) { // This query should success rather than returning a ResolveLock error. tk2.MustExec("update test_kill set c = c + 1 where id = 1") + + tk2.MustExec("rollback") } func (s *testPessimisticSuite) TestConcurrentInsert(c *C) { From 4bda04f15cdd971ccd9e512f62339e36fe2bd0d9 Mon Sep 17 00:00:00 2001 From: dongjunduo Date: Fri, 21 Jan 2022 02:35:46 -0600 Subject: [PATCH 03/21] planner: introduce historical statistic dumping triggered by analyze (#31032) close pingcap/tidb#18745 --- executor/analyze.go | 34 +++++++++++++++++ executor/analyze_test.go | 58 +++++++++++++++++++++++++++++ executor/infoschema_reader_test.go | 6 +-- session/bootstrap.go | 24 +++++++++++- session/bootstrap_test.go | 34 +++++++++++++++++ statistics/handle/dump.go | 59 ++++++++++++++++++++++++++++++ statistics/handle/dump_test.go | 35 ++++++++++++++++++ statistics/handle/handle.go | 51 ++++++++++++++++++++++++++ statistics/handle/handle_test.go | 27 ++++++++++++++ 9 files changed, 324 insertions(+), 4 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index 05f1ccd241b83..134b4ba303c04 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -178,6 +178,10 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { finishJobWithLogFn(ctx, results.Job, true) } else { finishJobWithLogFn(ctx, results.Job, false) + // Dump stats to historical storage. + if err := e.recordHistoricalStats(results.TableID.TableID); err != nil { + logutil.BgLogger().Error("record historical stats failed", zap.Error(err)) + } } } for _, task := range e.tasks { @@ -210,6 +214,10 @@ func (e *AnalyzeExec) Next(ctx context.Context, req *chunk.Chunk) error { if err != nil { logutil.Logger(ctx).Error("save global-level stats to storage failed", zap.Error(err)) } + // Dump stats to historical storage. + if err := e.recordHistoricalStats(globalStatsID.tableID); err != nil { + logutil.BgLogger().Error("record historical stats failed", zap.Error(err)) + } } } } @@ -262,6 +270,32 @@ func (e *AnalyzeExec) saveAnalyzeOptsV2() error { return nil } +func (e *AnalyzeExec) recordHistoricalStats(tableID int64) error { + statsHandle := domain.GetDomain(e.ctx).StatsHandle() + historicalStatsEnabled, err := statsHandle.CheckHistoricalStatsEnable() + if err != nil { + return errors.Errorf("check tidb_enable_historical_stats failed: %v", err) + } + if !historicalStatsEnabled { + return nil + } + + is := domain.GetDomain(e.ctx).InfoSchema() + tbl, existed := is.TableByID(tableID) + if !existed { + return errors.Errorf("cannot get table by id %d", tableID) + } + tblInfo := tbl.Meta() + dbInfo, existed := is.SchemaByTable(tblInfo) + if !existed { + return errors.Errorf("cannot get DBInfo by TableID %d", tableID) + } + if _, err := statsHandle.RecordHistoricalStatsToStorage(dbInfo.Name.O, tblInfo); err != nil { + return errors.Errorf("record table %s.%s's historical stats failed", dbInfo.Name.O, tblInfo.Name.O) + } + return nil +} + func getBuildStatsConcurrency(ctx sessionctx.Context) (int, error) { sessionVars := ctx.GetSessionVars() concurrency, err := variable.GetSessionOrGlobalSystemVar(sessionVars, variable.TiDBBuildStatsConcurrency) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 30e89b5b341cc..5355a3d54fc9d 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -16,6 +16,7 @@ package executor_test import ( "context" + "encoding/json" "fmt" "strconv" "strings" @@ -2568,3 +2569,60 @@ func TestAnalyzeColumnsErrorAndWarning(t *testing.T) { }(val) } } + +func TestRecordHistoryStatsAfterAnalyze(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_analyze_version = 2") + tk.MustExec("set global tidb_enable_historical_stats = 0") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10))") + + h := dom.StatsHandle() + is := dom.InfoSchema() + tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + + // 1. switch off the tidb_enable_historical_stats, and there is no records in table `mysql.stats_history` + rows := tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", tableInfo.Meta().ID)).Rows() + num, _ := strconv.Atoi(rows[0][0].(string)) + require.Equal(t, num, 0) + + tk.MustExec("analyze table t with 2 topn") + rows = tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", tableInfo.Meta().ID)).Rows() + num, _ = strconv.Atoi(rows[0][0].(string)) + require.Equal(t, num, 0) + + // 2. switch on the tidb_enable_historical_stats and do analyze + tk.MustExec("set global tidb_enable_historical_stats = 1") + defer tk.MustExec("set global tidb_enable_historical_stats = 0") + tk.MustExec("analyze table t with 2 topn") + rows = tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", tableInfo.Meta().ID)).Rows() + num, _ = strconv.Atoi(rows[0][0].(string)) + require.GreaterOrEqual(t, num, 1) + + // 3. dump current stats json + dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil) + require.NoError(t, err) + jsOrigin, _ := json.Marshal(dumpJSONTable) + + // 4. get the historical stats json + rows = tk.MustQuery(fmt.Sprintf("select * from mysql.stats_history where table_id = '%d' and create_time = ("+ + "select create_time from mysql.stats_history where table_id = '%d' order by create_time desc limit 1) "+ + "order by seq_no", tableInfo.Meta().ID, tableInfo.Meta().ID)).Rows() + num = len(rows) + require.GreaterOrEqual(t, num, 1) + data := make([][]byte, num) + for i, row := range rows { + data[i] = []byte(row[1].(string)) + } + jsonTbl, err := handle.BlocksToJSONTable(data) + require.NoError(t, err) + jsCur, err := json.Marshal(jsonTbl) + require.NoError(t, err) + // 5. historical stats must be equal to the current stats + require.JSONEq(t, string(jsOrigin), string(jsCur)) +} diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index e572dba51dc62..1febaaaa8d90d 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -918,7 +918,7 @@ func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { tk.MustQuery("select TABLE_SCHEMA, sum(TABLE_SIZE) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'test' group by TABLE_SCHEMA;").Check(testkit.Rows( "test 2", )) - c.Assert(len(tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Rows()), Equals, 28) + c.Assert(len(tk.MustQuery("select TABLE_NAME from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql';").Rows()), Equals, 29) // More tests about the privileges. tk.MustExec("create user 'testuser'@'localhost'") @@ -944,14 +944,14 @@ func (s *testInfoschemaClusterTableSuite) TestTableStorageStats(c *C) { Hostname: "localhost", }, nil, nil), Equals, true) - tk.MustQuery("select count(1) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql'").Check(testkit.Rows("28")) + tk.MustQuery("select count(1) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql'").Check(testkit.Rows("29")) c.Assert(tk.Se.Auth(&auth.UserIdentity{ Username: "testuser3", Hostname: "localhost", }, nil, nil), Equals, true) - tk.MustQuery("select count(1) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql'").Check(testkit.Rows("28")) + tk.MustQuery("select count(1) from information_schema.TABLE_STORAGE_STATS where TABLE_SCHEMA = 'mysql'").Check(testkit.Rows("29")) } func (s *testInfoschemaTableSuite) TestSequences(c *C) { diff --git a/session/bootstrap.go b/session/bootstrap.go index 094997a6243b6..f408b8ca0a90e 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -376,6 +376,16 @@ const ( column_ids TEXT(19372), PRIMARY KEY (table_id) CLUSTERED );` + // CreateStatsHistory stores the historical stats. + CreateStatsHistory = `CREATE TABLE IF NOT EXISTS mysql.stats_history ( + table_id bigint(64) NOT NULL, + stats_data longblob NOT NULL, + seq_no bigint(64) NOT NULL comment 'sequence number of the gzipped data slice', + version bigint(64) NOT NULL comment 'stats version which corresponding to stats:version in EXPLAIN', + create_time datetime(6) NOT NULL, + UNIQUE KEY table_version_seq (table_id, version, seq_no), + KEY table_create_time (table_id, create_time, seq_no) + );` ) // bootstrap initiates system DB for a store. @@ -557,11 +567,13 @@ const ( version81 = 81 // version82 adds the mysql.analyze_options table version82 = 82 + // version83 adds the tables mysql.stats_history + version83 = 83 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. // please make sure this is the largest version -var currentBootstrapVersion int64 = version82 +var currentBootstrapVersion int64 = version83 var ( bootstrapVersion = []func(Session, int64){ @@ -647,6 +659,7 @@ var ( upgradeToVer80, upgradeToVer81, upgradeToVer82, + upgradeToVer83, } ) @@ -1702,6 +1715,13 @@ func upgradeToVer82(s Session, ver int64) { doReentrantDDL(s, CreateAnalyzeOptionsTable) } +func upgradeToVer83(s Session, ver int64) { + if ver >= version83 { + return + } + doReentrantDDL(s, CreateStatsHistory) +} + 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= %?`, @@ -1788,6 +1808,8 @@ func doDDLWorks(s Session) { mustExecute(s, CreateTableCacheMetaTable) // Create analyze_options table. mustExecute(s, CreateAnalyzeOptionsTable) + // Create stats_history table. + mustExecute(s, CreateStatsHistory) } // doDMLWorks executes DML statements in bootstrap stage. diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index b1a7747721c7d..3f9a47cf5293d 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -824,6 +824,40 @@ func TestUpgradeVersion75(t *testing.T) { require.Equal(t, "char(255)", strings.ToLower(row.GetString(1))) } +func TestUpgradeVersion83(t *testing.T) { + ctx := context.Background() + store, _ := createStoreAndBootstrap(t) + defer func() { require.NoError(t, store.Close()) }() + + domV83, err := BootstrapSession(store) + require.NoError(t, err) + defer domV83.Close() + seV83 := createSessionAndSetID(t, store) + ver, err := getBootstrapVersion(seV83) + require.NoError(t, err) + require.Equal(t, currentBootstrapVersion, ver) + + statsHistoryTblFields := []struct { + field string + tp string + }{ + {"table_id", "bigint(64)"}, + {"stats_data", "longblob"}, + {"seq_no", "bigint(64)"}, + {"version", "bigint(64)"}, + {"create_time", "datetime(6)"}, + } + rStatsHistoryTbl := mustExec(t, seV83, `desc mysql.stats_history`) + req := rStatsHistoryTbl.NewChunk(nil) + require.NoError(t, rStatsHistoryTbl.Next(ctx, req)) + require.Equal(t, 5, req.NumRows()) + for i := 0; i < 5; i++ { + row := req.GetRow(i) + require.Equal(t, statsHistoryTblFields[i].field, strings.ToLower(row.GetString(0))) + require.Equal(t, statsHistoryTblFields[i].tp, strings.ToLower(row.GetString(1))) + } +} + func TestForIssue23387(t *testing.T) { // For issue https://github.com/pingcap/tidb/issues/23387 saveCurrentBootstrapVersion := currentBootstrapVersion diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 4628c18ac1c3a..2d0979667f622 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -15,6 +15,10 @@ package handle import ( + "bytes" + "compress/gzip" + "encoding/json" + "io/ioutil" "time" "github.com/pingcap/errors" @@ -326,3 +330,58 @@ func TableStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, jsonTbl *J tbl.ExtendedStats = extendedStatsFromJSON(jsonTbl.ExtStats) return tbl, nil } + +// JSONTableToBlocks convert JSONTable to json, then compresses it to blocks by gzip. +func JSONTableToBlocks(jsTable *JSONTable, blockSize int) ([][]byte, error) { + data, err := json.Marshal(jsTable) + if err != nil { + return nil, errors.Trace(err) + } + var gzippedData bytes.Buffer + gzipWriter := gzip.NewWriter(&gzippedData) + if _, err := gzipWriter.Write(data); err != nil { + return nil, errors.Trace(err) + } + if err := gzipWriter.Close(); err != nil { + return nil, errors.Trace(err) + } + blocksNum := gzippedData.Len() / blockSize + if gzippedData.Len()%blockSize != 0 { + blocksNum = blocksNum + 1 + } + blocks := make([][]byte, blocksNum) + for i := 0; i < blocksNum-1; i++ { + blocks[i] = gzippedData.Bytes()[blockSize*i : blockSize*(i+1)] + } + blocks[blocksNum-1] = gzippedData.Bytes()[blockSize*(blocksNum-1):] + return blocks, nil +} + +// BlocksToJSONTable convert gzip-compressed blocks to JSONTable +func BlocksToJSONTable(blocks [][]byte) (*JSONTable, error) { + if len(blocks) == 0 { + return nil, errors.New("Block empty error") + } + data := blocks[0] + for i := 1; i < len(blocks); i++ { + data = append(data, blocks[i]...) + } + gzippedData := bytes.NewReader(data) + gzipReader, err := gzip.NewReader(gzippedData) + if err != nil { + return nil, err + } + if err := gzipReader.Close(); err != nil { + return nil, err + } + jsonStr, err := ioutil.ReadAll(gzipReader) + if err != nil { + return nil, errors.Trace(err) + } + jsonTbl := JSONTable{} + err = json.Unmarshal(jsonStr, &jsonTbl) + if err != nil { + return nil, errors.Trace(err) + } + return &jsonTbl, nil +} diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index 00ebd9b35fe9c..a3bcded88b7fb 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -381,3 +381,38 @@ func TestDumpVer2Stats(t *testing.T) { // the statistics.Table in the stats cache is the same as the unmarshalled statistics.Table requireTableEqual(t, statsCacheTbl, loadTbl) } + +func TestJSONTableToBlocks(t *testing.T) { + tk, dom, clean := createTestKitAndDom(t) + defer clean() + 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))") + tk.MustExec("insert into t value(1, 'aaa'), (3, 'aab'), (5, 'bba'), (2, 'bbb'), (4, 'cca'), (6, 'ccc')") + // mark column stats as needed + tk.MustExec("select * from t where a = 3") + tk.MustExec("select * from t where b = 'bbb'") + tk.MustExec("alter table t add index single(a)") + tk.MustExec("alter table t add index multi(a, b)") + tk.MustExec("analyze table t with 2 topn") + h := dom.StatsHandle() + is := dom.InfoSchema() + tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + + dumpJSONTable, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil) + require.NoError(t, err) + jsOrigin, _ := json.Marshal(dumpJSONTable) + + blockSize := 30 + js, err := h.DumpStatsToJSON("test", tableInfo.Meta(), nil) + require.NoError(t, err) + dumpJSONBlocks, err := handle.JSONTableToBlocks(js, blockSize) + require.NoError(t, err) + jsConverted, err := handle.BlocksToJSONTable(dumpJSONBlocks) + require.NoError(t, err) + jsonStr, err := json.Marshal(jsConverted) + require.NoError(t, err) + require.JSONEq(t, string(jsOrigin), string(jsonStr)) +} diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 2b07e9aebcb99..45db4f42df329 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1944,3 +1944,54 @@ func (h *Handle) GetPredicateColumns(tableID int64) ([]int64, error) { } return columnIDs, nil } + +// Max column size is 6MB. Refer https://docs.pingcap.com/tidb/dev/tidb-limitations/#limitation-on-a-single-column +const maxColumnSize = 6 << 20 + +// RecordHistoricalStatsToStorage records the given table's stats data to mysql.stats_history +func (h *Handle) RecordHistoricalStatsToStorage(dbName string, tableInfo *model.TableInfo) (uint64, error) { + ctx := context.Background() + js, err := h.DumpStatsToJSON(dbName, tableInfo, nil) + if err != nil { + return 0, errors.Trace(err) + } + version := uint64(0) + for _, value := range js.Columns { + version = uint64(*value.StatsVer) + if version != 0 { + break + } + } + blocks, err := JSONTableToBlocks(js, maxColumnSize) + if err != nil { + return version, errors.Trace(err) + } + h.mu.Lock() + defer h.mu.Unlock() + exec := h.mu.ctx.(sqlexec.SQLExecutor) + _, err = exec.ExecuteInternal(ctx, "begin pessimistic") + if err != nil { + return version, errors.Trace(err) + } + defer func() { + err = finishTransaction(ctx, exec, err) + }() + ts := time.Now().Format("2006-01-02 15:04:05.999999") + + const sql = "INSERT INTO mysql.stats_history(table_id, stats_data, seq_no, version, create_time) VALUES (%?, %?, %?, %?, %?)" + for i := 0; i < len(blocks); i++ { + if _, err := exec.ExecuteInternal(ctx, sql, tableInfo.ID, blocks[i], i, version, ts); err != nil { + return version, errors.Trace(err) + } + } + return version, nil +} + +// CheckHistoricalStatsEnable is used to check whether TiDBEnableHistoricalStats is enabled. +func (h *Handle) CheckHistoricalStatsEnable() (enable bool, err error) { + val, err := h.mu.ctx.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBEnableHistoricalStats) + if err != nil { + return false, errors.Trace(err) + } + return variable.TiDBOptOn(val), nil +} diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index ac1ac3ebbe969..ed151a08f2310 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -18,6 +18,7 @@ import ( "bytes" "fmt" "math" + "strconv" "strings" "testing" "time" @@ -40,6 +41,7 @@ import ( "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) @@ -3114,3 +3116,28 @@ func (s *testStatsSuite) TestIncrementalModifyCountUpdate(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/injectBaseCount"), IsNil) c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/injectBaseModifyCount"), IsNil) } + +func (s *testStatsSuite) TestRecordHistoricalStatsToStorage(c *C) { + defer cleanEnv(c, s.store, s.do) + tk := testkit.NewTestKit(c, s.store) + 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))") + tk.MustExec("insert into t value(1, 'aaa'), (3, 'aab'), (5, 'bba'), (2, 'bbb'), (4, 'cca'), (6, 'ccc')") + // mark column stats as needed + tk.MustExec("select * from t where a = 3") + tk.MustExec("select * from t where b = 'bbb'") + tk.MustExec("alter table t add index single(a)") + tk.MustExec("alter table t add index multi(a, b)") + tk.MustExec("analyze table t with 2 topn") + + tableInfo, err := s.do.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(c, err) + version, err := s.do.StatsHandle().RecordHistoricalStatsToStorage("t", tableInfo.Meta()) + require.NoError(c, err) + + rows := tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where version = '%d'", version)).Rows() + num, _ := strconv.Atoi(rows[0][0].(string)) + require.GreaterOrEqual(c, num, 1) +} From 453535600f6016b4faae3158277f92013f111a85 Mon Sep 17 00:00:00 2001 From: tangenta Date: Fri, 21 Jan 2022 17:55:46 +0800 Subject: [PATCH 04/21] util/topsql: remove unused functions (#31877) ref pingcap/tidb#28720, ref pingcap/tidb#31192 --- util/topsql/reporter/datamodel.go | 89 -------------------------- util/topsql/reporter/datamodel_test.go | 82 ------------------------ 2 files changed, 171 deletions(-) diff --git a/util/topsql/reporter/datamodel.go b/util/topsql/reporter/datamodel.go index cb294e2dd217b..aa8b0562bb802 100644 --- a/util/topsql/reporter/datamodel.go +++ b/util/topsql/reporter/datamodel.go @@ -111,15 +111,6 @@ func (ts tsItems) Swap(i, j int) { ts[i], ts[j] = ts[j], ts[i] } -func (ts tsItems) sorted() bool { - for n := 0; n < len(ts)-1; n++ { - if ts[n].timestamp > ts[n+1].timestamp { - return false - } - } - return true -} - // toProto converts the tsItems to the corresponding protobuf representation. func (ts tsItems) toProto() []*tipb.TopSQLRecordItem { capacity := len(ts) @@ -310,74 +301,6 @@ func (r *record) appendStmtStatsItem(timestamp uint64, item stmtstats.StatementS } } -// merge other record into r. -// Attention, this function depend on r is sorted, and will sort `other` by timestamp. -func (r *record) merge(other *record) { - if other == nil || len(other.tsItems) == 0 { - return - } - - if !other.tsItems.sorted() { - sort.Sort(other) // this may never happen - } - if len(r.tsItems) == 0 { - r.totalCPUTimeMs = other.totalCPUTimeMs - r.tsItems = other.tsItems - r.tsIndex = other.tsIndex - return - } - length := len(r.tsItems) + len(other.tsItems) - newTsItems := make(tsItems, 0, length) - i, j := 0, 0 - for i < len(r.tsItems) && j < len(other.tsItems) { - if r.tsItems[i].timestamp == other.tsItems[j].timestamp { - newItem := zeroTsItem() - newItem.timestamp = r.tsItems[i].timestamp - newItem.cpuTimeMs = r.tsItems[i].cpuTimeMs + other.tsItems[j].cpuTimeMs - r.tsItems[i].stmtStats.Merge(&other.tsItems[j].stmtStats) - newItem.stmtStats = r.tsItems[i].stmtStats - newTsItems = append(newTsItems, newItem) - i++ - j++ - } else if r.tsItems[i].timestamp < other.tsItems[j].timestamp { - newItem := zeroTsItem() - newItem.timestamp = r.tsItems[i].timestamp - newItem.cpuTimeMs = r.tsItems[i].cpuTimeMs - newItem.stmtStats = r.tsItems[i].stmtStats - newTsItems = append(newTsItems, newItem) - i++ - } else { - newItem := zeroTsItem() - newItem.timestamp = other.tsItems[j].timestamp - newItem.cpuTimeMs = other.tsItems[j].cpuTimeMs - newItem.stmtStats = other.tsItems[j].stmtStats - newTsItems = append(newTsItems, newItem) - j++ - } - } - if i < len(r.tsItems) { - newTsItems = append(newTsItems, r.tsItems[i:]...) - } - if j < len(other.tsItems) { - newTsItems = append(newTsItems, other.tsItems[j:]...) - } - r.tsItems = newTsItems - r.totalCPUTimeMs += other.totalCPUTimeMs - r.rebuildTsIndex() -} - -// rebuildTsIndex rebuilds the entire tsIndex based on tsItems. -func (r *record) rebuildTsIndex() { - if len(r.tsItems) == 0 { - r.tsIndex = map[uint64]int{} - return - } - r.tsIndex = make(map[uint64]int, len(r.tsItems)) - for index, item := range r.tsItems { - r.tsIndex[item.timestamp] = index - } -} - // toProto converts the record to the corresponding protobuf representation. func (r *record) toProto() tipb.TopSQLRecord { return tipb.TopSQLRecord{ @@ -405,18 +328,6 @@ func (rs records) Swap(i, j int) { rs[i], rs[j] = rs[j], rs[i] } -// topN returns the largest n records (by record.totalCPUTimeMs), other -// records are returned as evicted. -func (rs records) topN(n int) (top, evicted records) { - if len(rs) <= n { - return rs, nil - } - if err := quickselect.QuickSelect(rs, n); err != nil { - return rs, nil - } - return rs[:n], rs[n:] -} - // toProto converts the records to the corresponding protobuf representation. func (rs records) toProto() []tipb.TopSQLRecord { pb := make([]tipb.TopSQLRecord, 0, len(rs)) diff --git a/util/topsql/reporter/datamodel_test.go b/util/topsql/reporter/datamodel_test.go index ba8d5ae85b78e..a068863205049 100644 --- a/util/topsql/reporter/datamodel_test.go +++ b/util/topsql/reporter/datamodel_test.go @@ -46,24 +46,6 @@ func Test_tsItem_toProto(t *testing.T) { assert.Equal(t, uint64(4), pb.StmtKvExecCount[""]) } -func Test_tsItems_Sort(t *testing.T) { - items := tsItems{} - assert.True(t, items.sorted()) - items = nil - assert.True(t, items.sorted()) - items = tsItems{ - {timestamp: 2}, - {timestamp: 3}, - {timestamp: 1}, - } - assert.False(t, items.sorted()) - sort.Sort(items) - assert.True(t, items.sorted()) - assert.Equal(t, uint64(1), items[0].timestamp) - assert.Equal(t, uint64(2), items[1].timestamp) - assert.Equal(t, uint64(3), items[2].timestamp) -} - func Test_tsItems_toProto(t *testing.T) { items := &tsItems{{}, {}, {}} pb := items.toProto() @@ -145,56 +127,6 @@ func Test_record_append(t *testing.T) { assert.Equal(t, uint64(30000), r.tsItems[2].stmtStats.SumDurationNs) } -func Test_record_merge(t *testing.T) { - r1 := record{ - totalCPUTimeMs: 1 + 2 + 3, - tsItems: tsItems{ - {timestamp: 1, cpuTimeMs: 1, stmtStats: *stmtstats.NewStatementStatsItem()}, - {timestamp: 2, cpuTimeMs: 2, stmtStats: *stmtstats.NewStatementStatsItem()}, - {timestamp: 3, cpuTimeMs: 3, stmtStats: *stmtstats.NewStatementStatsItem()}, - }, - } - r1.rebuildTsIndex() - r2 := record{ - totalCPUTimeMs: 6 + 5 + 4, - tsItems: tsItems{ - {timestamp: 6, cpuTimeMs: 6, stmtStats: *stmtstats.NewStatementStatsItem()}, - {timestamp: 5, cpuTimeMs: 5, stmtStats: *stmtstats.NewStatementStatsItem()}, - {timestamp: 4, cpuTimeMs: 4, stmtStats: *stmtstats.NewStatementStatsItem()}, - }, - } - r2.rebuildTsIndex() - r1.merge(&r2) - assert.Equal(t, uint64(4), r2.tsItems[0].timestamp) - assert.Equal(t, uint64(5), r2.tsItems[1].timestamp) - assert.Equal(t, uint64(6), r2.tsItems[2].timestamp) - assert.Len(t, r1.tsItems, 6) - assert.Len(t, r1.tsIndex, 6) - assert.Equal(t, uint64(1), r1.tsItems[0].timestamp) - assert.Equal(t, uint64(2), r1.tsItems[1].timestamp) - assert.Equal(t, uint64(3), r1.tsItems[2].timestamp) - assert.Equal(t, uint64(4), r1.tsItems[3].timestamp) - assert.Equal(t, uint64(5), r1.tsItems[4].timestamp) - assert.Equal(t, uint64(6), r1.tsItems[5].timestamp) - assert.Equal(t, uint64(1+2+3+4+5+6), r1.totalCPUTimeMs) -} - -func Test_record_rebuildTsIndex(t *testing.T) { - r := record{tsIndex: map[uint64]int{1: 1}} - r.rebuildTsIndex() - assert.Empty(t, r.tsIndex) - r.tsItems = tsItems{ - {timestamp: 1, cpuTimeMs: 1}, - {timestamp: 2, cpuTimeMs: 2}, - {timestamp: 3, cpuTimeMs: 3}, - } - r.rebuildTsIndex() - assert.Len(t, r.tsIndex, 3) - assert.Equal(t, 0, r.tsIndex[1]) - assert.Equal(t, 1, r.tsIndex[2]) - assert.Equal(t, 2, r.tsIndex[3]) -} - func Test_record_toProto(t *testing.T) { r := record{ sqlDigest: []byte("SQL-1"), @@ -220,20 +152,6 @@ func Test_records_Sort(t *testing.T) { assert.Equal(t, uint64(1), rs[2].totalCPUTimeMs) } -func Test_records_topN(t *testing.T) { - rs := records{ - {totalCPUTimeMs: 1}, - {totalCPUTimeMs: 3}, - {totalCPUTimeMs: 2}, - } - top, evicted := rs.topN(2) - assert.Len(t, top, 2) - assert.Len(t, evicted, 1) - assert.Equal(t, uint64(3), top[0].totalCPUTimeMs) - assert.Equal(t, uint64(2), top[1].totalCPUTimeMs) - assert.Equal(t, uint64(1), evicted[0].totalCPUTimeMs) -} - func Test_records_toProto(t *testing.T) { rs := records{{}, {}} pb := rs.toProto() From 2d522dd59001da28fe92aa11b868a555354dfcca Mon Sep 17 00:00:00 2001 From: Zhi Qi <30543181+LittleFall@users.noreply.github.com> Date: Fri, 21 Jan 2022 21:42:07 +0800 Subject: [PATCH 05/21] planner: Add MPPPartitionType `PassThroughType` to mark PassThrough Exchange. (#31766) * fix * add tests * simple * fix test * rename collect to passthrough * add more comments * add more tests * fix test * add more tests * rename * try fix wtf check_dev * add some comment * make rh --- planner/core/fragment.go | 1 + planner/core/physical_plan_test.go | 50 +++ planner/core/task.go | 38 +- .../integration_serial_suite_out.json | 15 +- planner/core/testdata/plan_suite_in.json | 37 ++ planner/core/testdata/plan_suite_out.json | 401 ++++++++++++++++++ planner/property/physical_property.go | 19 + 7 files changed, 544 insertions(+), 17 deletions(-) diff --git a/planner/core/fragment.go b/planner/core/fragment.go index df841f4a91f2c..837032e19619e 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -126,6 +126,7 @@ func (f *Fragment) init(p PhysicalPlan) error { } f.TableScan = x case *PhysicalExchangeReceiver: + // TODO: after we support partial merge, we should check whether all the target exchangeReceiver is same. f.singleton = x.children[0].(*PhysicalExchangeSender).ExchangeType == tipb.ExchangeType_PassThrough f.ExchangeReceivers = append(f.ExchangeReceivers, x) case *PhysicalUnionAll: diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index bc145e4a14ad7..f77ef93e20de1 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -18,6 +18,7 @@ import ( "context" "fmt" "math" + "strings" . "github.com/pingcap/check" "github.com/pingcap/tidb/domain" @@ -2039,3 +2040,52 @@ func (s *testPlanSuite) TestIssue30965(c *C) { " └─Selection(Probe) 0.00 cop[tikv] eq(test.t30965.a, 3), eq(test.t30965.c, 5)", " └─TableRowIDScan 10.00 cop[tikv] table:t30965 keep order:false, stats:pseudo")) } + +func (s *testPlanSuite) TestMPPSinglePartitionType(c *C) { + var ( + input []string + output []struct { + SQL string + Plan []string + } + ) + s.testData.GetTestCases(c, &input, &output) + store, dom, err := newStoreWithBootstrap() + c.Assert(err, IsNil) + defer func() { + dom.Close() + store.Close() + }() + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists employee") + tk.MustExec("create table employee(empid int, deptid int, salary decimal(10,2))") + tk.MustExec("set tidb_enforce_mpp=1") + + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + c.Assert(exists, IsTrue) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "employee" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + for i, ts := range input { + s.testData.OnRecord(func() { + output[i].SQL = ts + }) + if strings.HasPrefix(ts, "set") { + tk.MustExec(ts) + continue + } + s.testData.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format='brief'" + ts).Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/task.go b/planner/core/task.go index 90fbcedacaa9b..0c0dfd00d9005 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -745,7 +745,7 @@ func (p *PhysicalHashJoin) convertPartitionKeysIfNeed(lTask, rTask *mppTask) (*m if lChanged { nlTask := lTask.copy().(*mppTask) nlTask.p = lProj - nlTask = nlTask.enforceExchangerImpl(&property.PhysicalProperty{ + nlTask = nlTask.enforceExchanger(&property.PhysicalProperty{ TaskTp: property.MppTaskType, MPPPartitionTp: property.HashType, MPPPartitionCols: lPartKeys, @@ -757,7 +757,7 @@ func (p *PhysicalHashJoin) convertPartitionKeysIfNeed(lTask, rTask *mppTask) (*m if rChanged { nrTask := rTask.copy().(*mppTask) nrTask.p = rProj - nrTask = nrTask.enforceExchangerImpl(&property.PhysicalProperty{ + nrTask = nrTask.enforceExchanger(&property.PhysicalProperty{ TaskTp: property.MppTaskType, MPPPartitionTp: property.HashType, MPPPartitionCols: rPartKeys, @@ -1610,7 +1610,7 @@ func BuildFinalModeAggregation( } // TODO: Refactor the way of constructing aggregation functions. - // This fop loop is ugly, but I do not find a proper way to reconstruct + // This for loop is ugly, but I do not find a proper way to reconstruct // it right away. // group_concat is special when pushing down, it cannot take the two phase execution if no distinct but with orderBy, and other cases are also different: @@ -2075,6 +2075,20 @@ func (p *PhysicalHashAgg) cpuCostDivisor(hasDistinct bool) (float64, float64) { return math.Min(float64(finalCon), float64(partialCon)), float64(finalCon + partialCon) } +func (p *PhysicalHashAgg) attach2TaskForMpp1Phase(mpp *mppTask) task { + inputRows := mpp.count() + // 1-phase agg: when the partition columns can be satisfied, where the plan does not need to enforce Exchange + // only push down the original agg + proj := p.convertAvgForMPP() + attachPlan2Task(p.self, mpp) + if proj != nil { + attachPlan2Task(proj, mpp) + } + mpp.addCost(p.GetCost(inputRows, false, true)) + p.cost = mpp.cost() + return mpp +} + func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { t := tasks[0].copy() mpp, ok := t.(*mppTask) @@ -2095,6 +2109,7 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { p.cost = mpp.cost() return mpp case Mpp2Phase: + // TODO: when partition property is matched by sub-plan, we actually needn't do extra an exchange and final agg. proj := p.convertAvgForMPP() partialAgg, finalAgg := p.newPartialAggregate(kv.TiFlash, true) if partialAgg == nil { @@ -2149,6 +2164,10 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { finalAgg.SetCost(t.cost()) return t case MppScalar: + prop := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.SinglePartitionType} + if !mpp.needEnforceExchanger(prop) { + return p.attach2TaskForMpp1Phase(mpp) + } proj := p.convertAvgForMPP() partialAgg, finalAgg := p.newPartialAggregate(kv.TiFlash, true) if finalAgg == nil { @@ -2158,8 +2177,7 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { if partialAgg != nil { attachPlan2Task(partialAgg, mpp) } - prop := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType} - newMpp := mpp.enforceExchangerImpl(prop) + newMpp := mpp.enforceExchanger(prop) attachPlan2Task(finalAgg, newMpp) if proj == nil { proj = PhysicalProjection{ @@ -2345,17 +2363,21 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { return rt } -func (t *mppTask) needEnforce(prop *property.PhysicalProperty) bool { +func (t *mppTask) needEnforceExchanger(prop *property.PhysicalProperty) bool { switch prop.MPPPartitionTp { case property.AnyType: return false case property.BroadcastType: return true + case property.SinglePartitionType: + return t.partTp != property.SinglePartitionType default: if t.partTp != property.HashType { return true } // TODO: consider equalivant class + // TODO: `prop.IsSubsetOf` is enough, instead of equal. + // for example, if already partitioned by hash(B,C), then same (A,B,C) must distribute on a same node. if len(prop.MPPPartitionCols) != len(t.hashCols) { return true } @@ -2373,7 +2395,7 @@ func (t *mppTask) enforceExchanger(prop *property.PhysicalProperty) *mppTask { t.p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because operator `Sort` is not supported now.") return &mppTask{} } - if !t.needEnforce(prop) { + if !t.needEnforceExchanger(prop) { return t } return t.copy().(*mppTask).enforceExchangerImpl(prop) @@ -2390,7 +2412,7 @@ func (t *mppTask) enforceExchangerImpl(prop *property.PhysicalProperty) *mppTask } ctx := t.p.SCtx() sender := PhysicalExchangeSender{ - ExchangeType: tipb.ExchangeType(prop.MPPPartitionTp), + ExchangeType: prop.MPPPartitionTp.ToExchangeType(), HashCols: prop.MPPPartitionCols, }.Init(ctx, t.p.statsInfo()) sender.SetChildren(t.p) diff --git a/planner/core/testdata/integration_serial_suite_out.json b/planner/core/testdata/integration_serial_suite_out.json index 8e88ca5157d3f..112bb95e43c64 100644 --- a/planner/core/testdata/integration_serial_suite_out.json +++ b/planner/core/testdata/integration_serial_suite_out.json @@ -2755,15 +2755,12 @@ "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", " └─Projection 1.00 batchCop[tiflash] Column#5", " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct Column#4)->Column#5", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:Column#4, ", - " └─Projection 1.00 batchCop[tiflash] Column#4", - " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4", - " └─ExchangeReceiver 1.00 batchCop[tiflash] ", - " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + " └─Projection 1.00 batchCop[tiflash] Column#4", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.t.value)->Column#4", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.t.value, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" ] }, { diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index 09233f05dd121..ce4c38cbc695d 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -746,5 +746,42 @@ "cases": [ "select * from t where t.a < 3 and t.a < 3" ] + }, + { + "name": "TestMPPSinglePartitionType", + "cases": [ + // test normal aggregation, MPP2Phase. + "select count(*) from employee group by deptid+1", + // test normal aggregation, MPPScalar. + "select count(distinct deptid) a from employee", + // test normal join, Broadcast. + "select * from employee join employee e1 using(deptid)", + // test redundant collect exchange can be eliminated. + "select count(distinct a) from (select count(distinct deptid) a from employee) x", + // test mppScalar agg below mpp2Phase agg. + "select count(a) from (select count(distinct deptid) a, count(distinct empid) b from employee) x group by b+1", + // test mppScalar agg below mpp1Phase agg, TODO: add hint to enforce + // test mppScalar agg below mppTiDB agg, TODO: add hint to enforce + "select count(a) from (select count(distinct deptid) a, count(distinct empid) b from employee) x group by b", + // test mppScalar agg below join + "select * from employee join (select count(distinct deptid) a, count(distinct empid) b from employee) e1", + "select * from employee e1 join (select count(distinct deptid) a from employee) e2 on e1.deptid = e2.a", + "select * from (select count(distinct deptid) a from employee) e1 join employee e2 on e1.a = e2.deptid", + "select * from (select count(distinct deptid) a from employee) e1 join (select count(distinct deptid) b from employee) e2 on e1.a=e2.b", + // test mpp2Phase agg below join + "select * from employee e1 join employee e2 on e1.deptid = e2.deptid", + "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join employee e2 on e1.d = e2.deptid", + "select * from employee e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.deptid = e2.d", + "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.d = e2.d", + // non-broadcast join + "set tidb_broadcast_join_threshold_count=1", + "set tidb_broadcast_join_threshold_size=1", + "select * from (select count(distinct deptid) a from employee) e1 join employee e2 on e1.a = e2.deptid", + "select * from (select count(distinct deptid) a from employee) e1 join (select count(distinct deptid) b from employee) e2 on e1.a=e2.b", + "select * from employee e1 join employee e2 on e1.deptid = e2.deptid", + "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join employee e2 on e1.d = e2.deptid", + "select * from employee e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.deptid = e2.d", + "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.d = e2.d" + ] } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 013d86e227342..92f425c428531 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -2677,5 +2677,406 @@ ] } ] + }, + { + "Name": "TestMPPSinglePartitionType", + "Cases": [ + { + "SQL": "select count(*) from employee group by deptid+1", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#5", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#12, funcs:sum(Column#13)->Column#5", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: N/A]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#14, funcs:count(1)->Column#13", + " └─Projection 10000.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#14", + " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select count(distinct deptid) a from employee", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from employee join employee e1 using(deptid)", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 cop[tiflash] test.employee.deptid, test.employee.empid, test.employee.salary, test.employee.empid, test.employee.salary", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.employee.deptid, test.employee.deptid)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:employee keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 cop[tiflash] table:e1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select count(distinct a) from (select count(distinct deptid) a from employee) x", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#6", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct Column#5)->Column#6", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select count(a) from (select count(distinct deptid) a, count(distinct empid) b from employee) x group by b+1", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#7", + " └─HashAgg 1.00 batchCop[tiflash] group by:Column#12, funcs:sum(Column#13)->Column#7", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: N/A]", + " └─HashAgg 1.00 batchCop[tiflash] group by:Column#15, funcs:count(Column#14)->Column#13", + " └─Projection 1.00 batchCop[tiflash] Column#5, plus(Column#6, 1)->Column#15", + " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5, funcs:count(distinct test.employee.empid)->Column#6", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, test.employee.empid, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select count(a) from (select count(distinct deptid) a, count(distinct empid) b from employee) x group by b", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#7", + " └─HashAgg 1.00 batchCop[tiflash] group by:Column#6, funcs:count(Column#5)->Column#7", + " └─Projection 1.00 batchCop[tiflash] Column#5, Column#6", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5, funcs:count(distinct test.employee.empid)->Column#6", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, test.employee.empid, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from employee join (select count(distinct deptid) a, count(distinct empid) b from employee) e1", + "Plan": [ + "TableReader 10000.00 root data:ExchangeSender", + "└─ExchangeSender 10000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 10000.00 batchCop[tiflash] CARTESIAN inner join", + " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 1.00 batchCop[tiflash] Column#9, Column#10", + " │ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#9, funcs:count(distinct test.employee.empid)->Column#10", + " │ └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, test.employee.empid, ", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─TableFullScan(Probe) 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from employee e1 join (select count(distinct deptid) a from employee) e2 on e1.deptid = e2.a", + "Plan": [ + "TableReader 1.25 root data:ExchangeSender", + "└─ExchangeSender 1.25 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 1.25 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#9)]", + " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 1.00 batchCop[tiflash] Column#9", + " │ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#9", + " │ └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:e1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from (select count(distinct deptid) a from employee) e1 join employee e2 on e1.a = e2.deptid", + "Plan": [ + "TableReader 1.25 root data:ExchangeSender", + "└─ExchangeSender 1.25 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.25 batchCop[tiflash] Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", + " └─HashJoin 1.25 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#5)]", + " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 1.00 batchCop[tiflash] Column#5", + " │ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " │ └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:e2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from (select count(distinct deptid) a from employee) e1 join (select count(distinct deptid) b from employee) e2 on e1.a=e2.b", + "Plan": [ + "TableReader 1.00 root data:ExchangeSender", + "└─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 1.00 batchCop[tiflash] inner join, equal:[eq(Column#5, Column#10)]", + " ├─ExchangeReceiver(Build) 1.00 batchCop[tiflash] ", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 1.00 batchCop[tiflash] Column#5", + " │ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " │ └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─Projection(Probe) 1.00 batchCop[tiflash] Column#10", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#10", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from employee e1 join employee e2 on e1.deptid = e2.deptid", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.employee.deptid, test.employee.deptid)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:e1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 cop[tiflash] table:e2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join employee e2 on e1.d = e2.deptid", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#6, Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", + " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#6)]", + " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", + " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", + " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#5, funcs:firstrow(Column#15)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: N/A]", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#27, funcs:count(Column#25)->Column#14, funcs:firstrow(Column#26)->Column#15", + " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#27", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:e2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from employee e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.deptid = e2.d", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#10)]", + " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", + " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#10, Column#9", + " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 batchCop[tiflash] Column#9, test.employee.deptid", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#9, funcs:firstrow(Column#15)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: N/A]", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#27, funcs:count(Column#25)->Column#14, funcs:firstrow(Column#26)->Column#15", + " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#27", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:e1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.d = e2.d", + "Plan": [ + "TableReader 6400.00 root data:ExchangeSender", + "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6400.00 batchCop[tiflash] inner join, equal:[eq(Column#6, Column#12)]", + " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", + " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: Broadcast", + " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", + " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#17, funcs:sum(Column#18)->Column#5, funcs:firstrow(Column#19)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: N/A]", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#43, funcs:count(Column#41)->Column#18, funcs:firstrow(Column#42)->Column#19", + " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#43", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─Projection(Probe) 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#12, Column#11", + " └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " └─Projection 8000.00 batchCop[tiflash] Column#11, test.employee.deptid", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#20, funcs:sum(Column#21)->Column#11, funcs:firstrow(Column#22)->test.employee.deptid", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: N/A]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#46, funcs:count(Column#44)->Column#21, funcs:firstrow(Column#45)->Column#22", + " └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#46", + " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + ] + }, + { + "SQL": "set tidb_broadcast_join_threshold_count=1", + "Plan": null + }, + { + "SQL": "set tidb_broadcast_join_threshold_size=1", + "Plan": null + }, + { + "SQL": "select * from (select count(distinct deptid) a from employee) e1 join employee e2 on e1.a = e2.deptid", + "Plan": [ + "Projection 1.25 root Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", + "└─HashJoin 1.25 root inner join, equal:[eq(test.employee.deptid, Column#5)]", + " ├─TableReader(Build) 1.00 root data:ExchangeSender", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " │ └─Projection 1.00 batchCop[tiflash] Column#5", + " │ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " │ └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " │ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " │ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 cop[tiflash] table:e2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from (select count(distinct deptid) a from employee) e1 join (select count(distinct deptid) b from employee) e2 on e1.a=e2.b", + "Plan": [ + "HashJoin 1.00 root inner join, equal:[eq(Column#5, Column#10)]", + "├─TableReader(Build) 1.00 root data:ExchangeSender", + "│ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + "│ └─Projection 1.00 batchCop[tiflash] Column#10", + "│ └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#10", + "│ └─ExchangeReceiver 1.00 batchCop[tiflash] ", + "│ └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + "│ └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", + "│ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + "└─TableReader(Probe) 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 batchCop[tiflash] Column#5", + " └─HashAgg 1.00 batchCop[tiflash] funcs:count(distinct test.employee.deptid)->Column#5", + " └─ExchangeReceiver 1.00 batchCop[tiflash] ", + " └─ExchangeSender 1.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 batchCop[tiflash] group by:test.employee.deptid, ", + " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from employee e1 join employee e2 on e1.deptid = e2.deptid", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 cop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 cop[tiflash] inner join, equal:[eq(test.employee.deptid, test.employee.deptid)]", + " ├─ExchangeReceiver(Build) 9990.00 cop[tiflash] ", + " │ └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: N/A]", + " │ └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", + " │ └─TableFullScan 10000.00 cop[tiflash] table:e1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 cop[tiflash] ", + " └─ExchangeSender 9990.00 cop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: N/A]", + " └─Selection 9990.00 cop[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 cop[tiflash] table:e2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join employee e2 on e1.d = e2.deptid", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] Column#6, Column#5, test.employee.empid, test.employee.deptid, test.employee.salary", + " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#6)]", + " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", + " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: N/A]", + " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", + " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#5, funcs:firstrow(Column#15)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: N/A]", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, funcs:count(Column#27)->Column#14, funcs:firstrow(Column#28)->Column#15", + " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#29", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#26, collate: N/A]", + " └─Projection 9990.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, cast(test.employee.deptid, bigint(20))->Column#26", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:e2 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from employee e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.deptid = e2.d", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, Column#10, Column#9", + " └─HashJoin 8000.00 batchCop[tiflash] inner join, equal:[eq(test.employee.deptid, Column#10)]", + " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", + " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#10, collate: N/A]", + " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#10, Column#9", + " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 batchCop[tiflash] Column#9, test.employee.deptid", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#13, funcs:sum(Column#14)->Column#9, funcs:firstrow(Column#15)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#13, collate: N/A]", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#29, funcs:count(Column#27)->Column#14, funcs:firstrow(Column#28)->Column#15", + " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#29", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 batchCop[tiflash] ", + " └─ExchangeSender 9990.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#26, collate: N/A]", + " └─Projection 9990.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, test.employee.salary, cast(test.employee.deptid, bigint(20))->Column#26", + " └─Selection 9990.00 batchCop[tiflash] not(isnull(test.employee.deptid))", + " └─TableFullScan 10000.00 batchCop[tiflash] table:e1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.d = e2.d", + "Plan": [ + "TableReader 6400.00 root data:ExchangeSender", + "└─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6400.00 batchCop[tiflash] inner join, equal:[eq(Column#6, Column#12)]", + " ├─ExchangeReceiver(Build) 6400.00 batchCop[tiflash] ", + " │ └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#6, collate: N/A]", + " │ └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#6, Column#5", + " │ └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " │ └─Projection 8000.00 batchCop[tiflash] Column#5, test.employee.deptid", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#17, funcs:sum(Column#18)->Column#5, funcs:firstrow(Column#19)->test.employee.deptid", + " │ └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " │ └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#17, collate: N/A]", + " │ └─HashAgg 8000.00 batchCop[tiflash] group by:Column#43, funcs:count(Column#41)->Column#18, funcs:firstrow(Column#42)->Column#19", + " │ └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#43", + " │ └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 6400.00 batchCop[tiflash] ", + " └─ExchangeSender 6400.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#12, collate: N/A]", + " └─Projection 6400.00 batchCop[tiflash] plus(test.employee.deptid, 1)->Column#12, Column#11", + " └─Selection 6400.00 batchCop[tiflash] not(isnull(plus(test.employee.deptid, 1)))", + " └─Projection 8000.00 batchCop[tiflash] Column#11, test.employee.deptid", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#20, funcs:sum(Column#21)->Column#11, funcs:firstrow(Column#22)->test.employee.deptid", + " └─ExchangeReceiver 8000.00 batchCop[tiflash] ", + " └─ExchangeSender 8000.00 batchCop[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: N/A]", + " └─HashAgg 8000.00 batchCop[tiflash] group by:Column#46, funcs:count(Column#44)->Column#21, funcs:firstrow(Column#45)->Column#22", + " └─Projection 10000.00 batchCop[tiflash] test.employee.empid, test.employee.deptid, plus(test.employee.deptid, 1)->Column#46", + " └─TableFullScan 10000.00 batchCop[tiflash] table:employee keep order:false, stats:pseudo" + ] + } + ] } ] diff --git a/planner/property/physical_property.go b/planner/property/physical_property.go index 1e6e3aaeefae1..1b15a965eb245 100644 --- a/planner/property/physical_property.go +++ b/planner/property/physical_property.go @@ -18,10 +18,12 @@ import ( "bytes" "fmt" + "github.com/pingcap/log" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" + "github.com/pingcap/tipb/go-tipb" ) // wholeTaskTypes records all possible kinds of task that a plan can return. For Agg, TopN and Limit, we will try to get @@ -51,8 +53,25 @@ const ( BroadcastType // HashType requires current task to shuffle its data according to some columns. HashType + // SinglePartitionType requires all the task pass the data to one node (tidb/tiflash). + SinglePartitionType ) +// ToExchangeType generates ExchangeType from MPPPartitionType +func (t MPPPartitionType) ToExchangeType() tipb.ExchangeType { + switch t { + case BroadcastType: + return tipb.ExchangeType_Broadcast + case HashType: + return tipb.ExchangeType_Hash + case SinglePartitionType: + return tipb.ExchangeType_PassThrough + default: + log.Warn("generate an exchange with any partition type, which is illegal.") + return tipb.ExchangeType_PassThrough + } +} + // MPPPartitionColumn is the column that will be used in MPP Hash Exchange type MPPPartitionColumn struct { Col *expression.Column From 622b8583e8086505a7a53929d1dae5bcc700bd9f Mon Sep 17 00:00:00 2001 From: fengou1 <85682690+fengou1@users.noreply.github.com> Date: Fri, 21 Jan 2022 23:07:46 +0800 Subject: [PATCH 06/21] br: update unit test - take local timestamp for TSO calculation instead of const variable (#31361) close pingcap/tidb#30449 --- br/pkg/task/backup_test.go | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/br/pkg/task/backup_test.go b/br/pkg/task/backup_test.go index 816d4837203d3..0800b95dbed2c 100644 --- a/br/pkg/task/backup_test.go +++ b/br/pkg/task/backup_test.go @@ -24,10 +24,13 @@ func TestParseTSString(t *testing.T) { require.NoError(t, err) require.Equal(t, uint64(400036290571534337), ts) - _, offset := time.Now().Local().Zone() - ts, err = parseTSString("2018-05-11 01:42:23") + ts, err = parseTSString("2021-01-01 01:42:23") require.NoError(t, err) - require.Equal(t, uint64(400032515489792000-(offset*1000)<<18), ts) + localTime := time.Date(2021, time.Month(1), 1, 1, 42, 23, 0, time.Local) + + localTimestamp := localTime.Unix() + localTSO := uint64((localTimestamp << 18) * 1000) + require.Equal(t, localTSO, ts) } func TestParseCompressionType(t *testing.T) { From 4ae96872ff2bde97187a5c4c713482525f66ac09 Mon Sep 17 00:00:00 2001 From: Chengpeng Yan <41809508+Reminiscent@users.noreply.github.com> Date: Sat, 22 Jan 2022 00:09:47 +0800 Subject: [PATCH 07/21] planner: do not use like to build range when new collation is enabled (#31278) close pingcap/tidb#31174 --- .../explain_generate_column_substitute.result | 1 + cmd/explaintest/r/index_merge.result | 2 +- .../t/explain_generate_column_substitute.test | 1 + expression/integration_serial_test.go | 23 +++++++++++++++++++ util/collate/collate.go | 7 ++++-- util/ranger/checker.go | 10 ++++++++ 6 files changed, 41 insertions(+), 3 deletions(-) diff --git a/cmd/explaintest/r/explain_generate_column_substitute.result b/cmd/explaintest/r/explain_generate_column_substitute.result index 3cab89b08dfc5..fb209b6bcabd5 100644 --- a/cmd/explaintest/r/explain_generate_column_substitute.result +++ b/cmd/explaintest/r/explain_generate_column_substitute.result @@ -1,3 +1,4 @@ +set names utf8mb4; use test; drop table if exists t; create table t(a int, b real, c bigint as ((a+1)) virtual, e real as ((b+a))); diff --git a/cmd/explaintest/r/index_merge.result b/cmd/explaintest/r/index_merge.result index ddce511db13c8..014aac58825b5 100644 --- a/cmd/explaintest/r/index_merge.result +++ b/cmd/explaintest/r/index_merge.result @@ -322,7 +322,7 @@ drop view if exists v2; create view v2 as select /*+ use_index_merge(t1) */ * from t1 where c1 < 10 or c2 < 10 and c3 < 10; show create view v2; View Create View character_set_client collation_connection -v2 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`%` SQL SECURITY DEFINER VIEW `v2` (`c1`, `c2`, `c3`) AS SELECT /*+ USE_INDEX_MERGE(`t1` )*/ `test`.`t1`.`c1` AS `c1`,`test`.`t1`.`c2` AS `c2`,`test`.`t1`.`c3` AS `c3` FROM `test`.`t1` WHERE `c1`<10 OR `c2`<10 AND `c3`<10 utf8mb4 utf8mb4_general_ci +v2 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`%` SQL SECURITY DEFINER VIEW `v2` (`c1`, `c2`, `c3`) AS SELECT /*+ USE_INDEX_MERGE(`t1` )*/ `test`.`t1`.`c1` AS `c1`,`test`.`t1`.`c2` AS `c2`,`test`.`t1`.`c3` AS `c3` FROM `test`.`t1` WHERE `c1`<10 OR `c2`<10 AND `c3`<10 utf8mb4 utf8mb4_bin select * from v2 order by 1; c1 c2 c3 1 1 1 diff --git a/cmd/explaintest/t/explain_generate_column_substitute.test b/cmd/explaintest/t/explain_generate_column_substitute.test index 4e47ce34607ae..f71b05c59f5c7 100644 --- a/cmd/explaintest/t/explain_generate_column_substitute.test +++ b/cmd/explaintest/t/explain_generate_column_substitute.test @@ -1,3 +1,4 @@ +set names utf8mb4; use test; drop table if exists t; create table t(a int, b real, c bigint as ((a+1)) virtual, e real as ((b+a))); diff --git a/expression/integration_serial_test.go b/expression/integration_serial_test.go index d43a5342d8b23..01013e56a0283 100644 --- a/expression/integration_serial_test.go +++ b/expression/integration_serial_test.go @@ -96,6 +96,29 @@ func TestIssue17891(t *testing.T) { tk.MustExec("create table test(id int, value set ('a','b','c') charset utf8mb4 collate utf8mb4_general_ci default 'a,B ,C');") } +func TestIssue31174(t *testing.T) { + collate.SetNewCollationEnabledForTest(true) + defer collate.SetNewCollationEnabledForTest(false) + + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(4) collate utf8_general_ci primary key /*T![clustered_index] clustered */);") + tk.MustExec("insert into t values('`?');") + // The 'like' condition can not be used to construct the range. + tk.HasPlan("select * from t where a like '`%';", "TableFullScan") + tk.MustQuery("select * from t where a like '`%';").Check(testkit.Rows("`?")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(4) collate binary primary key /*T![clustered_index] clustered */);") + tk.MustExec("insert into t values('`?');") + tk.HasPlan("select * from t where a like '`%';", "TableRangeScan") + tk.MustQuery("select * from t where a like '`%';").Check(testkit.Rows("`?\x00\x00")) +} + func TestIssue20268(t *testing.T) { collate.SetNewCollationEnabledForTest(true) defer collate.SetNewCollationEnabledForTest(false) diff --git a/util/collate/collate.go b/util/collate/collate.go index f74bacb96130f..1b31abcb62ffb 100644 --- a/util/collate/collate.go +++ b/util/collate/collate.go @@ -331,10 +331,13 @@ func IsCICollation(collate string) bool { collate == "utf8_unicode_ci" || collate == "utf8mb4_unicode_ci" } -// IsBinCollation returns if the collation is 'xx_bin'. +// IsBinCollation returns if the collation is 'xx_bin' or 'bin'. +// The function is to determine whether the sortkey of a char type of data under the collation is equal to the data itself, +// and both xx_bin and collationBin are satisfied. func IsBinCollation(collate string) bool { return collate == charset.CollationASCII || collate == charset.CollationLatin1 || - collate == charset.CollationUTF8 || collate == charset.CollationUTF8MB4 + collate == charset.CollationUTF8 || collate == charset.CollationUTF8MB4 || + collate == charset.CollationBin } // CollationToProto converts collation from string to int32(used by protocol). diff --git a/util/ranger/checker.go b/util/ranger/checker.go index 8431bc05abef6..3f667b5a8733e 100644 --- a/util/ranger/checker.go +++ b/util/ranger/checker.go @@ -112,6 +112,16 @@ func (c *conditionChecker) checkScalarFunction(scalar *expression.ScalarFunction func (c *conditionChecker) checkLikeFunc(scalar *expression.ScalarFunction) bool { _, collation := scalar.CharsetAndCollation() + if collate.NewCollationEnabled() && !collate.IsBinCollation(collation) { + // The algorithm constructs the range in byte-level: for example, ab% is mapped to [ab, ac] by adding 1 to the last byte. + // However, this is incorrect for non-binary collation strings because the sort key order is not the same as byte order. + // For example, "`%" is mapped to the range [`, a](where ` is 0x60 and a is 0x61). + // Because the collation utf8_general_ci is case-insensitive, a and A have the same sort key. + // Finally, the range comes to be [`, A], which is actually an empty range. + // See https://github.com/pingcap/tidb/issues/31174 for more details. + // In short, when the column type is non-binary collation string, we cannot use `like` expressions to generate the range. + return false + } if !collate.CompatibleCollate(scalar.GetArgs()[0].GetType().Collate, collation) { return false } From bb3e90915dc6a54c51a7495f7579275984c09d93 Mon Sep 17 00:00:00 2001 From: rebelice Date: Sat, 22 Jan 2022 01:29:46 +0800 Subject: [PATCH 08/21] bindinfo: update lastUpdateTime with invalid bind (#31775) close pingcap/tidb#31703 --- bindinfo/handle.go | 11 +++++++---- bindinfo/handle_test.go | 25 +++++++++++++++++++++++++ 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 773597a7f7eec..d57d2edb1da0f 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -159,15 +159,18 @@ func (h *BindHandle) Update(fullLoad bool) (err error) { continue } hash, meta, err := h.newBindRecord(row) - if err != nil { - logutil.BgLogger().Debug("[sql-bind] failed to generate bind record from data row", zap.Error(err)) - continue - } + // Update lastUpdateTime to the newest one. + // Even if this one is an invalid bind. if meta.Bindings[0].UpdateTime.Compare(lastUpdateTime) > 0 { lastUpdateTime = meta.Bindings[0].UpdateTime } + if err != nil { + logutil.BgLogger().Debug("[sql-bind] failed to generate bind record from data row", zap.Error(err)) + continue + } + oldRecord := newCache.getBindRecord(hash, meta.OriginalSQL, meta.Db) newRecord := merge(oldRecord, meta).removeDeletedBindings() if len(newRecord.Bindings) > 0 { diff --git a/bindinfo/handle_test.go b/bindinfo/handle_test.go index 398b0641579ca..8522d95902c1c 100644 --- a/bindinfo/handle_test.go +++ b/bindinfo/handle_test.go @@ -99,6 +99,31 @@ func TestBindingLastUpdateTime(t *testing.T) { tk.MustQuery(`show global status like 'last_plan_binding_update_time';`).Check(testkit.Rows()) } +func TestBindingLastUpdateTimeWithInvalidBind(t *testing.T) { + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + + rows0 := tk.MustQuery("show status like 'last_plan_binding_update_time';").Rows() + updateTime0 := rows0[0][1] + require.Equal(t, updateTime0, "0000-00-00 00:00:00") + + tk.MustExec("insert into mysql.bind_info values('select * from `test` . `t`', 'select * from `test` . `t` use index(`idx`)', 'test', 'using', '2000-01-01 09:00:00', '2000-01-01 09:00:00', '', '','" + + bindinfo.Manual + "')") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + tk.MustExec("admin reload bindings;") + + rows1 := tk.MustQuery("show status like 'last_plan_binding_update_time';").Rows() + updateTime1 := rows1[0][1] + require.Equal(t, updateTime1, "2000-01-01 09:00:00.000") + + rows2 := tk.MustQuery("show global bindings").Rows() + require.Equal(t, len(rows2), 0) +} + func TestBindParse(t *testing.T) { store, _, clean := testkit.CreateMockStoreAndDomain(t) defer clean() From f28cb77500e9cae75cf5d02cd879fff72ed6bf41 Mon Sep 17 00:00:00 2001 From: fengou1 <85682690+fengou1@users.noreply.github.com> Date: Sat, 22 Jan 2022 02:49:46 +0800 Subject: [PATCH 09/21] br: disable the batch ddl by default (#31845) close pingcap/tidb#31835 --- br/pkg/task/restore.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index 64fa8bacc26c8..b4ecfc69c7c6a 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -47,7 +47,7 @@ const ( defaultPDConcurrency = 1 defaultBatchFlushInterval = 16 * time.Second defaultDDLConcurrency = 16 - defaultFlagDdlBatchSize = 128 + defaultFlagDdlBatchSize = 1 ) // RestoreCommonConfig is the common configuration for all BR restore tasks. From 5af13e8a9f72ddb4312138b2665db52fb9f3688d Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 22 Jan 2022 09:15:46 +0800 Subject: [PATCH 10/21] *: improve import (#31887) ref pingcap/tidb#31716 --- parser/charset/encoding_bin.go | 1 + parser/charset/encoding_table.go | 3 ++- util/collate/collate_bench_test.go | 3 +-- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/parser/charset/encoding_bin.go b/parser/charset/encoding_bin.go index 6546f302cef45..3a9bf2e4f8968 100644 --- a/parser/charset/encoding_bin.go +++ b/parser/charset/encoding_bin.go @@ -15,6 +15,7 @@ package charset import ( "bytes" + "golang.org/x/text/encoding" ) diff --git a/parser/charset/encoding_table.go b/parser/charset/encoding_table.go index 2780272296acb..24d61b07f9ba4 100644 --- a/parser/charset/encoding_table.go +++ b/parser/charset/encoding_table.go @@ -14,6 +14,8 @@ package charset import ( + "strings" + "golang.org/x/text/encoding" "golang.org/x/text/encoding/charmap" "golang.org/x/text/encoding/japanese" @@ -21,7 +23,6 @@ import ( "golang.org/x/text/encoding/simplifiedchinese" "golang.org/x/text/encoding/traditionalchinese" "golang.org/x/text/encoding/unicode" - "strings" ) // Lookup returns the encoding with the specified label, and its canonical diff --git a/util/collate/collate_bench_test.go b/util/collate/collate_bench_test.go index 69c594dd2d7b7..e3397758f6123 100644 --- a/util/collate/collate_bench_test.go +++ b/util/collate/collate_bench_test.go @@ -16,9 +16,8 @@ package collate import ( "math/rand" - "testing" - _ "net/http/pprof" + "testing" ) const short = 2 << 4 From 7f442079acc4bb4f8ce3feb5575258ff233f33fe Mon Sep 17 00:00:00 2001 From: Ding Zengxian Date: Sat, 22 Jan 2022 10:27:46 +0800 Subject: [PATCH 11/21] *: Minimize file and directory permissions (#31740) ref pingcap/tidb#31310 --- br/cmd/tidb-lightning-ctl/main.go | 2 +- br/pkg/lightning/backend/local/local.go | 4 ++-- br/pkg/lightning/checkpoints/checkpoints.go | 2 +- cmd/pluginpkg/pluginpkg.go | 2 +- executor/select_into.go | 3 ++- parser/goyacc/main.go | 2 +- store/mockstore/unistore/lockstore/load_dump.go | 2 +- store/mockstore/unistore/mock.go | 2 +- store/mockstore/unistore/tikv/mvcc.go | 2 +- util/disk/tempDir.go | 4 ++-- 10 files changed, 13 insertions(+), 12 deletions(-) diff --git a/br/cmd/tidb-lightning-ctl/main.go b/br/cmd/tidb-lightning-ctl/main.go index 43891c8fb0fb3..b877fecb2f3c9 100644 --- a/br/cmd/tidb-lightning-ctl/main.go +++ b/br/cmd/tidb-lightning-ctl/main.go @@ -254,7 +254,7 @@ func checkpointDump(ctx context.Context, cfg *config.Config, dumpFolder string) } defer cpdb.Close() - if err := os.MkdirAll(dumpFolder, 0o755); err != nil { + if err := os.MkdirAll(dumpFolder, 0o750); err != nil { return errors.Trace(err) } diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index b4a5826e5651f..77f6b6d2e8702 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -592,7 +592,7 @@ func (local *local) OpenEngine(ctx context.Context, cfg *backend.EngineConfig, e return errors.Trace(err) } if !common.IsDirExists(sstDir) { - if err := os.Mkdir(sstDir, 0o755); err != nil { + if err := os.Mkdir(sstDir, 0o750); err != nil { return errors.Trace(err) } } @@ -1526,7 +1526,7 @@ func (local *local) ResetEngine(ctx context.Context, engineUUID uuid.UUID) error localEngine.db = db localEngine.engineMeta = engineMeta{} if !common.IsDirExists(localEngine.sstDir) { - if err := os.Mkdir(localEngine.sstDir, 0o755); err != nil { + if err := os.Mkdir(localEngine.sstDir, 0o750); err != nil { return errors.Trace(err) } } diff --git a/br/pkg/lightning/checkpoints/checkpoints.go b/br/pkg/lightning/checkpoints/checkpoints.go index a39a24098cbd4..47ced55db41c5 100644 --- a/br/pkg/lightning/checkpoints/checkpoints.go +++ b/br/pkg/lightning/checkpoints/checkpoints.go @@ -990,7 +990,7 @@ func (cpdb *FileCheckpointsDB) save() error { // because `os.WriteFile` is not atomic, directly write into it may reset the file // to an empty file if write is not finished. tmpPath := cpdb.path + ".tmp" - if err := os.WriteFile(tmpPath, serialized, 0o644); err != nil { // nolint:gosec + if err := os.WriteFile(tmpPath, serialized, 0o600); err != nil { return errors.Trace(err) } if err := os.Rename(tmpPath, cpdb.path); err != nil { diff --git a/cmd/pluginpkg/pluginpkg.go b/cmd/pluginpkg/pluginpkg.go index 8390b10adbb10..24a7c6a0a5dd6 100644 --- a/cmd/pluginpkg/pluginpkg.go +++ b/cmd/pluginpkg/pluginpkg.go @@ -126,7 +126,7 @@ func main() { } genFileName := filepath.Join(pkgDir, filepath.Base(pkgDir)+".gen.go") - genFile, err := os.OpenFile(genFileName, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0755) + genFile, err := os.OpenFile(genFileName, os.O_RDWR|os.O_CREATE|os.O_TRUNC, 0700) // # nosec G302 if err != nil { log.Printf("generate code failure during prepare output file, %+v\n", err) os.Exit(1) diff --git a/executor/select_into.go b/executor/select_into.go index cb0a2d5356551..5003f1dbf9fd1 100644 --- a/executor/select_into.go +++ b/executor/select_into.go @@ -52,7 +52,8 @@ func (s *SelectIntoExec) Open(ctx context.Context) error { return errors.New("unsupported SelectInto type") } - f, err := os.OpenFile(s.intoOpt.FileName, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0666) + // MySQL-compatible behavior: allow files to be group-readable + f, err := os.OpenFile(s.intoOpt.FileName, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0640) // # nosec G302 if err != nil { return errors.Trace(err) } diff --git a/parser/goyacc/main.go b/parser/goyacc/main.go index 1b8fae47cd756..22d78f2998e91 100644 --- a/parser/goyacc/main.go +++ b/parser/goyacc/main.go @@ -324,7 +324,7 @@ func main1(in string) (err error) { } if fn := *oXErrorsGen; fn != "" { - f, err := os.OpenFile(fn, os.O_RDWR|os.O_CREATE, 0666) + f, err := os.OpenFile(fn, os.O_RDWR|os.O_CREATE, 0600) if err != nil { return err } diff --git a/store/mockstore/unistore/lockstore/load_dump.go b/store/mockstore/unistore/lockstore/load_dump.go index dca102ab0b0ed..f0192331ecd48 100644 --- a/store/mockstore/unistore/lockstore/load_dump.go +++ b/store/mockstore/unistore/lockstore/load_dump.go @@ -96,7 +96,7 @@ func (ls *MemStore) writeItem(writer *bufio.Writer, data []byte) error { // DumpToFile dumps the meta to a file func (ls *MemStore) DumpToFile(fileName string, meta []byte) error { tmpFileName := fileName + ".tmp" - f, err := os.OpenFile(tmpFileName, os.O_CREATE|os.O_TRUNC|os.O_RDWR, 0666) + f, err := os.OpenFile(tmpFileName, os.O_CREATE|os.O_TRUNC|os.O_RDWR, 0600) if err != nil { return errors.Trace(err) } diff --git a/store/mockstore/unistore/mock.go b/store/mockstore/unistore/mock.go index 37bfaf5473d5d..06bfe7396abc4 100644 --- a/store/mockstore/unistore/mock.go +++ b/store/mockstore/unistore/mock.go @@ -34,7 +34,7 @@ func New(path string) (*RPCClient, pd.Client, *Cluster, error) { persistent = false } - if err := os.MkdirAll(path, 0777); err != nil { + if err := os.MkdirAll(path, 0750); err != nil { return nil, nil, nil, err } diff --git a/store/mockstore/unistore/tikv/mvcc.go b/store/mockstore/unistore/tikv/mvcc.go index c25e2e2104b28..a0d943a5647da 100644 --- a/store/mockstore/unistore/tikv/mvcc.go +++ b/store/mockstore/unistore/tikv/mvcc.go @@ -122,7 +122,7 @@ type lockEntryHdr struct { func (store *MVCCStore) dumpMemLocks() error { tmpFileName := store.dir + "/lock_store.tmp" - f, err := os.OpenFile(tmpFileName, os.O_CREATE|os.O_TRUNC|os.O_RDWR, 0666) + f, err := os.OpenFile(tmpFileName, os.O_CREATE|os.O_TRUNC|os.O_RDWR, 0600) if err != nil { return errors.Trace(err) } diff --git a/util/disk/tempDir.go b/util/disk/tempDir.go index 0603b54fd007c..1a615ccd647b1 100644 --- a/util/disk/tempDir.go +++ b/util/disk/tempDir.go @@ -64,7 +64,7 @@ func InitializeTempDir() error { tempDir := config.GetGlobalConfig().TempStoragePath _, err := os.Stat(tempDir) if err != nil && !os.IsExist(err) { - err = os.MkdirAll(tempDir, 0755) + err = os.MkdirAll(tempDir, 0750) if err != nil { return err } @@ -118,7 +118,7 @@ func CleanUp() { func CheckAndCreateDir(path string) error { _, err := os.Stat(path) if err != nil && !os.IsExist(err) { - err = os.MkdirAll(path, 0755) + err = os.MkdirAll(path, 0750) if err != nil { return err } From 9c73e5ed23afc5e53e9c0ab816d2ac194c2ec92c Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sat, 22 Jan 2022 20:31:46 +0800 Subject: [PATCH 12/21] *: remove the t.Run (#31895) ref pingcap/tidb#31716 --- domain/db_test.go | 3 +-- domain/domain_test.go | 6 ++---- domain/main_test.go | 9 --------- executor/executor_pkg_test.go | 21 ++++++--------------- 4 files changed, 9 insertions(+), 30 deletions(-) diff --git a/domain/db_test.go b/domain/db_test.go index eee241ee7b0d4..990ec2ad41a70 100644 --- a/domain/db_test.go +++ b/domain/db_test.go @@ -25,8 +25,7 @@ import ( "github.com/stretchr/testify/require" ) -// SubTestDomainSession is batched in TestDomainSerial -func SubTestDomainSession(t *testing.T) { +func TestDomainSession(t *testing.T) { lease := 50 * time.Millisecond store, err := mockstore.NewMockStore() require.NoError(t, err) diff --git a/domain/domain_test.go b/domain/domain_test.go index 4d9e96ebdd913..bed887c72133f 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -45,8 +45,7 @@ import ( "go.etcd.io/etcd/integration" ) -// SubTestInfo is batched in TestDomainSerial -func SubTestInfo(t *testing.T) { +func TestInfo(t *testing.T) { if runtime.GOOS == "windows" { t.Skip("integration.NewClusterV3 will create file contains a colon which is not allowed on Windows") } @@ -154,8 +153,7 @@ func SubTestInfo(t *testing.T) { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/domain/infosync/FailPlacement")) } -// SubTestDomain is batched in TestDomainSerial -func SubTestDomain(t *testing.T) { +func TestDomain(t *testing.T) { store, err := mockstore.NewMockStore() require.NoError(t, err) diff --git a/domain/main_test.go b/domain/main_test.go index 969d50e4a0f67..aaa108626c8f3 100644 --- a/domain/main_test.go +++ b/domain/main_test.go @@ -17,7 +17,6 @@ package domain_test import ( "testing" - . "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/util/testbridge" "go.uber.org/goleak" ) @@ -30,11 +29,3 @@ func TestMain(m *testing.M) { } goleak.VerifyTestMain(m, opts...) } - -// TestDomainSerial handles tests in serial -func TestDomainSerial(t *testing.T) { - // these tests should run in serial for failpoint is global - t.Run("info", SubTestInfo) - t.Run("domain", SubTestDomain) - t.Run("domainSession", SubTestDomainSession) -} diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 51453e177b568..afe244bcf7a82 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -98,16 +98,7 @@ func (msm *mockSessionManager) SetServerID(serverID uint64) { msm.serverID = serverID } -func TestExecutorPkg(t *testing.T) { - t.Run("ShowProcessList", SubTestShowProcessList) - t.Run("BuildKvRangesForIndexJoinWithoutCwc", SubTestBuildKvRangesForIndexJoinWithoutCwc) - t.Run("GetFieldsFromLine", SubTestGetFieldsFromLine) - t.Run("SlowQueryRuntimeStats", SubTestSlowQueryRuntimeStats) - t.Run("AggPartialResultMapperB", SubTestAggPartialResultMapperB) - t.Run("FilterTemporaryTableKeys", SubTestFilterTemporaryTableKeys) -} - -func SubTestShowProcessList(t *testing.T) { +func TestShowProcessList(t *testing.T) { // Compose schema. names := []string{"Id", "User", "Host", "db", "Command", "Time", "State", "Info"} ftypes := []byte{mysql.TypeLonglong, mysql.TypeVarchar, mysql.TypeVarchar, @@ -180,7 +171,7 @@ func buildSchema(names []string, ftypes []byte) *expression.Schema { return schema } -func SubTestBuildKvRangesForIndexJoinWithoutCwc(t *testing.T) { +func TestBuildKvRangesForIndexJoinWithoutCwc(t *testing.T) { indexRanges := make([]*ranger.Range, 0, 6) indexRanges = append(indexRanges, generateIndexRange(1, 1, 1, 1, 1)) indexRanges = append(indexRanges, generateIndexRange(1, 1, 2, 1, 1)) @@ -224,7 +215,7 @@ func generateDatumSlice(vals ...int64) []types.Datum { return datums } -func SubTestGetFieldsFromLine(t *testing.T) { +func TestGetFieldsFromLine(t *testing.T) { tests := []struct { input string expected []string @@ -282,7 +273,7 @@ func assertEqualStrings(t *testing.T, got []field, expect []string) { } } -func SubTestSlowQueryRuntimeStats(t *testing.T) { +func TestSlowQueryRuntimeStats(t *testing.T) { stats := &slowQueryRuntimeStats{ totalFileNum: 2, readFileNum: 2, @@ -300,7 +291,7 @@ func SubTestSlowQueryRuntimeStats(t *testing.T) { // Test whether the actual buckets in Golang Map is same with the estimated number. // The test relies the implement of Golang Map. ref https://github.com/golang/go/blob/go1.13/src/runtime/map.go#L114 -func SubTestAggPartialResultMapperB(t *testing.T) { +func TestAggPartialResultMapperB(t *testing.T) { if runtime.Version() < `go1.13` { t.Skip("Unsupported version") } @@ -392,7 +383,7 @@ func getGrowing(m aggPartialResultMapper) bool { return value.oldbuckets != nil } -func SubTestFilterTemporaryTableKeys(t *testing.T) { +func TestFilterTemporaryTableKeys(t *testing.T) { vars := variable.NewSessionVars() const tableID int64 = 3 vars.TxnCtx = &variable.TransactionContext{ From 53b5cb6c9ff861883070add5b44f0f98690053ac Mon Sep 17 00:00:00 2001 From: disksing Date: Sun, 23 Jan 2022 12:35:46 +0800 Subject: [PATCH 13/21] ddl: fix data race and goroutine leak in test (#31896) close pingcap/tidb#31875 --- ddl/ddl_test.go | 2 -- ddl/ddl_tiflash_test.go | 17 +++++++++++++--- ddl/main_test.go | 35 ++++++++++++++++++++++++++++++++ ddl/options_test.go | 4 ++++ ddl/partition_test.go | 4 ++++ ddl/placement_policy_ddl_test.go | 4 ++++ ddl/table_test.go | 16 +++++++++++++++ 7 files changed, 77 insertions(+), 5 deletions(-) create mode 100644 ddl/main_test.go diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index ddec7f9fd01b9..1e0e7a210aa23 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -37,7 +37,6 @@ import ( "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" "github.com/stretchr/testify/require" - "github.com/tikv/client-go/v2/tikv" ) type DDLForTest interface { @@ -85,7 +84,6 @@ func TestT(t *testing.T) { conf.TiKVClient.AsyncCommit.AllowedClockDrift = 0 conf.Experimental.AllowsExpressionIndex = true }) - tikv.EnableFailpoints() _, err = infosync.GlobalInfoSyncerInit(context.Background(), "t", func() uint64 { return 1 }, nil, true) if err != nil { diff --git a/ddl/ddl_tiflash_test.go b/ddl/ddl_tiflash_test.go index ccbe96a8a3780..7569a0a19c17d 100644 --- a/ddl/ddl_tiflash_test.go +++ b/ddl/ddl_tiflash_test.go @@ -223,7 +223,13 @@ func (s *tiflashDDLTestSuite) SetPdLoop(tick int) func() { // Run all kinds of DDLs, and will create no redundant pd rules for TiFlash. func (s *tiflashDDLTestSuite) TestTiFlashNoRedundantPDRules(c *C) { - _, _, cluster, _ := unistore.New("") + rpcClient, pdClient, cluster, err := unistore.New("") + c.Assert(err, IsNil) + defer func() { + rpcClient.Close() + pdClient.Close() + cluster.Close() + }() for _, store := range s.cluster.GetAllStores() { cluster.AddStore(store.Id, store.Address, store.Labels...) } @@ -535,7 +541,13 @@ func (s *tiflashDDLTestSuite) TestSetPlacementRuleNormal(c *C) { // When gc worker works, it will automatically remove pd rule for TiFlash. func (s *tiflashDDLTestSuite) TestSetPlacementRuleWithGCWorker(c *C) { - _, _, cluster, err := unistore.New("") + rpcClient, pdClient, cluster, err := unistore.New("") + c.Assert(err, IsNil) + defer func() { + rpcClient.Close() + pdClient.Close() + cluster.Close() + }() for _, store := range s.cluster.GetAllStores() { cluster.AddStore(store.Id, store.Address, store.Labels...) } @@ -545,7 +557,6 @@ func (s *tiflashDDLTestSuite) TestSetPlacementRuleWithGCWorker(c *C) { }() fCancelPD := s.SetPdLoop(10000) defer fCancelPD() - c.Assert(err, IsNil) gcWorker, err := gcworker.NewMockGCWorker(s.store) c.Assert(err, IsNil) // Make SetPdLoop take effects. diff --git a/ddl/main_test.go b/ddl/main_test.go new file mode 100644 index 0000000000000..9cd5e3f8f3d21 --- /dev/null +++ b/ddl/main_test.go @@ -0,0 +1,35 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl_test + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "github.com/tikv/client-go/v2/tikv" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.SetupForCommonTest() + tikv.EnableFailpoints() + + opts := []goleak.Option{ + goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + + goleak.VerifyTestMain(m, opts...) +} diff --git a/ddl/options_test.go b/ddl/options_test.go index 1fe257072eaab..ed9b9b89e087d 100644 --- a/ddl/options_test.go +++ b/ddl/options_test.go @@ -28,6 +28,10 @@ import ( func TestOptions(t *testing.T) { client, err := clientv3.NewFromURL("test") require.NoError(t, err) + defer func() { + err := client.Close() + require.NoError(t, err) + }() callback := &ddl.BaseCallback{} lease := time.Second * 3 store := &mock.Store{} diff --git a/ddl/partition_test.go b/ddl/partition_test.go index 68a55efc75d9c..66b3565b18a45 100644 --- a/ddl/partition_test.go +++ b/ddl/partition_test.go @@ -27,6 +27,10 @@ import ( func ExportTestDropAndTruncatePartition(t *testing.T) { store := testCreateStoreT(t, "test_store") + defer func() { + err := store.Close() + require.NoError(t, err) + }() d, err := testNewDDLAndStart( context.Background(), WithStore(store), diff --git a/ddl/placement_policy_ddl_test.go b/ddl/placement_policy_ddl_test.go index 00c6237e29f35..3ffb6986f3e5a 100644 --- a/ddl/placement_policy_ddl_test.go +++ b/ddl/placement_policy_ddl_test.go @@ -63,6 +63,10 @@ func (s *testDDLSuiteToVerify) TestPlacementPolicyInUse() { ctx := context.Background() d, err := testNewDDLAndStart(ctx, WithStore(store)) require.NoError(s.T(), err) + defer func() { + err := d.Stop() + require.NoError(s.T(), err) + }() sctx := testNewContext(d) db1, err := testSchemaInfo(d, "db1") diff --git a/ddl/table_test.go b/ddl/table_test.go index 3f5d100df2250..d7e2650ff5ad9 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -348,6 +348,10 @@ func testAlterNoCacheTable(t *testing.T, ctx sessionctx.Context, d *ddl, newSche func ExportTestRenameTables(t *testing.T) { store, err := mockstore.NewMockStore() + defer func() { + err := store.Close() + require.NoError(t, err) + }() require.NoError(t, err) ddl, err := testNewDDLAndStart( context.Background(), @@ -355,6 +359,10 @@ func ExportTestRenameTables(t *testing.T) { WithLease(testLease), ) require.NoError(t, err) + defer func() { + err := ddl.Stop() + require.NoError(t, err) + }() dbInfo, err := testSchemaInfo(ddl, "test_table") require.NoError(t, err) @@ -397,12 +405,20 @@ func ExportTestRenameTables(t *testing.T) { func TestCreateTables(t *testing.T) { store, err := mockstore.NewMockStore() require.NoError(t, err) + defer func() { + err := store.Close() + require.NoError(t, err) + }() ddl, err := testNewDDLAndStart( context.Background(), WithStore(store), WithLease(testLease), ) require.NoError(t, err) + defer func() { + err := ddl.Stop() + require.NoError(t, err) + }() dbInfo, err := testSchemaInfo(ddl, "test_table") require.NoError(t, err) From 701f126b27f041829413a5a6ab22b610846ad60b Mon Sep 17 00:00:00 2001 From: tison Date: Sun, 23 Jan 2022 13:19:46 +0800 Subject: [PATCH 14/21] br/pkg/backup: migrate test-infra to testify (#31288) close pingcap/tidb#28164 --- br/pkg/backup/client_test.go | 251 ++++++++++++++++++----------------- br/pkg/backup/main_test.go | 37 ++++++ br/pkg/backup/schema_test.go | 236 ++++++++++++++++---------------- br/pkg/pdutil/pd.go | 4 +- 4 files changed, 287 insertions(+), 241 deletions(-) create mode 100644 br/pkg/backup/main_test.go diff --git a/br/pkg/backup/client_test.go b/br/pkg/backup/client_test.go index e341f15417f55..e4799b3192c33 100644 --- a/br/pkg/backup/client_test.go +++ b/br/pkg/backup/client_test.go @@ -10,7 +10,6 @@ import ( "time" "github.com/golang/protobuf/proto" - . "github.com/pingcap/check" backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/kvproto/pkg/encryptionpb" "github.com/pingcap/kvproto/pkg/errorpb" @@ -23,9 +22,10 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/testkit" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" @@ -44,92 +44,88 @@ type testBackup struct { storage storage.ExternalStorage } -var _ = Suite(&testBackup{}) - -func TestT(t *testing.T) { - TestingT(t) -} - -func (r *testBackup) SetUpSuite(c *C) { +func createBackupSuite(t *testing.T) (s *testBackup, clean func()) { _, _, pdClient, err := testutils.NewMockTiKV("", nil) - c.Assert(err, IsNil) - r.mockPDClient = pdClient - r.ctx, r.cancel = context.WithCancel(context.Background()) + require.NoError(t, err) + s = new(testBackup) + s.mockPDClient = pdClient + s.ctx, s.cancel = context.WithCancel(context.Background()) mockMgr := &conn.Mgr{PdController: &pdutil.PdController{}} - mockMgr.SetPDClient(r.mockPDClient) + mockMgr.SetPDClient(s.mockPDClient) mockMgr.SetHTTP([]string{"test"}, nil) - r.backupClient, err = backup.NewBackupClient(r.ctx, mockMgr) - c.Assert(err, IsNil) - - r.cluster, err = mock.NewCluster() - c.Assert(err, IsNil) - base := c.MkDir() - r.storage, err = storage.NewLocalStorage(base) - c.Assert(err, IsNil) - //c.Assert(r.cluster.Start(), IsNil) - + s.backupClient, err = backup.NewBackupClient(s.ctx, mockMgr) + require.NoError(t, err) + + s.cluster, err = mock.NewCluster() + require.NoError(t, err) + base := t.TempDir() + s.storage, err = storage.NewLocalStorage(base) + require.NoError(t, err) + require.NoError(t, s.cluster.Start()) + + clean = func() { + mockMgr.Close() + s.cluster.Stop() + } + return } -func (r *testBackup) resetStorage(c *C) { - var err error - base := c.MkDir() - r.storage, err = storage.NewLocalStorage(base) - c.Assert(err, IsNil) -} +func TestGetTS(t *testing.T) { + s, clean := createBackupSuite(t) + defer clean() -func (r *testBackup) TestGetTS(c *C) { - var ( - err error - // mockPDClient' physical ts and current ts will have deviation - // so make this deviation tolerance 100ms - deviation = 100 - ) + // mockPDClient' physical ts and current ts will have deviation + // so make this deviation tolerance 100ms + deviation := 100 // timeago not work expectedDuration := 0 currentTS := time.Now().UnixNano() / int64(time.Millisecond) - ts, err := r.backupClient.GetTS(r.ctx, 0, 0) - c.Assert(err, IsNil) + ts, err := s.backupClient.GetTS(s.ctx, 0, 0) + require.NoError(t, err) pdTS := oracle.ExtractPhysical(ts) duration := int(currentTS - pdTS) - c.Assert(duration, Greater, expectedDuration-deviation) - c.Assert(duration, Less, expectedDuration+deviation) + require.Greater(t, duration, expectedDuration-deviation) + require.Less(t, duration, expectedDuration+deviation) // timeago = "1.5m" expectedDuration = 90000 currentTS = time.Now().UnixNano() / int64(time.Millisecond) - ts, err = r.backupClient.GetTS(r.ctx, 90*time.Second, 0) - c.Assert(err, IsNil) + ts, err = s.backupClient.GetTS(s.ctx, 90*time.Second, 0) + require.NoError(t, err) pdTS = oracle.ExtractPhysical(ts) duration = int(currentTS - pdTS) - c.Assert(duration, Greater, expectedDuration-deviation) - c.Assert(duration, Less, expectedDuration+deviation) + require.Greater(t, duration, expectedDuration-deviation) + require.Less(t, duration, expectedDuration+deviation) // timeago = "-1m" - _, err = r.backupClient.GetTS(r.ctx, -time.Minute, 0) - c.Assert(err, ErrorMatches, "negative timeago is not allowed.*") + _, err = s.backupClient.GetTS(s.ctx, -time.Minute, 0) + require.Error(t, err) + require.Regexp(t, "negative timeago is not allowed.*", err.Error()) // timeago = "1000000h" overflows - _, err = r.backupClient.GetTS(r.ctx, 1000000*time.Hour, 0) - c.Assert(err, ErrorMatches, ".*backup ts overflow.*") + _, err = s.backupClient.GetTS(s.ctx, 1000000*time.Hour, 0) + require.Error(t, err) + require.Regexp(t, ".*backup ts overflow.*", err.Error()) // timeago = "10h" exceed GCSafePoint - p, l, err := r.mockPDClient.GetTS(r.ctx) - c.Assert(err, IsNil) + p, l, err := s.mockPDClient.GetTS(s.ctx) + require.NoError(t, err) now := oracle.ComposeTS(p, l) - _, err = r.mockPDClient.UpdateGCSafePoint(r.ctx, now) - c.Assert(err, IsNil) - _, err = r.backupClient.GetTS(r.ctx, 10*time.Hour, 0) - c.Assert(err, ErrorMatches, ".*GC safepoint [0-9]+ exceed TS [0-9]+.*") + _, err = s.mockPDClient.UpdateGCSafePoint(s.ctx, now) + require.NoError(t, err) + _, err = s.backupClient.GetTS(s.ctx, 10*time.Hour, 0) + require.Error(t, err) + require.Regexp(t, ".*GC safepoint [0-9]+ exceed TS [0-9]+.*", err.Error()) // timeago and backupts both exists, use backupts backupts := oracle.ComposeTS(p+10, l) - ts, err = r.backupClient.GetTS(r.ctx, time.Minute, backupts) - c.Assert(err, IsNil) - c.Assert(ts, Equals, backupts) + ts, err = s.backupClient.GetTS(s.ctx, time.Minute, backupts) + require.NoError(t, err) + require.Equal(t, backupts, ts) } -func (r *testBackup) TestBuildTableRangeIntHandle(c *C) { +func TestBuildTableRangeIntHandle(t *testing.T) { type Case struct { ids []int64 trs []kv.KeyRange @@ -151,34 +147,34 @@ func (r *testBackup) TestBuildTableRangeIntHandle(c *C) { }}, } for _, cs := range cases { - c.Log(cs) + t.Log(cs) tbl := &model.TableInfo{Partition: &model.PartitionInfo{Enable: true}} for _, id := range cs.ids { tbl.Partition.Definitions = append(tbl.Partition.Definitions, model.PartitionDefinition{ID: id}) } ranges, err := backup.BuildTableRanges(tbl) - c.Assert(err, IsNil) - c.Assert(ranges, DeepEquals, cs.trs) + require.NoError(t, err) + require.Equal(t, cs.trs, ranges) } tbl := &model.TableInfo{ID: 7} ranges, err := backup.BuildTableRanges(tbl) - c.Assert(err, IsNil) - c.Assert(ranges, DeepEquals, []kv.KeyRange{ + require.NoError(t, err) + require.Equal(t, []kv.KeyRange{ {StartKey: tablecodec.EncodeRowKey(7, low), EndKey: tablecodec.EncodeRowKey(7, high)}, - }) + }, ranges) } -func (r *testBackup) TestBuildTableRangeCommonHandle(c *C) { +func TestBuildTableRangeCommonHandle(t *testing.T) { type Case struct { ids []int64 trs []kv.KeyRange } low, err_l := codec.EncodeKey(nil, nil, []types.Datum{types.MinNotNullDatum()}...) - c.Assert(err_l, IsNil) + require.NoError(t, err_l) high, err_h := codec.EncodeKey(nil, nil, []types.Datum{types.MaxValueDatum()}...) - c.Assert(err_h, IsNil) + require.NoError(t, err_h) high = kv.Key(high).PrefixNext() cases := []Case{ {ids: []int64{1}, trs: []kv.KeyRange{ @@ -195,26 +191,26 @@ func (r *testBackup) TestBuildTableRangeCommonHandle(c *C) { }}, } for _, cs := range cases { - c.Log(cs) + t.Log(cs) tbl := &model.TableInfo{Partition: &model.PartitionInfo{Enable: true}, IsCommonHandle: true} for _, id := range cs.ids { tbl.Partition.Definitions = append(tbl.Partition.Definitions, model.PartitionDefinition{ID: id}) } ranges, err := backup.BuildTableRanges(tbl) - c.Assert(err, IsNil) - c.Assert(ranges, DeepEquals, cs.trs) + require.NoError(t, err) + require.Equal(t, cs.trs, ranges) } tbl := &model.TableInfo{ID: 7, IsCommonHandle: true} ranges, err_r := backup.BuildTableRanges(tbl) - c.Assert(err_r, IsNil) - c.Assert(ranges, DeepEquals, []kv.KeyRange{ + require.NoError(t, err_r) + require.Equal(t, []kv.KeyRange{ {StartKey: tablecodec.EncodeRowKey(7, low), EndKey: tablecodec.EncodeRowKey(7, high)}, - }) + }, ranges) } -func (r *testBackup) TestOnBackupRegionErrorResponse(c *C) { +func TestOnBackupRegionErrorResponse(t *testing.T) { type Case struct { storeID uint64 bo *tikv.Backoffer @@ -241,18 +237,21 @@ func (r *testBackup) TestOnBackupRegionErrorResponse(c *C) { {storeID: 1, backupTS: 421123291611137, resp: newBackupRegionErrorResp(&errorpb.Error{ProposalInMergingMode: &errorpb.ProposalInMergingMode{}}), exceptedBackoffMs: 1000, exceptedErr: false}, } for _, cs := range cases { - c.Log(cs) + t.Log(cs) _, backoffMs, err := backup.OnBackupResponse(cs.storeID, cs.bo, cs.backupTS, cs.lockResolver, cs.resp) - c.Assert(backoffMs, Equals, cs.exceptedBackoffMs) + require.Equal(t, cs.exceptedBackoffMs, backoffMs) if cs.exceptedErr { - c.Assert(err, NotNil) + require.Error(t, err) } else { - c.Assert(err, IsNil) + require.NoError(t, err) } } } -func (r *testBackup) TestSendCreds(c *C) { +func TestSendCreds(t *testing.T) { + s, clean := createBackupSuite(t) + defer clean() + accessKey := "ab" secretAccessKey := "cd" backendOpt := storage.BackendOptions{ @@ -262,16 +261,16 @@ func (r *testBackup) TestSendCreds(c *C) { }, } backend, err := storage.ParseBackend("s3://bucket/prefix/", &backendOpt) - c.Assert(err, IsNil) + require.NoError(t, err) opts := &storage.ExternalStorageOptions{ SendCredentials: true, } - _, err = storage.New(r.ctx, backend, opts) - c.Assert(err, IsNil) + _, err = storage.New(s.ctx, backend, opts) + require.NoError(t, err) access_key := backend.GetS3().AccessKey - c.Assert(access_key, Equals, "ab") + require.Equal(t, "ab", access_key) secret_access_key := backend.GetS3().SecretAccessKey - c.Assert(secret_access_key, Equals, "cd") + require.Equal(t, "cd", secret_access_key) backendOpt = storage.BackendOptions{ S3: storage.S3BackendOptions{ @@ -280,24 +279,27 @@ func (r *testBackup) TestSendCreds(c *C) { }, } backend, err = storage.ParseBackend("s3://bucket/prefix/", &backendOpt) - c.Assert(err, IsNil) + require.NoError(t, err) opts = &storage.ExternalStorageOptions{ SendCredentials: false, } - _, err = storage.New(r.ctx, backend, opts) - c.Assert(err, IsNil) + _, err = storage.New(s.ctx, backend, opts) + require.NoError(t, err) access_key = backend.GetS3().AccessKey - c.Assert(access_key, Equals, "") + require.Equal(t, "", access_key) secret_access_key = backend.GetS3().SecretAccessKey - c.Assert(secret_access_key, Equals, "") + require.Equal(t, "", secret_access_key) } -func (r *testBackup) TestskipUnsupportedDDLJob(c *C) { - tk := testkit.NewTestKit(c, r.cluster.Storage) +func TestSkipUnsupportedDDLJob(t *testing.T) { + s, clean := createBackupSuite(t) + defer clean() + + tk := testkit.NewTestKit(t, s.cluster.Storage) tk.MustExec("CREATE DATABASE IF NOT EXISTS test_db;") tk.MustExec("CREATE TABLE IF NOT EXISTS test_db.test_table (c1 INT);") - lastTS, err := r.cluster.GetOracle().GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) - c.Assert(err, IsNil, Commentf("Error get last ts: %s", err)) + lastTS, err := s.cluster.GetOracle().GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + require.NoErrorf(t, err, "Error get last ts: %s", err) tk.MustExec("RENAME TABLE test_db.test_table to test_db.test_table1;") tk.MustExec("DROP TABLE test_db.test_table1;") tk.MustExec("DROP DATABASE test_db;") @@ -312,58 +314,61 @@ func (r *testBackup) TestskipUnsupportedDDLJob(c *C) { tk.MustExec("ALTER TABLE tb attributes \"merge_option=allow\"") tk.MustExec("ALTER TABLE tb PARTITION p0 attributes \"merge_option=deny\"") - ts, err := r.cluster.GetOracle().GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) - c.Assert(err, IsNil, Commentf("Error get ts: %s", err)) + ts, err := s.cluster.GetOracle().GetTimestamp(context.Background(), &oracle.Option{TxnScope: oracle.GlobalTxnScope}) + require.NoErrorf(t, err, "Error get ts: %s", err) cipher := backuppb.CipherInfo{CipherType: encryptionpb.EncryptionMethod_PLAINTEXT} - metaWriter := metautil.NewMetaWriter(r.storage, metautil.MetaFileSize, false, &cipher) + metaWriter := metautil.NewMetaWriter(s.storage, metautil.MetaFileSize, false, &cipher) ctx := context.Background() metaWriter.StartWriteMetasAsync(ctx, metautil.AppendDDL) - err = backup.WriteBackupDDLJobs(metaWriter, r.cluster.Storage, lastTS, ts) - c.Assert(err, IsNil, Commentf("Error get ddl jobs: %s", err)) + err = backup.WriteBackupDDLJobs(metaWriter, s.cluster.Storage, lastTS, ts) + require.NoErrorf(t, err, "Error get ddl jobs: %s", err) err = metaWriter.FinishWriteMetas(ctx, metautil.AppendDDL) - c.Assert(err, IsNil, Commentf("Flush failed", err)) + require.NoError(t, err, "Flush failed", err) err = metaWriter.FlushBackupMeta(ctx) - c.Assert(err, IsNil, Commentf("Finially flush backupmeta failed", err)) + require.NoError(t, err, "Finally flush backup meta failed", err) - metaBytes, err := r.storage.ReadFile(ctx, metautil.MetaFile) - c.Assert(err, IsNil) + metaBytes, err := s.storage.ReadFile(ctx, metautil.MetaFile) + require.NoError(t, err) mockMeta := &backuppb.BackupMeta{} err = proto.Unmarshal(metaBytes, mockMeta) - c.Assert(err, IsNil) + require.NoError(t, err) // check the schema version - metaReader := metautil.NewMetaReader(mockMeta, r.storage, &cipher) + metaReader := metautil.NewMetaReader(mockMeta, s.storage, &cipher) allDDLJobsBytes, err := metaReader.ReadDDLs(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) var allDDLJobs []*model.Job err = json.Unmarshal(allDDLJobsBytes, &allDDLJobs) - c.Assert(err, IsNil) - c.Assert(len(allDDLJobs), Equals, 8) + require.NoError(t, err) + require.Len(t, allDDLJobs, 8) } -func (r *testBackup) TestCheckBackupIsLocked(c *C) { +func TestCheckBackupIsLocked(t *testing.T) { + s, clean := createBackupSuite(t) + defer clean() + ctx := context.Background() - r.resetStorage(c) // check passed with an empty storage - err := backup.CheckBackupStorageIsLocked(ctx, r.storage) - c.Assert(err, IsNil) + err := backup.CheckBackupStorageIsLocked(ctx, s.storage) + require.NoError(t, err) // check passed with only a lock file - err = r.storage.WriteFile(ctx, metautil.LockFile, nil) - c.Assert(err, IsNil) - err = backup.CheckBackupStorageIsLocked(ctx, r.storage) - c.Assert(err, IsNil) + err = s.storage.WriteFile(ctx, metautil.LockFile, nil) + require.NoError(t, err) + err = backup.CheckBackupStorageIsLocked(ctx, s.storage) + require.NoError(t, err) // check passed with a lock file and other non-sst files. - err = r.storage.WriteFile(ctx, "1.txt", nil) - c.Assert(err, IsNil) - err = backup.CheckBackupStorageIsLocked(ctx, r.storage) - c.Assert(err, IsNil) + err = s.storage.WriteFile(ctx, "1.txt", nil) + require.NoError(t, err) + err = backup.CheckBackupStorageIsLocked(ctx, s.storage) + require.NoError(t, err) // check failed - err = r.storage.WriteFile(ctx, "1.sst", nil) - c.Assert(err, IsNil) - err = backup.CheckBackupStorageIsLocked(ctx, r.storage) - c.Assert(err, ErrorMatches, "backup lock file and sst file exist in(.+)") + err = s.storage.WriteFile(ctx, "1.sst", nil) + require.NoError(t, err) + err = backup.CheckBackupStorageIsLocked(ctx, s.storage) + require.Error(t, err) + require.Regexp(t, "backup lock file and sst file exist in(.+)", err.Error()) } diff --git a/br/pkg/backup/main_test.go b/br/pkg/backup/main_test.go new file mode 100644 index 0000000000000..ee9d4be00aca6 --- /dev/null +++ b/br/pkg/backup/main_test.go @@ -0,0 +1,37 @@ +// 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package backup + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.SetupForCommonTest() + opts := []goleak.Option{ + goleak.IgnoreTopFunction("github.com/klauspost/compress/zstd.(*blockDec).startDecoder"), + goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb.(*DB).compactionError"), + goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb.(*DB).mCompaction"), + goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb.(*DB).mpoolDrain"), + goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb.(*DB).tCompaction"), + goleak.IgnoreTopFunction("github.com/pingcap/goleveldb/leveldb/util.(*BufferPool).drain"), + goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + } + goleak.VerifyTestMain(m, opts...) +} diff --git a/br/pkg/backup/schema_test.go b/br/pkg/backup/schema_test.go index c858d556f98d0..dc4038bf98051 100644 --- a/br/pkg/backup/schema_test.go +++ b/br/pkg/backup/schema_test.go @@ -8,9 +8,9 @@ import ( "math" "strings" "sync/atomic" + "testing" "github.com/golang/protobuf/proto" - . "github.com/pingcap/check" backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/kvproto/pkg/encryptionpb" filter "github.com/pingcap/tidb-tools/pkg/table-filter" @@ -20,42 +20,35 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -var _ = Suite(&testBackupSchemaSuite{}) - -type testBackupSchemaSuite struct { - mock *mock.Cluster -} - -func (s *testBackupSchemaSuite) SetUpSuite(c *C) { +func createMockCluster(t *testing.T) (m *mock.Cluster, clean func()) { var err error - s.mock, err = mock.NewCluster() - c.Assert(err, IsNil) - c.Assert(s.mock.Start(), IsNil) -} - -func (s *testBackupSchemaSuite) TearDownSuite(c *C) { - s.mock.Stop() - testleak.AfterTest(c)() + m, err = mock.NewCluster() + require.NoError(t, err) + require.NoError(t, m.Start()) + clean = func() { + m.Stop() + } + return } -func (s *testBackupSchemaSuite) GetRandomStorage(c *C) storage.ExternalStorage { - base := c.MkDir() +func GetRandomStorage(t *testing.T) storage.ExternalStorage { + base := t.TempDir() es, err := storage.NewLocalStorage(base) - c.Assert(err, IsNil) + require.NoError(t, err) return es } -func (s *testBackupSchemaSuite) GetSchemasFromMeta(c *C, es storage.ExternalStorage) []*metautil.Table { +func GetSchemasFromMeta(t *testing.T, es storage.ExternalStorage) []*metautil.Table { ctx := context.Background() metaBytes, err := es.ReadFile(ctx, metautil.MetaFile) - c.Assert(err, IsNil) + require.NoError(t, err) mockMeta := &backuppb.BackupMeta{} err = proto.Unmarshal(metaBytes, mockMeta) - c.Assert(err, IsNil) + require.NoError(t, err) metaReader := metautil.NewMetaReader(mockMeta, es, &backuppb.CipherInfo{ @@ -66,7 +59,7 @@ func (s *testBackupSchemaSuite) GetSchemasFromMeta(c *C, es storage.ExternalStor output := make(chan *metautil.Table, 4) go func() { err = metaReader.ReadSchemasFiles(ctx, output) - c.Assert(err, IsNil) + require.NoError(t, err) close(output) }() @@ -95,33 +88,36 @@ func (sp *simpleProgress) get() int64 { return atomic.LoadInt64(&sp.counter) } -func (s *testBackupSchemaSuite) TestBuildBackupRangeAndSchema(c *C) { - tk := testkit.NewTestKit(c, s.mock.Storage) +func TestBuildBackupRangeAndSchema(t *testing.T) { + m, clean := createMockCluster(t) + defer clean() + + tk := testkit.NewTestKit(t, m.Storage) // Table t1 is not exist. testFilter, err := filter.Parse([]string{"test.t1"}) - c.Assert(err, IsNil) + require.NoError(t, err) _, backupSchemas, err := backup.BuildBackupRangeAndSchema( - s.mock.Storage, testFilter, math.MaxUint64) - c.Assert(err, IsNil) - c.Assert(backupSchemas, IsNil) + m.Storage, testFilter, math.MaxUint64) + require.NoError(t, err) + require.Nil(t, backupSchemas) // Database is not exist. fooFilter, err := filter.Parse([]string{"foo.t1"}) - c.Assert(err, IsNil) + require.NoError(t, err) _, backupSchemas, err = backup.BuildBackupRangeAndSchema( - s.mock.Storage, fooFilter, math.MaxUint64) - c.Assert(err, IsNil) - c.Assert(backupSchemas, IsNil) + m.Storage, fooFilter, math.MaxUint64) + require.NoError(t, err) + require.Nil(t, backupSchemas) // Empty database. // Filter out system tables manually. noFilter, err := filter.Parse([]string{"*.*", "!mysql.*"}) - c.Assert(err, IsNil) + require.NoError(t, err) _, backupSchemas, err = backup.BuildBackupRangeAndSchema( - s.mock.Storage, noFilter, math.MaxUint64) - c.Assert(err, IsNil) - c.Assert(backupSchemas, IsNil) + m.Storage, noFilter, math.MaxUint64) + require.NoError(t, err) + require.Nil(t, backupSchemas) tk.MustExec("use test") tk.MustExec("drop table if exists t1;") @@ -129,30 +125,30 @@ func (s *testBackupSchemaSuite) TestBuildBackupRangeAndSchema(c *C) { tk.MustExec("insert into t1 values (10);") _, backupSchemas, err = backup.BuildBackupRangeAndSchema( - s.mock.Storage, testFilter, math.MaxUint64) - c.Assert(err, IsNil) - c.Assert(backupSchemas.Len(), Equals, 1) + m.Storage, testFilter, math.MaxUint64) + require.NoError(t, err) + require.Equal(t, 1, backupSchemas.Len()) updateCh := new(simpleProgress) skipChecksum := false - es := s.GetRandomStorage(c) + es := GetRandomStorage(t) cipher := backuppb.CipherInfo{ CipherType: encryptionpb.EncryptionMethod_PLAINTEXT, } metaWriter := metautil.NewMetaWriter(es, metautil.MetaFileSize, false, &cipher) ctx := context.Background() err = backupSchemas.BackupSchemas( - ctx, metaWriter, s.mock.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) - c.Assert(updateCh.get(), Equals, int64(1)) - c.Assert(err, IsNil) + ctx, metaWriter, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + require.Equal(t, int64(1), updateCh.get()) + require.NoError(t, err) err = metaWriter.FlushBackupMeta(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) - schemas := s.GetSchemasFromMeta(c, es) - c.Assert(len(schemas), Equals, 1) + schemas := GetSchemasFromMeta(t, es) + require.Len(t, schemas, 1) // Cluster returns a dummy checksum (all fields are 1). - c.Assert(schemas[0].Crc64Xor, Not(Equals), 0, Commentf("%v", schemas[0])) - c.Assert(schemas[0].TotalKvs, Not(Equals), 0, Commentf("%v", schemas[0])) - c.Assert(schemas[0].TotalBytes, Not(Equals), 0, Commentf("%v", schemas[0])) + require.NotZerof(t, schemas[0].Crc64Xor, "%v", schemas[0]) + require.NotZerof(t, schemas[0].TotalKvs, "%v", schemas[0]) + require.NotZerof(t, schemas[0].TotalBytes, "%v", schemas[0]) tk.MustExec("drop table if exists t2;") tk.MustExec("create table t2 (a int);") @@ -160,34 +156,37 @@ func (s *testBackupSchemaSuite) TestBuildBackupRangeAndSchema(c *C) { tk.MustExec("insert into t2 values (11);") _, backupSchemas, err = backup.BuildBackupRangeAndSchema( - s.mock.Storage, noFilter, math.MaxUint64) - c.Assert(err, IsNil) - c.Assert(backupSchemas.Len(), Equals, 2) + m.Storage, noFilter, math.MaxUint64) + require.NoError(t, err) + require.Equal(t, 2, backupSchemas.Len()) updateCh.reset() - es2 := s.GetRandomStorage(c) + es2 := GetRandomStorage(t) metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, &cipher) err = backupSchemas.BackupSchemas( - ctx, metaWriter2, s.mock.Storage, nil, math.MaxUint64, 2, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) - c.Assert(updateCh.get(), Equals, int64(2)) - c.Assert(err, IsNil) + ctx, metaWriter2, m.Storage, nil, math.MaxUint64, 2, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + require.Equal(t, int64(2), updateCh.get()) + require.NoError(t, err) err = metaWriter2.FlushBackupMeta(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) - schemas = s.GetSchemasFromMeta(c, es2) + schemas = GetSchemasFromMeta(t, es2) - c.Assert(len(schemas), Equals, 2) + require.Len(t, schemas, 2) // Cluster returns a dummy checksum (all fields are 1). - c.Assert(schemas[0].Crc64Xor, Not(Equals), 0, Commentf("%v", schemas[0])) - c.Assert(schemas[0].TotalKvs, Not(Equals), 0, Commentf("%v", schemas[0])) - c.Assert(schemas[0].TotalBytes, Not(Equals), 0, Commentf("%v", schemas[0])) - c.Assert(schemas[1].Crc64Xor, Not(Equals), 0, Commentf("%v", schemas[1])) - c.Assert(schemas[1].TotalKvs, Not(Equals), 0, Commentf("%v", schemas[1])) - c.Assert(schemas[1].TotalBytes, Not(Equals), 0, Commentf("%v", schemas[1])) + require.NotZerof(t, schemas[0].Crc64Xor, "%v", schemas[0]) + require.NotZerof(t, schemas[0].TotalKvs, "%v", schemas[0]) + require.NotZerof(t, schemas[0].TotalBytes, "%v", schemas[0]) + require.NotZerof(t, schemas[1].Crc64Xor, "%v", schemas[1]) + require.NotZerof(t, schemas[1].TotalKvs, "%v", schemas[1]) + require.NotZerof(t, schemas[1].TotalBytes, "%v", schemas[1]) } -func (s *testBackupSchemaSuite) TestBuildBackupRangeAndSchemaWithBrokenStats(c *C) { - tk := testkit.NewTestKit(c, s.mock.Storage) +func TestBuildBackupRangeAndSchemaWithBrokenStats(t *testing.T) { + m, clean := createMockCluster(t) + defer clean() + + tk := testkit.NewTestKit(t, m.Storage) tk.MustExec("use test") tk.MustExec("drop table if exists t3;") tk.MustExec("create table t3 (a char(1));") @@ -203,11 +202,11 @@ func (s *testBackupSchemaSuite) TestBuildBackupRangeAndSchemaWithBrokenStats(c * `) f, err := filter.Parse([]string{"test.t3"}) - c.Assert(err, IsNil) + require.NoError(t, err) - _, backupSchemas, err := backup.BuildBackupRangeAndSchema(s.mock.Storage, f, math.MaxUint64) - c.Assert(err, IsNil) - c.Assert(backupSchemas.Len(), Equals, 1) + _, backupSchemas, err := backup.BuildBackupRangeAndSchema(m.Storage, f, math.MaxUint64) + require.NoError(t, err) + require.Equal(t, 1, backupSchemas.Len()) skipChecksum := false updateCh := new(simpleProgress) @@ -216,57 +215,60 @@ func (s *testBackupSchemaSuite) TestBuildBackupRangeAndSchemaWithBrokenStats(c * CipherType: encryptionpb.EncryptionMethod_PLAINTEXT, } - es := s.GetRandomStorage(c) + es := GetRandomStorage(t) metaWriter := metautil.NewMetaWriter(es, metautil.MetaFileSize, false, &cipher) ctx := context.Background() err = backupSchemas.BackupSchemas( - ctx, metaWriter, s.mock.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) - c.Assert(err, IsNil) + ctx, metaWriter, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + require.NoError(t, err) err = metaWriter.FlushBackupMeta(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) - schemas := s.GetSchemasFromMeta(c, es) - c.Assert(err, IsNil) - c.Assert(schemas, HasLen, 1) + schemas := GetSchemasFromMeta(t, es) + require.NoError(t, err) + require.Len(t, schemas, 1) // the stats should be empty, but other than that everything should be backed up. - c.Assert(schemas[0].Stats, IsNil) - c.Assert(schemas[0].Crc64Xor, Not(Equals), 0) - c.Assert(schemas[0].TotalKvs, Not(Equals), 0) - c.Assert(schemas[0].TotalBytes, Not(Equals), 0) - c.Assert(schemas[0].Info, NotNil) - c.Assert(schemas[0].DB, NotNil) + require.Nil(t, schemas[0].Stats) + require.NotZerof(t, schemas[0].Crc64Xor, "%v", schemas[0]) + require.NotZerof(t, schemas[0].TotalKvs, "%v", schemas[0]) + require.NotZerof(t, schemas[0].TotalBytes, "%v", schemas[0]) + require.NotNil(t, schemas[0].Info) + require.NotNil(t, schemas[0].DB) // recover the statistics. tk.MustExec("analyze table t3;") - _, backupSchemas, err = backup.BuildBackupRangeAndSchema(s.mock.Storage, f, math.MaxUint64) - c.Assert(err, IsNil) - c.Assert(backupSchemas.Len(), Equals, 1) + _, backupSchemas, err = backup.BuildBackupRangeAndSchema(m.Storage, f, math.MaxUint64) + require.NoError(t, err) + require.Equal(t, 1, backupSchemas.Len()) updateCh.reset() - statsHandle := s.mock.Domain.StatsHandle() - es2 := s.GetRandomStorage(c) + statsHandle := m.Domain.StatsHandle() + es2 := GetRandomStorage(t) metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, &cipher) err = backupSchemas.BackupSchemas( - ctx, metaWriter2, s.mock.Storage, statsHandle, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) - c.Assert(err, IsNil) + ctx, metaWriter2, m.Storage, statsHandle, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, skipChecksum, updateCh) + require.NoError(t, err) err = metaWriter2.FlushBackupMeta(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) - schemas2 := s.GetSchemasFromMeta(c, es2) - c.Assert(schemas2, HasLen, 1) + schemas2 := GetSchemasFromMeta(t, es2) + require.Len(t, schemas2, 1) // the stats should now be filled, and other than that the result should be equivalent to the first backup. - c.Assert(schemas2[0].Stats, NotNil) - c.Assert(schemas2[0].Crc64Xor, Equals, schemas[0].Crc64Xor) - c.Assert(schemas2[0].TotalKvs, Equals, schemas[0].TotalKvs) - c.Assert(schemas2[0].TotalBytes, Equals, schemas[0].TotalBytes) - c.Assert(schemas2[0].Info, DeepEquals, schemas[0].Info) - c.Assert(schemas2[0].DB, DeepEquals, schemas[0].DB) + require.NotNil(t, schemas2[0].Stats) + require.Equal(t, schemas[0].Crc64Xor, schemas2[0].Crc64Xor) + require.Equal(t, schemas[0].TotalKvs, schemas2[0].TotalKvs) + require.Equal(t, schemas[0].TotalBytes, schemas2[0].TotalBytes) + require.Equal(t, schemas[0].Info, schemas2[0].Info) + require.Equal(t, schemas[0].DB, schemas2[0].DB) } -func (s *testBackupSchemaSuite) TestBackupSchemasForSystemTable(c *C) { - tk := testkit.NewTestKit(c, s.mock.Storage) - es2 := s.GetRandomStorage(c) +func TestBackupSchemasForSystemTable(t *testing.T) { + m, clean := createMockCluster(t) + defer clean() + + tk := testkit.NewTestKit(t, m.Storage) + es2 := GetRandomStorage(t) systemTablesCount := 32 tablePrefix := "systable" @@ -277,10 +279,10 @@ func (s *testBackupSchemaSuite) TestBackupSchemasForSystemTable(c *C) { } f, err := filter.Parse([]string{"mysql.systable*"}) - c.Assert(err, IsNil) - _, backupSchemas, err := backup.BuildBackupRangeAndSchema(s.mock.Storage, f, math.MaxUint64) - c.Assert(err, IsNil) - c.Assert(backupSchemas.Len(), Equals, systemTablesCount) + require.NoError(t, err) + _, backupSchemas, err := backup.BuildBackupRangeAndSchema(m.Storage, f, math.MaxUint64) + require.NoError(t, err) + require.Equal(t, systemTablesCount, backupSchemas.Len()) ctx := context.Background() cipher := backuppb.CipherInfo{ @@ -289,16 +291,16 @@ func (s *testBackupSchemaSuite) TestBackupSchemasForSystemTable(c *C) { updateCh := new(simpleProgress) metaWriter2 := metautil.NewMetaWriter(es2, metautil.MetaFileSize, false, &cipher) - err = backupSchemas.BackupSchemas(ctx, metaWriter2, s.mock.Storage, nil, + err = backupSchemas.BackupSchemas(ctx, metaWriter2, m.Storage, nil, math.MaxUint64, 1, variable.DefChecksumTableConcurrency, true, updateCh) - c.Assert(err, IsNil) + require.NoError(t, err) err = metaWriter2.FlushBackupMeta(ctx) - c.Assert(err, IsNil) + require.NoError(t, err) - schemas2 := s.GetSchemasFromMeta(c, es2) - c.Assert(schemas2, HasLen, systemTablesCount) + schemas2 := GetSchemasFromMeta(t, es2) + require.Len(t, schemas2, systemTablesCount) for _, schema := range schemas2 { - c.Assert(schema.DB.Name, Equals, utils.TemporaryDBName("mysql")) - c.Assert(strings.HasPrefix(schema.Info.Name.O, tablePrefix), Equals, true) + require.Equal(t, utils.TemporaryDBName("mysql"), schema.DB.Name) + require.Equal(t, true, strings.HasPrefix(schema.Info.Name.O, tablePrefix)) } } diff --git a/br/pkg/pdutil/pd.go b/br/pkg/pdutil/pd.go index 3f4c45d1deefa..a3487187f8b09 100644 --- a/br/pkg/pdutil/pd.go +++ b/br/pkg/pdutil/pd.go @@ -703,7 +703,9 @@ func (p *PdController) doRemoveSchedulersWith( // Close close the connection to pd. func (p *PdController) Close() { p.pdClient.Close() - close(p.schedulerPauseCh) + if p.schedulerPauseCh != nil { + close(p.schedulerPauseCh) + } } // FetchPDVersion get pd version From 505517718040010b5d028f0424040c7f7c094d8a Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sun, 23 Jan 2022 16:53:47 +0800 Subject: [PATCH 15/21] *: clean code (#31903) ref pingcap/tidb#31716 --- br/pkg/lightning/common/util_test.go | 2 +- br/pkg/summary/collector.go | 2 +- sessionctx/binloginfo/binloginfo_test.go | 2 +- store/mockstore/unistore/cophandler/topn.go | 2 +- table/tables/tables_test.go | 2 +- util/checksum/checksum_test.go | 4 ++-- util/tracing/util_test.go | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/br/pkg/lightning/common/util_test.go b/br/pkg/lightning/common/util_test.go index 3915a318b06c5..3668626276938 100644 --- a/br/pkg/lightning/common/util_test.go +++ b/br/pkg/lightning/common/util_test.go @@ -22,7 +22,7 @@ import ( "net/http/httptest" "time" - sqlmock "github.com/DATA-DOG/go-sqlmock" + "github.com/DATA-DOG/go-sqlmock" . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/common" diff --git a/br/pkg/summary/collector.go b/br/pkg/summary/collector.go index 6c82bf54fba25..b0a3735db0c88 100644 --- a/br/pkg/summary/collector.go +++ b/br/pkg/summary/collector.go @@ -51,7 +51,7 @@ type LogCollector interface { type logFunc func(msg string, fields ...zap.Field) -var collector LogCollector = NewLogCollector(log.Info) +var collector = NewLogCollector(log.Info) // InitCollector initilize global collector instance. func InitCollector( // revive:disable-line:flag-parameter diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 6178944ddffa9..0eee0d77eafdc 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -42,7 +42,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" - binlog "github.com/pingcap/tipb/go-binlog" + "github.com/pingcap/tipb/go-binlog" "github.com/stretchr/testify/require" "google.golang.org/grpc" ) diff --git a/store/mockstore/unistore/cophandler/topn.go b/store/mockstore/unistore/cophandler/topn.go index 185317cbec1ee..54c0853a7199a 100644 --- a/store/mockstore/unistore/cophandler/topn.go +++ b/store/mockstore/unistore/cophandler/topn.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" - tipb "github.com/pingcap/tipb/go-tipb" + "github.com/pingcap/tipb/go-tipb" ) type sortRow struct { diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index a3d13a3bfa89e..bc2d8440f7e04 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -36,7 +36,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/testutil" - binlog "github.com/pingcap/tipb/go-binlog" + "github.com/pingcap/tipb/go-binlog" "github.com/stretchr/testify/require" "google.golang.org/grpc" ) diff --git a/util/checksum/checksum_test.go b/util/checksum/checksum_test.go index b3d205954c7ff..57367b6eff37a 100644 --- a/util/checksum/checksum_test.go +++ b/util/checksum/checksum_test.go @@ -479,7 +479,7 @@ func assertUnderlyingWrite(t *testing.T, encrypt bool, f io.WriteCloser, fc func } func underlyingReadAt(f io.ReaderAt, encrypt bool, ctrCipher *encrypt2.CtrCipher, n, off int) error { - var underlying io.ReaderAt = f + var underlying = f if encrypt { underlying = encrypt2.NewReader(underlying, ctrCipher) } @@ -492,7 +492,7 @@ func underlyingReadAt(f io.ReaderAt, encrypt bool, ctrCipher *encrypt2.CtrCipher func assertReadAtFunc(t *testing.T, encrypt bool, ctrCipher *encrypt2.CtrCipher) func(off int64, r []byte, assertErr error, assertN int, assertString string, f io.ReaderAt) { return func(off int64, r []byte, assertErr error, assertN int, assertString string, f io.ReaderAt) { - var underlying io.ReaderAt = f + var underlying = f if encrypt { underlying = encrypt2.NewReader(underlying, ctrCipher) } diff --git a/util/tracing/util_test.go b/util/tracing/util_test.go index 7b54289041723..0e7760713803a 100644 --- a/util/tracing/util_test.go +++ b/util/tracing/util_test.go @@ -18,7 +18,7 @@ import ( "context" "testing" - basictracer "github.com/opentracing/basictracer-go" + "github.com/opentracing/basictracer-go" "github.com/opentracing/opentracing-go" "github.com/pingcap/tidb/util/tracing" "github.com/stretchr/testify/require" From 60f0948fb49985e71702c41d3b5cb6dede4abc58 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Sun, 23 Jan 2022 17:27:46 +0800 Subject: [PATCH 16/21] session: fix data race in the TestUpgradeVersion83 (#31889) close pingcap/tidb#31831 --- session/bootstrap_test.go | 39 +++--------------- session/bootstrap_upgrade_test.go | 57 ++++++++++++++++++++++++++ session/index_usage_sync_lease_test.go | 3 ++ 3 files changed, 65 insertions(+), 34 deletions(-) create mode 100644 session/bootstrap_upgrade_test.go diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index 3f9a47cf5293d..e1921df899f2c 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -32,6 +32,11 @@ import ( "github.com/stretchr/testify/require" ) +// This test file have many problem. +// 1. Please use testkit to create dom, session and store. +// 2. Don't use createStoreAndBootstrap and BootstrapSession together. It will cause data race. +// Please do not add any test here. You can add test case at the bootstrap_update_test.go. After All problem fixed, +// We will overwrite this file by update_test.go. func TestBootstrap(t *testing.T) { store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() @@ -824,40 +829,6 @@ func TestUpgradeVersion75(t *testing.T) { require.Equal(t, "char(255)", strings.ToLower(row.GetString(1))) } -func TestUpgradeVersion83(t *testing.T) { - ctx := context.Background() - store, _ := createStoreAndBootstrap(t) - defer func() { require.NoError(t, store.Close()) }() - - domV83, err := BootstrapSession(store) - require.NoError(t, err) - defer domV83.Close() - seV83 := createSessionAndSetID(t, store) - ver, err := getBootstrapVersion(seV83) - require.NoError(t, err) - require.Equal(t, currentBootstrapVersion, ver) - - statsHistoryTblFields := []struct { - field string - tp string - }{ - {"table_id", "bigint(64)"}, - {"stats_data", "longblob"}, - {"seq_no", "bigint(64)"}, - {"version", "bigint(64)"}, - {"create_time", "datetime(6)"}, - } - rStatsHistoryTbl := mustExec(t, seV83, `desc mysql.stats_history`) - req := rStatsHistoryTbl.NewChunk(nil) - require.NoError(t, rStatsHistoryTbl.Next(ctx, req)) - require.Equal(t, 5, req.NumRows()) - for i := 0; i < 5; i++ { - row := req.GetRow(i) - require.Equal(t, statsHistoryTblFields[i].field, strings.ToLower(row.GetString(0))) - require.Equal(t, statsHistoryTblFields[i].tp, strings.ToLower(row.GetString(1))) - } -} - func TestForIssue23387(t *testing.T) { // For issue https://github.com/pingcap/tidb/issues/23387 saveCurrentBootstrapVersion := currentBootstrapVersion diff --git a/session/bootstrap_upgrade_test.go b/session/bootstrap_upgrade_test.go new file mode 100644 index 0000000000000..815bfebc08b75 --- /dev/null +++ b/session/bootstrap_upgrade_test.go @@ -0,0 +1,57 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package session_test + +import ( + "context" + "strings" + "testing" + + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" +) + +func TestUpgradeVersion83(t *testing.T) { + ctx := context.Background() + store, _, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + ver, err := session.GetBootstrapVersion(tk.Session()) + require.NoError(t, err) + require.Equal(t, session.CurrentBootstrapVersion, ver) + + statsHistoryTblFields := []struct { + field string + tp string + }{ + {"table_id", "bigint(64)"}, + {"stats_data", "longblob"}, + {"seq_no", "bigint(64)"}, + {"version", "bigint(64)"}, + {"create_time", "datetime(6)"}, + } + rStatsHistoryTbl, err := tk.Exec(`desc mysql.stats_history`) + require.NoError(t, err) + req := rStatsHistoryTbl.NewChunk(nil) + require.NoError(t, rStatsHistoryTbl.Next(ctx, req)) + require.Equal(t, 5, req.NumRows()) + for i := 0; i < 5; i++ { + row := req.GetRow(i) + require.Equal(t, statsHistoryTblFields[i].field, strings.ToLower(row.GetString(0))) + require.Equal(t, statsHistoryTblFields[i].tp, strings.ToLower(row.GetString(1))) + } +} diff --git a/session/index_usage_sync_lease_test.go b/session/index_usage_sync_lease_test.go index 53327f45cf308..ede3aca8d01e6 100644 --- a/session/index_usage_sync_lease_test.go +++ b/session/index_usage_sync_lease_test.go @@ -20,6 +20,9 @@ import ( "github.com/stretchr/testify/require" ) +var GetBootstrapVersion = getBootstrapVersion +var CurrentBootstrapVersion = currentBootstrapVersion + func TestIndexUsageSyncLease(t *testing.T) { store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() From afa1335f21895db2f5c28f53a00113b1e5f3baa2 Mon Sep 17 00:00:00 2001 From: tison Date: Mon, 24 Jan 2022 10:45:47 +0800 Subject: [PATCH 17/21] br/pkg/lightning: migrate test-infra to testify (#31910) close pingcap/tidb#28239, close pingcap/tidb#28242, close pingcap/tidb#28243, close pingcap/tidb#28244 --- br/pkg/lightning/metric/metric_test.go | 31 ++---- br/pkg/lightning/tikv/tikv_test.go | 96 ++++++++++++------- .../lightning/verification/checksum_test.go | 39 +++----- br/pkg/lightning/worker/worker_test.go | 35 +++---- 4 files changed, 97 insertions(+), 104 deletions(-) diff --git a/br/pkg/lightning/metric/metric_test.go b/br/pkg/lightning/metric/metric_test.go index 18bc94e8517b2..65c81d8eaa8e8 100644 --- a/br/pkg/lightning/metric/metric_test.go +++ b/br/pkg/lightning/metric/metric_test.go @@ -18,43 +18,32 @@ import ( "errors" "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/lightning/metric" "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" ) -type testMetricSuite struct{} - -func (s *testMetricSuite) SetUpSuite(c *C) {} -func (s *testMetricSuite) TearDownSuite(c *C) {} - -var _ = Suite(&testMetricSuite{}) - -func TestMetric(t *testing.T) { - TestingT(t) -} - -func (s *testMetricSuite) TestReadCounter(c *C) { +func TestReadCounter(t *testing.T) { counter := prometheus.NewCounter(prometheus.CounterOpts{}) counter.Add(1256.0) counter.Add(2214.0) - c.Assert(metric.ReadCounter(counter), Equals, 3470.0) + require.Equal(t, 3470.0, metric.ReadCounter(counter)) } -func (s *testMetricSuite) TestReadHistogramSum(c *C) { +func TestReadHistogramSum(t *testing.T) { histogram := prometheus.NewHistogram(prometheus.HistogramOpts{}) histogram.Observe(11131.5) histogram.Observe(15261.0) - c.Assert(metric.ReadHistogramSum(histogram), Equals, 26392.5) + require.Equal(t, 26392.5, metric.ReadHistogramSum(histogram)) } -func (s *testMetricSuite) TestRecordEngineCount(c *C) { +func TestRecordEngineCount(t *testing.T) { metric.RecordEngineCount("table1", nil) metric.RecordEngineCount("table1", errors.New("mock error")) successCounter, err := metric.ProcessedEngineCounter.GetMetricWithLabelValues("table1", "success") - c.Assert(err, IsNil) - c.Assert(metric.ReadCounter(successCounter), Equals, 1.0) + require.NoError(t, err) + require.Equal(t, 1.0, metric.ReadCounter(successCounter)) failureCount, err := metric.ProcessedEngineCounter.GetMetricWithLabelValues("table1", "failure") - c.Assert(err, IsNil) - c.Assert(metric.ReadCounter(failureCount), Equals, 1.0) + require.NoError(t, err) + require.Equal(t, 1.0, metric.ReadCounter(failureCount)) } diff --git a/br/pkg/lightning/tikv/tikv_test.go b/br/pkg/lightning/tikv/tikv_test.go index f74b20c5ce57e..6c8d7a976f1d4 100644 --- a/br/pkg/lightning/tikv/tikv_test.go +++ b/br/pkg/lightning/tikv/tikv_test.go @@ -1,3 +1,17 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + package tikv_test import ( @@ -9,18 +23,15 @@ import ( "net/url" "sort" "sync" + "testing" "github.com/coreos/go-semver/semver" - . "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/import_sstpb" "github.com/pingcap/tidb/br/pkg/lightning/common" kv "github.com/pingcap/tidb/br/pkg/lightning/tikv" + "github.com/stretchr/testify/require" ) -type tikvSuite struct{} - -var _ = Suite(&tikvSuite{}) - var ( // Samples from importer backend for testing the Check***Version functions. // No need keep these versions in sync. @@ -30,7 +41,7 @@ var ( requiredMaxTiKVVersion = *semver.New("6.0.0") ) -func (s *tikvSuite) TestForAllStores(c *C) { +func TestForAllStores(t *testing.T) { server := httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { _, err := w.Write([]byte(` { @@ -84,7 +95,7 @@ func (s *tikvSuite) TestForAllStores(c *C) { ] } `)) - c.Assert(err, IsNil) + require.NoError(t, err) })) defer server.Close() @@ -100,11 +111,10 @@ func (s *tikvSuite) TestForAllStores(c *C) { allStoresLock.Unlock() return nil }) - c.Assert(err, IsNil) + require.NoError(t, err) sort.Slice(allStores, func(i, j int) bool { return allStores[i].Address < allStores[j].Address }) - - c.Assert(allStores, DeepEquals, []*kv.Store{ + require.Equal(t, []*kv.Store{ { Address: "127.0.0.1:20160", Version: "3.0.0-beta.1", @@ -125,10 +135,10 @@ func (s *tikvSuite) TestForAllStores(c *C) { Version: "3.0.1", State: kv.StoreStateOffline, }, - }) + }, allStores) } -func (s *tikvSuite) TestFetchModeFromMetrics(c *C) { +func TestFetchModeFromMetrics(t *testing.T) { testCases := []struct { metrics string mode import_sstpb.SwitchMode @@ -149,69 +159,77 @@ func (s *tikvSuite) TestFetchModeFromMetrics(c *C) { } for _, tc := range testCases { - comment := Commentf("test case '%s'", tc.metrics) + comment := fmt.Sprintf("test case '%s'", tc.metrics) mode, err := kv.FetchModeFromMetrics(tc.metrics) if tc.isErr { - c.Assert(err, NotNil, comment) + require.Error(t, err, comment) } else { - c.Assert(err, IsNil, comment) - c.Assert(mode, Equals, tc.mode, comment) + require.NoError(t, err, comment) + require.Equal(t, tc.mode, mode, comment) } } } -func (s *tikvSuite) TestCheckPDVersion(c *C) { +func TestCheckPDVersion(t *testing.T) { var version string ctx := context.Background() mockServer := httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { - c.Assert(req.URL.Path, Equals, "/pd/api/v1/version") + require.Equal(t, "/pd/api/v1/version", req.URL.Path) w.WriteHeader(http.StatusOK) _, err := w.Write([]byte(version)) - c.Assert(err, IsNil) + require.NoError(t, err) })) mockURL, err := url.Parse(mockServer.URL) - c.Assert(err, IsNil) + require.NoError(t, err) tls := common.NewTLSFromMockServer(mockServer) version = `{ "version": "v4.0.0-rc.2-451-g760fb650" }` - c.Assert(kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion), IsNil) + require.NoError(t, kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion)) version = `{ "version": "v4.0.0" }` - c.Assert(kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion), IsNil) + require.NoError(t, kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion)) version = `{ "version": "v9999.0.0" }` - c.Assert(kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion), ErrorMatches, "PD version too new.*") + err = kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion) + require.Error(t, err) + require.Regexp(t, "PD version too new.*", err.Error()) version = `{ "version": "v6.0.0" }` - c.Assert(kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion), ErrorMatches, "PD version too new.*") + err = kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion) + require.Error(t, err) + require.Regexp(t, "PD version too new.*", err.Error()) version = `{ "version": "v6.0.0-beta" }` - c.Assert(kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion), ErrorMatches, "PD version too new.*") + err = kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion) + require.Error(t, err) + require.Regexp(t, "PD version too new.*", err.Error()) version = `{ "version": "v1.0.0" }` - c.Assert(kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion), ErrorMatches, "PD version too old.*") + err = kv.CheckPDVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion) + require.Error(t, err) + require.Regexp(t, "PD version too old.*", err.Error()) } -func (s *tikvSuite) TestCheckTiKVVersion(c *C) { +func TestCheckTiKVVersion(t *testing.T) { var versions []string ctx := context.Background() mockServer := httptest.NewTLSServer(http.HandlerFunc(func(w http.ResponseWriter, req *http.Request) { - c.Assert(req.URL.Path, Equals, "/pd/api/v1/stores") + require.Equal(t, "/pd/api/v1/stores", req.URL.Path) w.WriteHeader(http.StatusOK) stores := make([]map[string]interface{}, 0, len(versions)) @@ -227,25 +245,33 @@ func (s *tikvSuite) TestCheckTiKVVersion(c *C) { "count": len(versions), "stores": stores, }) - c.Assert(err, IsNil) + require.NoError(t, err) })) mockURL, err := url.Parse(mockServer.URL) - c.Assert(err, IsNil) + require.NoError(t, err) tls := common.NewTLSFromMockServer(mockServer) versions = []string{"4.1.0", "v4.1.0-alpha-9-ga27a7dd"} - c.Assert(kv.CheckTiKVVersion(ctx, tls, mockURL.Host, requiredMinTiKVVersion, requiredMaxTiKVVersion), IsNil) + require.NoError(t, kv.CheckTiKVVersion(ctx, tls, mockURL.Host, requiredMinTiKVVersion, requiredMaxTiKVVersion)) versions = []string{"9999.0.0", "4.0.0"} - c.Assert(kv.CheckTiKVVersion(ctx, tls, mockURL.Host, requiredMinTiKVVersion, requiredMaxTiKVVersion), ErrorMatches, `TiKV \(at tikv0\.test:20160\) version too new.*`) + err = kv.CheckTiKVVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion) + require.Error(t, err) + require.Regexp(t, `TiKV \(at tikv0\.test:20160\) version too new.*`, err.Error()) versions = []string{"4.0.0", "1.0.0"} - c.Assert(kv.CheckTiKVVersion(ctx, tls, mockURL.Host, requiredMinTiKVVersion, requiredMaxTiKVVersion), ErrorMatches, `TiKV \(at tikv1\.test:20160\) version too old.*`) + err = kv.CheckTiKVVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion) + require.Error(t, err) + require.Regexp(t, `TiKV \(at tikv1\.test:20160\) version too old.*`, err.Error()) versions = []string{"6.0.0"} - c.Assert(kv.CheckTiKVVersion(ctx, tls, mockURL.Host, requiredMinTiKVVersion, requiredMaxTiKVVersion), ErrorMatches, `TiKV \(at tikv0\.test:20160\) version too new.*`) + err = kv.CheckTiKVVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion) + require.Error(t, err) + require.Regexp(t, `TiKV \(at tikv0\.test:20160\) version too new.*`, err.Error()) versions = []string{"6.0.0-beta"} - c.Assert(kv.CheckTiKVVersion(ctx, tls, mockURL.Host, requiredMinTiKVVersion, requiredMaxTiKVVersion), ErrorMatches, `TiKV \(at tikv0\.test:20160\) version too new.*`) + err = kv.CheckTiKVVersion(ctx, tls, mockURL.Host, requiredMinPDVersion, requiredMaxPDVersion) + require.Error(t, err) + require.Regexp(t, `TiKV \(at tikv0\.test:20160\) version too new.*`, err.Error()) } diff --git a/br/pkg/lightning/verification/checksum_test.go b/br/pkg/lightning/verification/checksum_test.go index fb9c714c7a736..58484eecfea16 100644 --- a/br/pkg/lightning/verification/checksum_test.go +++ b/br/pkg/lightning/verification/checksum_test.go @@ -18,34 +18,23 @@ import ( "encoding/json" "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/verification" + "github.com/stretchr/testify/require" ) -type testKVChcksumSuite struct{} - -func (s *testKVChcksumSuite) SetUpSuite(c *C) {} -func (s *testKVChcksumSuite) TearDownSuite(c *C) {} - -var _ = Suite(&testKVChcksumSuite{}) - -func TestKVChcksum(t *testing.T) { - TestingT(t) -} - func uint64NotEqual(a uint64, b uint64) bool { return a != b } -func (s *testKVChcksumSuite) TestChcksum(c *C) { +func TestChcksum(t *testing.T) { checksum := verification.NewKVChecksum(0) - c.Assert(checksum.Sum(), Equals, uint64(0)) + require.Equal(t, uint64(0), checksum.Sum()) // checksum on nothing checksum.Update([]common.KvPair{}) - c.Assert(checksum.Sum(), Equals, uint64(0)) + require.Equal(t, uint64(0), checksum.Sum()) checksum.Update(nil) - c.Assert(checksum.Sum(), Equals, uint64(0)) + require.Equal(t, uint64(0), checksum.Sum()) // checksum on real data excpectChecksum := uint64(4850203904608948940) @@ -67,18 +56,18 @@ func (s *testKVChcksumSuite) TestChcksum(c *C) { for _, kv := range kvs { kvBytes += uint64(len(kv.Key) + len(kv.Val)) } - c.Assert(checksum.SumSize(), Equals, kvBytes) - c.Assert(checksum.SumKVS(), Equals, uint64(len(kvs))) - c.Assert(checksum.Sum(), Equals, excpectChecksum) + require.Equal(t, kvBytes, checksum.SumSize()) + require.Equal(t, uint64(len(kvs)), checksum.SumKVS()) + require.Equal(t, excpectChecksum, checksum.Sum()) // recompute on same key-value checksum.Update(kvs) - c.Assert(checksum.SumSize(), Equals, kvBytes<<1) - c.Assert(checksum.SumKVS(), Equals, uint64(len(kvs))<<1) - c.Assert(uint64NotEqual(checksum.Sum(), excpectChecksum), IsTrue) + require.Equal(t, kvBytes<<1, checksum.SumSize()) + require.Equal(t, uint64(len(kvs))<<1, checksum.SumKVS()) + require.True(t, uint64NotEqual(checksum.Sum(), excpectChecksum)) } -func (s *testKVChcksumSuite) TestChecksumJSON(c *C) { +func TestChecksumJSON(t *testing.T) { testStruct := &struct { Checksum verification.KVChecksum }{ @@ -87,6 +76,6 @@ func (s *testKVChcksumSuite) TestChecksumJSON(c *C) { res, err := json.Marshal(testStruct) - c.Assert(err, IsNil) - c.Assert(res, BytesEquals, []byte(`{"Checksum":{"checksum":7890,"size":123,"kvs":456}}`)) + require.NoError(t, err) + require.Equal(t, []byte(`{"Checksum":{"checksum":7890,"size":123,"kvs":456}}`), res) } diff --git a/br/pkg/lightning/worker/worker_test.go b/br/pkg/lightning/worker/worker_test.go index 11a09fd849569..dd1eccda1552d 100644 --- a/br/pkg/lightning/worker/worker_test.go +++ b/br/pkg/lightning/worker/worker_test.go @@ -18,39 +18,28 @@ import ( "context" "testing" - . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/lightning/worker" + "github.com/stretchr/testify/require" ) -type testWorkerPool struct{} - -func (s *testWorkerPool) SetUpSuite(c *C) {} -func (s *testWorkerPool) TearDownSuite(c *C) {} - -var _ = Suite(&testWorkerPool{}) - -func TestNewRestoreWorkerPool(t *testing.T) { - TestingT(t) -} - -func (s *testWorkerPool) TestApplyRecycle(c *C) { +func TestApplyRecycle(t *testing.T) { pool := worker.NewPool(context.Background(), 3, "test") w1, w2, w3 := pool.Apply(), pool.Apply(), pool.Apply() - c.Assert(w1.ID, Equals, int64(1)) - c.Assert(w2.ID, Equals, int64(2)) - c.Assert(w3.ID, Equals, int64(3)) - c.Assert(pool.HasWorker(), Equals, false) + require.Equal(t, int64(1), w1.ID) + require.Equal(t, int64(2), w2.ID) + require.Equal(t, int64(3), w3.ID) + require.Equal(t, false, pool.HasWorker()) pool.Recycle(w3) - c.Assert(pool.HasWorker(), Equals, true) - c.Assert(pool.Apply(), Equals, w3) + require.Equal(t, true, pool.HasWorker()) + require.Equal(t, w3, pool.Apply()) pool.Recycle(w2) - c.Assert(pool.Apply(), Equals, w2) + require.Equal(t, w2, pool.Apply()) pool.Recycle(w1) - c.Assert(pool.Apply(), Equals, w1) + require.Equal(t, w1, pool.Apply()) - c.Assert(pool.HasWorker(), Equals, false) + require.Equal(t, false, pool.HasWorker()) - c.Assert(func() { pool.Recycle(nil) }, PanicMatches, "invalid restore worker") + require.PanicsWithValue(t, "invalid restore worker", func() { pool.Recycle(nil) }) } From 4c3dfe89e5520a75e34d357e5439e61f684e5dff Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Mon, 24 Jan 2022 04:49:47 +0100 Subject: [PATCH 18/21] ddl: Partition duplicate name fix (#31782) close pingcap/tidb#31681 --- ddl/db_change_test.go | 2 +- ddl/db_partition_test.go | 37 +++++++++++++++++++++++++++++++++++++ ddl/db_test.go | 4 ++-- ddl/ddl_api.go | 16 +++++++++++++--- ddl/partition.go | 17 +++++++++++++---- 5 files changed, 66 insertions(+), 10 deletions(-) diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 110da2b44455c..32af0c9b02da7 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -1586,7 +1586,7 @@ func (s *testStateChangeSuite) TestDDLIfExists(c *C) { s.testParallelExecSQL(c, "alter table test_exists drop index if exists idx_c") // DROP PARTITION (ADD PARTITION tested in TestParallelAlterAddPartition) - _, err = s.se.Execute(context.Background(), "create table test_exists_2 (a int key) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20))") + _, err = s.se.Execute(context.Background(), "create table test_exists_2 (a int key) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20), partition p2 values less than (30))") c.Assert(err, IsNil) s.testParallelExecSQL(c, "alter table test_exists_2 drop partition if exists p1") } diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index f40c8daaf97d1..393a720411c4f 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -3552,3 +3552,40 @@ func TestRenameTables(t *testing.T) { ddl.ExportTestRenameTables(t) } + +func (s *testIntegrationSuite1) TestDuplicatePartitionNames(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("create database DuplicatePartitionNames") + defer tk.MustExec("drop database DuplicatePartitionNames") + tk.MustExec("use DuplicatePartitionNames") + + tk.MustExec("set @@tidb_enable_list_partition=on") + tk.MustExec("create table t1 (a int) partition by list (a) (partition p1 values in (1), partition p2 values in (2), partition p3 values in (3))") + tk.MustExec("insert into t1 values (1),(2),(3)") + tk.MustExec("alter table t1 truncate partition p1,p1") + tk.MustQuery("select * from t1").Sort().Check(testkit.Rows("2", "3")) + tk.MustExec("insert into t1 values (1)") + err := tk.ExecToErr("alter table t1 drop partition p1,p1") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[ddl:1507]Error in list of partitions to DROP") + err = tk.ExecToErr("alter table t1 drop partition p1,p9") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[ddl:1507]Error in list of partitions to DROP") + err = tk.ExecToErr("alter table t1 drop partition p1,p1,p1") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[ddl:1508]Cannot remove all partitions, use DROP TABLE instead") + err = tk.ExecToErr("alter table t1 drop partition p1,p9,p1") + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[ddl:1508]Cannot remove all partitions, use DROP TABLE instead") + tk.MustQuery("select * from t1").Sort().Check(testkit.Rows("1", "2", "3")) + tk.MustExec("alter table t1 drop partition p1") + tk.MustQuery("select * from t1").Sort().Check(testkit.Rows("2", "3")) + tk.MustQuery("Show create table t1").Check(testkit.Rows("" + + "t1 CREATE TABLE `t1` (\n" + + " `a` int(11) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin\n" + + "PARTITION BY LIST (`a`)\n" + + "(PARTITION `p2` VALUES IN (2),\n" + + " PARTITION `p3` VALUES IN (3))")) +} diff --git a/ddl/db_test.go b/ddl/db_test.go index 085c452267876..21f5a07c6bd56 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -4899,13 +4899,13 @@ func (s *testDBSuite4) TestIfExists(c *C) { // DROP PARTITION s.mustExec(tk, c, "drop table if exists t2") - s.mustExec(tk, c, "create table t2 (a int key) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20))") + s.mustExec(tk, c, "create table t2 (a int key) partition by range(a) (partition pNeg values less than (0), partition p0 values less than (10), partition p1 values less than (20))") sql = "alter table t2 drop partition p1" s.mustExec(tk, c, sql) tk.MustGetErrCode(sql, errno.ErrDropPartitionNonExistent) s.mustExec(tk, c, "alter table t2 drop partition if exists p1") c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1507|Error in list of partitions to p1")) + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1507|Error in list of partitions to DROP")) } func testAddIndexForGeneratedColumn(tk *testkit.TestKit, s *testDBSuite5, c *C) { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b6606355d7c10..34219b8f9abad 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3407,20 +3407,30 @@ func (d *ddl) TruncateTablePartition(ctx sessionctx.Context, ident ast.Ident, sp return errors.Trace(ErrPartitionMgmtOnNonpartitioned) } - pids := make([]int64, len(spec.PartitionNames)) + var pids []int64 if spec.OnAllPartitions { pids = make([]int64, len(meta.GetPartitionInfo().Definitions)) for i, def := range meta.GetPartitionInfo().Definitions { pids[i] = def.ID } } else { - for i, name := range spec.PartitionNames { + // MySQL allows duplicate partition names in truncate partition + // so we filter them out through a hash + pidMap := make(map[int64]bool) + for _, name := range spec.PartitionNames { pid, err := tables.FindPartitionByName(meta, name.L) if err != nil { return errors.Trace(err) } - pids[i] = pid + pidMap[pid] = true } + // linter makezero does not handle changing pids to zero length, + // so create a new var and then assign to pids... + newPids := make([]int64, 0, len(pidMap)) + for pid := range pidMap { + newPids = append(newPids, pid) + } + pids = newPids } job := &model.Job{ diff --git a/ddl/partition.go b/ddl/partition.go index 02fc91ac7dfae..7bfa3f250c074 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -936,22 +936,31 @@ func checkDropTablePartition(meta *model.TableInfo, partLowerNames []string) err if pi.Type != model.PartitionTypeRange && pi.Type != model.PartitionTypeList { return errOnlyOnRangeListPartition.GenWithStackByArgs("DROP") } + + // To be error compatible with MySQL, we need to do this first! + // see https://github.com/pingcap/tidb/issues/31681#issuecomment-1015536214 oldDefs := pi.Definitions + if len(oldDefs) <= len(partLowerNames) { + return errors.Trace(ErrDropLastPartition) + } + + dupCheck := make(map[string]bool) for _, pn := range partLowerNames { found := false for _, def := range oldDefs { if def.Name.L == pn { + if _, ok := dupCheck[pn]; ok { + return errors.Trace(ErrDropPartitionNonExistent.GenWithStackByArgs("DROP")) + } + dupCheck[pn] = true found = true break } } if !found { - return errors.Trace(ErrDropPartitionNonExistent.GenWithStackByArgs(pn)) + return errors.Trace(ErrDropPartitionNonExistent.GenWithStackByArgs("DROP")) } } - if len(oldDefs) == len(partLowerNames) { - return errors.Trace(ErrDropLastPartition) - } return nil } From 55fce4539a6b6188ff49148ad19567887a5b9a7a Mon Sep 17 00:00:00 2001 From: wjHuang Date: Mon, 24 Jan 2022 12:07:47 +0800 Subject: [PATCH 19/21] executor: don't fill generated columns at the non-public state (#31850) close pingcap/tidb#31735 --- ddl/db_test.go | 49 +++++++++++++++++++++++++++++++++++++++ executor/batch_checker.go | 3 ++- 2 files changed, 51 insertions(+), 1 deletion(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 21f5a07c6bd56..5968e7b920144 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -7711,3 +7711,52 @@ func (s *testDBSuite2) TestCreateTables(c *C) { err = d.BatchCreateTableWithInfo(tk.Se, model.NewCIStr("test"), infos, ddl.OnExistError) c.Check(terror.ErrorEqual(err, infoschema.ErrTableExists), IsTrue) } + +func (s *testSerialDBSuite) TestAddGeneratedColumnAndInsert(c *C) { + // For issue #31735. + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test_db") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (a int, unique kye(a))") + tk.MustExec("insert into t1 value (1), (10)") + + var checkErr error + tk1 := testkit.NewTestKit(c, s.store) + _, checkErr = tk1.Exec("use test_db") + + d := s.dom.DDL() + hook := &ddl.TestDDLCallback{Do: s.dom} + ctx := mock.NewContext() + ctx.Store = s.store + times := 0 + hook.OnJobUpdatedExported = func(job *model.Job) { + if checkErr != nil { + return + } + switch job.SchemaState { + case model.StateDeleteOnly: + _, checkErr = tk1.Exec("insert into t1 values (1) on duplicate key update a=a+1") + if checkErr == nil { + _, checkErr = tk1.Exec("replace into t1 values (2)") + } + case model.StateWriteOnly: + _, checkErr = tk1.Exec("insert into t1 values (2) on duplicate key update a=a+1") + if checkErr == nil { + _, checkErr = tk1.Exec("replace into t1 values (3)") + } + case model.StateWriteReorganization: + if checkErr == nil && job.SchemaState == model.StateWriteReorganization && times == 0 { + _, checkErr = tk1.Exec("insert into t1 values (3) on duplicate key update a=a+1") + if checkErr == nil { + _, checkErr = tk1.Exec("replace into t1 values (4)") + } + times++ + } + } + } + d.(ddl.DDLForTest).SetHook(hook) + + tk.MustExec("alter table t1 add column gc int as ((a+1))") + tk.MustQuery("select * from t1 order by a").Check(testkit.Rows("4 5", "10 11")) + c.Assert(checkErr, IsNil) +} diff --git a/executor/batch_checker.go b/executor/batch_checker.go index 7b214d7e54196..76eea024bc3d3 100644 --- a/executor/batch_checker.go +++ b/executor/batch_checker.go @@ -259,8 +259,9 @@ func getOldRow(ctx context.Context, sctx sessionctx.Context, txn kv.Transaction, } } } - if col.IsGenerated() { + if col.IsGenerated() && col.State == model.StatePublic { // only the virtual column needs fill back. + // Insert doesn't fill the generated columns at non-public state. if !col.GeneratedStored { val, err := genExprs[gIdx].Eval(chunk.MutRowFromDatums(oldRow).ToRow()) if err != nil { From 20d7589bf33b920eecbb3484bacde3c143817a71 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 24 Jan 2022 12:31:47 +0800 Subject: [PATCH 20/21] *: fix data race in the TestAnalyzeIncremental (#31908) close pingcap/tidb#31907 --- executor/analyze_test.go | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 5355a3d54fc9d..c167eff12bcf3 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -1013,10 +1013,8 @@ func TestAnalyzeIndex(t *testing.T) { } func TestAnalyzeIncremental(t *testing.T) { - store, clean := testkit.CreateMockStore(t) + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() - dom, err := session.BootstrapSession(store) - require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -1027,10 +1025,7 @@ func TestAnalyzeIncremental(t *testing.T) { func TestAnalyzeIncrementalStreaming(t *testing.T) { t.Skip("unistore hasn't support streaming yet.") - store, clean := testkit.CreateMockStore(t) - dom, err := session.BootstrapSession(store) - require.NoError(t, err) - + store, dom, clean := testkit.CreateMockStoreAndDomain(t) defer clean() tk := testkit.NewTestKit(t, store) tk.MustExec("use test") From 416acd08e856c404e8017b2fc18988dce71fb845 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20van=20Eeden?= Date: Mon, 24 Jan 2022 06:49:47 +0200 Subject: [PATCH 21/21] server: Improve failure message for unix socket issues (#31628) close pingcap/tidb#30663 --- server/server.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/server/server.go b/server/server.go index fa5dfc6c2198c..7d8cc59f29c52 100644 --- a/server/server.go +++ b/server/server.go @@ -317,10 +317,8 @@ func cleanupStaleSocket(socket string) error { return fmt.Errorf("unix socket %s exists and is functional, not removing it", socket) } - logutil.BgLogger().Warn("Unix socket exists and is nonfunctional, removing it", - zap.String("socket", socket), zap.Error(err)) - if err = os.Remove(socket); err != nil { - return fmt.Errorf("failed to remove socket file %s", socket) + if err2 := os.Remove(socket); err2 != nil { + return fmt.Errorf("failed to cleanup stale Unix socket file %s: %w", socket, err) } return nil