From e3f8d9805e9c7ce99cd9c1802d6b374ec9cc8dd1 Mon Sep 17 00:00:00 2001 From: David <8039876+AmoebaProtozoa@users.noreply.github.com> Date: Wed, 11 Jan 2023 17:36:33 +0800 Subject: [PATCH 1/8] lightning: fix codecPDClient (#40496) Signed-off-by: David <8039876+AmoebaProtozoa@users.noreply.github.com> Signed-off-by: David <8039876+AmoebaProtozoa@users.noreply.github.com> Co-authored-by: Ti Chi Robot Co-authored-by: Weizhen Wang --- br/pkg/lightning/backend/local/local.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index cc88fd6a89483..d1df848bef107 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -458,7 +458,7 @@ func NewLocalBackend( return backend.MakeBackend(nil), common.ErrCreateKVClient.Wrap(err).GenWithStackByArgs() } rpcCli := tikvclient.NewRPCClient(tikvclient.WithSecurity(tls.ToTiKVSecurityConfig())) - pdCliForTiKV := &tikvclient.CodecPDClient{Client: pdCtl.GetPDClient()} + pdCliForTiKV := tikvclient.NewCodecPDClient(tikvclient.ModeTxn, pdCtl.GetPDClient()) tikvCli, err := tikvclient.NewKVStore("lightning-local-backend", pdCliForTiKV, spkv, rpcCli) if err != nil { return backend.MakeBackend(nil), common.ErrCreateKVClient.Wrap(err).GenWithStackByArgs() From 2cf328bf65d0f7117783e8d31cdcf583aca95991 Mon Sep 17 00:00:00 2001 From: wjHuang Date: Wed, 11 Jan 2023 18:06:33 +0800 Subject: [PATCH 2/8] ddl: let concurrent truncate on the same table depend on the previous one (#40501) * done Signed-off-by: wjhuang2016 * refine test Signed-off-by: wjhuang2016 Signed-off-by: wjhuang2016 Co-authored-by: Ti Chi Robot --- ddl/db_test.go | 49 ++++++++++++++++++++++++++++++++ ddl/job_table.go | 2 ++ ddl/metadatalocktest/mdl_test.go | 2 -- 3 files changed, 51 insertions(+), 2 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 3380af7e0a2c5..629316af251a4 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1577,3 +1577,52 @@ func TestSetInvalidDefaultValueAfterModifyColumn(t *testing.T) { wg.Wait() require.EqualError(t, checkErr, "[ddl:1101]BLOB/TEXT/JSON column 'a' can't have a default value") } + +func TestMDLTruncateTable(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + tk := testkit.NewTestKit(t, store) + tk2 := testkit.NewTestKit(t, store) + tk3 := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int);") + tk.MustExec("begin") + tk.MustExec("select * from t for update") + + var wg sync.WaitGroup + + hook := &ddl.TestDDLCallback{Do: dom} + wg.Add(2) + var timetk2 time.Time + var timetk3 time.Time + + one := false + f := func(job *model.Job) { + if !one { + one = true + } else { + return + } + go func() { + tk3.MustExec("truncate table test.t") + timetk3 = time.Now() + wg.Done() + }() + } + + hook.OnJobUpdatedExported.Store(&f) + dom.DDL().SetHook(hook) + + go func() { + tk2.MustExec("truncate table test.t") + timetk2 = time.Now() + wg.Done() + }() + + time.Sleep(2 * time.Second) + timeMain := time.Now() + tk.MustExec("commit") + wg.Wait() + require.True(t, timetk2.After(timeMain)) + require.True(t, timetk3.After(timeMain)) +} diff --git a/ddl/job_table.go b/ddl/job_table.go index 740bb5c0b7da1..16dd6fa45f1e3 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -370,6 +370,8 @@ func job2UniqueIDs(job *model.Job, schema bool) string { } slices.Sort(s) return strings.Join(s, ",") + case model.ActionTruncateTable: + return strconv.FormatInt(job.TableID, 10) + "," + strconv.FormatInt(job.Args[0].(int64), 10) } if schema { return strconv.FormatInt(job.SchemaID, 10) diff --git a/ddl/metadatalocktest/mdl_test.go b/ddl/metadatalocktest/mdl_test.go index fd307968cad73..d7c05fa334508 100644 --- a/ddl/metadatalocktest/mdl_test.go +++ b/ddl/metadatalocktest/mdl_test.go @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -//go:build !featuretag - package metadatalocktest import ( From 2f13578ec5be41787cc2a5d96d98cfd923773b67 Mon Sep 17 00:00:00 2001 From: Zhou Kunqin <25057648+time-and-fate@users.noreply.github.com> Date: Wed, 11 Jan 2023 18:30:34 +0800 Subject: [PATCH 3/8] statistics: fix estimation error when ranges are too many and modify count is large (#40472) * fix * fmt * bazel update * update test result Co-authored-by: Weizhen Wang Co-authored-by: Ti Chi Robot --- statistics/BUILD.bazel | 1 + statistics/index.go | 10 ++-- statistics/selectivity_test.go | 84 +++++++++++++++++++++++++++++++++- 3 files changed, 90 insertions(+), 5 deletions(-) diff --git a/statistics/BUILD.bazel b/statistics/BUILD.bazel index 6a1b3d5a54921..e6992020197c3 100644 --- a/statistics/BUILD.bazel +++ b/statistics/BUILD.bazel @@ -112,6 +112,7 @@ go_test( "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_log//:log", "@com_github_stretchr_testify//require", + "@org_golang_x_exp//slices", "@org_uber_go_goleak//:goleak", "@org_uber_go_zap//:zap", ], diff --git a/statistics/index.go b/statistics/index.go index 78246942ffb99..d201aa8fdd14f 100644 --- a/statistics/index.go +++ b/statistics/index.go @@ -222,6 +222,7 @@ func (idx *Index) GetRowCount(sctx sessionctx.Context, coll *HistColl, indexRang totalCount := float64(0) isSingleCol := len(idx.Info.Columns) == 1 for _, indexRange := range indexRanges { + var count float64 lb, err := codec.EncodeKey(sc, nil, indexRange.LowVal...) if err != nil { return 0, err @@ -242,7 +243,7 @@ func (idx *Index) GetRowCount(sctx sessionctx.Context, coll *HistColl, indexRang totalCount++ continue } - count := idx.equalRowCount(lb, realtimeRowCount) + count = idx.equalRowCount(lb, realtimeRowCount) // If the current table row count has changed, we should scale the row count accordingly. count *= idx.GetIncreaseFactor(realtimeRowCount) totalCount += count @@ -262,7 +263,7 @@ func (idx *Index) GetRowCount(sctx sessionctx.Context, coll *HistColl, indexRang r := types.NewBytesDatum(rb) lowIsNull := bytes.Equal(lb, nullKeyBytes) if isSingleCol && lowIsNull { - totalCount += float64(idx.Histogram.NullCount) + count += float64(idx.Histogram.NullCount) } expBackoffSuccess := false // Due to the limitation of calcFraction and convertDatumToScalar, the histogram actually won't estimate anything. @@ -301,16 +302,17 @@ func (idx *Index) GetRowCount(sctx sessionctx.Context, coll *HistColl, indexRang } } if !expBackoffSuccess { - totalCount += idx.BetweenRowCount(l, r) + count += idx.BetweenRowCount(l, r) } // If the current table row count has changed, we should scale the row count accordingly. - totalCount *= idx.GetIncreaseFactor(realtimeRowCount) + count *= idx.GetIncreaseFactor(realtimeRowCount) // handling the out-of-range part if (idx.outOfRange(l) && !(isSingleCol && lowIsNull)) || idx.outOfRange(r) { totalCount += idx.Histogram.outOfRangeRowCount(&l, &r, modifyCount) } + totalCount += count } totalCount = mathutil.Clamp(totalCount, 0, float64(realtimeRowCount)) return totalCount, nil diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 08ac16612dd61..05a7413fa3d09 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/ranger" "github.com/stretchr/testify/require" + "golang.org/x/exp/slices" ) func TestCollationColumnEstimate(t *testing.T) { @@ -891,7 +892,7 @@ func prepareSelectivity(testKit *testkit.TestKit, dom *domain.Domain) (*statisti return statsTbl, nil } -func getRange(start, end int64) []*ranger.Range { +func getRange(start, end int64) ranger.Ranges { ran := &ranger.Range{ LowVal: []types.Datum{types.NewIntDatum(start)}, HighVal: []types.Datum{types.NewIntDatum(end)}, @@ -900,6 +901,21 @@ func getRange(start, end int64) []*ranger.Range { return []*ranger.Range{ran} } +func getRanges(start, end []int64) (res ranger.Ranges) { + if len(start) != len(end) { + return nil + } + for i := range start { + ran := &ranger.Range{ + LowVal: []types.Datum{types.NewIntDatum(start[i])}, + HighVal: []types.Datum{types.NewIntDatum(end[i])}, + Collators: collate.GetBinaryCollatorSlice(1), + } + res = append(res, ran) + } + return +} + func TestSelectivityGreedyAlgo(t *testing.T) { nodes := make([]*statistics.StatsNode, 3) nodes[0] = statistics.MockStatsNode(1, 3, 2) @@ -1075,3 +1091,69 @@ func TestGlobalStatsOutOfRangeEstimationAfterDelete(t *testing.T) { testKit.MustQuery(input[i]).Check(testkit.Rows(output[i].Result...)) } } + +func generateMapsForMockStatsTbl(statsTbl *statistics.Table) { + idx2Columns := make(map[int64][]int64) + colID2IdxIDs := make(map[int64][]int64) + for _, idxHist := range statsTbl.Indices { + ids := make([]int64, 0, len(idxHist.Info.Columns)) + for _, idxCol := range idxHist.Info.Columns { + ids = append(ids, int64(idxCol.Offset)) + } + colID2IdxIDs[ids[0]] = append(colID2IdxIDs[ids[0]], idxHist.ID) + idx2Columns[idxHist.ID] = ids + } + for _, idxIDs := range colID2IdxIDs { + slices.Sort(idxIDs) + } + statsTbl.Idx2ColumnIDs = idx2Columns + statsTbl.ColID2IdxIDs = colID2IdxIDs +} + +func TestIssue39593(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + testKit := testkit.NewTestKit(t, store) + + testKit.MustExec("use test") + testKit.MustExec("drop table if exists t") + testKit.MustExec("create table t(a int, b int, index idx(a, b))") + is := dom.InfoSchema() + tb, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + tblInfo := tb.Meta() + + // mock the statistics.Table + statsTbl := mockStatsTable(tblInfo, 540) + colValues, err := generateIntDatum(1, 54) + require.NoError(t, err) + for i := 1; i <= 2; i++ { + statsTbl.Columns[int64(i)] = &statistics.Column{ + Histogram: *mockStatsHistogram(int64(i), colValues, 10, types.NewFieldType(mysql.TypeLonglong)), + Info: tblInfo.Columns[i-1], + StatsLoadedStatus: statistics.NewStatsFullLoadStatus(), + StatsVer: 2, + } + } + idxValues, err := generateIntDatum(2, 3) + require.NoError(t, err) + tp := types.NewFieldType(mysql.TypeBlob) + statsTbl.Indices[1] = &statistics.Index{ + Histogram: *mockStatsHistogram(1, idxValues, 60, tp), + Info: tblInfo.Indices[0], + StatsVer: 2, + } + generateMapsForMockStatsTbl(statsTbl) + + sctx := testKit.Session() + idxID := tblInfo.Indices[0].ID + vals := []int64{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20} + count, err := statsTbl.GetRowCountByIndexRanges(sctx, idxID, getRanges(vals, vals)) + require.NoError(t, err) + // estimated row count without any changes + require.Equal(t, float64(360), count) + statsTbl.Count *= 10 + count, err = statsTbl.GetRowCountByIndexRanges(sctx, idxID, getRanges(vals, vals)) + require.NoError(t, err) + // estimated row count after mock modify on the table + require.Equal(t, float64(3600), count) +} From cefb26146b016a34739918ddd344972debb19ff9 Mon Sep 17 00:00:00 2001 From: crazycs Date: Wed, 11 Jan 2023 19:08:33 +0800 Subject: [PATCH 4/8] executor: fix issue of KEY_COLUMN_USAGE table return wrong result about foreign key column (#40491) Signed-off-by: crazycs520 Signed-off-by: crazycs520 Co-authored-by: Weizhen Wang Co-authored-by: Ti Chi Robot --- executor/fktest/foreign_key_test.go | 12 ++++++++++++ executor/infoschema_reader.go | 8 ++++---- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/executor/fktest/foreign_key_test.go b/executor/fktest/foreign_key_test.go index 1dc92d6954a2e..670c273b4cb1c 100644 --- a/executor/fktest/foreign_key_test.go +++ b/executor/fktest/foreign_key_test.go @@ -2733,3 +2733,15 @@ func TestForeignKeyAndMemoryTracker(t *testing.T) { tk.MustExec("update t1 set id=id+100000 where id=1") tk.MustQuery("select id,pid from t1 where id<3 or pid is null order by id").Check(testkit.Rows("2 1", "100001 ")) } + +func TestForeignKeyMetaInKeyColumnUsage(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@foreign_key_checks=1") + tk.MustExec("use test") + tk.MustExec("create table t1 (a int, b int, index(a, b));") + tk.MustExec("create table t2 (a int, b int, index(a, b), constraint fk foreign key(a, b) references t1(a, b));") + tk.MustQuery("select CONSTRAINT_NAME, TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME, REFERENCED_TABLE_SCHEMA, REFERENCED_TABLE_NAME, REFERENCED_COLUMN_NAME from " + + "INFORMATION_SCHEMA.KEY_COLUMN_USAGE where CONSTRAINT_SCHEMA='test' and TABLE_NAME='t2' and REFERENCED_TABLE_SCHEMA is not null and REFERENCED_COLUMN_NAME is not null;"). + Check(testkit.Rows("fk test t2 a test t1 a", "fk test t2 b test t1 b")) +} diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 472220bb2dcc6..601495b27aff5 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -1677,11 +1677,11 @@ func keyColumnUsageInTable(schema *model.DBInfo, table *model.TableInfo) [][]typ } } for _, fk := range table.ForeignKeys { - fkRefCol := "" - if len(fk.RefCols) > 0 { - fkRefCol = fk.RefCols[0].O - } for i, key := range fk.Cols { + fkRefCol := "" + if len(fk.RefCols) > i { + fkRefCol = fk.RefCols[i].O + } col := nameToCol[key.L] record := types.MakeDatums( infoschema.CatalogVal, // CONSTRAINT_CATALOG From 7beec042148d46679173f4521465a3b31272f4fd Mon Sep 17 00:00:00 2001 From: Song Gao Date: Wed, 11 Jan 2023 19:44:33 +0800 Subject: [PATCH 5/8] sysvar: add switch for plan replayer capture using historical stats (#40492) * add switch * fix * fix * fix Co-authored-by: Weizhen Wang Co-authored-by: Ti Chi Robot --- domain/plan_replayer.go | 2 +- domain/plan_replayer_dump.go | 8 ++++++-- executor/adapter.go | 6 ++++++ executor/analyzetest/analyze_test.go | 11 ++++++----- executor/plan_replayer.go | 2 +- server/plan_replayer.go | 2 +- sessionctx/variable/sysvar.go | 12 ++++++++++-- sessionctx/variable/tidb_vars.go | 4 ++++ util/replayer/replayer.go | 8 ++++---- 9 files changed, 39 insertions(+), 16 deletions(-) diff --git a/domain/plan_replayer.go b/domain/plan_replayer.go index 2bbb15772d56c..7d52f282ba56e 100644 --- a/domain/plan_replayer.go +++ b/domain/plan_replayer.go @@ -419,7 +419,7 @@ func (w *planReplayerTaskDumpWorker) HandleTask(task *PlanReplayerDumpTask) (suc return true } - file, fileName, err := replayer.GeneratePlanReplayerFile(task.IsCapture) + file, fileName, err := replayer.GeneratePlanReplayerFile(task.IsCapture, task.IsContinuesCapture, variable.EnableHistoricalStatsForCapture.Load()) if err != nil { logutil.BgLogger().Warn("[plan-replayer-capture] generate task file failed", zap.String("sqlDigest", taskKey.SQLDigest), diff --git a/domain/plan_replayer_dump.go b/domain/plan_replayer_dump.go index a0bb07581a6d7..01ab473e16a90 100644 --- a/domain/plan_replayer_dump.go +++ b/domain/plan_replayer_dump.go @@ -71,6 +71,8 @@ const ( PlanReplayerTaskMetaSQLDigest = "sqlDigest" // PlanReplayerTaskMetaPlanDigest indicates the plan digest of this task PlanReplayerTaskMetaPlanDigest = "planDigest" + // PlanReplayerTaskEnableHistoricalStats indicates whether the task is using historical stats + PlanReplayerTaskEnableHistoricalStats = "enableHistoricalStats" ) type tableNamePair struct { @@ -278,8 +280,9 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, return err } - // For capture task, we don't dump stats - if !task.IsCapture { + // For capture task, we dump stats in storage only if EnableHistoricalStatsForCapture is disabled. + // For manual plan replayer dump command, we directly dump stats in storage + if !variable.EnableHistoricalStatsForCapture.Load() || !task.IsCapture { // Dump stats if err = dumpStats(zw, pairs, do); err != nil { return err @@ -350,6 +353,7 @@ func dumpSQLMeta(zw *zip.Writer, task *PlanReplayerDumpTask) error { varMap[PlanReplayerTaskMetaIsContinues] = strconv.FormatBool(task.IsContinuesCapture) varMap[PlanReplayerTaskMetaSQLDigest] = task.SQLDigest varMap[PlanReplayerTaskMetaPlanDigest] = task.PlanDigest + varMap[PlanReplayerTaskEnableHistoricalStats] = strconv.FormatBool(variable.EnableHistoricalStatsForCapture.Load()) if err := toml.NewEncoder(cf).Encode(varMap); err != nil { return errors.AddStack(err) } diff --git a/executor/adapter.go b/executor/adapter.go index 444c358f96c97..09dc49f58d54f 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -2016,8 +2016,14 @@ func checkPlanReplayerCaptureTask(sctx sessionctx.Context, stmtNode ast.StmtNode return } tasks := handle.GetTasks() + if len(tasks) == 0 { + return + } _, sqlDigest := sctx.GetSessionVars().StmtCtx.SQLDigest() _, planDigest := sctx.GetSessionVars().StmtCtx.GetPlanDigest() + if sqlDigest == nil || planDigest == nil { + return + } key := replayer.PlanReplayerTaskKey{ SQLDigest: sqlDigest.String(), PlanDigest: planDigest.String(), diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index 2d520703e07d5..843200fea6cf9 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -2830,16 +2830,17 @@ PARTITION BY RANGE ( a ) ( "Warning 1105 Ignore columns and options when analyze partition in dynamic mode", "Warning 8244 Build global-level stats failed due to missing partition-level column stats: table `t` partition `p0` column `d`, please run analyze table to refresh columns of all partitions", )) - tk.MustQuery("select * from t where a > 1 and b > 1 and c > 1 and d > 1") - require.NoError(t, h.LoadNeededHistograms()) - tbl := h.GetTableStats(tableInfo) - require.Equal(t, 0, len(tbl.Columns)) + // flaky test, fix it later + //tk.MustQuery("select * from t where a > 1 and b > 1 and c > 1 and d > 1") + //require.NoError(t, h.LoadNeededHistograms()) + //tbl := h.GetTableStats(tableInfo) + //require.Equal(t, 0, len(tbl.Columns)) // ignore both p0's 3 buckets, persisted-partition-options' 1 bucket, just use table-level 2 buckets tk.MustExec("analyze table t partition p0") tk.MustQuery("select * from t where a > 1 and b > 1 and c > 1 and d > 1") require.NoError(t, h.LoadNeededHistograms()) - tbl = h.GetTableStats(tableInfo) + tbl := h.GetTableStats(tableInfo) require.Equal(t, 2, len(tbl.Columns[tableInfo.Columns[2].ID].Buckets)) } diff --git a/executor/plan_replayer.go b/executor/plan_replayer.go index ff102e20820b2..868b969e78247 100644 --- a/executor/plan_replayer.go +++ b/executor/plan_replayer.go @@ -130,7 +130,7 @@ func (e *PlanReplayerExec) registerCaptureTask(ctx context.Context) error { func (e *PlanReplayerExec) createFile() error { var err error - e.DumpInfo.File, e.DumpInfo.FileName, err = replayer.GeneratePlanReplayerFile(false) + e.DumpInfo.File, e.DumpInfo.FileName, err = replayer.GeneratePlanReplayerFile(false, false, false) if err != nil { return err } diff --git a/server/plan_replayer.go b/server/plan_replayer.go index 64629c6ee0070..30f7c4ae821c1 100644 --- a/server/plan_replayer.go +++ b/server/plan_replayer.go @@ -220,7 +220,7 @@ func isExists(path string) (bool, error) { } func handlePlanReplayerCaptureFile(content []byte, path string, handler downloadFileHandler) ([]byte, error) { - if !strings.Contains(handler.filePath, "capture_replayer") { + if !strings.HasPrefix(handler.filePath, "capture_replayer") { return content, nil } b := bytes.NewReader(content) diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 46a338a588762..726043595651d 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1164,7 +1164,15 @@ var defaultSysVars = []*SysVar{ PasswordReuseInterval.Store(TidbOptInt64(val, DefPasswordReuseTime)) return nil }}, - + {Scope: ScopeGlobal, Name: TiDBEnableHistoricalStatsForCapture, Value: BoolToOnOff(DefTiDBEnableHistoricalStatsForCapture), Type: TypeBool, + SetGlobal: func(ctx context.Context, vars *SessionVars, s string) error { + EnableHistoricalStatsForCapture.Store(TiDBOptOn(s)) + return nil + }, + GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { + return BoolToOnOff(EnableHistoricalStatsForCapture.Load()), nil + }, + }, {Scope: ScopeGlobal, Name: TiDBHistoricalStatsDuration, Value: DefTiDBHistoricalStatsDuration.String(), Type: TypeDuration, MinValue: int64(time.Minute * 10), MaxValue: uint64(time.Hour * 24 * 365), GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { return HistoricalStatsDuration.Load().String(), nil @@ -1187,7 +1195,7 @@ var defaultSysVars = []*SysVar{ return BoolToOnOff(vars.EnablePlanReplayedContinuesCapture), nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePlanReplayerCapture, Value: BoolToOnOff(false), Type: TypeBool, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnablePlanReplayerCapture, Value: BoolToOnOff(true), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnablePlanReplayerCapture = TiDBOptOn(val) return nil diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 810d5116ab5f7..04bb8739813ae 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -892,6 +892,8 @@ const ( PasswordReuseTime = "password_reuse_interval" // TiDBHistoricalStatsDuration indicates the duration to remain tidb historical stats TiDBHistoricalStatsDuration = "tidb_historical_stats_duration" + // TiDBEnableHistoricalStatsForCapture indicates whether use historical stats in plan replayer capture + TiDBEnableHistoricalStatsForCapture = "tidb_enable_historical_stats_for_capture" ) // TiDB intentional limits @@ -1146,6 +1148,7 @@ const ( DefPasswordReuseTime = 0 DefTiDBStoreBatchSize = 0 DefTiDBHistoricalStatsDuration = 7 * 24 * time.Hour + DefTiDBEnableHistoricalStatsForCapture = false DefTiDBTTLJobScheduleWindowStartTime = "00:00 +0000" DefTiDBTTLJobScheduleWindowEndTime = "23:59 +0000" DefTiDBTTLScanWorkerCount = 4 @@ -1225,6 +1228,7 @@ var ( IsSandBoxModeEnabled = atomic.NewBool(false) MaxPreparedStmtCountValue = atomic.NewInt64(DefMaxPreparedStmtCount) HistoricalStatsDuration = atomic.NewDuration(DefTiDBHistoricalStatsDuration) + EnableHistoricalStatsForCapture = atomic.NewBool(DefTiDBEnableHistoricalStatsForCapture) ) var ( diff --git a/util/replayer/replayer.go b/util/replayer/replayer.go index 39287ada70194..de7439bd724f2 100644 --- a/util/replayer/replayer.go +++ b/util/replayer/replayer.go @@ -33,13 +33,13 @@ type PlanReplayerTaskKey struct { } // GeneratePlanReplayerFile generates plan replayer file -func GeneratePlanReplayerFile(isCapture bool) (*os.File, string, error) { +func GeneratePlanReplayerFile(isCapture, isContinuesCapture, enableHistoricalStatsForCapture bool) (*os.File, string, error) { path := GetPlanReplayerDirName() err := os.MkdirAll(path, os.ModePerm) if err != nil { return nil, "", errors.AddStack(err) } - fileName, err := generatePlanReplayerFileName(isCapture) + fileName, err := generatePlanReplayerFileName(isCapture, isContinuesCapture, enableHistoricalStatsForCapture) if err != nil { return nil, "", errors.AddStack(err) } @@ -50,7 +50,7 @@ func GeneratePlanReplayerFile(isCapture bool) (*os.File, string, error) { return zf, fileName, err } -func generatePlanReplayerFileName(isCapture bool) (string, error) { +func generatePlanReplayerFileName(isCapture, isContinuesCapture, enableHistoricalStatsForCapture bool) (string, error) { // Generate key and create zip file time := time.Now().UnixNano() b := make([]byte, 16) @@ -60,7 +60,7 @@ func generatePlanReplayerFileName(isCapture bool) (string, error) { return "", err } key := base64.URLEncoding.EncodeToString(b) - if isCapture { + if isContinuesCapture || isCapture && enableHistoricalStatsForCapture { return fmt.Sprintf("capture_replayer_%v_%v.zip", key, time), nil } return fmt.Sprintf("replayer_%v_%v.zip", key, time), nil From defd80a6f5494ae9724224b062ef3937c576e6a1 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 11 Jan 2023 20:20:33 +0800 Subject: [PATCH 6/8] *: upgrade go1.19.5 (#40483) Signed-off-by: Weizhen Wang Signed-off-by: Weizhen Wang Co-authored-by: Ti Chi Robot --- WORKSPACE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/WORKSPACE b/WORKSPACE index 559746eab6b1d..627c7dd5c5575 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -35,7 +35,7 @@ go_download_sdk( "https://mirrors.aliyun.com/golang/{}", "https://dl.google.com/go/{}", ], - version = "1.19.3", + version = "1.19.5", ) go_register_toolchains( From f88714d1af06f3adf80a593f314eb80cb8025a31 Mon Sep 17 00:00:00 2001 From: Zak Zhao <57036248+joccau@users.noreply.github.com> Date: Wed, 11 Jan 2023 21:00:33 +0800 Subject: [PATCH 7/8] br: need init MDL variable when GetDomain (#40512) Signed-off-by: joccau Signed-off-by: joccau Co-authored-by: Ti Chi Robot --- br/pkg/gluetidb/glue.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/br/pkg/gluetidb/glue.go b/br/pkg/gluetidb/glue.go index c9756fe07ea89..45c8d84862351 100644 --- a/br/pkg/gluetidb/glue.go +++ b/br/pkg/gluetidb/glue.go @@ -136,6 +136,10 @@ func (g Glue) UseOneShotSession(store kv.Storage, closeDomain bool, fn func(glue if err != nil { return errors.Trace(err) } + if err = session.InitMDLVariable(store); err != nil { + return errors.Trace(err) + } + // because domain was created during the whole program exists. // and it will register br info to info syncer. // we'd better close it as soon as possible. From eb35c773b512e4e00c42caf7f04ea7397d00c127 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 11 Jan 2023 13:18:33 +0000 Subject: [PATCH 8/8] ddl: avoid commit conflicts when updating/delete from mysql.tidb_ddl_reorg. (#38738) * Added test case * ddl fix #38669. The issue was that mysql.tidb_ddl_reorg table was updated by an inner transaction after the outer transaction started, which then made a commit conflict in the outer transaction, when it deleted the same row. * Fixed typo in comment * Added test case for #24427 * Disabled tests for CI testing * Revert "Disabled tests for CI testing" This reverts commit 17c28f30ba8802c578fed5653107aa6995e17607. * Revert "Revert "Disabled tests for CI testing"" This reverts commit 65c84d94f7ab4440c739703312f5329619c6ccdc. * removed test skips * Clean up the tidb_ddl_reorg entry after DDL is completed * Use a cleanup job afterwards instead. * Fixed test * Moved cleanup before asyncNotify * More detailed test failure log * Refined test error message * Injecting timoeut to get stack traces from CI * Updated Debug Dump on timeout * Delete mulitple entries in tidb_ddl_reorg if needed * Linting * Linting * Added CI debug logs * Linting + CI debugs * fixed CI debug * Try to cleanup also if job.State == synced * check for non-error of runErr instead of error... * Use a new session, instead of reusing worker.sess * Also handle case when job == nil * Removed CI debug logs * Misssed change session from w.sess to newly created sess * Improved TestConcurrentDDLSwitch and added CI debug logs * Always cleaning up all orphan mysql.tidb_ddl_reorg entries * linting * Also cleanup if job is nil * Updated TestModifyColumnReorgInfo + CI debug logs * more CI debug * refactored the cleanupDDLReorgHandle code * Added missing cleanup in handleDDLJobQueue * Removed debug panic * Code cleanup * Test updates * Debug cleanup * Cleaned up test after removal of old non-concurrent DDL code merge * Linting * always wrap changes to tidb_ddl_reorg in an own transaction + fixed some typos * Minimum fix * Always update reorg meta, not only on error * Issue is here :) * Fixed newReorgHandler * Wrapped more tidb_ddl_reorg changes into separate transactions * linting * Removed updateDDLReorgStartHandle * cleanups * Made runInTxn a method on *session, instead of normal function * Update test * Final touches * Removed duplicate test * CleanupDDLReorgHandles should only be called from HandleJobDone. * Variable rename * Renamed 'delete' variabel name * Updated test * small revert * Removed timeout debugging code * Simplified the cleanup to only start a new txn and not a new session * Reverted the change of GetDDLInfo Co-authored-by: Ti Chi Robot --- ddl/backfilling.go | 24 +++++----- ddl/column.go | 17 +++++-- ddl/column_type_change_test.go | 15 ++++++ ddl/db_partition_test.go | 19 ++++++++ ddl/db_test.go | 5 +- ddl/ddl.go | 15 +++++- ddl/ddl_worker.go | 1 + ddl/index.go | 30 ++++++------ ddl/job_table.go | 86 +++++++++++++++++----------------- ddl/modify_column_test.go | 13 +++-- ddl/partition.go | 7 ++- ddl/reorg.go | 66 ++++++++++++-------------- parser/model/ddl.go | 3 ++ 13 files changed, 179 insertions(+), 122 deletions(-) diff --git a/ddl/backfilling.go b/ddl/backfilling.go index a7c23a545208e..aae3a9b75790e 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -146,7 +146,7 @@ func GetLeaseGoTime(currTime time.Time, lease time.Duration) types.Time { // Backfilling is time consuming, to accelerate this process, TiDB has built some sub // workers to do this in the DDL owner node. // -// DDL owner thread +// DDL owner thread (also see comments before runReorgJob func) // ^ // | (reorgCtx.doneCh) // | @@ -583,9 +583,10 @@ func (dc *ddlCtx) sendTasksAndWait(scheduler *backfillScheduler, totalAddedCount err = dc.isReorgRunnable(reorgInfo.Job.ID) } + // Update the reorg handle that has been processed. + err1 := reorgInfo.UpdateReorgMeta(nextKey, scheduler.sessPool) + if err != nil { - // Update the reorg handle that has been processed. - err1 := reorgInfo.UpdateReorgMeta(nextKey, scheduler.sessPool) metrics.BatchAddIdxHistogram.WithLabelValues(metrics.LblError).Observe(elapsedTime.Seconds()) logutil.BgLogger().Warn("[ddl] backfill worker handle batch tasks failed", @@ -614,7 +615,8 @@ func (dc *ddlCtx) sendTasksAndWait(scheduler *backfillScheduler, totalAddedCount zap.String("start key", hex.EncodeToString(startKey)), zap.String("next key", hex.EncodeToString(nextKey)), zap.Int64("batch added count", taskAddedCount), - zap.String("take time", elapsedTime.String())) + zap.String("take time", elapsedTime.String()), + zap.NamedError("updateHandleError", err1)) return nil } @@ -1320,15 +1322,15 @@ func GetMaxBackfillJob(sess *session, jobID, currEleID int64, currEleKey []byte) } // MoveBackfillJobsToHistoryTable moves backfill table jobs to the backfill history table. -func MoveBackfillJobsToHistoryTable(sessCtx sessionctx.Context, bfJob *BackfillJob) error { - sess, ok := sessCtx.(*session) +func MoveBackfillJobsToHistoryTable(sctx sessionctx.Context, bfJob *BackfillJob) error { + s, ok := sctx.(*session) if !ok { - return errors.Errorf("sess ctx:%#v convert session failed", sessCtx) + return errors.Errorf("sess ctx:%#v convert session failed", sctx) } - return runInTxn(sess, func(se *session) error { + return s.runInTxn(func(se *session) error { // TODO: Consider batch by batch update backfill jobs and insert backfill history jobs. - bJobs, err := GetBackfillJobs(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s'", + bJobs, err := GetBackfillJobs(se, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s'", bfJob.JobID, bfJob.EleID, bfJob.EleKey), "update_backfill_job") if err != nil { return errors.Trace(err) @@ -1342,13 +1344,13 @@ func MoveBackfillJobsToHistoryTable(sessCtx sessionctx.Context, bfJob *BackfillJ return errors.Trace(err) } startTS := txn.StartTS() - err = RemoveBackfillJob(sess, true, bJobs[0]) + err = RemoveBackfillJob(se, true, bJobs[0]) if err == nil { for _, bj := range bJobs { bj.State = model.JobStateCancelled bj.FinishTS = startTS } - err = AddBackfillHistoryJob(sess, bJobs) + err = AddBackfillHistoryJob(se, bJobs) } logutil.BgLogger().Info("[ddl] move backfill jobs to history table", zap.Int("job count", len(bJobs))) return errors.Trace(err) diff --git a/ddl/column.go b/ddl/column.go index 25ce1f81b9557..9893d6528038b 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -806,7 +806,13 @@ func doReorgWorkForModifyColumnMultiSchema(w *worker, d *ddlCtx, t *meta.Meta, j func doReorgWorkForModifyColumn(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, oldCol, changingCol *model.ColumnInfo, changingIdxs []*model.IndexInfo) (done bool, ver int64, err error) { job.ReorgMeta.ReorgTp = model.ReorgTypeTxn - rh := newReorgHandler(t, w.sess) + sctx, err1 := w.sessPool.get() + if err1 != nil { + err = errors.Trace(err1) + return + } + defer w.sessPool.put(sctx) + rh := newReorgHandler(newSession(sctx)) dbInfo, err := t.GetDatabase(job.SchemaID) if err != nil { return false, ver, errors.Trace(err) @@ -1291,8 +1297,8 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra if err != nil { return w.reformatErrors(err) } - if w.sessCtx.GetSessionVars().StmtCtx.GetWarnings() != nil && len(w.sessCtx.GetSessionVars().StmtCtx.GetWarnings()) != 0 { - warn := w.sessCtx.GetSessionVars().StmtCtx.GetWarnings() + warn := w.sessCtx.GetSessionVars().StmtCtx.GetWarnings() + if len(warn) != 0 { //nolint:forcetypeassert recordWarning = errors.Cause(w.reformatErrors(warn[0].Err)).(*terror.Error) } @@ -1376,8 +1382,9 @@ func (w *updateColumnWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t taskCtx.nextKey = nextKey taskCtx.done = taskDone - warningsMap := make(map[errors.ErrorID]*terror.Error, len(rowRecords)) - warningsCountMap := make(map[errors.ErrorID]int64, len(rowRecords)) + // Optimize for few warnings! + warningsMap := make(map[errors.ErrorID]*terror.Error, 2) + warningsCountMap := make(map[errors.ErrorID]int64, 2) for _, rowRecord := range rowRecords { taskCtx.scanCount++ diff --git a/ddl/column_type_change_test.go b/ddl/column_type_change_test.go index 4f79ce7782368..308a815773ce9 100644 --- a/ddl/column_type_change_test.go +++ b/ddl/column_type_change_test.go @@ -2421,3 +2421,18 @@ func TestColumnTypeChangeTimestampToInt(t *testing.T) { tk.MustExec("alter table t add index idx1(id, c1);") tk.MustExec("admin check table t") } + +func TestFixDDLTxnWillConflictWithReorgTxn(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("create table t (a int)") + tk.MustExec("set global tidb_ddl_enable_fast_reorg = OFF") + tk.MustExec("alter table t add index(a)") + tk.MustExec("set @@sql_mode=''") + tk.MustExec("insert into t values(128),(129)") + tk.MustExec("alter table t modify column a tinyint") + + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 2 warnings with this error code, first warning: constant 128 overflows tinyint")) +} diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 6be12283c920a..c61eeaf885aa6 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -4528,6 +4528,25 @@ func TestPartitionTableWithAnsiQuotes(t *testing.T) { ` PARTITION "pMax" VALUES LESS THAN (MAXVALUE,MAXVALUE))`)) } +func TestAlterModifyPartitionColTruncateWarning(t *testing.T) { + t.Skip("waiting for supporting Modify Partition Column again") + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "truncWarn" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk.MustExec(`set sql_mode = default`) + tk.MustExec(`create table t (a varchar(255)) partition by range columns (a) (partition p1 values less than ("0"), partition p2 values less than ("zzzz"))`) + tk.MustExec(`insert into t values ("123456"),(" 654321")`) + tk.MustContainErrMsg(`alter table t modify a varchar(5)`, "[types:1265]Data truncated for column 'a', value is '") + tk.MustExec(`set sql_mode = ''`) + tk.MustExec(`alter table t modify a varchar(5)`) + // Fix the duplicate warning, see https://github.com/pingcap/tidb/issues/38699 + tk.MustQuery(`show warnings`).Check(testkit.Rows(""+ + "Warning 1265 Data truncated for column 'a', value is ' 654321'", + "Warning 1265 Data truncated for column 'a', value is ' 654321'")) +} + func TestAlterModifyColumnOnPartitionedTableRename(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/ddl/db_test.go b/ddl/db_test.go index 629316af251a4..46cfe301ec4f4 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -618,10 +618,7 @@ func TestAddExpressionIndexRollback(t *testing.T) { // Check whether the reorg information is cleaned up. err := sessiontxn.NewTxn(context.Background(), ctx) require.NoError(t, err) - txn, err := ctx.Txn(true) - require.NoError(t, err) - m := meta.NewMeta(txn) - element, start, end, physicalID, err := ddl.NewReorgHandlerForTest(m, testkit.NewTestKit(t, store).Session()).GetDDLReorgHandle(currJob) + element, start, end, physicalID, err := ddl.NewReorgHandlerForTest(testkit.NewTestKit(t, store).Session()).GetDDLReorgHandle(currJob) require.True(t, meta.ErrDDLReorgElementNotExist.Equal(err)) require.Nil(t, element) require.Nil(t, start) diff --git a/ddl/ddl.go b/ddl/ddl.go index b89c8264fd125..9fa2a9f99cc10 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -1343,7 +1343,7 @@ func GetDDLInfo(s sessionctx.Context) (*Info, error) { return info, nil } - _, info.ReorgHandle, _, _, err = newReorgHandler(t, sess).GetDDLReorgHandle(reorgJob) + _, info.ReorgHandle, _, _, err = newReorgHandler(sess).GetDDLReorgHandle(reorgJob) if err != nil { if meta.ErrDDLReorgElementNotExist.Equal(err) { return info, nil @@ -1584,6 +1584,19 @@ func (s *session) session() sessionctx.Context { return s.Context } +func (s *session) runInTxn(f func(*session) error) (err error) { + err = s.begin() + if err != nil { + return err + } + err = f(s) + if err != nil { + s.rollback() + return + } + return errors.Trace(s.commit()) +} + // GetAllHistoryDDLJobs get all the done DDL jobs. func GetAllHistoryDDLJobs(m *meta.Meta) ([]*model.Job, error) { iterator, err := GetLastHistoryDDLJobsIterator(m) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index cc75cb43e50d7..5c700f6273a3b 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -699,6 +699,7 @@ func (w *worker) HandleJobDone(d *ddlCtx, job *model.Job, t *meta.Meta) error { if err != nil { return err } + CleanupDDLReorgHandles(job, w.sess) asyncNotify(d.ddlJobDoneCh) return nil } diff --git a/ddl/index.go b/ddl/index.go index 512c856faa8ef..ae42ad84aba2e 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -882,7 +882,13 @@ func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Jo func runReorgJobAndHandleErr(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table, indexInfo *model.IndexInfo, mergingTmpIdx bool) (done bool, ver int64, err error) { elements := []*meta.Element{{ID: indexInfo.ID, TypeKey: meta.IndexElementKey}} - rh := newReorgHandler(t, w.sess) + sctx, err1 := w.sessPool.get() + if err1 != nil { + err = err1 + return + } + defer w.sessPool.put(sctx) + rh := newReorgHandler(newSession(sctx)) dbInfo, err := t.GetDatabase(job.SchemaID) if err != nil { return false, ver, errors.Trace(err) @@ -1274,13 +1280,10 @@ func (w *baseIndexWorker) String() string { } func (w *baseIndexWorker) UpdateTask(bfJob *BackfillJob) error { - sess, ok := w.backfillCtx.sessCtx.(*session) - if !ok { - return errors.Errorf("sess ctx:%#v convert session failed", w.backfillCtx.sessCtx) - } + s := newSession(w.backfillCtx.sessCtx) - return runInTxn(sess, func(se *session) error { - jobs, err := GetBackfillJobs(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' and id = %d", + return s.runInTxn(func(se *session) error { + jobs, err := GetBackfillJobs(se, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' and id = %d", bfJob.JobID, bfJob.EleID, bfJob.EleKey, bfJob.ID), "update_backfill_task") if err != nil { return err @@ -1297,26 +1300,23 @@ func (w *baseIndexWorker) UpdateTask(bfJob *BackfillJob) error { return err } bfJob.InstanceLease = GetLeaseGoTime(currTime, InstanceLease) - return updateBackfillJob(sess, BackfillTable, bfJob, "update_backfill_task") + return updateBackfillJob(se, BackfillTable, bfJob, "update_backfill_task") }) } func (w *baseIndexWorker) FinishTask(bfJob *BackfillJob) error { - sess, ok := w.backfillCtx.sessCtx.(*session) - if !ok { - return errors.Errorf("sess ctx:%#v convert session failed", w.backfillCtx.sessCtx) - } - return runInTxn(sess, func(se *session) error { + s := newSession(w.backfillCtx.sessCtx) + return s.runInTxn(func(se *session) error { txn, err := se.txn() if err != nil { return errors.Trace(err) } bfJob.FinishTS = txn.StartTS() - err = RemoveBackfillJob(sess, false, bfJob) + err = RemoveBackfillJob(se, false, bfJob) if err != nil { return err } - return AddBackfillHistoryJob(sess, []*BackfillJob{bfJob}) + return AddBackfillHistoryJob(se, []*BackfillJob{bfJob}) }) } diff --git a/ddl/job_table.go b/ddl/job_table.go index 16dd6fa45f1e3..782abcc8b5765 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -432,15 +432,8 @@ func getDDLReorgHandle(sess *session, job *model.Job) (element *meta.Element, st return } -// updateDDLReorgStartHandle update the startKey of the handle. -func updateDDLReorgStartHandle(sess *session, job *model.Job, element *meta.Element, startKey kv.Key) error { - sql := fmt.Sprintf("update mysql.tidb_ddl_reorg set ele_id = %d, ele_type = %s, start_key = %s where job_id = %d", - element.ID, wrapKey2String(element.TypeKey), wrapKey2String(startKey), job.ID) - _, err := sess.execute(context.Background(), sql, "update_start_handle") - return err -} - // updateDDLReorgHandle update startKey, endKey physicalTableID and element of the handle. +// Caller should wrap this in a separate transaction, to avoid conflicts. func updateDDLReorgHandle(sess *session, jobID int64, startKey kv.Key, endKey kv.Key, physicalTableID int64, element *meta.Element) error { sql := fmt.Sprintf("update mysql.tidb_ddl_reorg set ele_id = %d, ele_type = %s, start_key = %s, end_key = %s, physical_id = %d where job_id = %d", element.ID, wrapKey2String(element.TypeKey), wrapKey2String(startKey), wrapKey2String(endKey), physicalTableID, jobID) @@ -449,28 +442,48 @@ func updateDDLReorgHandle(sess *session, jobID int64, startKey kv.Key, endKey kv } // initDDLReorgHandle initializes the handle for ddl reorg. -func initDDLReorgHandle(sess *session, jobID int64, startKey kv.Key, endKey kv.Key, physicalTableID int64, element *meta.Element) error { - sql := fmt.Sprintf("insert into mysql.tidb_ddl_reorg(job_id, ele_id, ele_type, start_key, end_key, physical_id) values (%d, %d, %s, %s, %s, %d)", +func initDDLReorgHandle(s *session, jobID int64, startKey kv.Key, endKey kv.Key, physicalTableID int64, element *meta.Element) error { + del := fmt.Sprintf("delete from mysql.tidb_ddl_reorg where job_id = %d", jobID) + ins := fmt.Sprintf("insert into mysql.tidb_ddl_reorg(job_id, ele_id, ele_type, start_key, end_key, physical_id) values (%d, %d, %s, %s, %s, %d)", jobID, element.ID, wrapKey2String(element.TypeKey), wrapKey2String(startKey), wrapKey2String(endKey), physicalTableID) - _, err := sess.execute(context.Background(), sql, "update_handle") - return err + return s.runInTxn(func(se *session) error { + _, err := se.execute(context.Background(), del, "init_handle") + if err != nil { + logutil.BgLogger().Info("initDDLReorgHandle failed to delete", zap.Int64("jobID", jobID), zap.Error(err)) + } + _, err = se.execute(context.Background(), ins, "init_handle") + return err + }) } // deleteDDLReorgHandle deletes the handle for ddl reorg. -func removeDDLReorgHandle(sess *session, job *model.Job, elements []*meta.Element) error { +func removeDDLReorgHandle(s *session, job *model.Job, elements []*meta.Element) error { if len(elements) == 0 { return nil } sql := fmt.Sprintf("delete from mysql.tidb_ddl_reorg where job_id = %d", job.ID) - _, err := sess.execute(context.Background(), sql, "remove_handle") - return err + return s.runInTxn(func(se *session) error { + _, err := se.execute(context.Background(), sql, "remove_handle") + return err + }) } // removeReorgElement removes the element from ddl reorg, it is the same with removeDDLReorgHandle, only used in failpoint -func removeReorgElement(sess *session, job *model.Job) error { +func removeReorgElement(s *session, job *model.Job) error { sql := fmt.Sprintf("delete from mysql.tidb_ddl_reorg where job_id = %d", job.ID) - _, err := sess.execute(context.Background(), sql, "remove_handle") - return err + return s.runInTxn(func(se *session) error { + _, err := se.execute(context.Background(), sql, "remove_handle") + return err + }) +} + +// cleanDDLReorgHandles removes handles that are no longer needed. +func cleanDDLReorgHandles(s *session, job *model.Job) error { + sql := "delete from mysql.tidb_ddl_reorg where job_id = " + strconv.FormatInt(job.ID, 10) + return s.runInTxn(func(se *session) error { + _, err := se.execute(context.Background(), sql, "clean_handle") + return err + }) } func wrapKey2String(key []byte) string { @@ -532,10 +545,10 @@ func AddBackfillHistoryJob(sess *session, backfillJobs []*BackfillJob) error { } // AddBackfillJobs adds the backfill jobs to the tidb_ddl_backfill table. -func AddBackfillJobs(sess *session, backfillJobs []*BackfillJob) error { +func AddBackfillJobs(s *session, backfillJobs []*BackfillJob) error { label := fmt.Sprintf("add_%s_job", BackfillTable) // Do runInTxn to get StartTS. - return runInTxn(newSession(sess), func(se *session) error { + return s.runInTxn(func(se *session) error { txn, err := se.txn() if err != nil { return errors.Trace(err) @@ -549,26 +562,13 @@ func AddBackfillJobs(sess *session, backfillJobs []*BackfillJob) error { if err != nil { return err } - _, err = sess.execute(context.Background(), sql, label) + _, err = se.execute(context.Background(), sql, label) return errors.Trace(err) }) } -func runInTxn(se *session, f func(*session) error) (err error) { - err = se.begin() - if err != nil { - return err - } - err = f(se) - if err != nil { - se.rollback() - return - } - return errors.Trace(se.commit()) -} - // GetBackfillJobsForOneEle batch gets the backfill jobs in the tblName table that contains only one element. -func GetBackfillJobsForOneEle(sess *session, batch int, excludedJobIDs []int64, lease time.Duration) ([]*BackfillJob, error) { +func GetBackfillJobsForOneEle(s *session, batch int, excludedJobIDs []int64, lease time.Duration) ([]*BackfillJob, error) { eJobIDsBuilder := strings.Builder{} for i, id := range excludedJobIDs { if i == 0 { @@ -584,14 +584,13 @@ func GetBackfillJobsForOneEle(sess *session, batch int, excludedJobIDs []int64, var err error var bJobs []*BackfillJob - s := newSession(sess) - err = runInTxn(s, func(se *session) error { - currTime, err := GetOracleTimeWithStartTS(s) + err = s.runInTxn(func(se *session) error { + currTime, err := GetOracleTimeWithStartTS(se) if err != nil { return err } - bJobs, err = GetBackfillJobs(sess, BackfillTable, + bJobs, err = GetBackfillJobs(se, BackfillTable, fmt.Sprintf("(exec_ID = '' or exec_lease < '%v') %s order by ddl_job_id, ele_key, ele_id limit %d", currTime.Add(-lease), eJobIDsBuilder.String(), batch), "get_backfill_job") return err @@ -614,17 +613,16 @@ func GetBackfillJobsForOneEle(sess *session, batch int, excludedJobIDs []int64, // GetAndMarkBackfillJobsForOneEle batch gets the backfill jobs in the tblName table that contains only one element, // and update these jobs with instance ID and lease. -func GetAndMarkBackfillJobsForOneEle(sess *session, batch int, jobID int64, uuid string, lease time.Duration) ([]*BackfillJob, error) { +func GetAndMarkBackfillJobsForOneEle(s *session, batch int, jobID int64, uuid string, lease time.Duration) ([]*BackfillJob, error) { var validLen int var bJobs []*BackfillJob - s := newSession(sess) - err := runInTxn(s, func(se *session) error { + err := s.runInTxn(func(se *session) error { currTime, err := GetOracleTimeWithStartTS(se) if err != nil { return err } - bJobs, err = GetBackfillJobs(sess, BackfillTable, + bJobs, err = GetBackfillJobs(se, BackfillTable, fmt.Sprintf("(exec_ID = '' or exec_lease < '%v') and ddl_job_id = %d order by ddl_job_id, ele_key, ele_id limit %d", currTime.Add(-lease), jobID, batch), "get_mark_backfill_job") if err != nil { @@ -645,7 +643,7 @@ func GetAndMarkBackfillJobsForOneEle(sess *session, batch int, jobID int64, uuid bJobs[i].InstanceID = uuid bJobs[i].InstanceLease = GetLeaseGoTime(currTime, lease) // TODO: batch update - if err = updateBackfillJob(sess, BackfillTable, bJobs[i], "get_mark_backfill_job"); err != nil { + if err = updateBackfillJob(se, BackfillTable, bJobs[i], "get_mark_backfill_job"); err != nil { return err } } diff --git a/ddl/modify_column_test.go b/ddl/modify_column_test.go index bd9c574970f71..6eb8e633be007 100644 --- a/ddl/modify_column_test.go +++ b/ddl/modify_column_test.go @@ -17,6 +17,7 @@ package ddl_test import ( "context" "fmt" + "strconv" "sync" "testing" "time" @@ -117,14 +118,18 @@ func TestModifyColumnReorgInfo(t *testing.T) { require.NoError(t, checkErr) // Check whether the reorg information is cleaned up when executing "modify column" failed. checkReorgHandle := func(gotElements, expectedElements []*meta.Element) { + require.Equal(t, len(expectedElements), len(gotElements)) for i, e := range gotElements { require.Equal(t, expectedElements[i], e) } + // check the consistency of the tables. + currJobID := strconv.FormatInt(currJob.ID, 10) + tk.MustQuery("select job_id, reorg, schema_ids, table_ids, type, processing from mysql.tidb_ddl_job where job_id = " + currJobID).Check(testkit.Rows()) + tk.MustQuery("select job_id from mysql.tidb_ddl_history where job_id = " + currJobID).Check(testkit.Rows(currJobID)) + tk.MustQuery("select job_id, ele_id, ele_type, physical_id from mysql.tidb_ddl_reorg where job_id = " + currJobID).Check(testkit.Rows()) require.NoError(t, sessiontxn.NewTxn(context.Background(), ctx)) - txn, err := ctx.Txn(true) - require.NoError(t, err) - m := meta.NewMeta(txn) - e, start, end, physicalID, err := ddl.NewReorgHandlerForTest(m, testkit.NewTestKit(t, store).Session()).GetDDLReorgHandle(currJob) + e, start, end, physicalID, err := ddl.NewReorgHandlerForTest(testkit.NewTestKit(t, store).Session()).GetDDLReorgHandle(currJob) + require.Error(t, err, "Error not ErrDDLReorgElementNotExists, found orphan row in tidb_ddl_reorg for job.ID %d: e: '%s', physicalID: %d, start: 0x%x end: 0x%x", currJob.ID, e, physicalID, start, end) require.True(t, meta.ErrDDLReorgElementNotExist.Equal(err)) require.Nil(t, e) require.Nil(t, start) diff --git a/ddl/partition.go b/ddl/partition.go index 5b67c82c5bf8b..1a3cab2e3eb01 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -1756,7 +1756,12 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( elements = append(elements, &meta.Element{ID: idxInfo.ID, TypeKey: meta.IndexElementKey}) } } - rh := newReorgHandler(t, w.sess) + sctx, err1 := w.sessPool.get() + if err1 != nil { + return ver, err1 + } + defer w.sessPool.put(sctx) + rh := newReorgHandler(newSession(sctx)) reorgInfo, err := getReorgInfoFromPartitions(d.jobContext(job.ID), d, rh, job, dbInfo, tbl, physicalTableIDs, elements) if err != nil || reorgInfo.first { diff --git a/ddl/reorg.go b/ddl/reorg.go index 7912560499344..e760e43c11221 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -141,11 +141,9 @@ func (rc *reorgCtx) increaseRowCount(count int64) { atomic.AddInt64(&rc.rowCount, count) } -func (rc *reorgCtx) getRowCountAndKey() (int64, kv.Key, *meta.Element) { +func (rc *reorgCtx) getRowCount() int64 { row := atomic.LoadInt64(&rc.rowCount) - h, _ := (rc.doneKey.Load()).(nullableKey) - element, _ := (rc.element.Load()).(*meta.Element) - return row, h.key, element + return row } // runReorgJob is used as a portal to do the reorganization work. @@ -232,7 +230,7 @@ func (w *worker) runReorgJob(rh *reorgHandler, reorgInfo *reorgInfo, tblInfo *mo d.removeReorgCtx(job) return dbterror.ErrCancelledDDLJob } - rowCount, _, _ := rc.getRowCountAndKey() + rowCount := rc.getRowCount() if err != nil { logutil.BgLogger().Warn("[ddl] run reorg job done", zap.Int64("handled rows", rowCount), zap.Error(err)) } else { @@ -252,17 +250,13 @@ func (w *worker) runReorgJob(rh *reorgHandler, reorgInfo *reorgInfo, tblInfo *mo } updateBackfillProgress(w, reorgInfo, tblInfo, 0) - if err1 := rh.RemoveDDLReorgHandle(job, reorgInfo.elements); err1 != nil { - logutil.BgLogger().Warn("[ddl] run reorg job done, removeDDLReorgHandle failed", zap.Error(err1)) - return errors.Trace(err1) - } case <-w.ctx.Done(): logutil.BgLogger().Info("[ddl] run reorg job quit") d.removeReorgCtx(job) // We return dbterror.ErrWaitReorgTimeout here too, so that outer loop will break. return dbterror.ErrWaitReorgTimeout case <-time.After(waitTimeout): - rowCount, doneKey, currentElement := rc.getRowCountAndKey() + rowCount := rc.getRowCount() job.SetRowCount(rowCount) updateBackfillProgress(w, reorgInfo, tblInfo, rowCount) @@ -271,17 +265,9 @@ func (w *worker) runReorgJob(rh *reorgHandler, reorgInfo *reorgInfo, tblInfo *mo rc.resetWarnings() - // Update a reorgInfo's handle. - // Since daemon-worker is triggered by timer to store the info half-way. - // you should keep these infos is read-only (like job) / atomic (like doneKey & element) / concurrent safe. - err := updateDDLReorgStartHandle(rh.s, job, currentElement, doneKey) logutil.BgLogger().Info("[ddl] run reorg job wait timeout", zap.Duration("wait time", waitTimeout), - zap.ByteString("element type", currentElement.TypeKey), - zap.Int64("element ID", currentElement.ID), - zap.Int64("total added row count", rowCount), - zap.String("done key", hex.EncodeToString(doneKey)), - zap.Error(err)) + zap.Int64("total added row count", rowCount)) // If timeout, we will return, check the owner and retry to wait job done again. return dbterror.ErrWaitReorgTimeout } @@ -640,10 +626,6 @@ func getReorgInfo(ctx *JobContext, d *ddlCtx, rh *reorgHandler, job *model.Job, failpoint.Inject("errorUpdateReorgHandle", func() (*reorgInfo, error) { return &info, errors.New("occur an error when update reorg handle") }) - err = rh.RemoveDDLReorgHandle(job, elements) - if err != nil { - return &info, errors.Trace(err) - } err = rh.InitDDLReorgHandle(job, start, end, pid, elements[0]) if err != nil { return &info, errors.Trace(err) @@ -750,27 +732,24 @@ func getReorgInfoFromPartitions(ctx *JobContext, d *ddlCtx, rh *reorgHandler, jo return &info, nil } +// UpdateReorgMeta creates a new transaction and updates tidb_ddl_reorg table, +// so the reorg can restart in case of issues. func (r *reorgInfo) UpdateReorgMeta(startKey kv.Key, pool *sessionPool) (err error) { if startKey == nil && r.EndKey == nil { return nil } - se, err := pool.get() + sctx, err := pool.get() if err != nil { return } - defer pool.put(se) + defer pool.put(sctx) - sess := newSession(se) + sess := newSession(sctx) err = sess.begin() if err != nil { return } - txn, err := sess.txn() - if err != nil { - sess.rollback() - return err - } - rh := newReorgHandler(meta.NewMeta(txn), sess) + rh := newReorgHandler(sess) err = updateDDLReorgHandle(rh.s, r.Job.ID, startKey, r.EndKey, r.PhysicalTableID, r.currElement) err1 := sess.commit() if err == nil { @@ -781,17 +760,16 @@ func (r *reorgInfo) UpdateReorgMeta(startKey kv.Key, pool *sessionPool) (err err // reorgHandler is used to handle the reorg information duration reorganization DDL job. type reorgHandler struct { - m *meta.Meta s *session } // NewReorgHandlerForTest creates a new reorgHandler, only used in test. -func NewReorgHandlerForTest(t *meta.Meta, sess sessionctx.Context) *reorgHandler { - return newReorgHandler(t, newSession(sess)) +func NewReorgHandlerForTest(sess sessionctx.Context) *reorgHandler { + return newReorgHandler(newSession(sess)) } -func newReorgHandler(t *meta.Meta, sess *session) *reorgHandler { - return &reorgHandler{m: t, s: sess} +func newReorgHandler(sess *session) *reorgHandler { + return &reorgHandler{s: sess} } // InitDDLReorgHandle initializes the job reorganization information. @@ -809,6 +787,20 @@ func (r *reorgHandler) RemoveDDLReorgHandle(job *model.Job, elements []*meta.Ele return removeDDLReorgHandle(r.s, job, elements) } +// CleanupDDLReorgHandles removes the job reorganization related handles. +func CleanupDDLReorgHandles(job *model.Job, s *session) { + if job != nil && !job.IsFinished() && !job.IsSynced() { + // Job is given, but it is neither finished nor synced; do nothing + return + } + + err := cleanDDLReorgHandles(s, job) + if err != nil { + // ignore error, cleanup is not that critical + logutil.BgLogger().Warn("Failed removing the DDL reorg entry in tidb_ddl_reorg", zap.String("job", job.String()), zap.Error(err)) + } +} + // GetDDLReorgHandle gets the latest processed DDL reorganize position. func (r *reorgHandler) GetDDLReorgHandle(job *model.Job) (element *meta.Element, startKey, endKey kv.Key, physicalTableID int64, err error) { return getDDLReorgHandle(r.s, job) diff --git a/parser/model/ddl.go b/parser/model/ddl.go index d14733d4df317..8eb26ca238d3f 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -668,6 +668,9 @@ func (job *Job) String() string { rowCount := job.GetRowCount() ret := fmt.Sprintf("ID:%d, Type:%s, State:%s, SchemaState:%s, SchemaID:%d, TableID:%d, RowCount:%d, ArgLen:%d, start time: %v, Err:%v, ErrCount:%d, SnapshotVersion:%v", job.ID, job.Type, job.State, job.SchemaState, job.SchemaID, job.TableID, rowCount, len(job.Args), TSConvert2Time(job.StartTS), job.Error, job.ErrorCount, job.SnapshotVer) + if job.ReorgMeta != nil { + ret += fmt.Sprintf(", UniqueWarnings:%d", len(job.ReorgMeta.Warnings)) + } if job.Type != ActionMultiSchemaChange && job.MultiSchemaInfo != nil { ret += fmt.Sprintf(", Multi-Schema Change:true, Revertible:%v", job.MultiSchemaInfo.Revertible) }