diff --git a/.bazelrc b/.bazelrc index d622cdfe057ed..a27e9186cefa6 100644 --- a/.bazelrc +++ b/.bazelrc @@ -12,7 +12,6 @@ build:release --config=ci build --incompatible_strict_action_env --incompatible_enable_cc_toolchain_resolution build:ci --remote_cache=http://172.16.4.21:8080/tidb --remote_timeout="10s" test:ci --verbose_failures -test:ci --test_timeout=180 test:ci --test_env=GO_TEST_WRAP_TESTV=1 --test_verbose_timeout_warnings test:ci --remote_cache=http://172.16.4.21:8080/tidb --remote_timeout="10s" test:ci --test_env=TZ=Asia/Shanghai --test_output=errors --experimental_ui_max_stdouterr_bytes=104857600 diff --git a/DEPS.bzl b/DEPS.bzl index ff6f339cb0c0c..e9b52f7359215 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -2347,8 +2347,8 @@ def go_deps(): name = "com_github_nxadm_tail", build_file_proto_mode = "disable_global", importpath = "github.com/nxadm/tail", - sum = "h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE=", - version = "v1.4.8", + sum = "h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78=", + version = "v1.4.4", ) go_repository( name = "com_github_oklog_ulid", @@ -2510,8 +2510,8 @@ def go_deps(): name = "com_github_pingcap_kvproto", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/kvproto", - sum = "h1:VKMmvYhtG28j1sCCBdq4s+V9UOYqNgQ6CQviQwOgTeg=", - version = "v0.0.0-20220705090230-a5d4ffd2ba33", + sum = "h1:PAXtUVMJnyQQS8t9GzihIFmh6FBXu0JziWbIVknLniA=", + version = "v0.0.0-20220711062932-08b02befd813", ) go_repository( name = "com_github_pingcap_log", @@ -2531,8 +2531,8 @@ def go_deps(): name = "com_github_pingcap_tipb", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/tipb", - sum = "h1:XaTE4ZhQbQtQZtAVzlZh/Pf6SjFfMSTe1ia2nGcl36Y=", - version = "v0.0.0-20220706024432-7be3cc83a7d5", + sum = "h1:hE1dQdnvxWCHhD0snX67paV9y6inq8TxVFbsKqjaTQk=", + version = "v0.0.0-20220714100504-7d3474676bc9", ) go_repository( name = "com_github_pkg_browser", @@ -3012,8 +3012,8 @@ def go_deps(): name = "com_github_tikv_client_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/client-go/v2", - sum = "h1:RpH/obpgyNKkXV4Wt8PqSdcUTnqWyExPcla+qdTVgi0=", - version = "v2.0.1-0.20220711061028-1c198aab9585", + sum = "h1:OSxo1R2y6iyAAxbUPL7h1HC/17CNuwNkW2lF1JySJ7k=", + version = "v2.0.1-0.20220718080214-86d51ba7eb02", ) go_repository( name = "com_github_tikv_pd_client", diff --git a/br/cmd/br/cmd.go b/br/cmd/br/cmd.go index 3c5bf674da430..f3aeb3393df52 100644 --- a/br/cmd/br/cmd.go +++ b/br/cmd/br/cmd.go @@ -36,6 +36,14 @@ var ( "*.*", fmt.Sprintf("!%s.*", utils.TemporaryDBName("*")), "!mysql.*", + "mysql.user", + "mysql.db", + "mysql.tables_priv", + "mysql.columns_priv", + "mysql.global_priv", + "mysql.global_grants", + "mysql.default_roles", + "mysql.role_edges", "!sys.*", "!INFORMATION_SCHEMA.*", "!PERFORMANCE_SCHEMA.*", diff --git a/br/cmd/br/restore.go b/br/cmd/br/restore.go index a228605fb51b7..b5a692c0dc03d 100644 --- a/br/cmd/br/restore.go +++ b/br/cmd/br/restore.go @@ -3,8 +3,11 @@ package main import ( + "fmt" + "github.com/pingcap/errors" "github.com/pingcap/log" + berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/gluetikv" "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/task" @@ -38,11 +41,31 @@ func runRestoreCommand(command *cobra.Command, cmdName string) error { } if err := task.RunRestore(GetDefaultContext(), tidbGlue, cmdName, &cfg); err != nil { log.Error("failed to restore", zap.Error(err)) + printWorkaroundOnFullRestoreError(command, err) return errors.Trace(err) } return nil } +// print workaround when we met not fresh or incompatible cluster error on full cluster restore +func printWorkaroundOnFullRestoreError(command *cobra.Command, err error) { + if !errors.ErrorEqual(err, berrors.ErrRestoreNotFreshCluster) && + !errors.ErrorEqual(err, berrors.ErrRestoreIncompatibleSys) { + return + } + fmt.Println("#######################################################################") + switch { + case errors.ErrorEqual(err, berrors.ErrRestoreNotFreshCluster): + fmt.Println("# the target cluster is not fresh, br cannot restore system tables.") + case errors.ErrorEqual(err, berrors.ErrRestoreIncompatibleSys): + fmt.Println("# the target cluster is not compatible with the backup data,") + fmt.Println("# br cannot restore system tables.") + } + fmt.Println("# you can use the following command to skip restoring system tables:") + fmt.Printf("# br restore %s --filter '*.*' --filter '!mysql.*' \n", command.Use) + fmt.Println("#######################################################################") +} + func runRestoreRawCommand(command *cobra.Command, cmdName string) error { cfg := task.RestoreRawConfig{ RawKvConfig: task.RawKvConfig{Config: task.Config{LogProgress: HasLogFile()}}, diff --git a/br/pkg/errors/errors.go b/br/pkg/errors/errors.go index f95be8e68ead5..355b7bad3668f 100644 --- a/br/pkg/errors/errors.go +++ b/br/pkg/errors/errors.go @@ -62,6 +62,8 @@ var ( ErrRestoreInvalidRange = errors.Normalize("invalid restore range", errors.RFCCodeText("BR:Restore:ErrRestoreInvalidRange")) ErrRestoreWriteAndIngest = errors.Normalize("failed to write and ingest", errors.RFCCodeText("BR:Restore:ErrRestoreWriteAndIngest")) ErrRestoreSchemaNotExists = errors.Normalize("schema not exists", errors.RFCCodeText("BR:Restore:ErrRestoreSchemaNotExists")) + ErrRestoreNotFreshCluster = errors.Normalize("cluster is not fresh", errors.RFCCodeText("BR:Restore:ErrRestoreNotFreshCluster")) + ErrRestoreIncompatibleSys = errors.Normalize("incompatible system table", errors.RFCCodeText("BR:Restore:ErrRestoreIncompatibleSys")) ErrUnsupportedSystemTable = errors.Normalize("the system table isn't supported for restoring yet", errors.RFCCodeText("BR:Restore:ErrUnsupportedSysTable")) ErrDatabasesAlreadyExisted = errors.Normalize("databases already existed in restored cluster", errors.RFCCodeText("BR:Restore:ErrDatabasesAlreadyExisted")) diff --git a/br/pkg/glue/BUILD.bazel b/br/pkg/glue/BUILD.bazel index 812b2b2c8b6a7..e51549b578927 100644 --- a/br/pkg/glue/BUILD.bazel +++ b/br/pkg/glue/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "//domain", "//kv", "//parser/model", + "//sessionctx", "@com_github_fatih_color//:color", "@com_github_pingcap_log//:log", "@com_github_tikv_pd_client//:client", diff --git a/br/pkg/glue/console_glue.go b/br/pkg/glue/console_glue.go index 5f00cb3a63e2e..3b64faa14b56f 100644 --- a/br/pkg/glue/console_glue.go +++ b/br/pkg/glue/console_glue.go @@ -23,12 +23,40 @@ type ConsoleOperations struct { ConsoleGlue } -// StartTask prints a task start information, and mark as finished when the returned function called. -func (ops ConsoleOperations) StartTask(message string) func() { +// An extra field appending to the task. +// return type is a {key: string, value: string} tuple. +type ExtraField func() [2]string + +// NOTE: +// Perhaps we'd better move these modifiers and terminal function to another package +// like `glue/termutil?` + +// WithTimeCost adds the task information of time costing for `ShowTask`. +func WithTimeCost() ExtraField { start := time.Now() + return func() [2]string { + return [2]string{"take", time.Since(start).String()} + } +} + +// WithConstExtraField adds an extra field with constant values. +func WithConstExtraField(key string, value interface{}) ExtraField { + return func() [2]string { + return [2]string{key, fmt.Sprint(value)} + } +} + +// ShowTask prints a task start information, and mark as finished when the returned function called. +// This is for TUI presenting. +func (ops ConsoleOperations) ShowTask(message string, extraFields ...ExtraField) func() { ops.Print(message) return func() { - ops.Printf("%s; take = %s\n", color.HiGreenString("DONE"), time.Since(start)) + fields := make([]string, 0, len(extraFields)) + for _, fieldFunc := range extraFields { + field := fieldFunc() + fields = append(fields, fmt.Sprintf("%s = %s", field[0], color.New(color.Bold).Sprint(field[1]))) + } + ops.Printf("%s; %s\n", color.HiGreenString("DONE"), strings.Join(fields, ", ")) } } diff --git a/br/pkg/logutil/logging.go b/br/pkg/logutil/logging.go index 8bc2ecd5653a3..028cfc00e5f43 100644 --- a/br/pkg/logutil/logging.go +++ b/br/pkg/logutil/logging.go @@ -282,18 +282,27 @@ func (kr StringifyKeys) String() string { if i > 0 { sb.WriteString(", ") } - sb.WriteString("[") - sb.WriteString(redact.Key(rng.StartKey)) - sb.WriteString(", ") - var endKey string - if len(rng.EndKey) == 0 { - endKey = "inf" - } else { - endKey = redact.Key(rng.EndKey) - } - sb.WriteString(redact.String(endKey)) - sb.WriteString(")") + sb.WriteString(StringifyRange(rng).String()) } sb.WriteString("}") return sb.String() } + +// StringifyRange is the wrapper for displaying a key range. +type StringifyRange kv.KeyRange + +func (rng StringifyRange) String() string { + sb := new(strings.Builder) + sb.WriteString("[") + sb.WriteString(redact.Key(rng.StartKey)) + sb.WriteString(", ") + var endKey string + if len(rng.EndKey) == 0 { + endKey = "inf" + } else { + endKey = redact.Key(rng.EndKey) + } + sb.WriteString(redact.String(endKey)) + sb.WriteString(")") + return sb.String() +} diff --git a/br/pkg/restore/client.go b/br/pkg/restore/client.go index 137b99a843302..d67adb90dffe5 100644 --- a/br/pkg/restore/client.go +++ b/br/pkg/restore/client.go @@ -37,11 +37,13 @@ import ( "github.com/pingcap/tidb/br/pkg/stream" "github.com/pingcap/tidb/br/pkg/summary" "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/config" ddlutil "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/pdtypes" "github.com/pingcap/tidb/tablecodec" @@ -147,6 +149,16 @@ type Client struct { storage storage.ExternalStorage + // if fullClusterRestore = true: + // - if there's system tables in the backup(backup data since br 5.1.0), the cluster should be a fresh cluster + // without user database or table. and system tables about privileges is restored together with user data. + // - if there no system tables in the backup(backup data from br < 5.1.0), restore all user data just like + // previous version did. + // if fullClusterRestore = false, restore all user data just like previous version did. + // fullClusterRestore = true when there is no explicit filter setting, and it's full restore or point command + // with a full backup data. + // todo: maybe change to an enum + fullClusterRestore bool // the query to insert rows into table `gc_delete_range`, lack of ts. deleteRangeQuery []string deleteRangeQueryCh chan string @@ -494,6 +506,14 @@ func (rc *Client) GetDatabase(name string) *utils.Database { return rc.databases[name] } +// HasBackedUpSysDB whether we have backed up system tables +// br backs system tables up since 5.1.0 +func (rc *Client) HasBackedUpSysDB() bool { + temporaryDB := utils.TemporaryDBName(mysql.SystemDB) + _, backedUp := rc.databases[temporaryDB.O] + return backedUp +} + // GetPlacementPolicies returns policies. func (rc *Client) GetPlacementPolicies() (*sync.Map, error) { policies := &sync.Map{} @@ -834,6 +854,101 @@ func (rc *Client) createTablesInWorkerPool(ctx context.Context, dom *domain.Doma return eg.Wait() } +// CheckTargetClusterFresh check whether the target cluster is fresh or not +// if there's no user dbs or tables, we take it as a fresh cluster, although +// user may have created some users or made other changes. +func (rc *Client) CheckTargetClusterFresh(ctx context.Context) error { + log.Info("checking whether target cluster is fresh") + userDBs := GetExistedUserDBs(rc.dom) + if len(userDBs) == 0 { + return nil + } + + const maxPrintCount = 10 + userTableOrDBNames := make([]string, 0, maxPrintCount+1) + addName := func(name string) bool { + if len(userTableOrDBNames) == maxPrintCount { + userTableOrDBNames = append(userTableOrDBNames, "...") + return false + } + userTableOrDBNames = append(userTableOrDBNames, name) + return true + } +outer: + for _, db := range userDBs { + if !addName(db.Name.L) { + break outer + } + for _, tbl := range db.Tables { + if !addName(tbl.Name.L) { + break outer + } + } + } + log.Error("not fresh cluster", zap.Strings("user tables", userTableOrDBNames)) + return errors.Annotate(berrors.ErrRestoreNotFreshCluster, "user db/tables: "+strings.Join(userTableOrDBNames, ", ")) +} + +func (rc *Client) CheckSysTableCompatibility(dom *domain.Domain, tables []*metautil.Table) error { + log.Info("checking target cluster system table compatibility with backed up data") + privilegeTablesInBackup := make([]*metautil.Table, 0) + for _, table := range tables { + decodedSysDBName, ok := utils.GetSysDBCIStrName(table.DB.Name) + if ok && utils.IsSysDB(decodedSysDBName.L) && sysPrivilegeTableSet[table.Info.Name.L] { + privilegeTablesInBackup = append(privilegeTablesInBackup, table) + } + } + sysDB := model.NewCIStr(mysql.SystemDB) + for _, table := range privilegeTablesInBackup { + ti, err := rc.GetTableSchema(dom, sysDB, table.Info.Name) + if err != nil { + log.Error("missing table on target cluster", zap.Stringer("table", table.Info.Name)) + return errors.Annotate(berrors.ErrRestoreIncompatibleSys, "missed system table: "+table.Info.Name.O) + } + backupTi := table.Info + if len(ti.Columns) != len(backupTi.Columns) { + log.Error("column count mismatch", + zap.Stringer("table", table.Info.Name), + zap.Int("col in cluster", len(ti.Columns)), + zap.Int("col in backup", len(backupTi.Columns))) + return errors.Annotatef(berrors.ErrRestoreIncompatibleSys, + "column count mismatch, table: %s, col in cluster: %d, col in backup: %d", + table.Info.Name.O, len(ti.Columns), len(backupTi.Columns)) + } + backupColMap := make(map[string]*model.ColumnInfo) + for i := range backupTi.Columns { + col := backupTi.Columns[i] + backupColMap[col.Name.L] = col + } + // order can be different but type must compatible + for i := range ti.Columns { + col := ti.Columns[i] + backupCol := backupColMap[col.Name.L] + if backupCol == nil { + log.Error("missing column in backup data", + zap.Stringer("table", table.Info.Name), + zap.String("col", fmt.Sprintf("%s %s", col.Name, col.FieldType.String()))) + return errors.Annotatef(berrors.ErrRestoreIncompatibleSys, + "missing column in backup data, table: %s, col: %s %s", + table.Info.Name.O, + col.Name, col.FieldType.String()) + } + if !utils.IsTypeCompatible(backupCol.FieldType, col.FieldType) { + log.Error("incompatible column", + zap.Stringer("table", table.Info.Name), + zap.String("col in cluster", fmt.Sprintf("%s %s", col.Name, col.FieldType.String())), + zap.String("col in backup", fmt.Sprintf("%s %s", backupCol.Name, backupCol.FieldType.String()))) + return errors.Annotatef(berrors.ErrRestoreIncompatibleSys, + "incompatible column, table: %s, col in cluster: %s %s, col in backup: %s %s", + table.Info.Name.O, + col.Name, col.FieldType.String(), + backupCol.Name, backupCol.FieldType.String()) + } + } + } + return nil +} + // ExecDDLs executes the queries of the ddl jobs. func (rc *Client) ExecDDLs(ctx context.Context, ddlJobs []*model.Job) error { // Sort the ddl jobs by schema version in ascending order. @@ -1467,6 +1582,14 @@ func (rc *Client) getRuleID(tableID int64) string { return "restore-t" + strconv.FormatInt(tableID, 10) } +// IsFull returns whether this backup is full. +func (rc *Client) IsFull() bool { + failpoint.Inject("mock-incr-backup-data", func() { + failpoint.Return(false) + }) + return !rc.IsIncremental() +} + // IsIncremental returns whether this backup is incremental. func (rc *Client) IsIncremental() bool { return !(rc.backupMeta.StartVersion == rc.backupMeta.EndVersion || @@ -2188,6 +2311,22 @@ func (rc *Client) SaveSchemas( return nil } +// InitFullClusterRestore init fullClusterRestore and set SkipGrantTable as needed +func (rc *Client) InitFullClusterRestore(explicitFilter bool) { + rc.fullClusterRestore = !explicitFilter && rc.IsFull() + + log.Info("full cluster restore", zap.Bool("value", rc.fullClusterRestore)) + + if rc.fullClusterRestore { + // have to skip grant table, in order to NotifyUpdatePrivilege + config.GetGlobalConfig().Security.SkipGrantTable = true + } +} + +func (rc *Client) IsFullClusterRestore() bool { + return rc.fullClusterRestore +} + // MockClient create a fake client used to test. func MockClient(dbs map[string]*utils.Database) *Client { return &Client{databases: dbs} diff --git a/br/pkg/restore/client_test.go b/br/pkg/restore/client_test.go index c7374351b2d4f..64da7f2a9e579 100644 --- a/br/pkg/restore/client_test.go +++ b/br/pkg/restore/client_test.go @@ -12,12 +12,15 @@ import ( "testing" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/import_sstpb" "github.com/pingcap/kvproto/pkg/metapb" + berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/gluetidb" "github.com/pingcap/tidb/br/pkg/metautil" "github.com/pingcap/tidb/br/pkg/mock" "github.com/pingcap/tidb/br/pkg/restore" + "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" @@ -105,6 +108,157 @@ func TestIsOnline(t *testing.T) { require.True(t, client.IsOnline()) } +func getStartedMockedCluster(t *testing.T) *mock.Cluster { + t.Helper() + cluster, err := mock.NewCluster() + require.NoError(t, err) + err = cluster.Start() + require.NoError(t, err) + return cluster +} + +func TestCheckTargetClusterFresh(t *testing.T) { + // cannot use shared `mc`, other parallel case may change it. + cluster := getStartedMockedCluster(t) + defer cluster.Stop() + + g := gluetidb.New() + client := restore.NewRestoreClient(cluster.PDClient, nil, defaultKeepaliveCfg, false) + err := client.Init(g, cluster.Storage) + require.NoError(t, err) + + ctx := context.Background() + require.NoError(t, client.CheckTargetClusterFresh(ctx)) + + require.NoError(t, client.CreateDatabase(ctx, &model.DBInfo{Name: model.NewCIStr("user_db")})) + require.True(t, berrors.ErrRestoreNotFreshCluster.Equal(client.CheckTargetClusterFresh(ctx))) +} + +func TestCheckTargetClusterFreshWithTable(t *testing.T) { + // cannot use shared `mc`, other parallel case may change it. + cluster := getStartedMockedCluster(t) + defer cluster.Stop() + + g := gluetidb.New() + client := restore.NewRestoreClient(cluster.PDClient, nil, defaultKeepaliveCfg, false) + err := client.Init(g, cluster.Storage) + require.NoError(t, err) + + ctx := context.Background() + info, err := cluster.Domain.GetSnapshotInfoSchema(math.MaxUint64) + require.NoError(t, err) + dbSchema, isExist := info.SchemaByName(model.NewCIStr("test")) + require.True(t, isExist) + intField := types.NewFieldType(mysql.TypeLong) + intField.SetCharset("binary") + table := &metautil.Table{ + DB: dbSchema, + Info: &model.TableInfo{ + ID: int64(1), + Name: model.NewCIStr("t"), + Columns: []*model.ColumnInfo{{ + ID: 1, + Name: model.NewCIStr("id"), + FieldType: *intField, + State: model.StatePublic, + }}, + Charset: "utf8mb4", + Collate: "utf8mb4_bin", + }, + } + _, _, err = client.CreateTables(cluster.Domain, []*metautil.Table{table}, 0) + require.NoError(t, err) + + require.True(t, berrors.ErrRestoreNotFreshCluster.Equal(client.CheckTargetClusterFresh(ctx))) +} + +func TestCheckSysTableCompatibility(t *testing.T) { + cluster := mc + g := gluetidb.New() + client := restore.NewRestoreClient(cluster.PDClient, nil, defaultKeepaliveCfg, false) + err := client.Init(g, cluster.Storage) + require.NoError(t, err) + + info, err := cluster.Domain.GetSnapshotInfoSchema(math.MaxUint64) + require.NoError(t, err) + dbSchema, isExist := info.SchemaByName(model.NewCIStr(mysql.SystemDB)) + require.True(t, isExist) + tmpSysDB := dbSchema.Clone() + tmpSysDB.Name = utils.TemporaryDBName(mysql.SystemDB) + sysDB := model.NewCIStr(mysql.SystemDB) + userTI, err := client.GetTableSchema(cluster.Domain, sysDB, model.NewCIStr("user")) + require.NoError(t, err) + + // column count mismatch + mockedUserTI := userTI.Clone() + mockedUserTI.Columns = mockedUserTI.Columns[:len(mockedUserTI.Columns)-1] + err = client.CheckSysTableCompatibility(cluster.Domain, []*metautil.Table{{ + DB: tmpSysDB, + Info: mockedUserTI, + }}) + require.True(t, berrors.ErrRestoreIncompatibleSys.Equal(err)) + + // column order mismatch(success) + mockedUserTI = userTI.Clone() + mockedUserTI.Columns[4], mockedUserTI.Columns[5] = mockedUserTI.Columns[5], mockedUserTI.Columns[4] + err = client.CheckSysTableCompatibility(cluster.Domain, []*metautil.Table{{ + DB: tmpSysDB, + Info: mockedUserTI, + }}) + require.NoError(t, err) + + // missing column + mockedUserTI = userTI.Clone() + mockedUserTI.Columns[0].Name = model.NewCIStr("new-name") + err = client.CheckSysTableCompatibility(cluster.Domain, []*metautil.Table{{ + DB: tmpSysDB, + Info: mockedUserTI, + }}) + require.True(t, berrors.ErrRestoreIncompatibleSys.Equal(err)) + + // incompatible column type + mockedUserTI = userTI.Clone() + mockedUserTI.Columns[0].FieldType.SetFlen(2000) // Columns[0] is `Host` char(255) + err = client.CheckSysTableCompatibility(cluster.Domain, []*metautil.Table{{ + DB: tmpSysDB, + Info: mockedUserTI, + }}) + require.True(t, berrors.ErrRestoreIncompatibleSys.Equal(err)) + + // compatible + mockedUserTI = userTI.Clone() + err = client.CheckSysTableCompatibility(cluster.Domain, []*metautil.Table{{ + DB: tmpSysDB, + Info: mockedUserTI, + }}) + require.NoError(t, err) +} + +func TestInitFullClusterRestore(t *testing.T) { + cluster := mc + g := gluetidb.New() + client := restore.NewRestoreClient(cluster.PDClient, nil, defaultKeepaliveCfg, false) + err := client.Init(g, cluster.Storage) + require.NoError(t, err) + + // explicit filter + client.InitFullClusterRestore(true) + require.False(t, client.IsFullClusterRestore()) + + client.InitFullClusterRestore(false) + require.True(t, client.IsFullClusterRestore()) + // set it to false again + client.InitFullClusterRestore(true) + require.False(t, client.IsFullClusterRestore()) + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/br/pkg/restore/mock-incr-backup-data", "return(true)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/br/pkg/restore/mock-incr-backup-data")) + }() + client.InitFullClusterRestore(false) + require.False(t, client.IsFullClusterRestore()) +} + func TestPreCheckTableClusterIndex(t *testing.T) { m := mc g := gluetidb.New() diff --git a/br/pkg/restore/db.go b/br/pkg/restore/db.go index a7b29a3e90a00..1e45725f920fc 100644 --- a/br/pkg/restore/db.go +++ b/br/pkg/restore/db.go @@ -461,6 +461,8 @@ func GetExistedUserDBs(dom *domain.Domain) []*model.DBInfo { if tidbutil.IsMemOrSysDB(dbName) { continue } else if dbName == "test" && len(db.Tables) == 0 { + // tidb create test db on fresh cluster + // if it's empty we don't take it as user db continue } else { existedDatabases = append(existedDatabases, db) diff --git a/br/pkg/restore/systable_restore.go b/br/pkg/restore/systable_restore.go index b0f588fa2286c..edbf1629f2e80 100644 --- a/br/pkg/restore/systable_restore.go +++ b/br/pkg/restore/systable_restore.go @@ -5,6 +5,7 @@ package restore import ( "context" "fmt" + "strings" "github.com/pingcap/errors" "github.com/pingcap/log" @@ -33,16 +34,7 @@ var unRecoverableTable = map[string]struct{}{ "tidb": {}, "global_variables": {}, - // all user related tables cannot be recovered for now. "column_stats_usage": {}, - "columns_priv": {}, - "db": {}, - "default_roles": {}, - "global_grants": {}, - "global_priv": {}, - "role_edges": {}, - "tables_priv": {}, - "user": {}, "capture_plan_baselines_blacklist": {}, // gc info don't need to recover. "gc_delete_range": {}, @@ -52,6 +44,17 @@ var unRecoverableTable = map[string]struct{}{ "schema_index_usage": {}, } +var sysPrivilegeTableSet = map[string]bool{ + "user": true, // since v1.0.0 + "db": true, // since v1.0.0 + "tables_priv": true, // since v1.0.0 + "columns_priv": true, // since v1.0.0 + "default_roles": true, // since v3.0.0 + "role_edges": true, // since v3.0.0 + "global_priv": true, // since v3.0.8 + "global_grants": true, // since v5.0.3 +} + func isUnrecoverableTable(tableName string) bool { _, ok := unRecoverableTable[tableName] return ok @@ -90,7 +93,7 @@ func (rc *Client) RestoreSystemSchemas(ctx context.Context, f filter.Filter) { for _, table := range originDatabase.Tables { tableName := table.Info.Name if f.MatchTable(sysDB, tableName.O) { - if err := rc.replaceTemporaryTableToSystable(ctx, tableName.L, db); err != nil { + if err := rc.replaceTemporaryTableToSystable(ctx, table.Info, db); err != nil { log.Warn("error during merging temporary tables into system tables", logutil.ShortError(err), zap.Stringer("table", tableName), @@ -139,18 +142,24 @@ func (rc *Client) afterSystemTablesReplaced(tables []string) error { for _, table := range tables { switch { case table == "user": - // We cannot execute `rc.dom.NotifyUpdatePrivilege` here, because there isn't - // sessionctx.Context provided by the glue. - // TODO: update the glue type and allow we retrieve a session context from it. - err = multierr.Append(err, errors.Annotatef(berrors.ErrUnsupportedSystemTable, - "restored user info may not take effect, until you should execute `FLUSH PRIVILEGES` manually")) + if rc.fullClusterRestore { + log.Info("privilege system table restored, please reconnect to make it effective") + err = rc.dom.NotifyUpdatePrivilege() + } else { + // to make it compatible with older version + // todo: should we allow restore system table in non-fresh cluster in later br version? + // if we don't, we can check it at first place. + err = multierr.Append(err, errors.Annotatef(berrors.ErrUnsupportedSystemTable, + "restored user info may not take effect, until you should execute `FLUSH PRIVILEGES` manually")) + } } } return err } // replaceTemporaryTableToSystable replaces the temporary table to real system table. -func (rc *Client) replaceTemporaryTableToSystable(ctx context.Context, tableName string, db *database) error { +func (rc *Client) replaceTemporaryTableToSystable(ctx context.Context, ti *model.TableInfo, db *database) error { + tableName := ti.Name.L execSQL := func(sql string) error { // SQLs here only contain table name and database name, seems it is no need to redact them. if err := rc.db.se.Execute(ctx, sql); err != nil { @@ -188,11 +197,26 @@ func (rc *Client) replaceTemporaryTableToSystable(ctx context.Context, tableName } if db.ExistingTables[tableName] != nil { - log.Info("table existing, using replace into for restore", + if rc.fullClusterRestore && sysPrivilegeTableSet[tableName] { + log.Info("full cluster restore, delete existing data", + zap.String("table", tableName), zap.Stringer("schema", db.Name)) + deleteSQL := fmt.Sprintf("DELETE FROM %s;", utils.EncloseDBAndTable(db.Name.L, tableName)) + if err := execSQL(deleteSQL); err != nil { + return err + } + } + log.Info("replace into existing table", zap.String("table", tableName), zap.Stringer("schema", db.Name)) - replaceIntoSQL := fmt.Sprintf("REPLACE INTO %s SELECT * FROM %s;", + // target column order may different with source cluster + columnNames := make([]string, 0, len(ti.Columns)) + for _, col := range ti.Columns { + columnNames = append(columnNames, utils.EncloseName(col.Name.L)) + } + colListStr := strings.Join(columnNames, ",") + replaceIntoSQL := fmt.Sprintf("REPLACE INTO %s(%s) SELECT %s FROM %s;", utils.EncloseDBAndTable(db.Name.L, tableName), + colListStr, colListStr, utils.EncloseDBAndTable(db.TemporaryName.L, tableName)) return execSQL(replaceIntoSQL) } diff --git a/br/pkg/stream/stream_mgr.go b/br/pkg/stream/stream_mgr.go index 0eccada182f06..c7604b62b2de2 100644 --- a/br/pkg/stream/stream_mgr.go +++ b/br/pkg/stream/stream_mgr.go @@ -36,6 +36,8 @@ import ( const ( streamBackupMetaPrefix = "v1/backupmeta" + streamBackupGlobalCheckpointPrefix = "v1/global_checkpoint" + metaDataWorkerPoolSize = 128 ) @@ -43,6 +45,10 @@ func GetStreamBackupMetaPrefix() string { return streamBackupMetaPrefix } +func GetStreamBackupGlobalCheckpointPrefix() string { + return streamBackupGlobalCheckpointPrefix +} + // appendTableObserveRanges builds key ranges corresponding to `tblIDS`. func appendTableObserveRanges(tblIDs []int64) []kv.KeyRange { krs := make([]kv.KeyRange, 0, len(tblIDs)) diff --git a/br/pkg/streamhelper/advancer.go b/br/pkg/streamhelper/advancer.go index e20516285e962..601a364cf51bb 100644 --- a/br/pkg/streamhelper/advancer.go +++ b/br/pkg/streamhelper/advancer.go @@ -109,14 +109,14 @@ func NewCheckpointAdvancer(env Env) *CheckpointAdvancer { // you may need to change the config `AdvancingByCache`. func (c *CheckpointAdvancer) disableCache() { c.cache = NoOPCheckpointCache{} - c.state = fullScan{} + c.state = &fullScan{} } // enable the cache. // also check `AdvancingByCache` in the config. func (c *CheckpointAdvancer) enableCache() { c.cache = NewCheckpoints() - c.state = fullScan{} + c.state = &fullScan{} } // UpdateConfig updates the config for the advancer. @@ -185,6 +185,7 @@ func (c *CheckpointAdvancer) tryAdvance(ctx context.Context, rst RangesSharesTS) defer c.recordTimeCost("try advance", zap.Uint64("checkpoint", rst.TS), zap.Int("len", len(rst.Ranges)))() defer func() { if err != nil { + log.Warn("failed to advance", logutil.ShortError(err), zap.Object("target", rst.Zap())) c.cache.InsertRanges(rst) } }() @@ -225,11 +226,19 @@ func (c *CheckpointAdvancer) tryAdvance(ctx context.Context, rst RangesSharesTS) // CalculateGlobalCheckpointLight tries to advance the global checkpoint by the cache. func (c *CheckpointAdvancer) CalculateGlobalCheckpointLight(ctx context.Context) (uint64, error) { - log.Info("advancer with cache: current tree", zap.Stringer("ct", c.cache)) + log.Info("[log backup advancer hint] advancer with cache: current tree", zap.Stringer("ct", c.cache)) rsts := c.cache.PopRangesWithGapGT(config.DefaultTryAdvanceThreshold) if len(rsts) == 0 { return 0, nil } + samples := rsts + if len(rsts) > 3 { + samples = rsts[:3] + } + for _, sample := range samples { + log.Info("[log backup advancer hint] sample range.", zap.Object("range", sample.Zap()), zap.Int("total-len", len(rsts))) + } + workers := utils.NewWorkerPool(uint(config.DefaultMaxConcurrencyAdvance), "regions") eg, cx := errgroup.WithContext(ctx) for _, rst := range rsts { @@ -242,7 +251,6 @@ func (c *CheckpointAdvancer) CalculateGlobalCheckpointLight(ctx context.Context) if err != nil { return 0, err } - log.Info("advancer with cache: new tree", zap.Stringer("cache", c.cache)) ts := c.cache.CheckpointTS() return ts, nil } @@ -420,6 +428,7 @@ func (c *CheckpointAdvancer) advanceCheckpointBy(ctx context.Context, getCheckpo if err != nil { return err } + log.Info("get checkpoint", zap.Uint64("old", c.lastCheckpoint), zap.Uint64("new", cp)) if cp < c.lastCheckpoint { log.Warn("failed to update global checkpoint: stale", zap.Uint64("old", c.lastCheckpoint), zap.Uint64("new", cp)) } diff --git a/br/pkg/streamhelper/collector.go b/br/pkg/streamhelper/collector.go index 1df39d0633d68..d7c42adbfdda0 100644 --- a/br/pkg/streamhelper/collector.go +++ b/br/pkg/streamhelper/collector.go @@ -192,8 +192,16 @@ func (c *storeCollector) sendPendingRequests(ctx context.Context) error { for _, checkpoint := range cps.Checkpoints { if checkpoint.Err != nil { log.Debug("failed to get region checkpoint", zap.Stringer("err", checkpoint.Err)) + if checkpoint.Err.EpochNotMatch != nil { + metrics.RegionCheckpointFailure.WithLabelValues("epoch-not-match").Inc() + } + if checkpoint.Err.NotLeader != nil { + metrics.RegionCheckpointFailure.WithLabelValues("not-leader").Inc() + } + metrics.RegionCheckpointRequest.WithLabelValues("fail").Inc() c.inconsistent = append(c.inconsistent, c.regionMap[checkpoint.Region.Id]) } else { + metrics.RegionCheckpointRequest.WithLabelValues("success").Inc() if c.onSuccess != nil { c.onSuccess(checkpoint.Checkpoint, c.regionMap[checkpoint.Region.Id]) } diff --git a/br/pkg/streamhelper/regioniter.go b/br/pkg/streamhelper/regioniter.go index b2bfa0820316c..9f71ebf38b472 100644 --- a/br/pkg/streamhelper/regioniter.go +++ b/br/pkg/streamhelper/regioniter.go @@ -5,13 +5,18 @@ package streamhelper import ( "bytes" "context" + "fmt" "time" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/redact" "github.com/pingcap/tidb/br/pkg/utils" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/metrics" ) const ( @@ -44,6 +49,13 @@ type RegionIter struct { PageSize int } +func (r *RegionIter) String() string { + return fmt.Sprintf("RegionIter:%s;%v;from=%s", + logutil.StringifyKeys([]kv.KeyRange{{StartKey: r.currentStartKey, EndKey: r.endKey}}), + r.infScanFinished, + redact.Key(r.startKey)) +} + // IterateRegion creates an iterater over the region range. func IterateRegion(cli RegionScanner, startKey, endKey []byte) *RegionIter { return &RegionIter{ @@ -85,8 +97,17 @@ func CheckRegionConsistency(startKey, endKey []byte, regions []RegionWithLeader) // Next get the next page of regions. func (r *RegionIter) Next(ctx context.Context) ([]RegionWithLeader, error) { var rs []RegionWithLeader - state := utils.InitialRetryState(30, 500*time.Millisecond, 500*time.Millisecond) - err := utils.WithRetry(ctx, func() error { + state := utils.InitialRetryState(8, 500*time.Millisecond, 500*time.Millisecond) + err := utils.WithRetry(ctx, func() (retErr error) { + defer func() { + if retErr != nil { + log.Warn("failed with trying to scan regions", logutil.ShortError(retErr), + logutil.Key("start", r.currentStartKey), + logutil.Key("end", r.endKey), + ) + } + metrics.RegionCheckpointFailure.WithLabelValues("retryable-scan-region").Inc() + }() regions, err := r.cli.RegionScan(ctx, r.currentStartKey, r.endKey, r.PageSize) if err != nil { return err @@ -115,8 +136,11 @@ func (r *RegionIter) Next(ctx context.Context) ([]RegionWithLeader, error) { // Done checks whether the iteration is done. func (r *RegionIter) Done() bool { + // special case: we want to scan to the end of key space. + // at this time, comparing currentStartKey and endKey may be misleading when + // they are both "". if len(r.endKey) == 0 { return r.infScanFinished } - return bytes.Compare(r.currentStartKey, r.endKey) >= 0 + return r.infScanFinished || bytes.Compare(r.currentStartKey, r.endKey) >= 0 } diff --git a/br/pkg/streamhelper/regioniter_test.go b/br/pkg/streamhelper/regioniter_test.go new file mode 100644 index 0000000000000..42d4445f70679 --- /dev/null +++ b/br/pkg/streamhelper/regioniter_test.go @@ -0,0 +1,184 @@ +// Copyright 2022 PingCAP, Inc. Licensed under Apache-2.0. + +package streamhelper_test + +import ( + "bytes" + "context" + "fmt" + "strings" + "testing" + + "github.com/pingcap/kvproto/pkg/metapb" + "github.com/pingcap/tidb/br/pkg/logutil" + "github.com/pingcap/tidb/br/pkg/redact" + "github.com/pingcap/tidb/br/pkg/streamhelper" + "github.com/pingcap/tidb/kv" + "github.com/stretchr/testify/require" +) + +type constantRegions []streamhelper.RegionWithLeader + +func regionToRange(region streamhelper.RegionWithLeader) kv.KeyRange { + return kv.KeyRange{ + StartKey: region.Region.StartKey, + EndKey: region.Region.EndKey, + } +} + +func (c constantRegions) EqualsTo(other []streamhelper.RegionWithLeader) bool { + if len(c) != len(other) { + return false + } + for i := 0; i < len(c); i++ { + r1 := regionToRange(c[i]) + r2 := regionToRange(other[i]) + + equals := bytes.Equal(r1.StartKey, r2.StartKey) && bytes.Equal(r1.EndKey, r2.EndKey) + if !equals { + return false + } + } + return true +} + +func (c constantRegions) String() string { + segs := make([]string, 0, len(c)) + for _, region := range c { + segs = append(segs, fmt.Sprintf("%d%s", region.Region.Id, logutil.StringifyRange(regionToRange(region)))) + } + return strings.Join(segs, ";") +} + +// RegionScan gets a list of regions, starts from the region that contains key. +// Limit limits the maximum number of regions returned. +func (c constantRegions) RegionScan(ctx context.Context, key []byte, endKey []byte, limit int) ([]streamhelper.RegionWithLeader, error) { + result := make([]streamhelper.RegionWithLeader, 0, limit) + for _, region := range c { + if overlaps(kv.KeyRange{StartKey: key, EndKey: endKey}, kv.KeyRange{StartKey: region.Region.StartKey, EndKey: region.Region.EndKey}) && len(result) < limit { + result = append(result, region) + } else if bytes.Compare(region.Region.StartKey, key) > 0 { + break + } + } + fmt.Printf("all = %s\n", c) + fmt.Printf("start = %s, end = %s, result = %s\n", redact.Key(key), redact.Key(endKey), constantRegions(result)) + return result, nil +} + +func makeSubrangeRegions(keys ...string) constantRegions { + if len(keys) == 0 { + return nil + } + id := uint64(1) + regions := make([]streamhelper.RegionWithLeader, 0, len(keys)+1) + start := keys[0] + for _, key := range keys[1:] { + region := streamhelper.RegionWithLeader{ + Region: &metapb.Region{ + Id: id, + StartKey: []byte(start), + EndKey: []byte(key), + }, + } + id++ + start = key + regions = append(regions, region) + } + return constantRegions(regions) + +} + +func useRegions(keys ...string) constantRegions { + ks := []string{""} + ks = append(ks, keys...) + ks = append(ks, "") + return makeSubrangeRegions(ks...) +} + +func manyRegions(from, to int) []string { + regions := []string{} + for i := from; i < to; i++ { + regions = append(regions, fmt.Sprintf("%06d", i)) + } + return regions +} + +func appendInitial(a []string) []string { + return append([]string{""}, a...) +} + +func appendFinal(a []string) []string { + return append(a, "") +} + +func TestRegionIterator(t *testing.T) { + type Case struct { + // boundary of regions, doesn't include the initial key (implicitly "") + // or the final key (implicitly +inf) + // Example: + // ["0001", "0002"] => [Region("", "0001"), Region("0001", "0002"), Region("0002", "")] + RegionBoundary []string + StartKey string + EndKey string + // border of required regions, include the initial key and the final key. + // Example: + // ["0001", "0002", ""] => [Region("0001", "0002"), Region("0002", "")] + RequiredRegionBoundary []string + } + + run := func(t *testing.T, c Case) { + req := require.New(t) + regions := useRegions(c.RegionBoundary...) + requiredRegions := makeSubrangeRegions(c.RequiredRegionBoundary...) + ctx := context.Background() + + collected := make([]streamhelper.RegionWithLeader, 0, len(c.RequiredRegionBoundary)) + iter := streamhelper.IterateRegion(regions, []byte(c.StartKey), []byte(c.EndKey)) + for !iter.Done() { + regions, err := iter.Next(ctx) + req.NoError(err) + collected = append(collected, regions...) + } + req.True(requiredRegions.EqualsTo(collected), "%s :: %s", requiredRegions, collected) + } + + cases := []Case{ + { + RegionBoundary: []string{"0001", "0003", "0008", "0078"}, + StartKey: "0077", + EndKey: "0079", + RequiredRegionBoundary: []string{"0008", "0078", ""}, + }, + { + RegionBoundary: []string{"0001", "0005", "0008", "0097"}, + StartKey: "0000", + EndKey: "0008", + RequiredRegionBoundary: []string{"", "0001", "0005", "0008"}, + }, + { + RegionBoundary: manyRegions(0, 10000), + StartKey: "000001", + EndKey: "005000", + RequiredRegionBoundary: manyRegions(1, 5001), + }, + { + RegionBoundary: manyRegions(0, 10000), + StartKey: "000100", + EndKey: "", + RequiredRegionBoundary: appendFinal(manyRegions(100, 10000)), + }, + { + RegionBoundary: manyRegions(0, 10000), + StartKey: "", + EndKey: "003000", + RequiredRegionBoundary: appendInitial(manyRegions(0, 3001)), + }, + } + + for i, c := range cases { + t.Run(fmt.Sprintf("case#%d", i), func(t *testing.T) { + run(t, c) + }) + } +} diff --git a/br/pkg/streamhelper/tsheap.go b/br/pkg/streamhelper/tsheap.go index 64669a151467a..75d674c56b18e 100644 --- a/br/pkg/streamhelper/tsheap.go +++ b/br/pkg/streamhelper/tsheap.go @@ -3,6 +3,7 @@ package streamhelper import ( + "encoding/hex" "fmt" "strings" "sync" @@ -12,8 +13,10 @@ import ( "github.com/pingcap/errors" berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/logutil" + "github.com/pingcap/tidb/br/pkg/redact" "github.com/pingcap/tidb/kv" "github.com/tikv/client-go/v2/oracle" + "go.uber.org/zap/zapcore" ) // CheckpointsCache is the heap-like cache for checkpoints. @@ -88,6 +91,26 @@ type RangesSharesTS struct { Ranges []kv.KeyRange } +func (rst *RangesSharesTS) Zap() zapcore.ObjectMarshaler { + return zapcore.ObjectMarshalerFunc(func(oe zapcore.ObjectEncoder) error { + rngs := rst.Ranges + if len(rst.Ranges) > 3 { + rngs = rst.Ranges[:3] + } + + oe.AddUint64("checkpoint", rst.TS) + return oe.AddArray("items", zapcore.ArrayMarshalerFunc(func(ae zapcore.ArrayEncoder) error { + return ae.AppendObject(zapcore.ObjectMarshalerFunc(func(oe1 zapcore.ObjectEncoder) error { + for _, rng := range rngs { + oe1.AddString("start-key", redact.String(hex.EncodeToString(rng.StartKey))) + oe1.AddString("end-key", redact.String(hex.EncodeToString(rng.EndKey))) + } + return nil + })) + })) + }) +} + func (rst *RangesSharesTS) String() string { // Make a more friendly string. return fmt.Sprintf("@%sR%d", oracle.GetTimeFromTS(rst.TS).Format("0405"), len(rst.Ranges)) diff --git a/br/pkg/streamhelper/tsheap_test.go b/br/pkg/streamhelper/tsheap_test.go index 843dbf3f42f09..6453ee36a60ab 100644 --- a/br/pkg/streamhelper/tsheap_test.go +++ b/br/pkg/streamhelper/tsheap_test.go @@ -101,6 +101,10 @@ func TestMergeRanges(t *testing.T) { parameter: []kv.KeyRange{r("", "01"), r("01", ""), r("", "02"), r("", "03"), r("01", "02")}, expected: []kv.KeyRange{r("", "")}, }, + { + parameter: []kv.KeyRange{r("", ""), r("", "01"), r("01", ""), r("01", "02")}, + expected: []kv.KeyRange{r("", "")}, + }, } for i, c := range cases { diff --git a/br/pkg/task/BUILD.bazel b/br/pkg/task/BUILD.bazel index b01d47de6de22..4acaf84014b03 100644 --- a/br/pkg/task/BUILD.bazel +++ b/br/pkg/task/BUILD.bazel @@ -39,6 +39,7 @@ go_library( "//statistics/handle", "//types", "//util/mathutil", + "//util/sqlexec", "//util/table-filter", "@com_github_docker_go_units//:go-units", "@com_github_fatih_color//:color", @@ -78,7 +79,6 @@ go_test( deps = [ "//br/pkg/conn", "//br/pkg/metautil", - "//br/pkg/pdutil", "//br/pkg/restore", "//br/pkg/storage", "//br/pkg/stream", diff --git a/br/pkg/task/common.go b/br/pkg/task/common.go index 7deada25bd562..d9479764f0ff9 100644 --- a/br/pkg/task/common.go +++ b/br/pkg/task/common.go @@ -208,6 +208,9 @@ type Config struct { GRPCKeepaliveTimeout time.Duration `json:"grpc-keepalive-timeout" toml:"grpc-keepalive-timeout"` CipherInfo backuppb.CipherInfo `json:"-" toml:"-"` + + // whether there's explicit filter + ExplicitFilter bool `json:"-" toml:"-"` } // DefineCommonFlags defines the flags common to all BRIE commands. @@ -467,6 +470,7 @@ func (cfg *Config) ParseFromFlags(flags *pflag.FlagSet) error { cfg.Tables = make(map[string]struct{}) var caseSensitive bool if filterFlag := flags.Lookup(flagFilter); filterFlag != nil { + cfg.ExplicitFilter = flags.Changed(flagFilter) cfg.FilterStr = filterFlag.Value.(pflag.SliceValue).GetSlice() cfg.TableFilter, err = filter.Parse(cfg.FilterStr) if err != nil { diff --git a/br/pkg/task/restore.go b/br/pkg/task/restore.go index a9007fad55923..09fe19348ef24 100644 --- a/br/pkg/task/restore.go +++ b/br/pkg/task/restore.go @@ -471,6 +471,22 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf return errors.Trace(err) } + // todo: move this check into InitFullClusterRestore, we should move restore config into a separate package + // to avoid import cycle problem which we won't do it in this pr, then refactor this + // + // if it's point restore and reached here, then cmdName=FullRestoreCmd and len(cfg.FullBackupStorage) > 0 + if cmdName == FullRestoreCmd { + client.InitFullClusterRestore(cfg.ExplicitFilter) + } + if client.IsFullClusterRestore() && client.HasBackedUpSysDB() { + if err = client.CheckTargetClusterFresh(ctx); err != nil { + return errors.Trace(err) + } + if err = client.CheckSysTableCompatibility(mgr.GetDomain(), tables); err != nil { + return errors.Trace(err) + } + } + sp := utils.BRServiceSafePoint{ BackupTS: restoreTS, TTL: utils.DefaultBRGCSafePointTTL, diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index 04b52adec2154..5974c75f226ad 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -17,12 +17,14 @@ package task import ( "bytes" "context" + "encoding/binary" "fmt" "net/http" "strings" "sync" "time" + "github.com/docker/go-units" "github.com/fatih/color" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" @@ -884,7 +886,7 @@ func RunStreamTruncate(c context.Context, g glue.Glue, cmdName string, cfg *Stre return err } } - readMetaDone := console.StartTask("Reading Metadata... ") + readMetaDone := console.ShowTask("Reading Metadata... ", glue.WithTimeCost()) metas := restore.StreamMetadataSet{ BeforeDoWriteBack: func(path string, last, current *backuppb.Metadata) (skip bool) { log.Info("Updating metadata.", zap.String("file", path), @@ -922,7 +924,7 @@ func RunStreamTruncate(c context.Context, g glue.Glue, cmdName string, cfg *Stre removed := metas.RemoveDataBefore(shiftUntilTS) // remove metadata - removeMetaDone := console.StartTask("Removing metadata... ") + removeMetaDone := console.ShowTask("Removing metadata... ", glue.WithTimeCost()) if !cfg.DryRun { if err := metas.DoWriteBack(ctx, storage); err != nil { return err @@ -931,8 +933,11 @@ func RunStreamTruncate(c context.Context, g glue.Glue, cmdName string, cfg *Stre removeMetaDone() // remove log - clearDataFileDone := console.StartTask( - fmt.Sprintf("Clearing data files done. kv-count = %v, total-size = %v", kvCount, totalSize)) + clearDataFileDone := console.ShowTask( + "Clearing data files... ", glue.WithTimeCost(), + glue.WithConstExtraField("kv-count", kvCount), + glue.WithConstExtraField("kv-size", fmt.Sprintf("%d(%s)", totalSize, units.HumanSize(float64(totalSize)))), + ) worker := utils.NewWorkerPool(128, "delete files") wg := new(sync.WaitGroup) for _, f := range removed { @@ -1264,7 +1269,7 @@ func getLogRange( logMinTS := mathutil.Max(logStartTS, truncateTS) // get max global resolved ts from metas. - logMaxTS, err := getGlobalResolvedTS(ctx, s) + logMaxTS, err := getGlobalCheckpointFromStorage(ctx, s) if err != nil { return 0, 0, errors.Trace(err) } @@ -1273,6 +1278,25 @@ func getLogRange( return logMinTS, logMaxTS, nil } +func getGlobalCheckpointFromStorage(ctx context.Context, s storage.ExternalStorage) (uint64, error) { + var globalCheckPointTS uint64 = 0 + opt := storage.WalkOption{SubDir: stream.GetStreamBackupGlobalCheckpointPrefix()} + err := s.WalkDir(ctx, &opt, func(path string, size int64) error { + if !strings.HasSuffix(path, ".ts") { + return nil + } + + buff, err := s.ReadFile(ctx, path) + if err != nil { + return errors.Trace(err) + } + ts := binary.LittleEndian.Uint64(buff) + globalCheckPointTS = mathutil.Max(ts, globalCheckPointTS) + return nil + }) + return globalCheckPointTS, errors.Trace(err) +} + // getFullBackupTS gets the snapshot-ts of full bakcup func getFullBackupTS( ctx context.Context, diff --git a/br/pkg/task/stream_test.go b/br/pkg/task/stream_test.go index a6e7e6fcbe854..d129eeaf5b2c2 100644 --- a/br/pkg/task/stream_test.go +++ b/br/pkg/task/stream_test.go @@ -16,6 +16,7 @@ package task import ( "context" + "encoding/binary" "fmt" "path/filepath" "testing" @@ -194,3 +195,67 @@ func TestGetGlobalResolvedTS2(t *testing.T) { require.Nil(t, err) require.Equal(t, uint64(99), globalResolvedTS) } + +func fakeCheckpointFiles( + ctx context.Context, + tmpDir string, + infos []fakeGlobalCheckPoint, +) error { + cpDir := filepath.Join(tmpDir, stream.GetStreamBackupGlobalCheckpointPrefix()) + s, err := storage.NewLocalStorage(cpDir) + if err != nil { + return errors.Trace(err) + } + + // create normal files belong to global-checkpoint files + for _, info := range infos { + filename := fmt.Sprintf("%v.ts", info.storeID) + buff := make([]byte, 8) + binary.LittleEndian.PutUint64(buff, info.global_checkpoint) + if _, err := s.Create(ctx, filename); err != nil { + return errors.Trace(err) + } + if err := s.WriteFile(ctx, filename, buff); err != nil { + return errors.Trace(err) + } + } + + // create a file not belonging to global-checkpoint-ts files + filename := fmt.Sprintf("%v.tst", 1) + err = s.WriteFile(ctx, filename, []byte("ping")) + return errors.AddStack(err) +} + +type fakeGlobalCheckPoint struct { + storeID int64 + global_checkpoint uint64 +} + +func TestGetGlobalCheckpointFromStorage(t *testing.T) { + ctx := context.Background() + tmpdir := t.TempDir() + s, err := storage.NewLocalStorage(tmpdir) + require.Nil(t, err) + + infos := []fakeGlobalCheckPoint{ + { + storeID: 1, + global_checkpoint: 98, + }, + { + storeID: 2, + global_checkpoint: 90, + }, + { + storeID: 2, + global_checkpoint: 99, + }, + } + + err = fakeCheckpointFiles(ctx, tmpdir, infos) + require.Nil(t, err) + + ts, err := getGlobalCheckpointFromStorage(ctx, s) + require.Nil(t, err) + require.Equal(t, ts, uint64(99)) +} diff --git a/br/pkg/utils/BUILD.bazel b/br/pkg/utils/BUILD.bazel index b708ec2fa7979..f4fabeb825882 100644 --- a/br/pkg/utils/BUILD.bazel +++ b/br/pkg/utils/BUILD.bazel @@ -27,10 +27,13 @@ go_library( "//br/pkg/logutil", "//br/pkg/metautil", "//errno", + "//kv", "//parser/model", "//parser/mysql", "//parser/terror", + "//sessionctx", "//util", + "//util/sqlexec", "@com_github_cheggaaa_pb_v3//:pb", "@com_github_google_uuid//:uuid", "@com_github_pingcap_errors//:errors", @@ -58,6 +61,7 @@ go_test( name = "utils_test", srcs = [ "backoff_test.go", + "db_test.go", "env_test.go", "json_test.go", "key_test.go", @@ -72,11 +76,17 @@ go_test( "//br/pkg/errors", "//br/pkg/metautil", "//br/pkg/storage", + "//parser/ast", "//parser/model", + "//parser/mysql", "//statistics/handle", "//tablecodec", "//testkit/testsetup", + "//types", + "//util/chunk", + "//util/sqlexec", "@com_github_golang_protobuf//proto", + "@com_github_pingcap_errors//:errors", "@com_github_pingcap_kvproto//pkg/brpb", "@com_github_pingcap_kvproto//pkg/encryptionpb", "@com_github_stretchr_testify//require", diff --git a/br/pkg/utils/misc.go b/br/pkg/utils/misc.go new file mode 100644 index 0000000000000..43efb4755a26b --- /dev/null +++ b/br/pkg/utils/misc.go @@ -0,0 +1,77 @@ +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package utils + +import ( + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/types" +) + +// IsTypeCompatible checks whether type target is compatible with type src +// they're compatible if +// - same null/not null and unsigned flag(maybe we can allow src not null flag, target null flag later) +// - have same evaluation type +// - target's flen and decimal should be bigger or equals to src's +// - elements in target is superset of elements in src if they're enum or set type +// - same charset and collate if they're string types +func IsTypeCompatible(src types.FieldType, target types.FieldType) bool { + if mysql.HasNotNullFlag(src.GetFlag()) != mysql.HasNotNullFlag(target.GetFlag()) { + return false + } + if mysql.HasUnsignedFlag(src.GetFlag()) != mysql.HasUnsignedFlag(target.GetFlag()) { + return false + } + srcEType, dstEType := src.EvalType(), target.EvalType() + if srcEType != dstEType { + return false + } + + getFLenAndDecimal := func(tp types.FieldType) (int, int) { + // ref FieldType.CompactStr + defaultFlen, defaultDecimal := mysql.GetDefaultFieldLengthAndDecimal(tp.GetType()) + flen, decimal := tp.GetFlen(), tp.GetDecimal() + if flen == types.UnspecifiedLength { + flen = defaultFlen + } + if decimal == types.UnspecifiedLength { + decimal = defaultDecimal + } + return flen, decimal + } + srcFLen, srcDecimal := getFLenAndDecimal(src) + targetFLen, targetDecimal := getFLenAndDecimal(target) + if srcFLen > targetFLen || srcDecimal > targetDecimal { + return false + } + + // if they're not enum or set type, elems will be empty + // and if they're not string types, charset and collate will be empty, + // so we check them anyway. + srcElems := src.GetElems() + targetElems := target.GetElems() + if len(srcElems) > len(targetElems) { + return false + } + targetElemSet := make(map[string]struct{}) + for _, item := range targetElems { + targetElemSet[item] = struct{}{} + } + for _, item := range srcElems { + if _, ok := targetElemSet[item]; !ok { + return false + } + } + return src.GetCharset() == target.GetCharset() && + src.GetCollate() == target.GetCollate() +} diff --git a/br/pkg/utils/misc_test.go b/br/pkg/utils/misc_test.go new file mode 100644 index 0000000000000..6e6afd0eae074 --- /dev/null +++ b/br/pkg/utils/misc_test.go @@ -0,0 +1,138 @@ +// 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, +// See the License for the specific language governing permissions and +// limitations under the License. + +package utils + +import ( + "testing" + + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/types" + "github.com/stretchr/testify/require" +) + +func TestIsTypeCompatible(t *testing.T) { + { + // different unsigned flag + src := types.NewFieldType(mysql.TypeInt24) + src.AddFlag(mysql.UnsignedFlag) + target := types.NewFieldType(mysql.TypeInt24) + require.False(t, IsTypeCompatible(*src, *target)) + + src.DelFlag(mysql.UnsignedFlag) + target.AddFlag(mysql.UnsignedFlag) + require.False(t, IsTypeCompatible(*src, *target)) + } + { + // different not null flag + src := types.NewFieldType(mysql.TypeInt24) + src.AddFlag(mysql.NotNullFlag) + target := types.NewFieldType(mysql.TypeInt24) + require.False(t, IsTypeCompatible(*src, *target)) + + src.DelFlag(mysql.NotNullFlag) + target.AddFlag(mysql.NotNullFlag) + require.False(t, IsTypeCompatible(*src, *target)) + } + { + // different evaluation type + src := types.NewFieldType(mysql.TypeInt24) + target := types.NewFieldType(mysql.TypeFloat) + require.False(t, IsTypeCompatible(*src, *target)) + } + { + // src flen > target + src := types.NewFieldType(mysql.TypeInt24) + target := types.NewFieldType(mysql.TypeTiny) + require.False(t, IsTypeCompatible(*src, *target)) + } + { + // src flen > target + src := types.NewFieldType(mysql.TypeVarchar) + src.SetFlen(100) + target := types.NewFieldType(mysql.TypeVarchar) + target.SetFlag(99) + require.False(t, IsTypeCompatible(*src, *target)) + } + { + // src decimal > target + src := types.NewFieldType(mysql.TypeNewDecimal) + src.SetDecimal(5) + target := types.NewFieldType(mysql.TypeNewDecimal) + target.SetDecimal(4) + require.False(t, IsTypeCompatible(*src, *target)) + } + { + // src has more elements + src := types.NewFieldType(mysql.TypeEnum) + src.SetElems([]string{"a", "b"}) + target := types.NewFieldType(mysql.TypeEnum) + target.SetElems([]string{"a"}) + require.False(t, IsTypeCompatible(*src, *target)) + } + { + // incompatible enum + src := types.NewFieldType(mysql.TypeEnum) + src.SetElems([]string{"a", "b"}) + target := types.NewFieldType(mysql.TypeEnum) + target.SetElems([]string{"a", "c", "d"}) + require.False(t, IsTypeCompatible(*src, *target)) + } + { + // incompatible charset + src := types.NewFieldType(mysql.TypeVarchar) + src.SetCharset("gbk") + target := types.NewFieldType(mysql.TypeVarchar) + target.SetCharset("utf8") + require.False(t, IsTypeCompatible(*src, *target)) + } + { + // incompatible collation + src := types.NewFieldType(mysql.TypeVarchar) + src.SetCharset("utf8") + src.SetCollate("utf8_bin") + target := types.NewFieldType(mysql.TypeVarchar) + target.SetCharset("utf8") + target.SetCollate("utf8_general_ci") + require.False(t, IsTypeCompatible(*src, *target)) + } + { + src := types.NewFieldType(mysql.TypeVarchar) + src.SetFlen(10) + src.SetCharset("utf8") + src.SetCollate("utf8_bin") + target := types.NewFieldType(mysql.TypeVarchar) + target.SetFlen(11) + target.SetCharset("utf8") + target.SetCollate("utf8_bin") + require.True(t, IsTypeCompatible(*src, *target)) + } + { + src := types.NewFieldType(mysql.TypeBlob) + target := types.NewFieldType(mysql.TypeLongBlob) + require.True(t, IsTypeCompatible(*src, *target)) + } + { + src := types.NewFieldType(mysql.TypeEnum) + src.SetElems([]string{"a", "b"}) + target := types.NewFieldType(mysql.TypeEnum) + target.SetElems([]string{"a", "b", "c"}) + require.True(t, IsTypeCompatible(*src, *target)) + } + { + src := types.NewFieldType(mysql.TypeTimestamp) + target := types.NewFieldType(mysql.TypeTimestamp) + target.SetDecimal(3) + require.True(t, IsTypeCompatible(*src, *target)) + } +} diff --git a/br/tests/_utils/br_tikv_outage_util b/br/tests/_utils/br_tikv_outage_util index 7386425aff8d4..cd3283a27ff08 100644 --- a/br/tests/_utils/br_tikv_outage_util +++ b/br/tests/_utils/br_tikv_outage_util @@ -41,7 +41,7 @@ load() { check() { run_sql 'drop database if exists '$TEST_NAME';' - run_br restore full -s local://"$backup_dir" + run_br restore full --filter '*.*' --filter '!mysql.*' -s local://"$backup_dir" count=$(run_sql 'select count(*) from '$TEST_NAME'.usertable;' | tail -n 1 | awk '{print $2}') [ "$count" -eq 20000 ] } \ No newline at end of file diff --git a/br/tests/_utils/run_services b/br/tests/_utils/run_services index a0fbe006fd189..a7449cb229bf2 100644 --- a/br/tests/_utils/run_services +++ b/br/tests/_utils/run_services @@ -44,6 +44,11 @@ stop() { killall -v -9 "$svc" &>/dev/null || return 0 } +restart_services() { + stop_services + start_services +} + stop_services() { for svc in "br" "tidb-server" "tiflash" "TiFlashMain" "tikv-server" "pd-server" "cdc" "minio"; do stop $svc & diff --git a/br/tests/_utils/run_sql_file b/br/tests/_utils/run_sql_file new file mode 100755 index 0000000000000..59942940c076a --- /dev/null +++ b/br/tests/_utils/run_sql_file @@ -0,0 +1,28 @@ +#!/bin/bash +# +# Copyright 2022 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +set -euo pipefail + +SQL_FILE="$1" +shift + +echo "[$(date)] Executing SQL_FILE: $SQL_FILE" > "$TEST_DIR/sql_res.$TEST_NAME.txt" +cat $SQL_FILE | mysql -uroot -h127.0.0.1 -P4000 \ + --ssl-ca="$TEST_DIR/certs/ca.pem" \ + --ssl-cert="$TEST_DIR/certs/curl.pem" \ + --ssl-key="$TEST_DIR/certs/curl.key" \ + "$@" \ + --default-character-set utf8 -E | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" diff --git a/br/tests/br_azblob/_run.sh b/br/tests/br_azblob/_run.sh index 5cc12b6a492cf..30d08cc00343e 100644 --- a/br/tests/br_azblob/_run.sh +++ b/br/tests/br_azblob/_run.sh @@ -52,7 +52,7 @@ done # new version restore full echo "restore start..." -run_br restore full \ +run_br restore full --filter '*.*' --filter '!mysql.*' \ -s "azure://$CONTAINER/$DB?" \ --pd $PD_ADDR --azblob.endpoint="$AZBLOB_ENDPOINT" \ --azblob.account-name="$ACCOUNT_NAME" \ diff --git a/br/tests/br_backup_empty/run.sh b/br/tests/br_backup_empty/run.sh index 41ca818139ba4..e55dd25245414 100644 --- a/br/tests/br_backup_empty/run.sh +++ b/br/tests/br_backup_empty/run.sh @@ -28,7 +28,7 @@ fi # restore empty. echo "restore empty cluster start..." -run_br restore full -s "local://$TEST_DIR/empty_cluster" --pd $PD_ADDR --ratelimit 1024 +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/empty_cluster" --pd $PD_ADDR --ratelimit 1024 if [ $? -ne 0 ]; then echo "TEST: [$TEST_NAME] failed on restore empty cluster!" exit 1 @@ -55,7 +55,7 @@ done # restore empty. echo "restore empty db start..." -run_br restore full -s "local://$TEST_DIR/empty_db" --pd $PD_ADDR --ratelimit 1024 +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/empty_db" --pd $PD_ADDR --ratelimit 1024 if [ $? -ne 0 ]; then echo "TEST: [$TEST_NAME] failed on restore empty cluster!" exit 1 @@ -76,7 +76,7 @@ while [ $i -le $DB_COUNT ]; do done echo "restore empty table start..." -run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/empty_table" +run_br --pd $PD_ADDR restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/empty_table" # insert one row to make sure table is restored. run_sql "INSERT INTO ${DB}1.usertable1 VALUES (\"a\", \"b\");" diff --git a/br/tests/br_crypter/run.sh b/br/tests/br_crypter/run.sh index 11cef56b334e5..6dae71360b8fc 100755 --- a/br/tests/br_crypter/run.sh +++ b/br/tests/br_crypter/run.sh @@ -59,7 +59,7 @@ function test_crypter_plaintext(){ drop_db echo "restore with crypter method of plaintext" - run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$DB/plaintext" --crypter.method "PLAINTEXT" + run_br --pd $PD_ADDR restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB/plaintext" --crypter.method "PLAINTEXT" check_db_row } @@ -79,7 +79,7 @@ function test_crypter(){ echo "restore crypter method of $CRYPTER_METHOD with wrong key of $CRYPTER_WRONG_KEY" restore_fail=0 - run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$DB/$CRYPTER_METHOD" \ + run_br --pd $PD_ADDR restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB/$CRYPTER_METHOD" \ --crypter.method $CRYPTER_METHOD --crypter.key $CRYPTER_WRONG_KEY || restore_fail=1 if [ $restore_fail -ne 1 ]; then echo "TEST: [$TEST_NAME] test restore crypter with wrong key failed!" @@ -87,7 +87,7 @@ function test_crypter(){ fi echo "restore crypter method of $CRYPTER_METHOD with the key of $CRYPTER_KEY" - run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$DB/$CRYPTER_METHOD" \ + run_br --pd $PD_ADDR restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB/$CRYPTER_METHOD" \ --crypter.method $CRYPTER_METHOD --crypter.key $CRYPTER_KEY check_db_row @@ -103,7 +103,7 @@ function test_crypter(){ echo "backup crypter method of $CRYPTER_METHOD with the wrong key-file of $CRYPTER_WRONG_KEY_FILE" restore_fail=0 - run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \ + run_br --pd $PD_ADDR restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \ --crypter.method $CRYPTER_METHOD --crypter.key-file $CRYPTER_WRONG_KEY_FILE || restore_fail=1 if [ $restore_fail -ne 1 ]; then echo "TEST: [$TEST_NAME] test restore with wrong key-file failed!" @@ -111,7 +111,7 @@ function test_crypter(){ fi echo "restore crypter method of $CRYPTER_METHOD with the key-file of $CRYPTER_KEY_FILE" - run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \ + run_br --pd $PD_ADDR restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \ --crypter.method $CRYPTER_METHOD --crypter.key-file $CRYPTER_KEY_FILE check_db_row diff --git a/br/tests/br_crypter2/run.sh b/br/tests/br_crypter2/run.sh index 91de74376ebce..5f6bff262358b 100755 --- a/br/tests/br_crypter2/run.sh +++ b/br/tests/br_crypter2/run.sh @@ -70,7 +70,7 @@ run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_fil drop_db -run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \ +run_br --pd $PD_ADDR restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \ --check-requirements=false --crypter.method $CRYPTER_METHOD --crypter.key $CRYPTER_KEY check_db_row diff --git a/br/tests/br_full/run.sh b/br/tests/br_full/run.sh index 8d754dbd28af1..6c5c039069bd9 100755 --- a/br/tests/br_full/run.sh +++ b/br/tests/br_full/run.sh @@ -81,7 +81,7 @@ for ct in limit lz4 zstd; do echo "restore with $ct backup start..." export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/restore/restore-storage-error=1*return(\"connection refused\");github.com/pingcap/tidb/br/pkg/restore/restore-gRPC-error=1*return(true)" export GO_FAILPOINTS=$GO_FAILPOINTS";github.com/pingcap/tidb/br/pkg/restore/no-leader-error=3*return(true)" - run_br restore full -s "local://$TEST_DIR/$DB-$ct" --pd $PD_ADDR --ratelimit 1024 + run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB-$ct" --pd $PD_ADDR --ratelimit 1024 export GO_FAILPOINTS="" for i in $(seq $DB_COUNT); do diff --git a/br/tests/br_full_cluster_restore/full_data.sql b/br/tests/br_full_cluster_restore/full_data.sql new file mode 100644 index 0000000000000..77429bff2e657 --- /dev/null +++ b/br/tests/br_full_cluster_restore/full_data.sql @@ -0,0 +1,28 @@ +create database db1; +create table db1.t1(id int, val varchar(100)); +insert into db1.t1 values(1, 'a'), (2, 'b'); +create table db1.t2(id int, val varchar(100)); +insert into db1.t2 values(1, 'a'), (2, 'b'); +create database db2; +create table db2.t1(id int, val varchar(100)); +insert into db2.t1 values(1, 'a'), (2, 'b'); +create table db2.t2(id int, val varchar(100)); +insert into db2.t2 values(1, 'a'), (2, 'b'); + +-- user1 can select on db1 and select on db2.t1 and db2.t2 +create user user1 identified by '123456'; +grant select on db1.* to user1; -- mysql.db +grant select on db2.t1 to user1; -- mysql.tables_priv +grant select, update(val) on db2.t2 to user1; -- mysql.tables_priv mysql.columns_priv + +-- user2 default role is role1 which can do select on db1 +create role role1; +grant select on db1.* to role1; +create user user2 identified by '123456'; +grant role1 to user2; -- mysql.role_edges +set default role all to user2; -- mysql.default_roles +grant ROLE_ADMIN on *.* to user2; -- mysql.global_grants + +-- user3 can only login with ssl and select db1.t1 +create user user3 identified by '123456' require ssl; +grant select on db1.t1 to user3; diff --git a/br/tests/br_full_cluster_restore/run.sh b/br/tests/br_full_cluster_restore/run.sh new file mode 100644 index 0000000000000..257bce42f73d3 --- /dev/null +++ b/br/tests/br_full_cluster_restore/run.sh @@ -0,0 +1,87 @@ +#! /bin/bash + +set -eu + +# we need to keep backup data after restart service +backup_dir=$TEST_DIR/keep_${TEST_NAME} +incr_backup_dir=${backup_dir}_inc +res_file="$TEST_DIR/sql_res.$TEST_NAME.txt" +br_log_file=$TEST_DIR/br.log +source tests/_utils/run_services + +function run_sql_as() { + user=$1 + shift + SQL="$1" + shift + echo "[$(date)] Executing SQL with user $user: $SQL" + mysql -u$user -p123456 -h127.0.0.1 -P4000 \ + "$@" \ + --default-character-set utf8 -E -e "$SQL" > "$TEST_DIR/sql_res.$TEST_NAME.txt" 2>&1 +} + +restart_services + +unset BR_LOG_TO_TERM +run_sql_file tests/${TEST_NAME}/full_data.sql +run_br backup full --log-file $br_log_file -s "local://$backup_dir" + +# br.test will add coverage output, so we use head here +LAST_BACKUP_TS=$(run_br validate decode --log-file $br_log_file --field="end-version" -s "local://$backup_dir" 2>/dev/null | head -n1) +run_sql "insert into db2.t1 values(3, 'c'), (4, 'd'), (5, 'e');" +run_br backup full --log-file $br_log_file --lastbackupts $LAST_BACKUP_TS -s "local://$incr_backup_dir" + +run_sql "drop database db2" + +echo "--> cluster is not fresh" +run_br restore full --log-file $br_log_file -s "local://$backup_dir" > $res_file 2>&1 || true +check_contains "the target cluster is not fresh" + +echo "--> non full backup data" +run_br restore full --log-file $br_log_file -s "local://$incr_backup_dir" +run_sql "select count(*) from db2.t1" +check_contains "count(*): 3" + +echo "--> restore using filter" +run_sql "drop database db1; drop database db2" +run_br restore full --log-file $br_log_file --filter '*.*' --filter '!mysql.*' -s "local://$backup_dir" +run_sql "select count(*) from db2.t1" +check_contains "count(*): 2" + +echo "--> incompatible system table: column count mismatch" +restart_services +# mock incompatible manually +run_sql "alter table mysql.user add column xx int;" +run_br restore full --log-file $br_log_file -s "local://$backup_dir" > $res_file 2>&1 || true +check_contains "the target cluster is not compatible with the backup data" + +echo "--> incompatible system table: column type incompatible" +restart_services +# mock incompatible manually +run_sql "alter table mysql.tables_priv modify column Table_priv set('Select') DEFAULT NULL;" +run_br restore full --log-file $br_log_file -s "local://$backup_dir" > $res_file 2>&1 || true +check_contains "the target cluster is not compatible with the backup data" + +echo "--> full cluster restore" +restart_services +run_br restore full --log-file $br_log_file -s "local://$backup_dir" +run_sql_as user1 "select count(*) from db1.t1" +check_contains "count(*): 2" +run_sql_as user1 "select count(*) from db2.t1" +check_contains "count(*): 2" +# user2 can select on db1 but not db2 +run_sql_as user2 "select count(*) from db1.t1" +check_contains "count(*): 2" +run_sql_as user2 "select count(*) from db2.t1" || true +check_contains "SELECT command denied to user" +# user3 can only query db1.t1 using ssl +# ci env uses mariadb client, ssl flag is different with mysql client +run_sql_as user3 "select count(*) from db1.t1" || true +check_contains "Access denied for user" +run_sql_as user3 "select count(*) from db1.t1" --ssl +check_contains "count(*): 2" +run_sql_as user3 "select count(*) from db1.t2" --ssl || true +check_contains "SELECT command denied to user" + +echo "clean up kept backup" +rm -rf $backup_dir $incr_backup_dir diff --git a/br/tests/br_full_ddl/run.sh b/br/tests/br_full_ddl/run.sh index 9ae563c6b0aef..1433c1f71e9a6 100755 --- a/br/tests/br_full_ddl/run.sh +++ b/br/tests/br_full_ddl/run.sh @@ -97,7 +97,7 @@ if [[ "${cluster_index_before_backup}" != "${cluster_index_before_restore}" ]]; fi echo "restore full without stats..." -run_br restore full -s "local://$TEST_DIR/${DB}_disable_stats" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/${DB}_disable_stats" --pd $PD_ADDR curl $TIDB_IP:10080/stats/dump/$DB/$TABLE | jq '.columns.field0 | del(.last_update_version, .fm_sketch, .correlation)' > $RESOTRE_STAT # stats should not be equal because we disable stats by default. @@ -114,7 +114,7 @@ run_sql "DROP DATABASE $DB;" # restore full echo "restore start..." export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/restore/restore-createtables-error=return(true)" -run_br restore full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --log-file $RESTORE_LOG --ddl-batch-size=128 || { cat $RESTORE_LOG; } +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --log-file $RESTORE_LOG --ddl-batch-size=128 || { cat $RESTORE_LOG; } export GO_FAILPOINTS="" panic_count=$(cat $RESTORE_LOG | grep "panic"| wc -l) @@ -128,7 +128,7 @@ run_sql "DROP DATABASE $DB;" # restore full echo "restore start..." export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/pdutil/PDEnabledPauseConfig=return(true)" -run_br restore full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --log-file $LOG || { cat $LOG; exit 1; } +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --log-file $LOG || { cat $LOG; exit 1; } export GO_FAILPOINTS="" pause_count=$(cat $LOG | grep "pause configs successful"| wc -l | xargs) diff --git a/br/tests/br_full_index/run.sh b/br/tests/br_full_index/run.sh index be43a423e6549..030d2416c3822 100755 --- a/br/tests/br_full_index/run.sh +++ b/br/tests/br_full_index/run.sh @@ -57,7 +57,7 @@ done # restore full echo "restore start..." -run_br restore full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB" --pd $PD_ADDR for i in $(seq $DB_COUNT); do row_count_new[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}') diff --git a/br/tests/br_gcs/run.sh b/br/tests/br_gcs/run.sh index 058191864ad71..103db54c2ed74 100755 --- a/br/tests/br_gcs/run.sh +++ b/br/tests/br_gcs/run.sh @@ -98,7 +98,7 @@ done # new version restore full echo "restore start..." -run_br restore full -s "gcs://$BUCKET/$DB?" --pd $PD_ADDR --gcs.endpoint="http://$GCS_HOST:$GCS_PORT/storage/v1/" --check-requirements=false +run_br restore full --filter '*.*' --filter '!mysql.*' -s "gcs://$BUCKET/$DB?" --pd $PD_ADDR --gcs.endpoint="http://$GCS_HOST:$GCS_PORT/storage/v1/" --check-requirements=false for i in $(seq $DB_COUNT); do row_count_new[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}') @@ -126,7 +126,7 @@ for i in $(seq $DB_COUNT); do done echo "v4.0.8 version restore start..." -run_br restore full -s "gcs://$BUCKET/${DB}_old" --pd $PD_ADDR --gcs.endpoint="http://$GCS_HOST:$GCS_PORT/storage/v1/" --check-requirements=false +run_br restore full --filter '*.*' --filter '!mysql.*' -s "gcs://$BUCKET/${DB}_old" --pd $PD_ADDR --gcs.endpoint="http://$GCS_HOST:$GCS_PORT/storage/v1/" --check-requirements=false for i in $(seq $DB_COUNT); do row_count_new[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}') diff --git a/br/tests/br_history/run.sh b/br/tests/br_history/run.sh index 161b01f8c8ba5..728583265adc4 100755 --- a/br/tests/br_history/run.sh +++ b/br/tests/br_history/run.sh @@ -43,7 +43,7 @@ run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB" --timeago "10s" # restore full echo "restore start..." -run_br restore full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB" --pd $PD_ADDR for i in $(seq $DB_COUNT); do row_count_new[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}') diff --git a/br/tests/br_incremental_same_table/run.sh b/br/tests/br_incremental_same_table/run.sh index 67b502ae819d1..021f6b020f460 100755 --- a/br/tests/br_incremental_same_table/run.sh +++ b/br/tests/br_incremental_same_table/run.sh @@ -62,7 +62,7 @@ done # full restore echo "full restore start..." -run_br restore full -s "local://$TEST_DIR/$DB/full" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB/full" --pd $PD_ADDR row_count_full=$(run_sql "SELECT COUNT(*) FROM $DB.$TABLE;" | awk '/COUNT/{print $2}') # check full restore if [ "${row_count_full}" != "${ROW_COUNT}" ];then diff --git a/br/tests/br_insert_after_restore/run.sh b/br/tests/br_insert_after_restore/run.sh index 5d8aa947b43a4..a6a9872cc0003 100755 --- a/br/tests/br_insert_after_restore/run.sh +++ b/br/tests/br_insert_after_restore/run.sh @@ -48,7 +48,7 @@ run_sql "DROP DATABASE $DB;" # restore full echo "restore start..." -run_br restore full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB" --pd $PD_ADDR row_count_new=$(run_sql "SELECT COUNT(*) FROM $DB.$TABLE;" | awk '/COUNT/{print $2}') diff --git a/br/tests/br_restore_TDE_enable/run.sh b/br/tests/br_restore_TDE_enable/run.sh index 793a390951a6e..4b10f8d86744d 100755 --- a/br/tests/br_restore_TDE_enable/run.sh +++ b/br/tests/br_restore_TDE_enable/run.sh @@ -119,7 +119,7 @@ bin/tikv-ctl --config=tests/config/tikv.toml encryption-meta dump-file | grep "A RESTORE_LOG="restore.log" rm -f $RESTORE_LOG unset BR_LOG_TO_TERM - run_br restore full -s "s3://mybucket/$DB?$S3_KEY" --pd $PD_ADDR --s3.endpoint="http://$S3_ENDPOINT" \ + run_br restore full --filter '*.*' --filter '!mysql.*' -s "s3://mybucket/$DB?$S3_KEY" --pd $PD_ADDR --s3.endpoint="http://$S3_ENDPOINT" \ --log-file $RESTORE_LOG for i in $(seq $DB_COUNT); do diff --git a/br/tests/br_s3/run.sh b/br/tests/br_s3/run.sh index df06051358ae5..30474f44e7b88 100755 --- a/br/tests/br_s3/run.sh +++ b/br/tests/br_s3/run.sh @@ -118,7 +118,7 @@ for p in $(seq 2); do RESTORE_LOG="restore.log" rm -f $RESTORE_LOG unset BR_LOG_TO_TERM - ( run_br restore full -s "s3://mybucket/$DB?$S3_KEY" --pd $PD_ADDR --s3.endpoint="http://$S3_ENDPOINT" \ + ( run_br restore full --filter '*.*' --filter '!mysql.*' -s "s3://mybucket/$DB?$S3_KEY" --pd $PD_ADDR --s3.endpoint="http://$S3_ENDPOINT" \ --ratelimit 1 \ --log-file $RESTORE_LOG || \ ( cat $RESTORE_LOG && BR_LOG_TO_TERM=1 && exit 1 ) ) & diff --git a/br/tests/br_skip_checksum/run.sh b/br/tests/br_skip_checksum/run.sh index fc7e7c7ed3535..35c49d05a3fa9 100755 --- a/br/tests/br_skip_checksum/run.sh +++ b/br/tests/br_skip_checksum/run.sh @@ -38,7 +38,7 @@ done # restore full, skipping genreate checksum. echo "restore start..." -run_br restore full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --ratelimit 1024 --checksum=false +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --ratelimit 1024 --checksum=false for i in $(seq $DB_COUNT); do row_count_new[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}') @@ -63,7 +63,7 @@ for i in $(seq $DB_COUNT); do run_sql "DROP DATABASE $DB${i};" done echo "restore(with checksum) start..." -run_br restore full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --ratelimit 1024 +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --ratelimit 1024 for i in $(seq $DB_COUNT); do row_count_new[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}') diff --git a/br/tests/br_small_batch_size/run.sh b/br/tests/br_small_batch_size/run.sh index bb91a3f9ad3c3..645c65141e46f 100755 --- a/br/tests/br_small_batch_size/run.sh +++ b/br/tests/br_small_batch_size/run.sh @@ -69,7 +69,7 @@ run_sql "drop database $DB" echo "restore start..." GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/task/small-batch-size=return(2)" \ -run_br restore full -s "local://$backup_dir" --pd $PD_ADDR --ratelimit 1024 +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$backup_dir" --pd $PD_ADDR --ratelimit 1024 for i in $record_counts; do check_size "t$i" $i diff --git a/br/tests/br_split_region_fail/run.sh b/br/tests/br_split_region_fail/run.sh index 13b511313ebfa..6e8d93fdf646a 100644 --- a/br/tests/br_split_region_fail/run.sh +++ b/br/tests/br_split_region_fail/run.sh @@ -47,7 +47,7 @@ echo "restore start..." unset BR_LOG_TO_TERM GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/restore/not-leader-error=1*return(true)->1*return(false);\ github.com/pingcap/tidb/br/pkg/restore/somewhat-retryable-error=3*return(true)" \ -run_br restore full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --ratelimit 1024 --log-file $LOG || true +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --ratelimit 1024 --log-file $LOG || true BR_LOG_TO_TERM=1 grep "a error occurs on split region" $LOG && \ diff --git a/br/tests/br_table_filter/run.sh b/br/tests/br_table_filter/run.sh index 272e1fce25c5c..ee90fcc4639c9 100755 --- a/br/tests/br_table_filter/run.sh +++ b/br/tests/br_table_filter/run.sh @@ -39,7 +39,7 @@ echo 'Simple check' run_br backup full -f "$DB.*" -s "local://$TEST_DIR/$DB/full" --pd $PD_ADDR run_sql "drop schema $DB;" -run_br restore full -s "local://$TEST_DIR/$DB/full" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB/full" --pd $PD_ADDR run_sql "select c from $DB.one;" run_sql "select c from $DB.two;" @@ -53,7 +53,7 @@ echo 'Filtered backup check' run_br backup full -f "$DB.t*" -s "local://$TEST_DIR/$DB/t" --pd $PD_ADDR run_sql "drop schema $DB;" -run_br restore full -s "local://$TEST_DIR/$DB/t" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB/t" --pd $PD_ADDR ! run_sql "select c from $DB.one;" run_sql "select c from $DB.two;" @@ -105,10 +105,10 @@ run_sql 'select c from '"$DB"'.`the,special,table`;' echo 'Case sensitive backup check' run_sql "drop schema $DB;" -run_br restore full --case-sensitive -s "local://$TEST_DIR/$DB/full" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' --case-sensitive -s "local://$TEST_DIR/$DB/full" --pd $PD_ADDR run_br backup full --case-sensitive -f "$DB.[oF]*" -s "local://$TEST_DIR/$DB/of" --pd $PD_ADDR run_sql "drop schema $DB;" -run_br restore full --case-sensitive -s "local://$TEST_DIR/$DB/of" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' --case-sensitive -s "local://$TEST_DIR/$DB/of" --pd $PD_ADDR run_sql "select c from $DB.one;" ! run_sql "select c from $DB.two;" diff --git a/br/tests/br_table_partition/run.sh b/br/tests/br_table_partition/run.sh index 87cd79a49c26c..74eec09f85ff3 100755 --- a/br/tests/br_table_partition/run.sh +++ b/br/tests/br_table_partition/run.sh @@ -38,7 +38,7 @@ run_sql "DROP DATABASE $DB;" # restore full echo "restore start..." -run_br restore full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB" --pd $PD_ADDR for i in $(seq $TABLE_COUNT) _Hash _List; do run_sql "SHOW CREATE TABLE $DB.$TABLE${i};" | grep 'PARTITION' diff --git a/br/tests/br_tiflash/run.sh b/br/tests/br_tiflash/run.sh index 2d8b0d6419256..e507f23f03a04 100644 --- a/br/tests/br_tiflash/run.sh +++ b/br/tests/br_tiflash/run.sh @@ -52,7 +52,7 @@ rm -rf "/${TEST_DIR}/$DB" run_br backup full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR run_sql "DROP DATABASE $DB" -run_br restore full -s "local://$TEST_DIR/$DB" --pd $PD_ADDR +run_br restore full --filter '*.*' --filter '!mysql.*' -s "local://$TEST_DIR/$DB" --pd $PD_ADDR # wating for TiFlash sync sleep 100 diff --git a/config/config.go b/config/config.go index d5bca9f1c2692..81d2f021bfc25 100644 --- a/config/config.go +++ b/config/config.go @@ -954,7 +954,7 @@ var defaultConf = Config{ TrxSummary: DefaultTrxSummary(), LogBackup: LogBackup{ Advancer: logbackupconf.Default(), - Enabled: false, + Enabled: true, }, } diff --git a/ddl/column.go b/ddl/column.go index 792b1ca54c057..5c40c4d10e397 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -119,7 +119,7 @@ func onAddColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) if err != nil { if ifNotExists && infoschema.ErrColumnExists.Equal(err) { job.Warning = toTError(err) - job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo) + job.State = model.JobStateDone return ver, nil } return ver, errors.Trace(err) @@ -230,7 +230,7 @@ func onDropColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) if ifExists && dbterror.ErrCantDropFieldOrKey.Equal(err) { // Convert the "not exists" error to a warning. job.Warning = toTError(err) - job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) + job.State = model.JobStateDone return ver, nil } return ver, errors.Trace(err) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2a68a26fa1d77..e2a015301bcea 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -511,7 +511,7 @@ func checkMultiSchemaSpecs(_sctx sessionctx.Context, specs []*ast.DatabaseOption for _, spec := range specs { if spec.Tp == ast.DatabaseSetTiFlashReplica { if hasSetTiFlashReplica { - return dbterror.ErrRunMultiSchemaChanges + return dbterror.ErrRunMultiSchemaChanges.FastGenByArgs(model.ActionSetTiFlashReplica.String()) } hasSetTiFlashReplica = true } diff --git a/ddl/ddl_tiflash_test.go b/ddl/ddl_tiflash_test.go index 949b09f7738fd..2db35d19cb744 100644 --- a/ddl/ddl_tiflash_test.go +++ b/ddl/ddl_tiflash_test.go @@ -783,10 +783,10 @@ func TestAlterDatabaseErrorGrammar(t *testing.T) { defer tear() tk := testkit.NewTestKit(t, store) - tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 SET TIFLASH REPLICA 2 LOCATION LABELS 'a','b'", "[ddl:8200]Unsupported multi schema change") - tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 SET TIFLASH REPLICA 2", "[ddl:8200]Unsupported multi schema change") - tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 LOCATION LABELS 'a','b' SET TIFLASH REPLICA 2", "[ddl:8200]Unsupported multi schema change") - tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 LOCATION LABELS 'a','b' SET TIFLASH REPLICA 2 LOCATION LABELS 'a','b'", "[ddl:8200]Unsupported multi schema change") + tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 SET TIFLASH REPLICA 2 LOCATION LABELS 'a','b'", "[ddl:8200]Unsupported multi schema change for set tiflash replica") + tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 SET TIFLASH REPLICA 2", "[ddl:8200]Unsupported multi schema change for set tiflash replica") + tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 LOCATION LABELS 'a','b' SET TIFLASH REPLICA 2", "[ddl:8200]Unsupported multi schema change for set tiflash replica") + tk.MustGetErrMsg("ALTER DATABASE t SET TIFLASH REPLICA 1 LOCATION LABELS 'a','b' SET TIFLASH REPLICA 2 LOCATION LABELS 'a','b'", "[ddl:8200]Unsupported multi schema change for set tiflash replica") } func TestAlterDatabaseBasic(t *testing.T) { diff --git a/ddl/index.go b/ddl/index.go index 40b8b3cf11ebc..f89fa95ad9c0d 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -695,7 +695,7 @@ func onDropIndex(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { if err != nil { if ifExists && dbterror.ErrCantDropFieldOrKey.Equal(err) { job.Warning = toTError(err) - job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) + job.State = model.JobStateDone return ver, nil } return ver, errors.Trace(err) diff --git a/ddl/multi_schema_change.go b/ddl/multi_schema_change.go index ddb298c2f92e3..894b926ba512e 100644 --- a/ddl/multi_schema_change.go +++ b/ddl/multi_schema_change.go @@ -76,7 +76,7 @@ func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ve if err != nil { return ver, err } - sub.FromProxyJob(&proxyJob) + sub.FromProxyJob(&proxyJob, ver) return ver, nil } // The last rollback/cancelling sub-job is done. @@ -95,7 +95,7 @@ func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ve } proxyJob := sub.ToProxyJob(job) ver, err = w.runDDLJob(d, t, &proxyJob) - sub.FromProxyJob(&proxyJob) + sub.FromProxyJob(&proxyJob, ver) handleRevertibleException(job, sub, proxyJob.Error) return ver, err } @@ -106,16 +106,23 @@ func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ve if err != nil { return ver, err } + var schemaVersionGenerated = false subJobs := make([]model.SubJob, len(job.MultiSchemaInfo.SubJobs)) // Step the sub-jobs to the non-revertible states all at once. + // We only generate 1 schema version for these sub-job. for i, sub := range job.MultiSchemaInfo.SubJobs { if sub.IsFinished() { continue } subJobs[i] = *sub proxyJob := sub.ToProxyJob(job) + if schemaVersionGenerated { + proxyJob.MultiSchemaInfo.SkipVersion = true + } else { + schemaVersionGenerated = true + } ver, err = w.runDDLJob(d, t, &proxyJob) - sub.FromProxyJob(&proxyJob) + sub.FromProxyJob(&proxyJob, ver) if err != nil || proxyJob.Error != nil { for j := i - 1; j >= 0; j-- { job.MultiSchemaInfo.SubJobs[j] = &subJobs[j] @@ -137,11 +144,10 @@ func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ve } proxyJob := sub.ToProxyJob(job) ver, err = w.runDDLJob(d, t, &proxyJob) - sub.FromProxyJob(&proxyJob) + sub.FromProxyJob(&proxyJob, ver) return ver, err } - job.State = model.JobStateDone - return ver, err + return finishMultiSchemaJob(job, t) } func handleRevertibleException(job *model.Job, subJob *model.SubJob, err *terror.Error) { @@ -252,7 +258,7 @@ func fillMultiSchemaInfo(info *model.MultiSchemaInfo, job *model.Job) (err error info.AlterIndexes = append(info.AlterIndexes, idxName) case model.ActionRebaseAutoID, model.ActionModifyTableComment, model.ActionModifyTableCharsetAndCollate: default: - return dbterror.ErrRunMultiSchemaChanges + return dbterror.ErrRunMultiSchemaChanges.FastGenByArgs(job.Type.String()) } return nil } @@ -358,3 +364,17 @@ func rollingBackMultiSchemaChange(job *model.Job) error { job.State = model.JobStateRollingback return dbterror.ErrCancelledDDLJob } + +func finishMultiSchemaJob(job *model.Job, t *meta.Meta) (ver int64, err error) { + for _, sub := range job.MultiSchemaInfo.SubJobs { + if ver < sub.SchemaVer { + ver = sub.SchemaVer + } + } + tblInfo, err := t.GetTable(job.SchemaID, job.TableID) + if err != nil { + return ver, err + } + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) + return ver, err +} diff --git a/ddl/multi_schema_change_test.go b/ddl/multi_schema_change_test.go index 60ba161ef09d6..e0e166e62e9b5 100644 --- a/ddl/multi_schema_change_test.go +++ b/ddl/multi_schema_change_test.go @@ -1137,6 +1137,17 @@ func TestMultiSchemaChangeNoSubJobs(t *testing.T) { require.Equal(t, "create table", rs[0][3]) } +func TestMultiSchemaChangeUnsupportedType(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + + tk.MustExec("create table t (a int, b int);") + tk.MustGetErrMsg("alter table t add column c int, auto_id_cache = 1;", + "[ddl:8200]Unsupported multi schema change for modify auto id cache") +} + type cancelOnceHook struct { store kv.Storage triggered bool diff --git a/ddl/table.go b/ddl/table.go index 4391ccd416f49..b253554516976 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -1334,7 +1334,7 @@ func updateVersionAndTableInfo(d *ddlCtx, t *meta.Meta, job *model.Job, tblInfo default: } }) - if shouldUpdateVer { + if shouldUpdateVer && (job.MultiSchemaInfo == nil || !job.MultiSchemaInfo.SkipVersion) { ver, err = updateSchemaVersion(d, t, job) if err != nil { return 0, errors.Trace(err) diff --git a/domain/domain.go b/domain/domain.go index 8785a3c440aad..95d945b70b80a 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -27,6 +27,7 @@ import ( "github.com/ngaut/pools" "github.com/pingcap/errors" "github.com/pingcap/failpoint" + "github.com/pingcap/log" "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/br/pkg/streamhelper" "github.com/pingcap/tidb/config" @@ -903,6 +904,10 @@ func (do *Domain) Init(ddlLease time.Duration, sysExecutorFactory func(*Domain) func (do *Domain) initLogBackup(ctx context.Context, pdClient pd.Client) error { cfg := config.GetGlobalConfig() if cfg.LogBackup.Enabled { + if pdClient == nil || do.etcdClient == nil { + log.Warn("pd / etcd client not provided, won't begin Advancer.") + return nil + } env, err := streamhelper.TiDBEnv(pdClient, do.etcdClient, cfg) if err != nil { return err diff --git a/errors.toml b/errors.toml index 73646c46ddbdf..5043c913d0147 100755 --- a/errors.toml +++ b/errors.toml @@ -181,6 +181,11 @@ error = ''' restore checksum mismatch ''' +["BR:Restore:ErrRestoreIncompatibleSys"] +error = ''' +incompatible system table +''' + ["BR:Restore:ErrRestoreInvalidBackup"] error = ''' invalid backup @@ -206,6 +211,11 @@ error = ''' region does not have peer ''' +["BR:Restore:ErrRestoreNotFreshCluster"] +error = ''' +cluster is not fresh +''' + ["BR:Restore:ErrRestoreRangeMismatch"] error = ''' restore range mismatch diff --git a/executor/executor_test.go b/executor/executor_test.go index 1277386725a6b..2765af034e3d6 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -4361,7 +4361,6 @@ func TestGetResultRowsCount(t *testing.T) { require.True(t, ok) cnt := executor.GetResultRowsCount(tk.Session().GetSessionVars().StmtCtx, p) require.Equal(t, ca.row, cnt, fmt.Sprintf("sql: %v", ca.sql)) - require.Equal(t, cnt, ca.row, fmt.Sprintf("sql: %v", ca.sql)) } } diff --git a/executor/set_test.go b/executor/set_test.go index f91d1f1e80388..6f88d49d6ef0a 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -676,11 +676,11 @@ func TestSetVar(t *testing.T) { tk.MustQuery("show global variables like 'tidb_ignore_prepared_cache_close_stmt'").Check(testkit.Rows("tidb_ignore_prepared_cache_close_stmt OFF")) // test for tidb_enable_new_cost_interface - tk.MustQuery("select @@global.tidb_enable_new_cost_interface").Check(testkit.Rows("0")) // default value is 0 - tk.MustExec("set global tidb_enable_new_cost_interface=1") - tk.MustQuery("select @@global.tidb_enable_new_cost_interface").Check(testkit.Rows("1")) + tk.MustQuery("select @@global.tidb_enable_new_cost_interface").Check(testkit.Rows("1")) // default value is 1 tk.MustExec("set global tidb_enable_new_cost_interface=0") tk.MustQuery("select @@global.tidb_enable_new_cost_interface").Check(testkit.Rows("0")) + tk.MustExec("set global tidb_enable_new_cost_interface=1") + tk.MustQuery("select @@global.tidb_enable_new_cost_interface").Check(testkit.Rows("1")) // test for tidb_remove_orderby_in_subquery tk.MustQuery("select @@session.tidb_remove_orderby_in_subquery").Check(testkit.Rows("0")) // default value is 0 @@ -690,6 +690,14 @@ func TestSetVar(t *testing.T) { tk.MustExec("set global tidb_remove_orderby_in_subquery=1") tk.MustQuery("select @@global.tidb_remove_orderby_in_subquery").Check(testkit.Rows("1")) + // test for tidb_opt_skew_distinct_agg + tk.MustQuery("select @@session.tidb_opt_skew_distinct_agg").Check(testkit.Rows("0")) // default value is 0 + tk.MustExec("set session tidb_opt_skew_distinct_agg=1") + tk.MustQuery("select @@session.tidb_opt_skew_distinct_agg").Check(testkit.Rows("1")) + tk.MustQuery("select @@global.tidb_opt_skew_distinct_agg").Check(testkit.Rows("0")) // default value is 0 + tk.MustExec("set global tidb_opt_skew_distinct_agg=1") + tk.MustQuery("select @@global.tidb_opt_skew_distinct_agg").Check(testkit.Rows("1")) + // the value of max_allowed_packet should be a multiple of 1024 tk.MustExec("set @@global.max_allowed_packet=16385") tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect max_allowed_packet value: '16385'")) diff --git a/executor/showtest/show_test.go b/executor/showtest/show_test.go index 2994069bcb0f6..263cf33e7cf7c 100644 --- a/executor/showtest/show_test.go +++ b/executor/showtest/show_test.go @@ -1445,7 +1445,7 @@ func TestShowBuiltin(t *testing.T) { res := tk.MustQuery("show builtins;") require.NotNil(t, res) rows := res.Rows() - const builtinFuncNum = 275 + const builtinFuncNum = 276 require.Equal(t, len(rows), builtinFuncNum) require.Equal(t, rows[0][0].(string), "abs") require.Equal(t, rows[builtinFuncNum-1][0].(string), "yearweek") diff --git a/expression/builtin.go b/expression/builtin.go index 772ef5c5d48f1..b56b7208249f7 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -834,6 +834,7 @@ var funcs = map[string]functionClass{ ast.TiDBVersion: &tidbVersionFunctionClass{baseFunctionClass{ast.TiDBVersion, 0, 0}}, ast.TiDBIsDDLOwner: &tidbIsDDLOwnerFunctionClass{baseFunctionClass{ast.TiDBIsDDLOwner, 0, 0}}, ast.TiDBDecodePlan: &tidbDecodePlanFunctionClass{baseFunctionClass{ast.TiDBDecodePlan, 1, 1}}, + ast.TiDBDecodeBinaryPlan: &tidbDecodePlanFunctionClass{baseFunctionClass{ast.TiDBDecodeBinaryPlan, 1, 1}}, ast.TiDBDecodeSQLDigests: &tidbDecodeSQLDigestsFunctionClass{baseFunctionClass{ast.TiDBDecodeSQLDigests, 1, 2}}, // TiDB Sequence function. diff --git a/expression/builtin_info.go b/expression/builtin_info.go index 1e61039b5d018..446f38fd9c409 100644 --- a/expression/builtin_info.go +++ b/expression/builtin_info.go @@ -25,6 +25,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/privilege" @@ -938,8 +939,12 @@ func (c *tidbDecodePlanFunctionClass) getFunction(ctx sessionctx.Context, args [ if err != nil { return nil, err } - sig := &builtinTiDBDecodePlanSig{bf} - return sig, nil + if c.funcName == ast.TiDBDecodePlan { + return &builtinTiDBDecodePlanSig{bf}, nil + } else if c.funcName == ast.TiDBDecodeBinaryPlan { + return &builtinTiDBDecodeBinaryPlanSig{bf}, nil + } + return nil, errors.New("unknown decode plan function") } type builtinTiDBDecodePlanSig struct { @@ -964,6 +969,29 @@ func (b *builtinTiDBDecodePlanSig) evalString(row chunk.Row) (string, bool, erro return planTree, false, nil } +type builtinTiDBDecodeBinaryPlanSig struct { + baseBuiltinFunc +} + +func (b *builtinTiDBDecodeBinaryPlanSig) Clone() builtinFunc { + newSig := &builtinTiDBDecodeBinaryPlanSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (b *builtinTiDBDecodeBinaryPlanSig) evalString(row chunk.Row) (string, bool, error) { + planString, isNull, err := b.args[0].EvalString(b.ctx, row) + if isNull || err != nil { + return "", isNull, err + } + planTree, err := plancodec.DecodeBinaryPlan(planString) + if err != nil { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(err) + return "", false, nil + } + return planTree, false, nil +} + type nextValFunctionClass struct { baseFunctionClass } diff --git a/go.mod b/go.mod index bd2953a3d9019..0fe46bb0fc296 100644 --- a/go.mod +++ b/go.mod @@ -46,11 +46,11 @@ require ( github.com/pingcap/errors v0.11.5-0.20211224045212-9687c2b0f87c github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3 github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20220705090230-a5d4ffd2ba33 + github.com/pingcap/kvproto v0.0.0-20220711062932-08b02befd813 github.com/pingcap/log v1.1.0 github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e - github.com/pingcap/tipb v0.0.0-20220706024432-7be3cc83a7d5 + github.com/pingcap/tipb v0.0.0-20220714100504-7d3474676bc9 github.com/prometheus/client_golang v1.12.2 github.com/prometheus/client_model v0.2.0 github.com/prometheus/common v0.32.1 @@ -62,7 +62,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.1-0.20220711061028-1c198aab9585 + github.com/tikv/client-go/v2 v2.0.1-0.20220718080214-86d51ba7eb02 github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710 github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index d586c031e94e9..e5a2bcdf61496 100644 --- a/go.sum +++ b/go.sum @@ -285,7 +285,6 @@ github.com/go-sql-driver/mysql v1.5.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LB github.com/go-sql-driver/mysql v1.6.0 h1:BCTh4TKNUYmOmMUcQ3IipzF5prigylS7XXjEkfCHuOE= github.com/go-sql-driver/mysql v1.6.0/go.mod h1:DCzpHaOWr8IXmIStZouvnhqoel9Qv2LBy8hT2VhHyBg= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= -github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0/go.mod h1:fyg7847qk6SyHyPtNmDHnmrv/HOrqktSC+C9fM+CJOE= github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= @@ -343,7 +342,6 @@ github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= -github.com/golang/snappy v0.0.2-0.20190904063534-ff6b7dc882cf/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -398,7 +396,6 @@ github.com/google/pprof v0.0.0-20201023163331-3e6fc7fc9c4c/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20201203190320-1bf35d6f28c2/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210122040257-d980be63207e/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210226084205-cbba55b83ad5/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= -github.com/google/pprof v0.0.0-20210407192527-94a9f03dee38/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= @@ -426,7 +423,6 @@ github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoA github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= -github.com/grpc-ecosystem/go-grpc-middleware v1.1.0/go.mod h1:f5nM7jw/oeRSadq3xCzHAvxcr8HZnzsqU6ILg/0NiiE= github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw= github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 h1:Ovs26xHkKqVztRpIrF/92BcuyuQ/YW4NSIpoGtfXNho= @@ -622,24 +618,18 @@ github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef/go.mod h1:7WjlapSfwQyo github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= github.com/nishanths/predeclared v0.2.2 h1:V2EPdZPliZymNAn79T8RkNApBjMmVKh5XRpLm/w98Vk= github.com/nishanths/predeclared v0.2.2/go.mod h1:RROzoN6TnGQupbC+lqggsOlcgysk3LMK/HI84Mp280c= +github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= -github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= -github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= -github.com/onsi/ginkgo v1.16.4/go.mod h1:dX+/inL/fNMqNlz0e9LfyB9TswhZpCVdJM/Z6Vvnwo0= github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= -github.com/onsi/ginkgo v1.16.5/go.mod h1:+E8gABHa3K6zRBolWtd+ROzc/U5bkGt0FwiG042wbpU= -github.com/onsi/ginkgo/v2 v2.0.0/go.mod h1:vw5CSIxN1JObi/U8gcbwft7ZxR2dgaR70JSE3/PpL4c= github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= -github.com/onsi/gomega v1.17.0/go.mod h1:HnhC7FXeEQY45zxNK3PPoIUhzk/80Xly9PcubAlGdZY= github.com/onsi/gomega v1.18.1 h1:M1GfJqGRrBrrGGsbxzV5dqM2U2ApXefZCQpkukxYRLE= -github.com/onsi/gomega v1.18.1/go.mod h1:0q+aL8jAiMXy9hbwj2mr5GziHiwhAIQpFmmtT5hitRs= github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7lZWlQw5UXuoo= github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= @@ -673,9 +663,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20220302110454-c696585a961b/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20220525022339-6aaebf466305/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= -github.com/pingcap/kvproto v0.0.0-20220705090230-a5d4ffd2ba33 h1:VKMmvYhtG28j1sCCBdq4s+V9UOYqNgQ6CQviQwOgTeg= -github.com/pingcap/kvproto v0.0.0-20220705090230-a5d4ffd2ba33/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= +github.com/pingcap/kvproto v0.0.0-20220711062932-08b02befd813 h1:PAXtUVMJnyQQS8t9GzihIFmh6FBXu0JziWbIVknLniA= +github.com/pingcap/kvproto v0.0.0-20220711062932-08b02befd813/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= @@ -684,8 +673,8 @@ github.com/pingcap/log v1.1.0 h1:ELiPxACz7vdo1qAvvaWJg1NrYFoY6gqAh/+Uo6aXdD8= github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 h1:HYbcxtnkN3s5tqrZ/z3eJS4j3Db8wMphEm1q10lY/TM= github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4/go.mod h1:sDCsM39cGiv2vwunZkaFA917vVkqDTGSPbbV7z4Oops= -github.com/pingcap/tipb v0.0.0-20220706024432-7be3cc83a7d5 h1:XaTE4ZhQbQtQZtAVzlZh/Pf6SjFfMSTe1ia2nGcl36Y= -github.com/pingcap/tipb v0.0.0-20220706024432-7be3cc83a7d5/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= +github.com/pingcap/tipb v0.0.0-20220714100504-7d3474676bc9 h1:hE1dQdnvxWCHhD0snX67paV9y6inq8TxVFbsKqjaTQk= +github.com/pingcap/tipb v0.0.0-20220714100504-7d3474676bc9/go.mod h1:A7mrd7WHBl1o63LE2bIBGEJMTNWXqhgmYiOvMLxozfs= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4 h1:49lOXmGaUpV9Fz3gd7TFZY106KVlPVa5jcYD1gaQf98= github.com/pkg/browser v0.0.0-20180916011732-0a3d74bf9ce4/go.mod h1:4OwLy04Bl9Ef3GJJCoec+30X3LQs/0/m4HFRt/2LUSA= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -800,8 +789,8 @@ github.com/tdakkota/asciicheck v0.1.1 h1:PKzG7JUTUmVspQTDqtkX9eSiLGossXTybutHwTX github.com/tdakkota/asciicheck v0.1.1/go.mod h1:yHp0ai0Z9gUljN3o0xMhYJnH/IcvkdTBOX2fmJ93JEM= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tikv/client-go/v2 v2.0.1-0.20220711061028-1c198aab9585 h1:RpH/obpgyNKkXV4Wt8PqSdcUTnqWyExPcla+qdTVgi0= -github.com/tikv/client-go/v2 v2.0.1-0.20220711061028-1c198aab9585/go.mod h1:uoZHYWKB+PsDueEnZ0EvF5zvNJPEauEWN26Tgi7qvNI= +github.com/tikv/client-go/v2 v2.0.1-0.20220718080214-86d51ba7eb02 h1:OSxo1R2y6iyAAxbUPL7h1HC/17CNuwNkW2lF1JySJ7k= +github.com/tikv/client-go/v2 v2.0.1-0.20220718080214-86d51ba7eb02/go.mod h1:UmDQEoeHXza8RSHBXVFERpxH54VBOf8yKZVphyel3l4= github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710 h1:jxgmKOscXSjaFEKQGRyY5qOpK8hLqxs2irb/uDJMtwk= github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710/go.mod h1:AtvppPwkiyUgQlR1W9qSqfTB+OsOIu19jDCOxOsPkmU= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= @@ -1045,7 +1034,6 @@ golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v golang.org/x/net v0.0.0-20210316092652-d523dce5a7f4/go.mod h1:RBQZq4jEuRlivfhVLdyRGr576XBO4/greRjx4P4O3yc= golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96bSt6lcn1PtDYWL6XObtHCRCNQM= golang.org/x/net v0.0.0-20210427231257-85d9c07bbe3a/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= -golang.org/x/net v0.0.0-20210428140749-89ef3d95e781/go.mod h1:OJAsFXCWl8Ukc7SiCT/9KSuxbyM7479/AVlXFRxuMCk= golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210610132358-84b48f89b13b/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= @@ -1137,7 +1125,6 @@ golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20201201145000-ef89a241ccb3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210104204734-6f8348627aad/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210112080510-489259a85091/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210220050731-9a76102bfb43/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1247,7 +1234,6 @@ golang.org/x/tools v0.0.0-20201110124207-079ba7bd75cd/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20201125231158-b5590deeca9b/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20201224043029-2b0845dc783e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= @@ -1482,4 +1468,3 @@ sourcegraph.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0/go.mod h1 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 h1:e1sMhtVq9AfcEy8AXNb8eSg6gbzfdpYhoNqnPJa+GzI= sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67/go.mod h1:L5q+DGLGOQFpo1snNEkLOJT2d1YTW66rWNzatr3He1k= stathat.com/c/consistent v1.0.0 h1:ezyc51EGcRPJUxfHGSgJjWzJdj3NiMU9pNfLNGiXV0c= -stathat.com/c/consistent v1.0.0/go.mod h1:QkzMWzcbB+yQBL2AttO6sgsQS/JSTapcDISJalmCDS0= diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 7b15d3d6bbf64..ad32c8280230b 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -9311,6 +9311,117 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Time spent on the RPC layer between TiDB and TiKV, including the part used in the TiDB batch client", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 16, + "y": 22 + }, + "hiddenSeries": false, + "id": 263, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_rpc_net_latency_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, store))", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "99-store{{store}}", + "refId": "A" + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_tikvclient_rpc_net_latency_seconds_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, store) / sum(rate(tidb_tikvclient_rpc_net_latency_seconds_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, store)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "avg-store{{store}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "RPC Layer Latency", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:132", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "$$hashKey": "object:133", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, diff --git a/metrics/log_backup.go b/metrics/log_backup.go index b477f447c2dbb..6c706b8027a79 100644 --- a/metrics/log_backup.go +++ b/metrics/log_backup.go @@ -48,4 +48,16 @@ var ( Help: "The batch size of scanning region or get region checkpoint.", Buckets: prometheus.ExponentialBuckets(1, 2.0, 12), }, []string{"type"}) + RegionCheckpointRequest = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "log_backup", + Name: "region_request", + Help: "The failure / success stat requesting region checkpoints.", + }, []string{"result"}) + RegionCheckpointFailure = prometheus.NewCounterVec(prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "log_backup", + Name: "region_request_failure", + Help: "The failure reasons of requesting region checkpoints.", + }, []string{"reason"}) ) diff --git a/metrics/metrics.go b/metrics/metrics.go index 4011e587cec71..1bf05749d6e92 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -195,6 +195,8 @@ func RegisterMetrics() { prometheus.MustRegister(AdvancerOwner) prometheus.MustRegister(AdvancerTickDuration) prometheus.MustRegister(GetCheckpointBatchSize) + prometheus.MustRegister(RegionCheckpointRequest) + prometheus.MustRegister(RegionCheckpointFailure) tikvmetrics.InitMetrics(TiDB, TiKVClient) tikvmetrics.RegisterMetrics() diff --git a/parser/ast/functions.go b/parser/ast/functions.go index ae61e06682656..b911734a993e7 100644 --- a/parser/ast/functions.go +++ b/parser/ast/functions.go @@ -257,6 +257,7 @@ const ( TiDBVersion = "tidb_version" TiDBIsDDLOwner = "tidb_is_ddl_owner" TiDBDecodePlan = "tidb_decode_plan" + TiDBDecodeBinaryPlan = "tidb_decode_binary_plan" TiDBDecodeSQLDigests = "tidb_decode_sql_digests" FormatBytes = "format_bytes" FormatNanoTime = "format_nano_time" diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 92c7280649318..1bbddf1382bf6 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -258,6 +258,9 @@ type MultiSchemaInfo struct { SubJobs []*SubJob `json:"sub_jobs"` Revertible bool `json:"revertible"` + // SkipVersion is used to control whether generating a new schema version for a sub-job. + SkipVersion bool `json:"-"` + AddColumns []CIStr `json:"-"` DropColumns []CIStr `json:"-"` ModifyColumns []CIStr `json:"-"` @@ -289,6 +292,7 @@ type SubJob struct { RowCount int64 `json:"row_count"` Warning *terror.Error `json:"warning"` CtxVars []interface{} `json:"-"` + SchemaVer int64 `json:"schema_version"` } // IsNormal returns true if the sub-job is normally running. @@ -342,7 +346,7 @@ func (sub *SubJob) ToProxyJob(parentJob *Job) Job { } // FromProxyJob converts a proxy job to a sub-job. -func (sub *SubJob) FromProxyJob(proxyJob *Job) { +func (sub *SubJob) FromProxyJob(proxyJob *Job, ver int64) { sub.Revertible = proxyJob.MultiSchemaInfo.Revertible sub.SchemaState = proxyJob.SchemaState sub.SnapshotVer = proxyJob.SnapshotVer @@ -350,6 +354,7 @@ func (sub *SubJob) FromProxyJob(proxyJob *Job) { sub.State = proxyJob.State sub.Warning = proxyJob.Warning sub.RowCount = proxyJob.RowCount + sub.SchemaVer = ver } // Job is for a DDL operation. diff --git a/parser/parser_test.go b/parser/parser_test.go index ee5c9fc7e384c..08b7c30595f97 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -4892,6 +4892,8 @@ func TestExplain(t *testing.T) { {"EXPLAIN FORMAT = JSON FOR CONNECTION 1", true, "EXPLAIN FORMAT = 'JSON' FOR CONNECTION 1"}, {"EXPLAIN FORMAT = JSON SELECT 1", true, "EXPLAIN FORMAT = 'JSON' SELECT 1"}, {"EXPLAIN FORMAT = 'hint' SELECT 1", true, "EXPLAIN FORMAT = 'hint' SELECT 1"}, + {"EXPLAIN ANALYZE FORMAT = 'verbose' SELECT 1", true, "EXPLAIN ANALYZE FORMAT = 'verbose' SELECT 1"}, + {"EXPLAIN ANALYZE FORMAT = 'binary' SELECT 1", true, "EXPLAIN ANALYZE FORMAT = 'binary' SELECT 1"}, {"EXPLAIN ALTER TABLE t1 ADD INDEX (a)", true, "EXPLAIN FORMAT = 'row' ALTER TABLE `t1` ADD INDEX(`a`)"}, {"EXPLAIN ALTER TABLE t1 ADD a varchar(255)", true, "EXPLAIN FORMAT = 'row' ALTER TABLE `t1` ADD COLUMN `a` VARCHAR(255)"}, } diff --git a/planner/core/access_object.go b/planner/core/access_object.go index b5aa4af1fd692..81296c39c9223 100644 --- a/planner/core/access_object.go +++ b/planner/core/access_object.go @@ -109,7 +109,7 @@ func (d DynamicPartitionAccessObjects) SetIntoPB(pb *tipb.ExplainOperator) { if len(obj.err) > 0 { continue } - pbObj := pbObjSlice[i] + pbObj := &pbObjSlice[i] pbObj.Database = obj.Database pbObj.Table = obj.Table pbObj.AllPartitions = obj.AllPartitions @@ -119,7 +119,11 @@ func (d DynamicPartitionAccessObjects) SetIntoPB(pb *tipb.ExplainOperator) { for i := range pbObjSlice { pbObjs.Objects = append(pbObjs.Objects, &pbObjSlice[i]) } - pb.AccessObject = &tipb.ExplainOperator_DynamicPartitionObjects{DynamicPartitionObjects: &pbObjs} + pb.AccessObjects = []*tipb.AccessObject{ + { + AccessObject: &tipb.AccessObject_DynamicPartitionObjects{DynamicPartitionObjects: &pbObjs}, + }, + } } // IndexAccess represents the index accessed by an operator. @@ -202,7 +206,11 @@ func (s *ScanAccessObject) SetIntoPB(pb *tipb.ExplainOperator) { for i := range s.Indexes { pbObj.Indexes = append(pbObj.Indexes, s.Indexes[i].ToPB()) } - pb.AccessObject = &tipb.ExplainOperator_ScanObject{ScanObject: &pbObj} + pb.AccessObjects = []*tipb.AccessObject{ + { + AccessObject: &tipb.AccessObject_ScanObject{ScanObject: &pbObj}, + }, + } } // OtherAccessObject represents other kinds of access. @@ -222,7 +230,11 @@ func (o OtherAccessObject) SetIntoPB(pb *tipb.ExplainOperator) { if pb == nil { return } - pb.AccessObject = &tipb.ExplainOperator_OtherObject{OtherObject: string(o)} + pb.AccessObjects = []*tipb.AccessObject{ + { + AccessObject: &tipb.AccessObject_OtherObject{OtherObject: string(o)}, + }, + } } // AccessObject implements dataAccesser interface. diff --git a/planner/core/binary_plan_test.go b/planner/core/binary_plan_test.go new file mode 100644 index 0000000000000..4e07bf0e51ffe --- /dev/null +++ b/planner/core/binary_plan_test.go @@ -0,0 +1,133 @@ +// 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 core_test + +import ( + "encoding/base64" + "fmt" + "regexp" + "testing" + + "github.com/golang/snappy" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/testdata" + "github.com/pingcap/tipb/go-tipb" + "github.com/stretchr/testify/require" +) + +func simplifyAndCheckBinaryPlan(t *testing.T, pb *tipb.ExplainData) { + if pb.Main != nil { + simplifyAndCheckBinaryOperator(t, pb.Main, pb.WithRuntimeStats) + } + for _, cte := range pb.Ctes { + if cte != nil { + simplifyAndCheckBinaryOperator(t, cte, pb.WithRuntimeStats) + } + } +} + +func simplifyAndCheckBinaryOperator(t *testing.T, pb *tipb.ExplainOperator, withRuntimeStats bool) { + if withRuntimeStats { + if pb.TaskType == tipb.TaskType_root { + require.NotEmpty(t, pb.RootBasicExecInfo) + } else if pb.TaskType != tipb.TaskType_unknown { + require.NotEmpty(t, pb.CopExecInfo) + } + } + pb.RootBasicExecInfo = "" + pb.RootGroupExecInfo = nil + pb.CopExecInfo = "" + match, err := regexp.MatchString("((Table|Index).*Scan)|CTEFullScan|Point_Get", pb.Name) + if err == nil && match { + require.NotNil(t, pb.AccessObjects) + } + // AccessObject field is an interface and json.Unmarshall can't handle it, so we don't check it against the json output. + pb.AccessObjects = nil + // MemoryBytes and DiskBytes are not stable sometimes. + pb.MemoryBytes = 0 + pb.DiskBytes = 0 + if len(pb.Children) > 0 { + for _, op := range pb.Children { + if op != nil { + simplifyAndCheckBinaryOperator(t, op, withRuntimeStats) + } + } + } +} + +func TestBinaryPlanInExplainAndSlowLog(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + // If we don't set this, it will be false sometimes and the cost in the result will be different. + tk.MustExec("set @@tidb_enable_chunk_rpc=true") + + var input []string + var output []struct { + SQL string + BinaryPlan *tipb.ExplainData + } + planSuiteData := core.GetBinaryPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + + for i, test := range input { + comment := fmt.Sprintf("case:%v sql:%s", i, test) + if len(test) < 7 || test[:7] != "explain" { + tk.MustExec(test) + testdata.OnRecord(func() { + output[i].SQL = test + output[i].BinaryPlan = nil + }) + continue + } + result := testdata.ConvertRowsToStrings(tk.MustQuery(test).Rows()) + require.Equal(t, len(result), 1, comment) + s := result[0] + + b, err := base64.StdEncoding.DecodeString(s) + require.NoError(t, err) + b, err = snappy.Decode(nil, b) + require.NoError(t, err) + binary := &tipb.ExplainData{} + err = binary.Unmarshal(b) + require.NoError(t, err) + testdata.OnRecord(func() { + output[i].SQL = test + output[i].BinaryPlan = binary + }) + simplifyAndCheckBinaryPlan(t, binary) + require.Equal(t, output[i].BinaryPlan, binary) + } +} + +func TestInvalidDecodeBinaryPlan(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + str1 := "some random bytes" + str2 := base64.StdEncoding.EncodeToString([]byte(str1)) + str3 := base64.StdEncoding.EncodeToString(snappy.Encode(nil, []byte(str1))) + + tk.MustQuery(`select tidb_decode_binary_plan('` + str1 + `')`).Check(testkit.Rows("")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 illegal base64 data at input byte 4")) + tk.MustQuery(`select tidb_decode_binary_plan('` + str2 + `')`).Check(testkit.Rows("")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 snappy: corrupt input")) + tk.MustQuery(`select tidb_decode_binary_plan('` + str3 + `')`).Check(testkit.Rows("")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 proto: illegal wireType 7")) +} diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 31ad78c12036c..77d3a8adc34cb 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -47,8 +47,10 @@ import ( "github.com/pingcap/tidb/util/kvcache" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/texttree" + "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) @@ -1201,6 +1203,8 @@ func (e *Explain) prepareSchema() error { fieldNames = []string{"dot contents"} case format == types.ExplainFormatHint: fieldNames = []string{"hint"} + case format == types.ExplainFormatBinary: + fieldNames = []string{"binary plan"} default: return errors.Errorf("explain format '%s' is not supported now", e.Format) } @@ -1257,6 +1261,10 @@ func (e *Explain) RenderResult() error { hints := GenHintsFromFlatPlan(flat) hints = append(hints, hint.ExtractTableHintsFromStmtNode(e.ExecStmt, nil)...) e.Rows = append(e.Rows, []string{hint.RestoreOptimizerHints(hints)}) + case types.ExplainFormatBinary: + flat := FlattenPhysicalPlan(e.TargetPlan, false) + str := BinaryPlanStrFromFlatPlan(e.ctx, flat) + e.Rows = append(e.Rows, []string{str}) default: return errors.Errorf("explain format '%s' is not supported now", e.Format) } @@ -1412,6 +1420,146 @@ func (e *Explain) getOperatorInfo(p Plan, id string) (string, string, string, st return estRows, estCost, accessObject, operatorInfo } +// BinaryPlanStrFromFlatPlan generates the compressed and encoded binary plan from a FlatPhysicalPlan. +func BinaryPlanStrFromFlatPlan(explainCtx sessionctx.Context, flat *FlatPhysicalPlan) string { + binary := binaryDataFromFlatPlan(explainCtx, flat) + if binary == nil { + return "" + } + proto, err := binary.Marshal() + if err != nil { + return "" + } + str := plancodec.Compress(proto) + return str +} + +func binaryDataFromFlatPlan(explainCtx sessionctx.Context, flat *FlatPhysicalPlan) *tipb.ExplainData { + if len(flat.Main) == 0 { + return nil + } + // Please see comments in EncodeFlatPlan() for this case. + // We keep consistency with EncodeFlatPlan() here. + if flat.InExecute { + return nil + } + res := &tipb.ExplainData{} + for _, op := range flat.Main { + // We assume that runtime stats are available to this plan tree if any operator in the "Main" has runtime stats. + rootStats, copStats, _, _ := getRuntimeInfo(explainCtx, op.Origin, nil) + if rootStats != nil || copStats != nil { + res.WithRuntimeStats = true + break + } + } + res.Main = binaryOpTreeFromFlatOps(explainCtx, flat.Main) + for _, explainedCTE := range flat.CTEs { + res.Ctes = append(res.Ctes, binaryOpTreeFromFlatOps(explainCtx, explainedCTE)) + } + return res +} + +func binaryOpTreeFromFlatOps(explainCtx sessionctx.Context, ops FlatPlanTree) *tipb.ExplainOperator { + s := make([]tipb.ExplainOperator, len(ops)) + for i, op := range ops { + binaryOpFromFlatOp(explainCtx, op, &s[i]) + for _, idx := range op.ChildrenIdx { + s[i].Children = append(s[i].Children, &s[idx]) + } + } + return &s[0] +} + +func binaryOpFromFlatOp(explainCtx sessionctx.Context, op *FlatOperator, out *tipb.ExplainOperator) { + out.Name = op.Origin.ExplainID().String() + switch op.Label { + case BuildSide: + out.Labels = []tipb.OperatorLabel{tipb.OperatorLabel_buildSide} + case ProbeSide: + out.Labels = []tipb.OperatorLabel{tipb.OperatorLabel_probeSide} + case SeedPart: + out.Labels = []tipb.OperatorLabel{tipb.OperatorLabel_seedPart} + case RecursivePart: + out.Labels = []tipb.OperatorLabel{tipb.OperatorLabel_recursivePart} + } + switch op.StoreType { + case kv.TiDB: + out.StoreType = tipb.StoreType_tidb + case kv.TiKV: + out.StoreType = tipb.StoreType_tikv + case kv.TiFlash: + out.StoreType = tipb.StoreType_tiflash + } + if op.IsRoot { + out.TaskType = tipb.TaskType_root + } else { + switch op.ReqType { + case Cop: + out.TaskType = tipb.TaskType_cop + case BatchCop: + out.TaskType = tipb.TaskType_batchCop + case MPP: + out.TaskType = tipb.TaskType_mpp + } + } + + // Runtime info + rootStats, copStats, memTracker, diskTracker := getRuntimeInfo(explainCtx, op.Origin, nil) + if statsInfo := op.Origin.statsInfo(); statsInfo != nil { + out.EstRows = statsInfo.RowCount + } + if op.IsPhysicalPlan { + p := op.Origin.(PhysicalPlan) + out.Cost = p.Cost() + } + if rootStats != nil { + basic, groups := rootStats.MergeStats() + out.RootBasicExecInfo = basic.String() + for _, group := range groups { + str := group.String() + if len(str) > 0 { + out.RootGroupExecInfo = append(out.RootGroupExecInfo, str) + } + } + out.ActRows = uint64(rootStats.GetActRows()) + } + if copStats != nil { + out.CopExecInfo = copStats.String() + out.ActRows = uint64(copStats.GetActRows()) + } + if memTracker != nil { + out.MemoryBytes = memTracker.MaxConsumed() + } else { + out.MemoryBytes = -1 + } + if diskTracker != nil { + out.DiskBytes = diskTracker.MaxConsumed() + } else { + out.DiskBytes = -1 + } + + // Operator info + if plan, ok := op.Origin.(dataAccesser); ok { + out.OperatorInfo = plan.OperatorInfo(false) + } else { + out.OperatorInfo = op.Origin.ExplainInfo() + } + + // Access object + switch p := op.Origin.(type) { + case dataAccesser: + ao := p.AccessObject() + if ao != nil { + ao.SetIntoPB(out) + } + case partitionAccesser: + ao := p.accessObject(explainCtx) + if ao != nil { + ao.SetIntoPB(out) + } + } +} + func (e *Explain) prepareDotInfo(p PhysicalPlan) { buffer := bytes.NewBufferString("") fmt.Fprintf(buffer, "\ndigraph %s {\n", p.ExplainID()) diff --git a/planner/core/enforce_mpp_test.go b/planner/core/enforce_mpp_test.go index dab97226f436e..5b39a08bdd8cb 100644 --- a/planner/core/enforce_mpp_test.go +++ b/planner/core/enforce_mpp_test.go @@ -441,3 +441,55 @@ func TestMPP2PhaseAggPushDown(t *testing.T) { require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } + +// Test skewed group distinct aggregate rewrite for MPP mode +func TestMPPSkewedGroupDistinctRewrite(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + // test table + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b bigint not null, c bigint, d date, e varchar(20))") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + + var input []string + var output []struct { + SQL string + Plan []string + Warn []string + } + enforceMPPSuiteData := plannercore.GetEnforceMPPSuiteData() + enforceMPPSuiteData.GetTestCases(t, &input, &output) + for i, tt := range input { + testdata.OnRecord(func() { + output[i].SQL = tt + }) + if strings.HasPrefix(tt, "set") || strings.HasPrefix(tt, "UPDATE") { + tk.MustExec(tt) + continue + } + testdata.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery(tt).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) + }) + res := tk.MustQuery(tt) + res.Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) + } +} diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 79cc45416fe80..60d794bd76e41 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2614,7 +2614,8 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // 1-phase agg // If there are no available partition cols, but still have group by items, that means group by items are all expressions or constants. // To avoid mess, we don't do any one-phase aggregation in this case. - if len(partitionCols) != 0 { + // If this is a skew distinct group agg, skip generating 1-phase agg, because skew data will cause performance issue + if len(partitionCols) != 0 && !la.ctx.GetSessionVars().EnableSkewDistinctAgg { childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: partitionCols, CanAddEnforcer: true, RejectSort: true} agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 9cce66aae3eb8..9c6fceea6c54e 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -289,30 +289,35 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl // compareTaskCost compares cost of curTask and bestTask and returns whether curTask's cost is smaller than bestTask's. func compareTaskCost(ctx sessionctx.Context, curTask, bestTask task) (curIsBetter bool, err error) { - if curTask.invalid() { + curCost, curInvalid, err := getTaskPlanCost(curTask) + if err != nil { + return false, err + } + bestCost, bestInvalid, err := getTaskPlanCost(bestTask) + if err != nil { + return false, err + } + if curInvalid { return false, nil } - if bestTask.invalid() { + if bestInvalid { return true, nil } - if ctx.GetSessionVars().EnableNewCostInterface { // use the new cost interface - curCost, err := getTaskPlanCost(curTask) - if err != nil { - return false, err - } - bestCost, err := getTaskPlanCost(bestTask) - if err != nil { - return false, err - } - return curCost < bestCost, nil - } - return curTask.cost() < bestTask.cost(), nil + return curCost < bestCost, nil } -func getTaskPlanCost(t task) (float64, error) { +// getTaskPlanCost returns the cost of this task. +// The new cost interface will be used if EnableNewCostInterface is true. +// The second returned value indicates whether this task is valid. +func getTaskPlanCost(t task) (float64, bool, error) { if t.invalid() { - return math.MaxFloat64, nil + return math.MaxFloat64, true, nil } + if !t.plan().SCtx().GetSessionVars().EnableNewCostInterface { + return t.cost(), false, nil + } + + // use the new cost interface var taskType property.TaskType switch t.(type) { case *rootTask: @@ -322,9 +327,10 @@ func getTaskPlanCost(t task) (float64, error) { case *mppTask: taskType = property.MppTaskType default: - return 0, errors.New("unknown task type") + return 0, false, errors.New("unknown task type") } - return t.plan().GetPlanCost(taskType, 0) + cost, err := t.plan().GetPlanCost(taskType, 0) + return cost, false, err } type physicalOptimizeOp struct { @@ -2198,6 +2204,7 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper HandleCols: ds.handleCols, tblCols: ds.TblCols, tblColHists: ds.TblColHists, + prop: prop, }.Init(ds.ctx, ds.blockOffset) ts.filterCondition = make([]expression.Expression, len(path.TableFilters)) copy(ts.filterCondition, path.TableFilters) @@ -2267,6 +2274,7 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper physicalTableID: ds.physicalTableID, tblColHists: ds.TblColHists, pkIsHandleCol: ds.getPKIsHandleCol(), + prop: prop, }.Init(ds.ctx, ds.blockOffset) statsTbl := ds.statisticTable if statsTbl.Indices[idx.ID] != nil { diff --git a/planner/core/flat_plan.go b/planner/core/flat_plan.go index fbdae38d4ef49..717b1f415c774 100644 --- a/planner/core/flat_plan.go +++ b/planner/core/flat_plan.go @@ -213,8 +213,11 @@ func (f *FlatPhysicalPlan) flattenRecursively(p Plan, info *operatorCtx, target childIdxs := make([]int, 0) var childIdx int childCtx := &operatorCtx{ - depth: info.depth + 1, - indent: texttree.Indent4Child(info.indent, info.isLastChild), + depth: info.depth + 1, + isRoot: info.isRoot, + storeType: info.storeType, + reqType: info.reqType, + indent: texttree.Indent4Child(info.indent, info.isLastChild), } // For physical operators, we just enumerate their children and collect their information. // Note that some physical operators are special, and they are handled below this part. @@ -268,9 +271,6 @@ func (f *FlatPhysicalPlan) flattenRecursively(p Plan, info *operatorCtx, target } for i := range children { - childCtx.isRoot = info.isRoot - childCtx.storeType = info.storeType - childCtx.reqType = info.reqType childCtx.label = label[i] childCtx.isLastChild = i == len(children)-1 target, childIdx = f.flattenRecursively(children[i], childCtx, target) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index dc46986bb004e..dfafb8a8a4acb 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -5576,24 +5576,24 @@ func TestIndexJoinCost(t *testing.T) { `│ └─Selection_17 9990.00 465000.00 cop[tikv] not(isnull(test.t_outer.a))`, `│ └─TableFullScan_16 10000.00 435000.00 cop[tikv] table:t_outer keep order:false, stats:pseudo`, `└─TableReader_8(Probe) 1.00 3.88 root data:TableRangeScan_7`, - ` └─TableRangeScan_7 1.00 0.00 cop[tikv] table:t_inner_pk range: decided by [test.t_outer.a], keep order:false, stats:pseudo`)) + ` └─TableRangeScan_7 1.00 30.00 cop[tikv] table:t_inner_pk range: decided by [test.t_outer.a], keep order:false, stats:pseudo`)) tk.MustQuery(`explain format=verbose select /*+ TIDB_INLJ(t_outer, t_inner_idx) */ t_inner_idx.a from t_outer, t_inner_idx where t_outer.a=t_inner_idx.a`).Check(testkit.Rows( // IndexJoin with inner IndexScan `IndexJoin_10 12487.50 235192.19 root inner join, inner:IndexReader_9, outer key:test.t_outer.a, inner key:test.t_inner_idx.a, equal cond:eq(test.t_outer.a, test.t_inner_idx.a)`, `├─TableReader_20(Build) 9990.00 36412.58 root data:Selection_19`, `│ └─Selection_19 9990.00 465000.00 cop[tikv] not(isnull(test.t_outer.a))`, `│ └─TableFullScan_18 10000.00 435000.00 cop[tikv] table:t_outer keep order:false, stats:pseudo`, `└─IndexReader_9(Probe) 1.25 5.89 root index:Selection_8`, - ` └─Selection_8 1.25 0.00 cop[tikv] not(isnull(test.t_inner_idx.a))`, - ` └─IndexRangeScan_7 1.25 0.00 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`)) + ` └─Selection_8 1.25 58.18 cop[tikv] not(isnull(test.t_inner_idx.a))`, + ` └─IndexRangeScan_7 1.25 54.43 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`)) tk.MustQuery(`explain format=verbose select /*+ TIDB_INLJ(t_outer, t_inner_idx) */ * from t_outer, t_inner_idx where t_outer.a=t_inner_idx.a`).Check(testkit.Rows( // IndexJoin with inner IndexLookup `IndexJoin_11 12487.50 531469.38 root inner join, inner:IndexLookUp_10, outer key:test.t_outer.a, inner key:test.t_inner_idx.a, equal cond:eq(test.t_outer.a, test.t_inner_idx.a)`, `├─TableReader_23(Build) 9990.00 36412.58 root data:Selection_22`, `│ └─Selection_22 9990.00 465000.00 cop[tikv] not(isnull(test.t_outer.a))`, `│ └─TableFullScan_21 10000.00 435000.00 cop[tikv] table:t_outer keep order:false, stats:pseudo`, `└─IndexLookUp_10(Probe) 1.25 35.55 root `, - ` ├─Selection_9(Build) 1.25 0.00 cop[tikv] not(isnull(test.t_inner_idx.a))`, - ` │ └─IndexRangeScan_7 1.25 0.00 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`, - ` └─TableRowIDScan_8(Probe) 1.25 0.00 cop[tikv] table:t_inner_idx keep order:false, stats:pseudo`)) + ` ├─Selection_9(Build) 1.25 75.08 cop[tikv] not(isnull(test.t_inner_idx.a))`, + ` │ └─IndexRangeScan_7 1.25 71.32 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`, + ` └─TableRowIDScan_8(Probe) 1.25 71.25 cop[tikv] table:t_inner_idx keep order:false, stats:pseudo`)) tk.MustQuery("explain format=verbose select /*+ inl_hash_join(t_outer, t_inner_idx) */ t_inner_idx.a from t_outer, t_inner_idx where t_outer.a=t_inner_idx.a").Check(testkit.Rows( `IndexHashJoin_12 12487.50 235192.19 root inner join, inner:IndexReader_9, outer key:test.t_outer.a, inner key:test.t_inner_idx.a, equal cond:eq(test.t_outer.a, test.t_inner_idx.a)`, @@ -5601,16 +5601,16 @@ func TestIndexJoinCost(t *testing.T) { `│ └─Selection_19 9990.00 465000.00 cop[tikv] not(isnull(test.t_outer.a))`, `│ └─TableFullScan_18 10000.00 435000.00 cop[tikv] table:t_outer keep order:false, stats:pseudo`, `└─IndexReader_9(Probe) 1.25 5.89 root index:Selection_8`, - ` └─Selection_8 1.25 0.00 cop[tikv] not(isnull(test.t_inner_idx.a))`, - ` └─IndexRangeScan_7 1.25 0.00 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`)) + ` └─Selection_8 1.25 58.18 cop[tikv] not(isnull(test.t_inner_idx.a))`, + ` └─IndexRangeScan_7 1.25 54.43 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`)) tk.MustQuery("explain format=verbose select /*+ inl_merge_join(t_outer, t_inner_idx) */ t_inner_idx.a from t_outer, t_inner_idx where t_outer.a=t_inner_idx.a").Check(testkit.Rows( `IndexMergeJoin_17 12487.50 229210.68 root inner join, inner:IndexReader_15, outer key:test.t_outer.a, inner key:test.t_inner_idx.a`, `├─TableReader_20(Build) 9990.00 36412.58 root data:Selection_19`, `│ └─Selection_19 9990.00 465000.00 cop[tikv] not(isnull(test.t_outer.a))`, `│ └─TableFullScan_18 10000.00 435000.00 cop[tikv] table:t_outer keep order:false, stats:pseudo`, `└─IndexReader_15(Probe) 1.25 5.89 root index:Selection_14`, - ` └─Selection_14 1.25 0.00 cop[tikv] not(isnull(test.t_inner_idx.a))`, - ` └─IndexRangeScan_13 1.25 0.00 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:true, stats:pseudo`)) + ` └─Selection_14 1.25 58.18 cop[tikv] not(isnull(test.t_inner_idx.a))`, + ` └─IndexRangeScan_13 1.25 54.43 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:true, stats:pseudo`)) } func TestHeuristicIndexSelection(t *testing.T) { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 81fe7209c7e78..633f3cef4e3bd 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -189,6 +189,10 @@ func (b *PlanBuilder) buildAggregation(ctx context.Context, p LogicalPlan, aggFu b.optFlag |= flagEliminateAgg b.optFlag |= flagEliminateProjection + if b.ctx.GetSessionVars().EnableSkewDistinctAgg { + b.optFlag |= flagSkewDistinctAgg + } + plan4Agg := LogicalAggregation{AggFuncs: make([]*aggregation.AggFuncDesc, 0, len(aggFuncList))}.Init(b.ctx, b.getSelectOffset()) if hint := b.TableHints(); hint != nil { plan4Agg.aggHints = hint.aggHints diff --git a/planner/core/main_test.go b/planner/core/main_test.go index 6ca488d8e7205..6b8bf974b6b6c 100644 --- a/planner/core/main_test.go +++ b/planner/core/main_test.go @@ -49,6 +49,7 @@ func TestMain(m *testing.M) { testDataMap.LoadTestSuiteData("testdata", "plan_suite_unexported") testDataMap.LoadTestSuiteData("testdata", "join_reorder_suite") testDataMap.LoadTestSuiteData("testdata", "flat_plan_suite") + testDataMap.LoadTestSuiteData("testdata", "binary_plan_suite") indexMergeSuiteData = testDataMap["index_merge_suite"] planSuiteUnexportedData = testDataMap["plan_suite_unexported"] @@ -122,3 +123,7 @@ func GetWindowPushDownSuiteData() testdata.TestData { func GetFlatPlanSuiteData() testdata.TestData { return testDataMap["flat_plan_suite"] } + +func GetBinaryPlanSuiteData() testdata.TestData { + return testDataMap["binary_plan_suite"] +} diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 8e5fc64373b0f..d69a9d99ece0b 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -60,6 +60,7 @@ const ( flagDecorrelate flagSemiJoinRewrite flagEliminateAgg + flagSkewDistinctAgg flagEliminateProjection flagMaxMinEliminate flagPredicatePushDown @@ -81,6 +82,7 @@ var optRuleList = []logicalOptRule{ &decorrelateSolver{}, &semiJoinRewriter{}, &aggregationEliminator{}, + &skewDistinctAggRewriter{}, &projectionEliminator{}, &maxMinEliminator{}, &ppdSolver{}, diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index ac00996af0d84..149f647123ab6 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -1921,6 +1921,47 @@ func TestIssue30965(t *testing.T) { " └─TableRowIDScan 10.00 cop[tikv] table:t30965 keep order:false, stats:pseudo")) } +func TestSkewDistinctAgg(t *testing.T) { + var ( + input []string + output []struct { + SQL string + Plan []string + } + ) + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + 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` int(11), `b` int(11), `c` int(11), `d` date)") + tk.MustExec("insert into t (a,b,c,d) value(1,4,5,'2019-06-01')") + tk.MustExec("insert into t (a,b,c,d) value(2,null,1,'2019-07-01')") + tk.MustExec("insert into t (a,b,c,d) value(3,4,5,'2019-08-01')") + tk.MustExec("insert into t (a,b,c,d) value(3,6,2,'2019-09-01')") + tk.MustExec("insert into t (a,b,c,d) value(10,4,null,'2020-06-01')") + tk.MustExec("insert into t (a,b,c,d) value(20,null,1,'2020-07-01')") + tk.MustExec("insert into t (a,b,c,d) value(30,4,5,'2020-08-01')") + tk.MustExec("insert into t (a,b,c,d) value(30,6,5,'2020-09-01')") + tk.MustQuery("select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) " + + "from t group by date_format(d,'%Y') order by df;").Check( + testkit.Rows("2019 9 3 3", "2020 90 3 2")) + tk.MustExec("set @@tidb_opt_skew_distinct_agg=1") + tk.MustQuery("select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) " + + "from t group by date_format(d,'%Y') order by df;").Check( + testkit.Rows("2019 9 3 3", "2020 90 3 2")) + + for i, ts := range input { + testdata.OnRecord(func() { + output[i].SQL = ts + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format='brief' " + ts).Rows()) + }) + tk.MustQuery("explain format='brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + } +} + func TestMPPSinglePartitionType(t *testing.T) { var ( input []string diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index e1b43e6bf0975..1aa700f9defd3 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -468,6 +468,7 @@ type PhysicalIndexScan struct { // tblColHists contains all columns before pruning, which are used to calculate row-size tblColHists *statistics.HistColl pkIsHandleCol *expression.Column + prop *property.PhysicalProperty } // Clone implements PhysicalPlan interface. @@ -569,6 +570,7 @@ type PhysicalTableScan struct { // tblCols and tblColHists contains all columns before pruning, which are used to calculate row-size tblCols []*expression.Column tblColHists *statistics.HistColl + prop *property.PhysicalProperty } // Clone implements PhysicalPlan interface. diff --git a/planner/core/plan_cost.go b/planner/core/plan_cost.go index 6dc4132fa2ba2..3a5f5aff903b3 100644 --- a/planner/core/plan_cost.go +++ b/planner/core/plan_cost.go @@ -404,7 +404,7 @@ func (p *PhysicalTableScan) GetPlanCost(taskType property.TaskType, costFlag uin switch p.ctx.GetSessionVars().CostModelVersion { case modelVer1: // scan cost: rows * row-size * scan-factor scanFactor := p.ctx.GetSessionVars().GetScanFactor(p.Table) - if p.Desc { + if p.Desc && p.prop != nil && p.prop.ExpectedCnt >= smallScanThreshold { scanFactor = p.ctx.GetSessionVars().GetDescScanFactor(p.Table) } selfCost = getCardinality(p, costFlag) * p.getScanRowSize() * scanFactor @@ -441,7 +441,7 @@ func (p *PhysicalIndexScan) GetPlanCost(taskType property.TaskType, costFlag uin switch p.ctx.GetSessionVars().CostModelVersion { case modelVer1: // scan cost: rows * row-size * scan-factor scanFactor := p.ctx.GetSessionVars().GetScanFactor(p.Table) - if p.Desc { + if p.Desc && p.prop != nil && p.prop.ExpectedCnt >= smallScanThreshold { scanFactor = p.ctx.GetSessionVars().GetDescScanFactor(p.Table) } selfCost = getCardinality(p, costFlag) * p.getScanRowSize() * scanFactor diff --git a/planner/core/rule_aggregation_skew_rewrite.go b/planner/core/rule_aggregation_skew_rewrite.go new file mode 100644 index 0000000000000..e36a3b7f0ea44 --- /dev/null +++ b/planner/core/rule_aggregation_skew_rewrite.go @@ -0,0 +1,262 @@ +// 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 core + +import ( + "context" + "fmt" + + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/parser/ast" +) + +type skewDistinctAggRewriter struct { +} + +// skewDistinctAggRewriter will rewrite group distinct aggregate into 2 level aggregates, e.g.: +// select S_NATIONKEY as s, count(S_SUPPKEY), count(distinct S_NAME) from supplier group by s; +// will be rewritten to +// select S_NATIONKEY as s, sum(c), count(S_NAME) from ( +// select S_NATIONKEY, S_NAME, count(S_SUPPKEY) c from supplier group by S_NATIONKEY, S_NAME +// ) as T group by s; +// +// If the group key is highly skewed and the distinct key has large number of distinct values +// (a.k.a. high cardinality), the query execution will be slow. This rule may help to ease the +// skew issue. +// +// The rewrite rule only applies to query that satisfies: +// - The aggregate has at least 1 group by column (the group key can be columns or expressions) +// - The aggregate has 1 and only 1 distinct aggregate function (limited to count, avg, sum) +// +// This rule is disabled by default. Use tidb_opt_skew_distinct_agg to enable the rule. +func (a *skewDistinctAggRewriter) rewriteSkewDistinctAgg(agg *LogicalAggregation, opt *logicalOptimizeOp) LogicalPlan { + // only group aggregate is applicable + if len(agg.GroupByItems) == 0 { + return nil + } + + // the number of distinct aggregate functions + nDistinct := 0 + // distinct columns collected from original aggregate + distinctCols := make([]expression.Expression, 0, 3 /* arbitrary value*/) + + for _, aggFunc := range agg.AggFuncs { + if aggFunc.HasDistinct { + nDistinct++ + distinctCols = append(distinctCols, aggFunc.Args...) + } + // TODO: support multiple DQA on same column, e.g. count(distinct x), sum(distinct x) + if nDistinct > 1 { + return nil + } + if !a.isQualifiedAgg(aggFunc) { + return nil + } + } + + // we only deal with single distinct aggregate for now, no more, no less + if nDistinct != 1 { + return nil + } + + // count(distinct a,b,c) group by d + // will generate a bottom agg with group by a,b,c,d + bottomAggGroupbyItems := make([]expression.Expression, 0, len(agg.GroupByItems)+len(distinctCols)) + bottomAggGroupbyItems = append(bottomAggGroupbyItems, agg.GroupByItems...) + bottomAggGroupbyItems = append(bottomAggGroupbyItems, distinctCols...) + + // aggregate functions for top aggregate + topAggFuncs := make([]*aggregation.AggFuncDesc, 0, len(agg.AggFuncs)) + // aggregate functions for bottom aggregate + bottomAggFuncs := make([]*aggregation.AggFuncDesc, 0, len(agg.AggFuncs)) + // output schema for top aggregate + topAggSchema := agg.schema.Clone() + // output schema for bottom aggregate + bottomAggSchema := expression.NewSchema(make([]*expression.Column, 0, agg.schema.Len())...) + + // we only care about non-distinct count() agg function + cntIndexes := make([]int, 0, 3) + + // now decompose original aggregate functions into 2 level aggregate functions, + // except distinct function. each agg function is in COMPLETE mode. + for i, aggFunc := range agg.AggFuncs { + // have to clone it to avoid unexpected modification by others, (︶︹︺) + newAggFunc := aggFunc.Clone() + if aggFunc.HasDistinct { + // TODO: support count(distinct a,b,c) + if len(aggFunc.Args) != 1 { + return nil + } + + for _, arg := range aggFunc.Args { + firstRow, err := aggregation.NewAggFuncDesc(agg.ctx, ast.AggFuncFirstRow, + []expression.Expression{arg}, false) + if err != nil { + return nil + } + bottomAggFuncs = append(bottomAggFuncs, firstRow) + bottomAggSchema.Append(arg.(*expression.Column)) + } + + // now the distinct is not needed anymore + newAggFunc.HasDistinct = false + topAggFuncs = append(topAggFuncs, newAggFunc) + } else { + // only count() will be decomposed to sum() + count(), the others will keep same + // original aggregate functions will go to bottom aggregate without any change + bottomAggFuncs = append(bottomAggFuncs, newAggFunc) + + // cast to Column, if failed, we know it is Constant, ignore the error message, + // we will later create a new schema column + aggCol, _ := newAggFunc.Args[0].(*expression.Column) + + // firstrow() doesn't change the input value and type + if newAggFunc.Name != ast.AggFuncFirstRow { + aggCol = &expression.Column{ + UniqueID: agg.ctx.GetSessionVars().AllocPlanColumnID(), + RetType: newAggFunc.RetTp, + } + } + bottomAggSchema.Append(aggCol) + + if newAggFunc.Name == ast.AggFuncCount { + cntIndexes = append(cntIndexes, i) + sumAggFunc, err := aggregation.NewAggFuncDesc(agg.ctx, ast.AggFuncSum, + []expression.Expression{aggCol}, false) + if err != nil { + return nil + } + topAggFuncs = append(topAggFuncs, sumAggFunc) + topAggSchema.Columns[i] = &expression.Column{ + UniqueID: agg.ctx.GetSessionVars().AllocPlanColumnID(), + RetType: sumAggFunc.RetTp, + } + } else { + topAggFunc := aggFunc.Clone() + topAggFunc.Args = make([]expression.Expression, 0, len(aggFunc.Args)) + topAggFunc.Args = append(topAggFunc.Args, aggCol) + topAggFuncs = append(topAggFuncs, topAggFunc) + } + } + } + + // now create the bottom and top aggregate operators + bottomAgg := LogicalAggregation{ + AggFuncs: bottomAggFuncs, + GroupByItems: bottomAggGroupbyItems, + aggHints: agg.aggHints, + }.Init(agg.ctx, agg.blockOffset) + bottomAgg.SetChildren(agg.children...) + bottomAgg.SetSchema(bottomAggSchema) + + topAgg := LogicalAggregation{ + AggFuncs: topAggFuncs, + GroupByItems: agg.GroupByItems, + aggHints: agg.aggHints, + }.Init(agg.ctx, agg.blockOffset) + topAgg.SetChildren(bottomAgg) + topAgg.SetSchema(topAggSchema) + + if len(cntIndexes) == 0 { + appendSkewDistinctAggRewriteTraceStep(agg, topAgg, opt) + return topAgg + } + + // it has count(), we have split it into sum()+count(), since sum() returns decimal + // we have to return a project operator that casts decimal to bigint + proj := LogicalProjection{ + Exprs: make([]expression.Expression, 0, len(agg.AggFuncs)), + }.Init(agg.ctx, agg.blockOffset) + for _, column := range topAggSchema.Columns { + proj.Exprs = append(proj.Exprs, column.Clone()) + } + + // wrap sum() with cast function to keep output data type same + for _, index := range cntIndexes { + exprType := proj.Exprs[index].GetType() + targetType := agg.schema.Columns[index].GetType() + if !exprType.Equal(targetType) { + proj.Exprs[index] = expression.BuildCastFunction(agg.ctx, proj.Exprs[index], targetType) + } + } + proj.SetSchema(agg.schema.Clone()) + proj.SetChildren(topAgg) + appendSkewDistinctAggRewriteTraceStep(agg, proj, opt) + return proj +} + +func (a *skewDistinctAggRewriter) isQualifiedAgg(aggFunc *aggregation.AggFuncDesc) bool { + if aggFunc.Mode != aggregation.CompleteMode { + return false + } + if len(aggFunc.OrderByItems) > 0 || len(aggFunc.Args) > 1 { + return false + } + + for _, arg := range aggFunc.Args { + if _, ok := arg.(*expression.Column); !ok { + if _, ok := arg.(*expression.Constant); !ok { + return false + } + } + } + + switch aggFunc.Name { + case ast.AggFuncFirstRow, ast.AggFuncCount, ast.AggFuncSum, ast.AggFuncMax, ast.AggFuncMin: + return true + case ast.AggFuncBitAnd, ast.AggFuncBitOr, ast.AggFuncBitXor: + return false + case ast.AggFuncAvg: + return aggFunc.HasDistinct + default: + return false + } +} + +func appendSkewDistinctAggRewriteTraceStep(agg *LogicalAggregation, result LogicalPlan, opt *logicalOptimizeOp) { + reason := func() string { + return fmt.Sprintf("%v_%v has a distinct agg function", agg.TP(), agg.ID()) + } + action := func() string { + return fmt.Sprintf("%v_%v is rewritten to a %v_%v", agg.TP(), agg.ID(), result.TP(), result.ID()) + } + + opt.appendStepToCurrent(agg.ID(), agg.TP(), reason, action) +} + +func (a *skewDistinctAggRewriter) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { + newChildren := make([]LogicalPlan, 0, len(p.Children())) + for _, child := range p.Children() { + newChild, err := a.optimize(ctx, child, opt) + if err != nil { + return nil, err + } + newChildren = append(newChildren, newChild) + } + p.SetChildren(newChildren...) + agg, ok := p.(*LogicalAggregation) + if !ok { + return p, nil + } + if newAgg := a.rewriteSkewDistinctAgg(agg, opt); newAgg != nil { + return newAgg, nil + } + return p, nil +} + +func (*skewDistinctAggRewriter) name() string { + return "skew_distinct_agg_rewrite" +} diff --git a/planner/core/testdata/binary_plan_suite_in.json b/planner/core/testdata/binary_plan_suite_in.json new file mode 100644 index 0000000000000..b2d81b30a5c1f --- /dev/null +++ b/planner/core/testdata/binary_plan_suite_in.json @@ -0,0 +1,22 @@ +[ + { + "name": "TestBinaryPlanInExplainAndSlowLog", + "cases": [ + "create table t(a int, b int, index ia(a))", + "create table t2(a int, b int)", + "insert into t value(1,1), (3,3)", + "explain analyze format = 'binary' insert into t2 value(2,2), (4,4)", + "explain analyze format = 'binary' replace into t2 value(2,2), (4,4)", + "explain analyze format = 'binary' select * from t", + "explain analyze format = 'binary' select sum(t.a) from t join t2", + "explain analyze format = 'binary' select a from t use index (ia) where a > 100", + "explain analyze format = 'binary' select a from t use index () where a > 100", + "explain analyze format = 'binary' with cte1 as (select a, b from t), cte2 as (select a, b from t2) select * from cte1 join cte2 on cte1.a = cte2.a", + "explain analyze format = 'binary' WITH RECURSIVE cte (n) AS( SELECT 1 UNION ALL SELECT n + 1 FROM cte WHERE n < 5)SELECT * FROM cte", + "explain analyze format = 'binary' update t2 set a = a + 1 where b > 10", + "explain analyze format = 'binary' insert into t2 value(2,2), (4,4)", + "explain analyze format = 'binary' insert into t2 select * from t", + "explain analyze format = 'binary' delete from t2" + ] + } +] diff --git a/planner/core/testdata/binary_plan_suite_out.json b/planner/core/testdata/binary_plan_suite_out.json new file mode 100644 index 0000000000000..bea5a39bb2b73 --- /dev/null +++ b/planner/core/testdata/binary_plan_suite_out.json @@ -0,0 +1,566 @@ +[ + { + "Name": "TestBinaryPlanInExplainAndSlowLog", + "Cases": [ + { + "SQL": "create table t(a int, b int, index ia(a))", + "BinaryPlan": null + }, + { + "SQL": "create table t2(a int, b int)", + "BinaryPlan": null + }, + { + "SQL": "insert into t value(1,1), (3,3)", + "BinaryPlan": null + }, + { + "SQL": "explain analyze format = 'binary' insert into t2 value(2,2), (4,4)", + "BinaryPlan": { + "main": { + "name": "Insert_1", + "task_type": 1, + "store_type": 1, + "operator_info": "N/A" + }, + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' replace into t2 value(2,2), (4,4)", + "BinaryPlan": { + "main": { + "name": "Insert_1", + "task_type": 1, + "store_type": 1, + "operator_info": "N/A" + }, + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' select * from t", + "BinaryPlan": { + "main": { + "name": "TableReader_5", + "children": [ + { + "name": "TableFullScan_4", + "cost": 570000, + "est_rows": 10000, + "act_rows": 2, + "task_type": 2, + "store_type": 2, + "operator_info": "keep order:false, stats:pseudo" + } + ], + "cost": 48834.666666666664, + "est_rows": 10000, + "act_rows": 2, + "task_type": 1, + "store_type": 1, + "operator_info": "data:TableFullScan_4" + }, + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' select sum(t.a) from t join t2", + "BinaryPlan": { + "main": { + "name": "HashAgg_8", + "children": [ + { + "name": "Projection_20", + "children": [ + { + "name": "HashJoin_10", + "children": [ + { + "name": "IndexReader_15", + "children": [ + { + "name": "IndexFullScan_14", + "cost": 435000, + "est_rows": 10000, + "act_rows": 2, + "task_type": 2, + "store_type": 2, + "operator_info": "keep order:false, stats:pseudo" + } + ], + "labels": [ + 2 + ], + "cost": 34418, + "est_rows": 10000, + "act_rows": 2, + "task_type": 1, + "store_type": 1, + "operator_info": "index:IndexFullScan_14" + }, + { + "name": "TableReader_17", + "children": [ + { + "name": "TableFullScan_16", + "cost": 570000, + "est_rows": 10000, + "act_rows": 4, + "task_type": 2, + "store_type": 2, + "operator_info": "keep order:false, stats:pseudo" + } + ], + "labels": [ + 1 + ], + "cost": 43418, + "est_rows": 10000, + "act_rows": 4, + "task_type": 1, + "store_type": 1, + "operator_info": "data:TableFullScan_16" + } + ], + "cost": 60107864, + "est_rows": 100000000, + "act_rows": 8, + "task_type": 1, + "store_type": 1, + "operator_info": "CARTESIAN inner join" + } + ], + "est_rows": 100000000, + "act_rows": 8, + "task_type": 1, + "store_type": 1, + "operator_info": "cast(test.t.a, decimal(10,0) BINARY)->Column#8" + } + ], + "cost": 120107897.001, + "est_rows": 1, + "act_rows": 1, + "task_type": 1, + "store_type": 1, + "operator_info": "funcs:sum(Column#8)->Column#7" + }, + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' select a from t use index (ia) where a > 100", + "BinaryPlan": { + "main": { + "name": "IndexReader_6", + "children": [ + { + "name": "IndexRangeScan_5", + "cost": 145000, + "est_rows": 3333.3333333333335, + "task_type": 2, + "store_type": 2, + "operator_info": "range:(100,+inf], keep order:false, stats:pseudo" + } + ], + "cost": 11473.555555555557, + "est_rows": 3333.3333333333335, + "task_type": 1, + "store_type": 1, + "operator_info": "index:IndexRangeScan_5" + }, + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' select a from t use index () where a > 100", + "BinaryPlan": { + "main": { + "name": "TableReader_7", + "children": [ + { + "name": "Selection_6", + "children": [ + { + "name": "TableFullScan_5", + "cost": 570000, + "est_rows": 10000, + "act_rows": 2, + "task_type": 2, + "store_type": 2, + "operator_info": "keep order:false, stats:pseudo" + } + ], + "cost": 600000, + "est_rows": 3333.3333333333335, + "task_type": 2, + "store_type": 2, + "operator_info": "gt(test.t.a, 100)" + } + ], + "cost": 41806.88888888889, + "est_rows": 3333.3333333333335, + "task_type": 1, + "store_type": 1, + "operator_info": "data:Selection_6" + }, + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' with cte1 as (select a, b from t), cte2 as (select a, b from t2) select * from cte1 join cte2 on cte1.a = cte2.a", + "BinaryPlan": { + "main": { + "name": "HashJoin_24", + "children": [ + { + "name": "Selection_26", + "children": [ + { + "name": "CTEFullScan_27", + "est_rows": 8000, + "act_rows": 2, + "task_type": 1, + "store_type": 1, + "operator_info": "data:CTE_0" + } + ], + "labels": [ + 2 + ], + "cost": 108852.66666666666, + "est_rows": 6400, + "act_rows": 2, + "task_type": 1, + "store_type": 1, + "operator_info": "not(isnull(test.t.a))" + }, + { + "name": "Selection_28", + "children": [ + { + "name": "CTEFullScan_29", + "est_rows": 8000, + "act_rows": 4, + "task_type": 1, + "store_type": 1, + "operator_info": "data:CTE_1" + } + ], + "labels": [ + 1 + ], + "cost": 108852.66666666666, + "est_rows": 6400, + "act_rows": 4, + "task_type": 1, + "store_type": 1, + "operator_info": "not(isnull(test.t2.a))" + } + ], + "cost": 241729.7333333333, + "est_rows": 8000, + "task_type": 1, + "store_type": 1, + "operator_info": "inner join, equal:[eq(test.t.a, test.t2.a)]" + }, + "ctes": [ + { + "name": "CTE_0", + "children": [ + { + "name": "Selection_13", + "children": [ + { + "name": "TableReader_16", + "children": [ + { + "name": "TableFullScan_15", + "cost": 570000, + "est_rows": 10000, + "act_rows": 2, + "task_type": 2, + "store_type": 2, + "operator_info": "keep order:false, stats:pseudo" + } + ], + "cost": 48834.666666666664, + "est_rows": 10000, + "act_rows": 2, + "task_type": 1, + "store_type": 1, + "operator_info": "data:TableFullScan_15" + } + ], + "labels": [ + 3 + ], + "cost": 84852.66666666666, + "est_rows": 8000, + "act_rows": 2, + "task_type": 1, + "store_type": 1, + "operator_info": "not(isnull(test.t.a))" + } + ], + "est_rows": 8000, + "act_rows": 2, + "task_type": 1, + "store_type": 1, + "operator_info": "Non-Recursive CTE" + }, + { + "name": "CTE_1", + "children": [ + { + "name": "Selection_18", + "children": [ + { + "name": "TableReader_21", + "children": [ + { + "name": "TableFullScan_20", + "cost": 570000, + "est_rows": 10000, + "act_rows": 4, + "task_type": 2, + "store_type": 2, + "operator_info": "keep order:false, stats:pseudo" + } + ], + "cost": 48834.666666666664, + "est_rows": 10000, + "act_rows": 4, + "task_type": 1, + "store_type": 1, + "operator_info": "data:TableFullScan_20" + } + ], + "labels": [ + 3 + ], + "cost": 84852.66666666666, + "est_rows": 8000, + "act_rows": 4, + "task_type": 1, + "store_type": 1, + "operator_info": "not(isnull(test.t2.a))" + } + ], + "est_rows": 8000, + "act_rows": 4, + "task_type": 1, + "store_type": 1, + "operator_info": "Non-Recursive CTE" + } + ], + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' WITH RECURSIVE cte (n) AS( SELECT 1 UNION ALL SELECT n + 1 FROM cte WHERE n < 5)SELECT * FROM cte", + "BinaryPlan": { + "main": { + "name": "CTEFullScan_17", + "est_rows": 1.8, + "act_rows": 5, + "task_type": 1, + "store_type": 1, + "operator_info": "data:CTE_0" + }, + "ctes": [ + { + "name": "CTE_0", + "children": [ + { + "name": "Projection_11", + "children": [ + { + "name": "TableDual_12", + "est_rows": 1, + "act_rows": 1, + "task_type": 1, + "store_type": 1, + "operator_info": "rows:1" + } + ], + "labels": [ + 3 + ], + "cost": 18.6, + "est_rows": 1, + "act_rows": 1, + "task_type": 1, + "store_type": 1, + "operator_info": "1->Column#2" + }, + { + "name": "Projection_13", + "children": [ + { + "name": "Selection_14", + "children": [ + { + "name": "CTETable_15", + "est_rows": 1, + "act_rows": 5, + "task_type": 1, + "store_type": 1, + "operator_info": "Scan on CTE_0" + } + ], + "cost": 3, + "est_rows": 0.8, + "act_rows": 4, + "task_type": 1, + "store_type": 1, + "operator_info": "lt(Column#3, 5)" + } + ], + "labels": [ + 4 + ], + "cost": 21.48, + "est_rows": 0.8, + "act_rows": 4, + "task_type": 1, + "store_type": 1, + "operator_info": "cast(plus(Column#3, 1), bigint(1) BINARY)->Column#5" + } + ], + "est_rows": 1.8, + "act_rows": 5, + "task_type": 1, + "store_type": 1, + "operator_info": "Recursive CTE" + } + ], + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' update t2 set a = a + 1 where b > 10", + "BinaryPlan": { + "main": { + "name": "Update_4", + "children": [ + { + "name": "TableReader_8", + "children": [ + { + "name": "Selection_7", + "children": [ + { + "name": "TableFullScan_6", + "cost": 570000, + "est_rows": 10000, + "act_rows": 4, + "task_type": 2, + "store_type": 2, + "operator_info": "keep order:false, stats:pseudo" + } + ], + "cost": 600000, + "est_rows": 3333.3333333333335, + "task_type": 2, + "store_type": 2, + "operator_info": "gt(test.t2.b, 10)" + } + ], + "cost": 45418, + "est_rows": 3333.3333333333335, + "task_type": 1, + "store_type": 1, + "operator_info": "data:Selection_7" + } + ], + "task_type": 1, + "store_type": 1, + "operator_info": "N/A" + }, + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' insert into t2 value(2,2), (4,4)", + "BinaryPlan": { + "main": { + "name": "Insert_1", + "task_type": 1, + "store_type": 1, + "operator_info": "N/A" + }, + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' insert into t2 select * from t", + "BinaryPlan": { + "main": { + "name": "Insert_1", + "children": [ + { + "name": "TableReader_7", + "children": [ + { + "name": "TableFullScan_6", + "cost": 570000, + "est_rows": 10000, + "act_rows": 2, + "task_type": 2, + "store_type": 2, + "operator_info": "keep order:false, stats:pseudo" + } + ], + "cost": 48834.666666666664, + "est_rows": 10000, + "act_rows": 2, + "task_type": 1, + "store_type": 1, + "operator_info": "data:TableFullScan_6" + } + ], + "task_type": 1, + "store_type": 1, + "operator_info": "N/A" + }, + "with_runtime_stats": true + } + }, + { + "SQL": "explain analyze format = 'binary' delete from t2", + "BinaryPlan": { + "main": { + "name": "Delete_3", + "children": [ + { + "name": "TableReader_6", + "children": [ + { + "name": "TableFullScan_5", + "cost": 570000, + "est_rows": 10000, + "act_rows": 8, + "task_type": 2, + "store_type": 2, + "operator_info": "keep order:false, stats:pseudo" + } + ], + "cost": 54251.333333333336, + "est_rows": 10000, + "act_rows": 8, + "task_type": 1, + "store_type": 1, + "operator_info": "data:TableFullScan_5" + } + ], + "task_type": 1, + "store_type": 1, + "operator_info": "N/A" + }, + "with_runtime_stats": true + } + } + ] + } +] diff --git a/planner/core/testdata/enforce_mpp_suite_in.json b/planner/core/testdata/enforce_mpp_suite_in.json index 3c70fa18e5a5f..faf58b69d395b 100644 --- a/planner/core/testdata/enforce_mpp_suite_in.json +++ b/planner/core/testdata/enforce_mpp_suite_in.json @@ -94,5 +94,19 @@ "EXPLAIN select o.o_id, count(*) from c, o where c.c_id=o.c_id group by o.o_id; -- 2. test agg push down, group by non-join column", "EXPLAIN select o.c_id, count(*) from c, o where c.c_id=o.c_id group by o.c_id; -- 3. test agg push down, group by join column" ] + }, + { + "name": "TestMPPSkewedGroupDistinctRewrite", + "cases": [ + "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_opt_skew_distinct_agg=1;", + "EXPLAIN select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) from t group by date_format(d,'%Y');", + "EXPLAIN select d, count(distinct c), sum(a), max(b), count(*) from t group by d;", + "EXPLAIN select date_format(d,'%Y') as df, count(distinct c) from t group by date_format(d,'%Y');", + "EXPLAIN select date_format(d,'%Y') as df, a, count(b), count(distinct c) from t group by date_format(d,'%Y'), a;", + "EXPLAIN select date_format(d,'%Y') as df, a, count(b), avg(distinct c) from t group by date_format(d,'%Y'), a;", + "EXPLAIN select d,e, min(b), count(distinct c), bit_or(a) from t group by e,d; -- bit agg func can't be pushed to TiFlash", + "EXPLAIN select a, count(b), avg(distinct c), count(distinct c) from t group by a; -- multi distinct funcs, bail out", + "EXPLAIN select count(b), count(distinct c) from t; -- single distinct func but no group key, bail out" + ] } ] diff --git a/planner/core/testdata/enforce_mpp_suite_out.json b/planner/core/testdata/enforce_mpp_suite_out.json index 4ef7f843bf651..73440140aa7b1 100644 --- a/planner/core/testdata/enforce_mpp_suite_out.json +++ b/planner/core/testdata/enforce_mpp_suite_out.json @@ -709,5 +709,166 @@ "Warn": null } ] + }, + { + "Name": "TestMPPSkewedGroupDistinctRewrite", + "Cases": [ + { + "SQL": "set @@tidb_allow_mpp=1;set @@tidb_enforce_mpp=1;set @@tidb_opt_skew_distinct_agg=1;", + "Plan": null, + "Warn": null + }, + { + "SQL": "EXPLAIN select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) from t group by date_format(d,'%Y');", + "Plan": [ + "TableReader_57 8000.00 root data:ExchangeSender_56", + "└─ExchangeSender_56 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] date_format(test.t.d, %Y)->Column#10, Column#7, cast(Column#14, bigint(21) BINARY)->Column#8, Column#9", + " └─Projection_52 8000.00 mpp[tiflash] Column#7, Column#14, Column#9, test.t.d", + " └─HashAgg_53 8000.00 mpp[tiflash] group by:Column#45, funcs:sum(Column#46)->Column#7, funcs:sum(Column#47)->Column#14, funcs:sum(Column#48)->Column#9, funcs:firstrow(Column#49)->test.t.d", + " └─ExchangeReceiver_55 8000.00 mpp[tiflash] ", + " └─ExchangeSender_54 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#45, collate: utf8mb4_bin]", + " └─HashAgg_51 8000.00 mpp[tiflash] group by:Column#59, funcs:sum(Column#55)->Column#46, funcs:sum(Column#56)->Column#47, funcs:count(Column#57)->Column#48, funcs:firstrow(Column#58)->Column#49", + " └─Projection_59 8000.00 mpp[tiflash] Column#12, cast(Column#13, decimal(20,0) BINARY)->Column#56, test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#59", + " └─Projection_38 8000.00 mpp[tiflash] Column#12, Column#13, test.t.c, test.t.d", + " └─HashAgg_39 8000.00 mpp[tiflash] group by:Column#30, test.t.c, funcs:sum(Column#31)->Column#12, funcs:sum(Column#32)->Column#13, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(Column#34)->test.t.d", + " └─ExchangeReceiver_41 8000.00 mpp[tiflash] ", + " └─ExchangeSender_40 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_37 8000.00 mpp[tiflash] group by:Column#53, Column#54, funcs:sum(Column#50)->Column#31, funcs:count(Column#51)->Column#32, funcs:firstrow(Column#52)->Column#34", + " └─Projection_58 10000.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#50, test.t.b, test.t.d, date_format(test.t.d, %Y)->Column#53, test.t.c", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select d, count(distinct c), sum(a), max(b), count(*) from t group by d;", + "Plan": [ + "TableReader_57 8000.00 root data:ExchangeSender_56", + "└─ExchangeSender_56 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] test.t.d, Column#7, Column#8, Column#9, cast(Column#14, bigint(21) BINARY)->Column#10", + " └─Projection_52 8000.00 mpp[tiflash] Column#7, Column#8, Column#9, Column#14, test.t.d", + " └─HashAgg_53 8000.00 mpp[tiflash] group by:test.t.d, funcs:sum(Column#45)->Column#7, funcs:sum(Column#46)->Column#8, funcs:max(Column#47)->Column#9, funcs:sum(Column#48)->Column#14, funcs:firstrow(test.t.d)->test.t.d", + " └─ExchangeReceiver_55 8000.00 mpp[tiflash] ", + " └─ExchangeSender_54 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.d, collate: binary]", + " └─HashAgg_51 8000.00 mpp[tiflash] group by:Column#58, funcs:count(Column#54)->Column#45, funcs:sum(Column#55)->Column#46, funcs:max(Column#56)->Column#47, funcs:sum(Column#57)->Column#48", + " └─Projection_59 8000.00 mpp[tiflash] test.t.c, Column#11, Column#12, cast(Column#13, decimal(20,0) BINARY)->Column#57, test.t.d", + " └─Projection_38 8000.00 mpp[tiflash] test.t.c, Column#11, Column#12, Column#13, test.t.d", + " └─HashAgg_39 8000.00 mpp[tiflash] group by:test.t.c, test.t.d, funcs:firstrow(test.t.c)->test.t.c, funcs:sum(Column#31)->Column#11, funcs:max(Column#32)->Column#12, funcs:sum(Column#33)->Column#13, funcs:firstrow(test.t.d)->test.t.d", + " └─ExchangeReceiver_41 8000.00 mpp[tiflash] ", + " └─ExchangeSender_40 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.d, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg_37 8000.00 mpp[tiflash] group by:Column#52, Column#53, funcs:sum(Column#50)->Column#31, funcs:max(Column#51)->Column#32, funcs:count(1)->Column#33", + " └─Projection_58 10000.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#50, test.t.b, test.t.d, test.t.c", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select date_format(d,'%Y') as df, count(distinct c) from t group by date_format(d,'%Y');", + "Plan": [ + "TableReader_56 8000.00 root data:ExchangeSender_55", + "└─ExchangeSender_55 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_7 8000.00 mpp[tiflash] date_format(test.t.d, %Y)->Column#8, Column#7", + " └─Projection_51 8000.00 mpp[tiflash] Column#7, test.t.d", + " └─HashAgg_52 8000.00 mpp[tiflash] group by:Column#28, funcs:sum(Column#29)->Column#7, funcs:firstrow(Column#30)->test.t.d", + " └─ExchangeReceiver_54 8000.00 mpp[tiflash] ", + " └─ExchangeSender_53 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#28, collate: utf8mb4_bin]", + " └─HashAgg_50 8000.00 mpp[tiflash] group by:Column#36, funcs:count(Column#34)->Column#29, funcs:firstrow(Column#35)->Column#30", + " └─Projection_58 8000.00 mpp[tiflash] test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#36", + " └─Projection_37 8000.00 mpp[tiflash] test.t.c, test.t.d", + " └─HashAgg_38 8000.00 mpp[tiflash] group by:Column#19, test.t.c, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(Column#21)->test.t.d", + " └─ExchangeReceiver_40 8000.00 mpp[tiflash] ", + " └─ExchangeSender_39 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c, collate: binary]", + " └─HashAgg_36 8000.00 mpp[tiflash] group by:Column#32, Column#33, funcs:firstrow(Column#31)->Column#21", + " └─Projection_57 10000.00 mpp[tiflash] test.t.d, date_format(test.t.d, %Y)->Column#32, test.t.c", + " └─TableFullScan_26 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select date_format(d,'%Y') as df, a, count(b), count(distinct c) from t group by date_format(d,'%Y'), a;", + "Plan": [ + "TableReader_57 8000.00 root data:ExchangeSender_56", + "└─ExchangeSender_56 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] date_format(test.t.d, %Y)->Column#9, test.t.a, cast(Column#12, bigint(21) BINARY)->Column#7, Column#8", + " └─Projection_52 8000.00 mpp[tiflash] Column#12, Column#8, test.t.a, test.t.d", + " └─HashAgg_53 8000.00 mpp[tiflash] group by:Column#43, test.t.a, funcs:sum(Column#44)->Column#12, funcs:sum(Column#45)->Column#8, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(Column#47)->test.t.d", + " └─ExchangeReceiver_55 8000.00 mpp[tiflash] ", + " └─ExchangeSender_54 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─HashAgg_51 8000.00 mpp[tiflash] group by:Column#56, Column#57, funcs:sum(Column#53)->Column#44, funcs:count(Column#54)->Column#45, funcs:firstrow(Column#55)->Column#47", + " └─Projection_59 8000.00 mpp[tiflash] cast(Column#11, decimal(20,0) BINARY)->Column#53, test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#56, test.t.a", + " └─Projection_38 8000.00 mpp[tiflash] Column#11, test.t.c, test.t.a, test.t.d", + " └─HashAgg_39 8000.00 mpp[tiflash] group by:Column#28, test.t.a, test.t.c, funcs:sum(Column#29)->Column#11, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(Column#32)->test.t.d", + " └─ExchangeReceiver_41 8000.00 mpp[tiflash] ", + " └─ExchangeSender_40 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg_37 8000.00 mpp[tiflash] group by:Column#50, Column#51, Column#52, funcs:count(Column#48)->Column#29, funcs:firstrow(Column#49)->Column#32", + " └─Projection_58 10000.00 mpp[tiflash] test.t.b, test.t.d, date_format(test.t.d, %Y)->Column#50, test.t.a, test.t.c", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select date_format(d,'%Y') as df, a, count(b), avg(distinct c) from t group by date_format(d,'%Y'), a;", + "Plan": [ + "TableReader_57 8000.00 root data:ExchangeSender_56", + "└─ExchangeSender_56 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_8 8000.00 mpp[tiflash] date_format(test.t.d, %Y)->Column#9, test.t.a, cast(Column#12, bigint(21) BINARY)->Column#7, Column#8", + " └─Projection_52 8000.00 mpp[tiflash] Column#12, div(Column#8, cast(case(eq(Column#46, 0), 1, Column#46), decimal(20,0) BINARY))->Column#8, test.t.a, test.t.d", + " └─HashAgg_53 8000.00 mpp[tiflash] group by:Column#47, test.t.a, funcs:sum(Column#48)->Column#12, funcs:sum(Column#49)->Column#46, funcs:sum(Column#50)->Column#8, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(Column#52)->test.t.d", + " └─ExchangeReceiver_55 8000.00 mpp[tiflash] ", + " └─ExchangeSender_54 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─HashAgg_51 8000.00 mpp[tiflash] group by:Column#62, Column#63, funcs:sum(Column#58)->Column#48, funcs:count(Column#59)->Column#49, funcs:sum(Column#60)->Column#50, funcs:firstrow(Column#61)->Column#52", + " └─Projection_59 8000.00 mpp[tiflash] cast(Column#11, decimal(20,0) BINARY)->Column#58, test.t.c, cast(test.t.c, decimal(24,4) BINARY)->Column#60, test.t.d, date_format(test.t.d, %Y)->Column#62, test.t.a", + " └─Projection_38 8000.00 mpp[tiflash] Column#11, test.t.c, test.t.a, test.t.d", + " └─HashAgg_39 8000.00 mpp[tiflash] group by:Column#28, test.t.a, test.t.c, funcs:sum(Column#29)->Column#11, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(Column#32)->test.t.d", + " └─ExchangeReceiver_41 8000.00 mpp[tiflash] ", + " └─ExchangeSender_40 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg_37 8000.00 mpp[tiflash] group by:Column#55, Column#56, Column#57, funcs:count(Column#53)->Column#29, funcs:firstrow(Column#54)->Column#32", + " └─Projection_58 10000.00 mpp[tiflash] test.t.b, test.t.d, date_format(test.t.d, %Y)->Column#55, test.t.a, test.t.c", + " └─TableFullScan_27 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "EXPLAIN select d,e, min(b), count(distinct c), bit_or(a) from t group by e,d; -- bit agg func can't be pushed to TiFlash", + "Plan": [ + "Projection_4 8000.00 root test.t.d, test.t.e, Column#7, Column#8, Column#9", + "└─HashAgg_6 8000.00 root group by:test.t.d, test.t.e, funcs:min(test.t.b)->Column#7, funcs:count(distinct test.t.c)->Column#8, funcs:bit_or(test.t.a)->Column#9, funcs:firstrow(test.t.d)->test.t.d, funcs:firstrow(test.t.e)->test.t.e", + " └─TableReader_10 10000.00 root data:TableFullScan_9", + " └─TableFullScan_9 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "Aggregation can not be pushed to tiflash because AggFunc `bit_or` is not supported now", + "Aggregation can not be pushed to tiflash because AggFunc `bit_or` is not supported now" + ] + }, + { + "SQL": "EXPLAIN select a, count(b), avg(distinct c), count(distinct c) from t group by a; -- multi distinct funcs, bail out", + "Plan": [ + "Projection_4 8000.00 root test.t.a, Column#7, Column#8, Column#9", + "└─HashAgg_6 8000.00 root group by:Column#14, funcs:count(Column#10)->Column#7, funcs:avg(distinct Column#11)->Column#8, funcs:count(distinct Column#12)->Column#9, funcs:firstrow(Column#13)->test.t.a", + " └─Projection_11 10000.00 root test.t.b, cast(test.t.c, decimal(24,4) BINARY)->Column#11, test.t.c, test.t.a, test.t.a", + " └─TableReader_10 10000.00 root data:TableFullScan_9", + " └─TableFullScan_9 10000.00 cop[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "Aggregation can not be pushed to storage layer in mpp mode because it contains agg function with distinct", + "Aggregation can not be pushed to storage layer in mpp mode because it contains agg function with distinct" + ] + }, + { + "SQL": "EXPLAIN select count(b), count(distinct c) from t; -- single distinct func but no group key, bail out", + "Plan": [ + "TableReader_30 1.00 root data:ExchangeSender_29", + "└─ExchangeSender_29 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_25 1.00 mpp[tiflash] Column#7, Column#8", + " └─HashAgg_26 1.00 mpp[tiflash] funcs:sum(Column#10)->Column#7, funcs:count(distinct test.t.c)->Column#8", + " └─ExchangeReceiver_28 1.00 mpp[tiflash] ", + " └─ExchangeSender_27 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg_24 1.00 mpp[tiflash] group by:test.t.c, funcs:count(test.t.b)->Column#10", + " └─TableFullScan_13 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + } + ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index fe0f0d0a6586a..eb3ed02b3886a 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1810,7 +1810,7 @@ "SQL": "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", "Plan": [ "Selection_6 0.01 8.60 root eq(test.t2.b, 2), in(test.t2.c, 1, 2, 3, 4, 5)", - "└─Point_Get_5 1.00 0.00 root table:t2, index:idx_a(a) " + "└─Point_Get_5 1.00 5.60 root table:t2, index:idx_a(a) " ], "Warnings": [ "Note 1105 unique index idx_a of t2 is selected since the path only has point ranges with double scan" @@ -1867,7 +1867,7 @@ "Plan": [ "Delete_4 N/A N/A root N/A", "└─Selection_7 2.00 9.80 root in(test.t1.g, 3, 4)", - " └─Point_Get_6 1.00 0.00 root table:t1, index:f(f) " + " └─Point_Get_6 1.00 6.80 root table:t1, index:f(f) " ], "Warnings": [ "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" @@ -1903,9 +1903,9 @@ "SQL": "select * from t where a > 1 order by f", "Plan": [ "IndexLookUp_14 3333.33 136747.00 root ", - "├─Selection_13(Build) 3333.33 0.00 cop[tikv] gt(test.t.a, 1)", + "├─Selection_13(Build) 3333.33 585000.00 cop[tikv] gt(test.t.a, 1)", "│ └─IndexFullScan_11 10000.00 555000.00 cop[tikv] table:t, index:f(f) keep order:true, stats:pseudo", - "└─TableRowIDScan_12(Probe) 3333.33 555000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "└─TableRowIDScan_12(Probe) 3333.33 370000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}" @@ -1916,7 +1916,7 @@ "Plan": [ "IndexLookUp_10 3333.33 86674.83 root ", "├─IndexRangeScan_8(Build) 3333.33 185000.00 cop[tikv] table:t, index:f(f) range:(1,+inf], keep order:false, stats:pseudo", - "└─TableRowIDScan_9(Probe) 3333.33 185000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "└─TableRowIDScan_9(Probe) 3333.33 370000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -1937,8 +1937,8 @@ "Plan": [ "IndexLookUp_15 3.33 206.74 root ", "├─IndexRangeScan_12(Build) 10.00 570.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo", - "└─Selection_14(Probe) 3.33 0.00 cop[tikv] gt(test.t.f, 3)", - " └─TableRowIDScan_13 10.00 570.00 cop[tikv] table:t keep order:false, stats:pseudo" + "└─Selection_14(Probe) 3.33 1140.00 cop[tikv] gt(test.t.f, 3)", + " └─TableRowIDScan_13 10.00 1110.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -1950,7 +1950,7 @@ "Sort_5 10.00 353.68 root test.t.f", "└─IndexLookUp_13 10.00 230.01 root ", " ├─IndexRangeScan_11(Build) 10.00 570.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo", - " └─TableRowIDScan_12(Probe) 10.00 570.00 cop[tikv] table:t keep order:false, stats:pseudo" + " └─TableRowIDScan_12(Probe) 10.00 1110.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -1960,9 +1960,9 @@ "SQL": "select * from t where d = 3 order by c, e", "Plan": [ "IndexLookUp_15 10.00 57222.78 root ", - "├─Selection_14(Build) 10.00 0.00 cop[tikv] eq(test.t.d, 3)", + "├─Selection_14(Build) 10.00 855000.00 cop[tikv] eq(test.t.d, 3)", "│ └─IndexFullScan_12 10000.00 825000.00 cop[tikv] table:t, index:c_d_e(c, d, e) keep order:true, stats:pseudo", - "└─TableRowIDScan_13(Probe) 10.00 825000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "└─TableRowIDScan_13(Probe) 10.00 1110.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,c_d_e] remain after pruning paths for t given Prop{SortItems: [{test.t.c asc} {test.t.e asc}], TaskTp: rootTask}" @@ -2019,7 +2019,7 @@ "Plan": [ "IndexLookUp_7 3.00 57.91 root ", "├─IndexRangeScan_5(Build) 3.00 171.00 cop[tikv] table:t, index:idx_b(b) range:(5,+inf], keep order:false", - "└─TableRowIDScan_6(Probe) 3.00 171.00 cop[tikv] table:t keep order:false" + "└─TableRowIDScan_6(Probe) 3.00 108.00 cop[tikv] table:t keep order:false" ], "Warnings": [ "Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -2460,7 +2460,7 @@ "Plan": [ "TopN_7 1.00 13.22 root test.t3.a, offset:0, count:1", "└─TableReader_16 1.00 10.22 root data:TopN_15", - " └─TopN_15 1.00 0.00 cop[tikv] test.t3.a, offset:0, count:1", + " └─TopN_15 1.00 117.00 cop[tikv] test.t3.a, offset:0, count:1", " └─TableFullScan_14 3.00 108.00 cop[tikv] table:t3 keep order:false" ] }, @@ -2469,7 +2469,7 @@ "Plan": [ "TopN_7 1.00 13.22 root test.t3.b, offset:0, count:1", "└─TableReader_16 1.00 10.22 root data:TopN_15", - " └─TopN_15 1.00 0.00 cop[tikv] test.t3.b, offset:0, count:1", + " └─TopN_15 1.00 117.00 cop[tikv] test.t3.b, offset:0, count:1", " └─TableFullScan_14 3.00 108.00 cop[tikv] table:t3 keep order:false" ] }, @@ -2481,7 +2481,7 @@ " └─Projection_22 3.00 76.80 mpp[tiflash] Column#4", " └─HashAgg_8 3.00 57.00 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", " └─ExchangeReceiver_21 3.00 48.00 mpp[tiflash] ", - " └─ExchangeSender_20 3.00 48.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", + " └─ExchangeSender_20 3.00 45.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", " └─TableFullScan_19 3.00 45.00 mpp[tiflash] table:t2 keep order:false" ] }, @@ -2531,7 +2531,7 @@ " └─ExchangeSender_45 3.00 195.38 mpp[tiflash] ExchangeType: PassThrough", " └─HashJoin_42 3.00 195.38 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─ExchangeReceiver_21(Build) 3.00 57.00 mpp[tiflash] ", - " │ └─ExchangeSender_20 3.00 57.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─ExchangeSender_20 3.00 54.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Selection_19 3.00 54.00 mpp[tiflash] not(isnull(test.t1.a))", " │ └─TableFullScan_18 3.00 45.00 mpp[tiflash] table:t1 keep order:false", " └─Selection_23(Probe) 3.00 54.00 mpp[tiflash] not(isnull(test.t2.a))", @@ -2549,7 +2549,7 @@ " └─ExchangeSender_38 3.00 204.38 mpp[tiflash] ExchangeType: PassThrough", " └─HashJoin_29 3.00 204.38 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─ExchangeReceiver_35(Build) 3.00 66.00 mpp[tiflash] ", - " │ └─ExchangeSender_34 3.00 66.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─ExchangeSender_34 3.00 63.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Selection_33 3.00 63.00 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", " │ └─TableFullScan_32 3.00 54.00 mpp[tiflash] table:t1 keep order:false", " └─Selection_37(Probe) 3.00 54.00 mpp[tiflash] not(isnull(test.t2.a))", diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index 627621aa1bfe6..d04c9d8944158 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -747,6 +747,14 @@ "select * from t where t.a < 3 and t.a < 3" ] }, + { + "name": "TestSkewDistinctAgg", + "cases": [ + "select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) from t group by date_format(d,'%Y')", + "select d, a, count(*), count(b), count(distinct c) from t group by d, a", + "select d, sum(a), count(b), avg(distinct c) from t group by d" + ] + }, { "name": "TestMPPSinglePartitionType", "cases": [ diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 49b92cb6e9afa..40c13bde56680 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -2678,6 +2678,46 @@ } ] }, + { + "Name": "TestSkewDistinctAgg", + "Cases": [ + { + "SQL": "select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) from t group by date_format(d,'%Y')", + "Plan": [ + "Projection 8000.00 root date_format(test.t.d, %Y)->Column#9, Column#6, cast(Column#13, bigint(21) BINARY)->Column#7, Column#8", + "└─HashAgg 8000.00 root group by:Column#29, funcs:sum(Column#25)->Column#6, funcs:sum(Column#26)->Column#13, funcs:count(Column#27)->Column#8, funcs:firstrow(Column#28)->test.t.d", + " └─Projection 8000.00 root Column#11, cast(Column#12, decimal(20,0) BINARY)->Column#26, test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#29", + " └─HashAgg 8000.00 root group by:Column#23, Column#24, funcs:sum(Column#19)->Column#11, funcs:count(Column#20)->Column#12, funcs:firstrow(Column#21)->test.t.c, funcs:firstrow(Column#22)->test.t.d", + " └─Projection 10000.00 root cast(test.t.a, decimal(10,0) BINARY)->Column#19, test.t.b, test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#23, test.t.c", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select d, a, count(*), count(b), count(distinct c) from t group by d, a", + "Plan": [ + "Projection 8000.00 root test.t.d, test.t.a, cast(Column#10, bigint(21) BINARY)->Column#6, cast(Column#12, bigint(21) BINARY)->Column#7, Column#8", + "└─HashAgg 8000.00 root group by:Column#23, Column#24, funcs:sum(Column#18)->Column#10, funcs:sum(Column#19)->Column#12, funcs:count(Column#20)->Column#8, funcs:firstrow(Column#21)->test.t.a, funcs:firstrow(Column#22)->test.t.d", + " └─Projection 8000.00 root cast(Column#9, decimal(20,0) BINARY)->Column#18, cast(Column#11, decimal(20,0) BINARY)->Column#19, test.t.c, test.t.a, test.t.d, test.t.d, test.t.a", + " └─HashAgg 8000.00 root group by:test.t.a, test.t.c, test.t.d, funcs:count(1)->Column#9, funcs:count(test.t.b)->Column#11, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(test.t.d)->test.t.d", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select d, sum(a), count(b), avg(distinct c) from t group by d", + "Plan": [ + "Projection 8000.00 root test.t.d, Column#6, cast(Column#11, bigint(21) BINARY)->Column#7, Column#8", + "└─HashAgg 8000.00 root group by:Column#20, funcs:sum(Column#16)->Column#6, funcs:sum(Column#17)->Column#11, funcs:avg(Column#18)->Column#8, funcs:firstrow(Column#19)->test.t.d", + " └─Projection 8000.00 root Column#9, cast(Column#10, decimal(20,0) BINARY)->Column#17, cast(test.t.c, decimal(15,4) BINARY)->Column#18, test.t.d, test.t.d", + " └─HashAgg 8000.00 root group by:test.t.c, test.t.d, funcs:sum(Column#12)->Column#9, funcs:count(Column#13)->Column#10, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.d)->test.t.d", + " └─TableReader 8000.00 root data:HashAgg", + " └─HashAgg 8000.00 cop[tikv] group by:test.t.c, test.t.d, funcs:sum(test.t.a)->Column#12, funcs:count(test.t.b)->Column#13", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + } + ] + }, { "Name": "TestMPPSinglePartitionType", "Cases": [ diff --git a/session/session.go b/session/session.go index 91296a0c5c643..03b54e43eb892 100644 --- a/session/session.go +++ b/session/session.go @@ -2644,6 +2644,7 @@ func CreateSession4TestWithOpt(store kv.Storage, opt *Opt) (Session, error) { // initialize session variables for test. s.GetSessionVars().InitChunkSize = 2 s.GetSessionVars().MaxChunkSize = 32 + s.GetSessionVars().EnablePaging = variable.DefTiDBEnablePaging err = s.GetSessionVars().SetSystemVar(variable.CharacterSetConnection, "utf8mb4") } return s, err diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 4fa4876cb4b0f..352713d9dbf19 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -689,6 +689,9 @@ type SessionVars struct { // AllowDistinctAggPushDown can be set true to allow agg with distinct push down to tikv/tiflash. AllowDistinctAggPushDown bool + // EnableSkewDistinctAgg can be set true to allow skew distinct aggregate rewrite + EnableSkewDistinctAgg bool + // MultiStatementMode permits incorrect client library usage. Not recommended to be turned on. MultiStatementMode int @@ -1425,6 +1428,7 @@ func NewSessionVars() *SessionVars { StatsLoadSyncWait: StatsLoadSyncWait.Load(), EnableLegacyInstanceScope: DefEnableLegacyInstanceScope, RemoveOrderbyInSubquery: DefTiDBRemoveOrderbyInSubquery, + EnableSkewDistinctAgg: DefTiDBSkewDistinctAgg, MaxAllowedPacket: DefMaxAllowedPacket, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 27beb3b5ae306..925a2ed47d4de 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -161,6 +161,10 @@ var defaultSysVars = []*SysVar{ s.AllowDistinctAggPushDown = TiDBOptOn(val) return nil }}, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptSkewDistinctAgg, Value: BoolToOnOff(DefTiDBSkewDistinctAgg), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + s.EnableSkewDistinctAgg = TiDBOptOn(val) + return nil + }}, {Scope: ScopeSession, Name: TiDBOptWriteRowID, Value: BoolToOnOff(DefOptWriteRowID), Type: TypeBool, skipInit: true, SetSession: func(s *SessionVars, val string) error { s.AllowWriteRowID = TiDBOptOn(val) return nil @@ -1619,7 +1623,7 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNewCostInterface, Value: BoolToOnOff(false), Hidden: true, Type: TypeBool, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNewCostInterface, Value: BoolToOnOff(true), Hidden: true, Type: TypeBool, SetSession: func(vars *SessionVars, s string) error { vars.EnableNewCostInterface = TiDBOptOn(s) return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index f4b82dbc47398..6b2ebbf181854 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -50,6 +50,9 @@ const ( // TiDBOptDistinctAggPushDown is used to decide whether agg with distinct should be pushed to tikv/tiflash. TiDBOptDistinctAggPushDown = "tidb_opt_distinct_agg_push_down" + // TiDBOptSkewDistinctAgg is used to indicate the distinct agg has data skew + TiDBOptSkewDistinctAgg = "tidb_opt_skew_distinct_agg" + // TiDBBCJThresholdSize is used to limit the size of small table for mpp broadcast join. // Its unit is bytes, if the size of small table is larger than it, we will not use bcj. TiDBBCJThresholdSize = "tidb_broadcast_join_threshold_size" @@ -940,6 +943,7 @@ const ( DefTiDBBatchPendingTiFlashCount = 4000 DefRCReadCheckTS = false DefTiDBRemoveOrderbyInSubquery = false + DefTiDBSkewDistinctAgg = false DefTiDBReadStaleness = 0 DefTiDBGCMaxWaitTime = 24 * 60 * 60 DefMaxAllowedPacket uint64 = 67108864 diff --git a/store/gcworker/BUILD.bazel b/store/gcworker/BUILD.bazel index 9ac77dd4ceb30..f43e28c02958b 100644 --- a/store/gcworker/BUILD.bazel +++ b/store/gcworker/BUILD.bazel @@ -6,6 +6,7 @@ go_library( importpath = "github.com/pingcap/tidb/store/gcworker", visibility = ["//visibility:public"], deps = [ + "//br/pkg/utils", "//ddl", "//ddl/label", "//ddl/placement", diff --git a/telemetry/BUILD.bazel b/telemetry/BUILD.bazel index 05df184817252..7cce3b2401d35 100644 --- a/telemetry/BUILD.bazel +++ b/telemetry/BUILD.bazel @@ -18,6 +18,7 @@ go_library( importpath = "github.com/pingcap/tidb/telemetry", visibility = ["//visibility:public"], deps = [ + "//br/pkg/utils", "//config", "//domain/infosync", "//infoschema", diff --git a/telemetry/data_feature_usage.go b/telemetry/data_feature_usage.go index 9020a7d10651b..0a2b77499aa32 100644 --- a/telemetry/data_feature_usage.go +++ b/telemetry/data_feature_usage.go @@ -49,6 +49,7 @@ type featureUsage struct { GlobalKill bool `json:"globalKill"` MultiSchemaChange *m.MultiSchemaChangeUsageCounter `json:"multiSchemaChange"` LogBackup bool `json:"logBackup"` + EnablePaging bool `json:"enablePaging"` } type placementPolicyUsage struct { @@ -85,6 +86,8 @@ func getFeatureUsage(ctx context.Context, sctx sessionctx.Context) (*featureUsag usage.LogBackup = getLogBackupUsageInfo(sctx) + usage.EnablePaging = getPagingUsageInfo(sctx) + return &usage, nil } @@ -287,3 +290,10 @@ func getGlobalKillUsageInfo() bool { func getLogBackupUsageInfo(ctx sessionctx.Context) bool { return utils.CheckLogBackupEnabled(ctx) } + +// getPagingUsageInfo gets the value of system variable `tidb_enable_paging`. +// This variable is set to true as default since v6.2.0. We want to know many +// users set it to false manually. +func getPagingUsageInfo(ctx sessionctx.Context) bool { + return ctx.GetSessionVars().EnablePaging +} diff --git a/telemetry/data_feature_usage_test.go b/telemetry/data_feature_usage_test.go index d89928bbcc19d..50617fbbf7de9 100644 --- a/telemetry/data_feature_usage_test.go +++ b/telemetry/data_feature_usage_test.go @@ -279,3 +279,18 @@ func TestGlobalKillUsageInfo(t *testing.T) { require.NoError(t, err) require.False(t, usage.GlobalKill) } + +func TestPagingUsageInfo(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + usage, err := telemetry.GetFeatureUsage(tk.Session()) + require.NoError(t, err) + require.True(t, usage.EnablePaging == variable.DefTiDBEnablePaging) + + tk.Session().GetSessionVars().EnablePaging = false + usage, err = telemetry.GetFeatureUsage(tk.Session()) + require.NoError(t, err) + require.False(t, usage.EnablePaging) +} diff --git a/types/explain_format.go b/types/explain_format.go index 6c8ef6f72d546..2599f7bb046ed 100644 --- a/types/explain_format.go +++ b/types/explain_format.go @@ -31,6 +31,8 @@ var ( ExplainFormatTraditional = "traditional" // ExplainFormatTrueCardCost indicates the optimizer to use true cardinality to calculate the cost. ExplainFormatTrueCardCost = "true_card_cost" + // ExplainFormatBinary prints the proto for binary plan. + ExplainFormatBinary = "binary" // ExplainFormats stores the valid formats for explain statement, used by validator. ExplainFormats = []string{ @@ -42,5 +44,6 @@ var ( ExplainFormatVerbose, ExplainFormatTraditional, ExplainFormatTrueCardCost, + ExplainFormatBinary, } ) diff --git a/util/dbterror/ddl_terror.go b/util/dbterror/ddl_terror.go index b65f516bf71c4..2e90d25ffdd2c 100644 --- a/util/dbterror/ddl_terror.go +++ b/util/dbterror/ddl_terror.go @@ -33,7 +33,7 @@ var ( // ErrCancelledDDLJob means the DDL job is cancelled. ErrCancelledDDLJob = ClassDDL.NewStd(mysql.ErrCancelledDDLJob) // ErrRunMultiSchemaChanges means we run multi schema changes. - ErrRunMultiSchemaChanges = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "multi schema change"), nil)) + ErrRunMultiSchemaChanges = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "multi schema change for %s"), nil)) // ErrOperateSameColumn means we change the same columns multiple times in a DDL. ErrOperateSameColumn = ClassDDL.NewStdErr(mysql.ErrUnsupportedDDLOperation, parser_mysql.Message(fmt.Sprintf(mysql.MySQLErrName[mysql.ErrUnsupportedDDLOperation].Raw, "operate same column '%s'"), nil)) // ErrOperateSameIndex means we change the same indexes multiple times in a DDL. diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index 30efc150f2214..db54f2c05c74a 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -86,6 +86,8 @@ const ( CommitBackoffTimeStr = "Commit_backoff_time" // BackoffTypesStr means the backoff type. BackoffTypesStr = "Backoff_types" + // SlowestCommitRPCDetailStr means the details of the slowest RPC during the transaction commit process. + SlowestCommitRPCDetailStr = "Slowest_commit_rpc_detail" // ResolveLockTimeStr means the time of resolving lock. ResolveLockTimeStr = "Resolve_lock_time" // LocalLatchWaitTimeStr means the time of waiting in local latch. @@ -98,6 +100,8 @@ const ( PrewriteRegionStr = "Prewrite_region" // TxnRetryStr means the count of transaction retry. TxnRetryStr = "Txn_retry" + // GetSnapshotTimeStr means the time spent on getting an engine snapshot. + GetSnapshotTimeStr = "Get_snapshot_time" // RocksdbDeleteSkippedCountStr means the count of rocksdb delete skipped count. RocksdbDeleteSkippedCountStr = "Rocksdb_delete_skipped_count" // RocksdbKeySkippedCountStr means the count of rocksdb key skipped count. @@ -108,6 +112,8 @@ const ( RocksdbBlockReadCountStr = "Rocksdb_block_read_count" // RocksdbBlockReadByteStr means the bytes of rocksdb block read. RocksdbBlockReadByteStr = "Rocksdb_block_read_byte" + // RocksdbBlockReadTimeStr means the time spent on rocksdb block read. + RocksdbBlockReadTimeStr = "Rocksdb_block_read_time" ) // String implements the fmt.Stringer interface. @@ -156,6 +162,12 @@ func (d ExecDetails) String() string { if len(commitDetails.Mu.BackoffTypes) > 0 { parts = append(parts, BackoffTypesStr+": "+fmt.Sprintf("%v", commitDetails.Mu.BackoffTypes)) } + if commitDetails.Mu.SlowestReqTotalTime > 0 { + parts = append(parts, SlowestCommitRPCDetailStr+": {total:"+strconv.FormatFloat(commitDetails.Mu.SlowestReqTotalTime.Seconds(), 'f', 3, 64)+ + "s, region_id: "+strconv.FormatUint(commitDetails.Mu.SlowestRegion, 10)+ + ", store: "+commitDetails.Mu.SlowestStoreAddr+ + ", "+commitDetails.Mu.SlowestExecDetails.String()+"}") + } commitDetails.Mu.Unlock() resolveLockTime := atomic.LoadInt64(&commitDetails.ResolveLock.ResolveLockTime) if resolveLockTime > 0 { @@ -186,6 +198,9 @@ func (d ExecDetails) String() string { if scanDetail.TotalKeys > 0 { parts = append(parts, TotalKeysStr+": "+strconv.FormatInt(scanDetail.TotalKeys, 10)) } + if scanDetail.GetSnapshotDuration > 0 { + parts = append(parts, GetSnapshotTimeStr+": "+strconv.FormatFloat(scanDetail.GetSnapshotDuration.Seconds(), 'f', 3, 64)) + } if scanDetail.RocksdbDeleteSkippedCount > 0 { parts = append(parts, RocksdbDeleteSkippedCountStr+": "+strconv.FormatUint(scanDetail.RocksdbDeleteSkippedCount, 10)) } @@ -201,6 +216,9 @@ func (d ExecDetails) String() string { if scanDetail.RocksdbBlockReadByte > 0 { parts = append(parts, RocksdbBlockReadByteStr+": "+strconv.FormatUint(scanDetail.RocksdbBlockReadByte, 10)) } + if scanDetail.RocksdbBlockReadDuration > 0 { + parts = append(parts, RocksdbBlockReadTimeStr+": "+strconv.FormatFloat(scanDetail.RocksdbBlockReadDuration.Seconds(), 'f', 3, 64)) + } } return strings.Join(parts, " ") } @@ -893,6 +911,17 @@ func (e *RuntimeStatsWithCommit) String() string { } buf.WriteString("}") } + if e.Commit.Mu.SlowestReqTotalTime > 0 { + buf.WriteString(", slowest_commit_rpc: {total: ") + buf.WriteString(strconv.FormatFloat(e.Commit.Mu.SlowestReqTotalTime.Seconds(), 'f', 3, 64)) + buf.WriteString("s, region_id: ") + buf.WriteString(strconv.FormatUint(e.Commit.Mu.SlowestRegion, 10)) + buf.WriteString(", store: ") + buf.WriteString(e.Commit.Mu.SlowestStoreAddr) + buf.WriteString(", ") + buf.WriteString(e.Commit.Mu.SlowestExecDetails.String()) + buf.WriteString("}") + } e.Commit.Mu.Unlock() if e.Commit.ResolveLock.ResolveLockTime > 0 { buf.WriteString(", resolve_lock: ") @@ -939,17 +968,28 @@ func (e *RuntimeStatsWithCommit) String() string { buf.WriteString(", resolve_lock:") buf.WriteString(FormatDuration(time.Duration(e.LockKeys.ResolveLock.ResolveLockTime))) } + e.LockKeys.Mu.Lock() if e.LockKeys.BackoffTime > 0 { buf.WriteString(", backoff: {time: ") buf.WriteString(FormatDuration(time.Duration(e.LockKeys.BackoffTime))) - e.LockKeys.Mu.Lock() if len(e.LockKeys.Mu.BackoffTypes) > 0 { buf.WriteString(", type: ") buf.WriteString(e.formatBackoff(e.LockKeys.Mu.BackoffTypes)) } - e.LockKeys.Mu.Unlock() buf.WriteString("}") } + if e.LockKeys.Mu.SlowestReqTotalTime > 0 { + buf.WriteString(", slowest_rpc: {total: ") + buf.WriteString(strconv.FormatFloat(e.LockKeys.Mu.SlowestReqTotalTime.Seconds(), 'f', 3, 64)) + buf.WriteString("s, region_id: ") + buf.WriteString(strconv.FormatUint(e.LockKeys.Mu.SlowestRegion, 10)) + buf.WriteString(", store: ") + buf.WriteString(e.LockKeys.Mu.SlowestStoreAddr) + buf.WriteString(", ") + buf.WriteString(e.LockKeys.Mu.SlowestExecDetails.String()) + buf.WriteString("}") + } + e.LockKeys.Mu.Unlock() if e.LockKeys.LockRPCTime > 0 { buf.WriteString(", lock_rpc:") buf.WriteString(time.Duration(e.LockKeys.LockRPCTime).String()) @@ -962,6 +1002,7 @@ func (e *RuntimeStatsWithCommit) String() string { buf.WriteString(", retry_count:") buf.WriteString(strconv.FormatInt(int64(e.LockKeys.RetryCount), 10)) } + buf.WriteString("}") } return buf.String() diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index 8dc53473ee9f2..0495a833a13cd 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -39,14 +39,51 @@ func TestString(t *testing.T) { Mu: struct { sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails }{ CommitBackoffTime: int64(time.Second), BackoffTypes: []string{ "backoff1", "backoff2", }, + SlowestReqTotalTime: time.Second, + SlowestRegion: 1000, + SlowestStoreAddr: "tikv-1:20160", + SlowestExecDetails: util.TiKVExecDetails{ + TimeDetail: &util.TimeDetail{ + TotalRPCWallTime: 500 * time.Millisecond, + }, + ScanDetail: &util.ScanDetail{ + ProcessedKeys: 10, + TotalKeys: 100, + RocksdbDeleteSkippedCount: 1, + RocksdbKeySkippedCount: 1, + RocksdbBlockCacheHitCount: 1, + RocksdbBlockReadCount: 1, + RocksdbBlockReadByte: 100, + RocksdbBlockReadDuration: 20 * time.Millisecond, + }, + WriteDetail: &util.WriteDetail{ + StoreBatchWaitDuration: 10 * time.Microsecond, + ProposeSendWaitDuration: 20 * time.Microsecond, + PersistLogDuration: 30 * time.Microsecond, + RaftDbWriteLeaderWaitDuration: 40 * time.Microsecond, + RaftDbSyncLogDuration: 45 * time.Microsecond, + RaftDbWriteMemtableDuration: 50 * time.Microsecond, + CommitLogDuration: 60 * time.Microsecond, + ApplyBatchWaitDuration: 70 * time.Microsecond, + ApplyLogDuration: 80 * time.Microsecond, + ApplyMutexLockDuration: 90 * time.Microsecond, + ApplyWriteLeaderWaitDuration: 100 * time.Microsecond, + ApplyWriteWalDuration: 101 * time.Microsecond, + ApplyWriteMemtableDuration: 102 * time.Microsecond, + }, + }, }, WriteKeys: 1, WriteSize: 1, @@ -64,6 +101,7 @@ func TestString(t *testing.T) { RocksdbBlockCacheHitCount: 1, RocksdbBlockReadCount: 1, RocksdbBlockReadByte: 100, + RocksdbBlockReadDuration: time.Millisecond, }, TimeDetail: util.TimeDetail{ ProcessTime: 2*time.Second + 5*time.Millisecond, @@ -71,8 +109,14 @@ func TestString(t *testing.T) { }, } expected := "Cop_time: 1.003 Process_time: 2.005 Wait_time: 1 Backoff_time: 1 Request_count: 1 Prewrite_time: 1 Commit_time: 1 " + - "Get_commit_ts_time: 1 Get_latest_ts_time: 1 Commit_backoff_time: 1 Backoff_types: [backoff1 backoff2] Resolve_lock_time: 1 Local_latch_wait_time: 1 Write_keys: 1 Write_size: 1 Prewrite_region: 1 Txn_retry: 1 " + - "Process_keys: 10 Total_keys: 100 Rocksdb_delete_skipped_count: 1 Rocksdb_key_skipped_count: 1 Rocksdb_block_cache_hit_count: 1 Rocksdb_block_read_count: 1 Rocksdb_block_read_byte: 100" + "Get_commit_ts_time: 1 Get_latest_ts_time: 1 Commit_backoff_time: 1 Backoff_types: [backoff1 backoff2] " + + "Slowest_commit_rpc_detail: {total:1.000s, region_id: 1000, store: tikv-1:20160, tikv_wall_time: 500ms, " + + "scan_detail: {total_process_keys: 10, total_keys: 100, rocksdb: {delete_skipped_count: 1, key_skipped_count: 1, " + + "block: {cache_hit_count: 1, read_count: 1, read_byte: 100 Bytes, read_time: 20ms}}}, write_detail: " + + "{store_batch_wait: 10µs, propose_send_wait: 20µs, persist_log: {total: 30µs, write_leader_wait: 40µs, sync_log: 45µs, write_memtable: 50µs}, " + + "commit_log: 60µs, apply_batch_wait: 70µs, apply: {total:80µs, mutex_lock: 90µs, write_leader_wait: 100µs, write_wal: 101µs, write_memtable: 102µs}}} " + + "Resolve_lock_time: 1 Local_latch_wait_time: 1 Write_keys: 1 Write_size: 1 Prewrite_region: 1 Txn_retry: 1 " + + "Process_keys: 10 Total_keys: 100 Rocksdb_delete_skipped_count: 1 Rocksdb_key_skipped_count: 1 Rocksdb_block_cache_hit_count: 1 Rocksdb_block_read_count: 1 Rocksdb_block_read_byte: 100 Rocksdb_block_read_time: 0.001" require.Equal(t, expected, detail.String()) detail = &ExecDetails{} require.Equal(t, "", detail.String()) @@ -133,7 +177,7 @@ func TestCopRuntimeStats(t *testing.T) { cop.scanDetail.RocksdbBlockReadCount = 0 // Print all fields even though the value of some fields is 0. str := "tikv_task:{proc max:1s, min:2ns, avg: 500ms, p80:1s, p95:1s, iters:4, tasks:2}, " + - "scan_detail: {total_process_keys: 0, total_process_keys_size: 0, total_keys: 15, rocksdb: {delete_skipped_count: 5, key_skipped_count: 0, block: {cache_hit_count: 10, read_count: 0, read_byte: 100 Bytes}}}" + "scan_detail: {total_keys: 15, rocksdb: {delete_skipped_count: 5, block: {cache_hit_count: 10, read_byte: 100 Bytes}}}" require.Equal(t, str, cop.String()) zeroScanDetail := util.ScanDetail{} @@ -185,11 +229,48 @@ func TestRuntimeStatsWithCommit(t *testing.T) { CommitTime: time.Second, Mu: struct { sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails }{ - CommitBackoffTime: int64(time.Second), - BackoffTypes: []string{"backoff1", "backoff2", "backoff1"}, + CommitBackoffTime: int64(time.Second), + BackoffTypes: []string{"backoff1", "backoff2", "backoff1"}, + SlowestReqTotalTime: time.Second, + SlowestRegion: 1000, + SlowestStoreAddr: "tikv-1:20160", + SlowestExecDetails: util.TiKVExecDetails{ + TimeDetail: &util.TimeDetail{ + TotalRPCWallTime: 500 * time.Millisecond, + }, + ScanDetail: &util.ScanDetail{ + ProcessedKeys: 10, + TotalKeys: 100, + RocksdbDeleteSkippedCount: 1, + RocksdbKeySkippedCount: 1, + RocksdbBlockCacheHitCount: 1, + RocksdbBlockReadCount: 1, + RocksdbBlockReadByte: 100, + RocksdbBlockReadDuration: 20 * time.Millisecond, + }, + WriteDetail: &util.WriteDetail{ + StoreBatchWaitDuration: 10 * time.Microsecond, + ProposeSendWaitDuration: 20 * time.Microsecond, + PersistLogDuration: 30 * time.Microsecond, + RaftDbWriteLeaderWaitDuration: 40 * time.Microsecond, + RaftDbSyncLogDuration: 45 * time.Microsecond, + RaftDbWriteMemtableDuration: 50 * time.Microsecond, + CommitLogDuration: 60 * time.Microsecond, + ApplyBatchWaitDuration: 70 * time.Microsecond, + ApplyLogDuration: 80 * time.Microsecond, + ApplyMutexLockDuration: 90 * time.Microsecond, + ApplyWriteLeaderWaitDuration: 100 * time.Microsecond, + ApplyWriteWalDuration: 101 * time.Microsecond, + ApplyWriteMemtableDuration: 102 * time.Microsecond, + }, + }, }, WriteKeys: 3, WriteSize: 66, @@ -202,7 +283,13 @@ func TestRuntimeStatsWithCommit(t *testing.T) { stats := &RuntimeStatsWithCommit{ Commit: commitDetail, } - expect := "commit_txn: {prewrite:1s, get_commit_ts:1s, commit:1s, backoff: {time: 1s, type: [backoff1 backoff2]}, resolve_lock: 1s, region_num:5, write_keys:3, write_byte:66, txn_retry:2}" + expect := "commit_txn: {prewrite:1s, get_commit_ts:1s, commit:1s, backoff: {time: 1s, type: [backoff1 backoff2]}, " + + "slowest_commit_rpc: {total: 1.000s, region_id: 1000, store: tikv-1:20160, tikv_wall_time: 500ms, scan_detail: " + + "{total_process_keys: 10, total_keys: 100, rocksdb: {delete_skipped_count: 1, key_skipped_count: 1, block: " + + "{cache_hit_count: 1, read_count: 1, read_byte: 100 Bytes, read_time: 20ms}}}, write_detail: " + + "{store_batch_wait: 10µs, propose_send_wait: 20µs, persist_log: {total: 30µs, write_leader_wait: 40µs, sync_log: 45µs, write_memtable: 50µs}, " + + "commit_log: 60µs, apply_batch_wait: 70µs, apply: {total:80µs, mutex_lock: 90µs, write_leader_wait: 100µs, write_wal: 101µs, write_memtable: 102µs}}}, " + + "resolve_lock: 1s, region_num:5, write_keys:3, write_byte:66, txn_retry:2}" require.Equal(t, expect, stats.String()) lockDetail := &util.LockKeysDetails{ @@ -212,12 +299,50 @@ func TestRuntimeStatsWithCommit(t *testing.T) { BackoffTime: int64(time.Second * 3), Mu: struct { sync.Mutex - BackoffTypes []string - }{BackoffTypes: []string{ - "backoff4", - "backoff5", - "backoff5", - }}, + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails + }{ + BackoffTypes: []string{ + "backoff4", + "backoff5", + "backoff5", + }, + SlowestReqTotalTime: time.Second, + SlowestRegion: 1000, + SlowestStoreAddr: "tikv-1:20160", + SlowestExecDetails: util.TiKVExecDetails{ + TimeDetail: &util.TimeDetail{ + TotalRPCWallTime: 500 * time.Millisecond, + }, + ScanDetail: &util.ScanDetail{ + ProcessedKeys: 10, + TotalKeys: 100, + RocksdbDeleteSkippedCount: 1, + RocksdbKeySkippedCount: 1, + RocksdbBlockCacheHitCount: 1, + RocksdbBlockReadCount: 1, + RocksdbBlockReadByte: 100, + RocksdbBlockReadDuration: 20 * time.Millisecond, + }, + WriteDetail: &util.WriteDetail{ + StoreBatchWaitDuration: 10 * time.Microsecond, + ProposeSendWaitDuration: 20 * time.Microsecond, + PersistLogDuration: 30 * time.Microsecond, + RaftDbWriteLeaderWaitDuration: 40 * time.Microsecond, + RaftDbSyncLogDuration: 45 * time.Microsecond, + RaftDbWriteMemtableDuration: 50 * time.Microsecond, + CommitLogDuration: 60 * time.Microsecond, + ApplyBatchWaitDuration: 70 * time.Microsecond, + ApplyLogDuration: 80 * time.Microsecond, + ApplyMutexLockDuration: 90 * time.Microsecond, + ApplyWriteLeaderWaitDuration: 100 * time.Microsecond, + ApplyWriteWalDuration: 101 * time.Microsecond, + ApplyWriteMemtableDuration: 102 * time.Microsecond, + }, + }}, LockRPCTime: int64(time.Second * 5), LockRPCCount: 50, RetryCount: 2, @@ -228,7 +353,13 @@ func TestRuntimeStatsWithCommit(t *testing.T) { stats = &RuntimeStatsWithCommit{ LockKeys: lockDetail, } - expect = "lock_keys: {time:1s, region:2, keys:10, resolve_lock:2s, backoff: {time: 3s, type: [backoff4 backoff5]}, lock_rpc:5s, rpc_count:50, retry_count:2}" + expect = "lock_keys: {time:1s, region:2, keys:10, resolve_lock:2s, backoff: {time: 3s, type: [backoff4 backoff5]}, " + + "slowest_rpc: {total: 1.000s, region_id: 1000, store: tikv-1:20160, tikv_wall_time: 500ms, scan_detail: " + + "{total_process_keys: 10, total_keys: 100, rocksdb: {delete_skipped_count: 1, key_skipped_count: 1, block: " + + "{cache_hit_count: 1, read_count: 1, read_byte: 100 Bytes, read_time: 20ms}}}, write_detail: " + + "{store_batch_wait: 10µs, propose_send_wait: 20µs, persist_log: {total: 30µs, write_leader_wait: 40µs, sync_log: 45µs, write_memtable: 50µs}, " + + "commit_log: 60µs, apply_batch_wait: 70µs, apply: {total:80µs, mutex_lock: 90µs, write_leader_wait: 100µs, write_wal: 101µs, write_memtable: 102µs}}}, " + + "lock_rpc:5s, rpc_count:50, retry_count:2}" require.Equal(t, expect, stats.String()) } diff --git a/util/memory/tracker.go b/util/memory/tracker.go index 233f201efebe3..f266a71495790 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -402,6 +402,8 @@ func (t *Tracker) BytesConsumed() int64 { } // MaxConsumed returns max number of bytes consumed during execution. +// Note: Don't make this method return -1 for special meanings in the future. Because binary plan has used -1 to +// distinguish between "0 bytes" and "N/A". ref: binaryOpFromFlatOp() func (t *Tracker) MaxConsumed() int64 { return t.maxConsumed.Load() } diff --git a/util/plancodec/binary_plan_decode.go b/util/plancodec/binary_plan_decode.go new file mode 100644 index 0000000000000..74f5c097bc94d --- /dev/null +++ b/util/plancodec/binary_plan_decode.go @@ -0,0 +1,300 @@ +// 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 plancodec + +import ( + "strconv" + "strings" + + "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/texttree" + "github.com/pingcap/tipb/go-tipb" +) + +// DecodeBinaryPlan decode the binary plan and display it similar to EXPLAIN ANALYZE statement. +func DecodeBinaryPlan(binaryPlan string) (string, error) { + protoBytes, err := decompress(binaryPlan) + if err != nil { + return "", err + } + pb := &tipb.ExplainData{} + err = pb.Unmarshal(protoBytes) + if err != nil { + return "", err + } + if pb.DiscardedDueToTooLong { + return planDiscardedDecoded, nil + } + // 1. decode the protobuf into strings + rows := decodeBinaryOperator(pb.Main, "", true, pb.WithRuntimeStats, nil) + for _, cte := range pb.Ctes { + rows = decodeBinaryOperator(cte, "", true, pb.WithRuntimeStats, rows) + } + if len(rows) == 0 { + return "", nil + } + + // 2. calculate the max length of each column and the total length + // Because the text tree part of the "id" column contains characters that consist of multiple bytes, we need the + // lengths calculated in bytes and runes both. Length in bytes is for preallocating memory. Length in runes is + // for padding space to align the content. + runeMaxLens, byteMaxLens := calculateMaxFieldLens(rows, pb.WithRuntimeStats) + singleRowLen := 0 + for _, fieldLen := range byteMaxLens { + singleRowLen += fieldLen + // every field begins with "| " and ends with " " + singleRowLen += 3 + } + // every row ends with " |\n" + singleRowLen += 3 + // length for a row * (row count + 1(for title row)) + totalBytes := singleRowLen * (len(rows) + 1) + // there is a "\n" at the beginning + totalBytes++ + + // 3. format the strings and get the final result + var b strings.Builder + b.Grow(totalBytes) + var titleFields []string + if pb.WithRuntimeStats { + titleFields = fullTitleFields + } else { + titleFields = noRuntimeStatsTitleFields + } + b.WriteString("\n") + for i, str := range titleFields { + b.WriteString("| ") + b.WriteString(str) + if len([]rune(str)) < runeMaxLens[i] { + // append spaces to align the content + b.WriteString(strings.Repeat(" ", runeMaxLens[i]-len([]rune(str)))) + } + b.WriteString(" ") + if i == len(titleFields)-1 { + b.WriteString(" |\n") + } + } + for _, row := range rows { + for i, str := range row { + b.WriteString("| ") + b.WriteString(str) + if len([]rune(str)) < runeMaxLens[i] { + // append spaces to align the content + b.WriteString(strings.Repeat(" ", runeMaxLens[i]-len([]rune(str)))) + } + b.WriteString(" ") + if i == len(titleFields)-1 { + b.WriteString(" |\n") + } + } + } + return b.String(), nil +} + +var ( + noRuntimeStatsTitleFields = []string{"id", "estRows", "estCost", "task", "access object", "operator info"} + fullTitleFields = []string{"id", "estRows", "estCost", "actRows", "task", "access object", "execution info", "operator info", "memory", "disk"} +) + +func calculateMaxFieldLens(rows [][]string, hasRuntimeStats bool) (runeLens, byteLens []int) { + runeLens = make([]int, len(rows[0])) + byteLens = make([]int, len(rows[0])) + for _, row := range rows { + for i, field := range row { + if runeLens[i] < len([]rune(field)) { + runeLens[i] = len([]rune(field)) + } + if byteLens[i] < len(field) { + byteLens[i] = len(field) + } + } + } + var titleFields []string + if hasRuntimeStats { + titleFields = fullTitleFields + } else { + titleFields = noRuntimeStatsTitleFields + } + for i := range byteLens { + if runeLens[i] < len([]rune(titleFields[i])) { + runeLens[i] = len([]rune(titleFields[i])) + } + if byteLens[i] < len(titleFields[i]) { + byteLens[i] = len(titleFields[i]) + } + } + return +} + +func decodeBinaryOperator(op *tipb.ExplainOperator, indent string, isLastChild, hasRuntimeStats bool, out [][]string) [][]string { + row := make([]string, 0, 10) + + // 1. extract the information and turn them into strings for display + explainID := texttree.PrettyIdentifier(op.Name+printDriverSide(op.Labels), indent, isLastChild) + estRows := strconv.FormatFloat(op.EstRows, 'f', 2, 64) + cost := strconv.FormatFloat(op.Cost, 'f', 2, 64) + var actRows, execInfo, memInfo, diskInfo string + if hasRuntimeStats { + actRows = strconv.FormatInt(int64(op.ActRows), 10) + execInfo = op.RootBasicExecInfo + groupExecInfo := strings.Join(op.RootGroupExecInfo, ",") + if len(groupExecInfo) > 0 { + if len(execInfo) > 0 { + execInfo += ", " + } + execInfo += groupExecInfo + } + if len(op.CopExecInfo) > 0 { + if len(execInfo) > 0 { + execInfo += ", " + } + execInfo += op.CopExecInfo + } + if op.MemoryBytes < 0 { + memInfo = "N/A" + } else { + memInfo = memory.FormatBytes(op.MemoryBytes) + } + if op.DiskBytes < 0 { + diskInfo = "N/A" + } else { + diskInfo = memory.FormatBytes(op.DiskBytes) + } + } + task := op.TaskType.String() + if op.TaskType != tipb.TaskType_unknown && op.TaskType != tipb.TaskType_root { + task = task + "[" + op.StoreType.String() + "]" + } + accessObject := printAccessObject(op.AccessObjects) + + // 2. append the strings to the slice + row = append(row, explainID, estRows, cost) + if hasRuntimeStats { + row = append(row, actRows) + } + row = append(row, task, accessObject) + if hasRuntimeStats { + row = append(row, execInfo) + } + row = append(row, op.OperatorInfo) + if hasRuntimeStats { + row = append(row, memInfo, diskInfo) + } + out = append(out, row) + + // 3. recursively process the children + children := make([]*tipb.ExplainOperator, len(op.Children)) + copy(children, op.Children) + if len(children) == 2 && + len(children[0].Labels) >= 1 && + children[0].Labels[0] == tipb.OperatorLabel_probeSide && + len(children[1].Labels) >= 1 && + children[1].Labels[0] == tipb.OperatorLabel_buildSide { + children[0], children[1] = children[1], children[0] + } + childIndent := texttree.Indent4Child(indent, isLastChild) + for i, child := range children { + out = decodeBinaryOperator(child, childIndent, i == len(children)-1, hasRuntimeStats, out) + } + return out +} + +func printDriverSide(labels []tipb.OperatorLabel) string { + strs := make([]string, 0, len(labels)) + for _, label := range labels { + switch label { + case tipb.OperatorLabel_empty: + strs = append(strs, "") + case tipb.OperatorLabel_buildSide: + strs = append(strs, "(Build)") + case tipb.OperatorLabel_probeSide: + strs = append(strs, "(Probe)") + case tipb.OperatorLabel_seedPart: + strs = append(strs, "(Seed Part)") + case tipb.OperatorLabel_recursivePart: + strs = append(strs, "(Recursive Part)") + } + } + return strings.Join(strs, "") +} + +func printDynamicPartitionObject(ao *tipb.DynamicPartitionAccessObject) string { + if ao == nil { + return "" + } + if ao.AllPartitions { + return "partition:all" + } else if len(ao.Partitions) == 0 { + return "partition:dual" + } + return "partition:" + strings.Join(ao.Partitions, ",") +} + +func printAccessObject(pbAccessObjs []*tipb.AccessObject) string { + strs := make([]string, 0, len(pbAccessObjs)) + for _, pbAccessObj := range pbAccessObjs { + switch ao := pbAccessObj.AccessObject.(type) { + case *tipb.AccessObject_DynamicPartitionObjects: + if ao == nil || ao.DynamicPartitionObjects == nil { + return "" + } + aos := ao.DynamicPartitionObjects.Objects + if len(aos) == 0 { + return "" + } + // If it only involves one table, just print the partitions. + if len(aos) == 1 { + return printDynamicPartitionObject(aos[0]) + } + var b strings.Builder + // If it involves multiple tables, we also need to print the table name. + for i, access := range aos { + if access == nil { + continue + } + if i != 0 { + b.WriteString(", ") + } + b.WriteString(printDynamicPartitionObject(access)) + b.WriteString(" of " + access.Table) + } + strs = append(strs, b.String()) + case *tipb.AccessObject_ScanObject: + if ao == nil || ao.ScanObject == nil { + return "" + } + scanAO := ao.ScanObject + var b strings.Builder + if len(scanAO.Table) > 0 { + b.WriteString("table:" + scanAO.Table) + } + if len(scanAO.Partitions) > 0 { + b.WriteString(", partition:" + strings.Join(scanAO.Partitions, ",")) + } + for _, index := range scanAO.Indexes { + if index.IsClusteredIndex { + b.WriteString(", clustered index:") + } else { + b.WriteString(", index:") + } + b.WriteString(index.Name + "(" + strings.Join(index.Cols, ", ") + ")") + } + strs = append(strs, b.String()) + case *tipb.AccessObject_OtherObject: + strs = append(strs, ao.OtherObject) + } + } + return strings.Join(strs, "") +} diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 041de2989241f..264ea838bde75 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -185,8 +185,12 @@ func TestAddStatement(t *testing.T) { LocalLatchTime: 50, Mu: struct { sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails }{ CommitBackoffTime: 1000, BackoffTypes: []string{boTxnLockName}, @@ -317,8 +321,12 @@ func TestAddStatement(t *testing.T) { LocalLatchTime: 5, Mu: struct { sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails }{ CommitBackoffTime: 100, BackoffTypes: []string{boTxnLockName}, @@ -603,8 +611,12 @@ func generateAnyExecInfo() *StmtExecInfo { LocalLatchTime: 10, Mu: struct { sync.Mutex - CommitBackoffTime int64 - BackoffTypes []string + CommitBackoffTime int64 + BackoffTypes []string + SlowestReqTotalTime time.Duration + SlowestRegion uint64 + SlowestStoreAddr string + SlowestExecDetails util.TiKVExecDetails }{ CommitBackoffTime: 200, BackoffTypes: []string{boTxnLockName},