diff --git a/.github/dependabot.yml b/.github/dependabot.yml new file mode 100644 index 0000000000000..bde24b721461e --- /dev/null +++ b/.github/dependabot.yml @@ -0,0 +1,14 @@ +version: 2 +updates: + - package-ecosystem: "gomod" + directory: "/" + schedule: + interval: "weekly" + day: "friday" + time: "18:00" + timezone: "Asia/Shanghai" + allow: + - dependency-name: "golang.org/*" + - dependency-name: "github.com/golangci/golangci-lint" + open-pull-requests-limit: 2 + diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 052e5ecab2b18..cb8e62687df8f 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -1388,108 +1388,3 @@ func TestDropBindBySQLDigest(t *testing.T) { tk.MustGetErrMsg(fmt.Sprintf("drop binding for sql digest '%s'", "1"), "can't find any binding for '1'") tk.MustGetErrMsg(fmt.Sprintf("drop binding for sql digest '%s'", ""), "sql digest is empty") } - -func TestCreateBindingFromHistory(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t1(id int primary key, a int, b int, key(a))") - tk.MustExec("create table t2(id int primary key, a int, b int, key(a))") - - var testCases = []struct { - sqls []string - hint string - }{ - { - sqls: []string{ - "select %s * from t1, t2 where t1.id = t2.id", - "select %s * from test.t1, t2 where t1.id = t2.id", - "select %s * from test.t1, test.t2 where t1.id = t2.id", - "select %s * from t1, test.t2 where t1.id = t2.id", - }, - hint: "/*+ merge_join(t1, t2) */", - }, - { - sqls: []string{ - "select %s * from t1 where a = 1", - "select %s * from test.t1 where a = 1", - }, - hint: "/*+ ignore_index(t, a) */", - }, - } - - for _, testCase := range testCases { - for _, bind := range testCase.sqls { - stmtsummary.StmtSummaryByDigestMap.Clear() - bindSQL := fmt.Sprintf(bind, testCase.hint) - tk.MustExec(bindSQL) - planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", bindSQL)).Rows() - tk.MustExec(fmt.Sprintf("create session binding from history using plan digest '%s'", planDigest[0][0])) - showRes := tk.MustQuery("show bindings").Rows() - require.Equal(t, len(showRes), 1) - require.Equal(t, planDigest[0][0], showRes[0][10]) - for _, sql := range testCase.sqls { - tk.MustExec(fmt.Sprintf(sql, "")) - tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) - } - } - showRes := tk.MustQuery("show bindings").Rows() - require.Equal(t, len(showRes), 1) - tk.MustExec(fmt.Sprintf("drop binding for sql digest '%s'", showRes[0][9])) - } - - // exception cases - tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", "1"), "can't find any plans for '1'") - tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", ""), "plan digest is empty") - tk.MustExec("create binding for select * from t1, t2 where t1.id = t2.id using select /*+ merge_join(t1, t2) */ * from t1, t2 where t1.id = t2.id") - showRes := tk.MustQuery("show bindings").Rows() - require.Equal(t, showRes[0][10], "") // plan digest should be nil by create for -} - -func TestCreateBindingForPrepareFromHistory(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(id int primary key, a int, key(a))") - - tk.MustExec("prepare stmt from 'select /*+ ignore_index(t,a) */ * from t where a = ?'") - tk.MustExec("set @a = 1") - tk.MustExec("execute stmt using @a") - planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", "select /*+ ignore_index(t,a) */ * from t where a = ? [arguments: 1]")).Rows() - showRes := tk.MustQuery("show bindings").Rows() - require.Equal(t, len(showRes), 0) - tk.MustExec(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0])) - showRes = tk.MustQuery("show bindings").Rows() - require.Equal(t, len(showRes), 1) - require.Equal(t, planDigest[0][0], showRes[0][10]) - tk.MustExec("execute stmt using @a") - tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) -} - -func TestErrorCasesCreateBindingFromHistory(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) - - tk.MustExec("use test") - tk.MustExec("drop table if exists t1, t2, t3") - tk.MustExec("create table t1(id int)") - tk.MustExec("create table t2(id int)") - tk.MustExec("create table t3(id int)") - - sql := "select * from t1 where t1.id in (select id from t2)" - tk.MustExec(sql) - planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows() - tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with sub query") - - sql = "select * from t1, t2, t3 where t1.id = t2.id and t2.id = t3.id" - tk.MustExec(sql) - planDigest = tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows() - tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with more than two table join") -} diff --git a/br/COMPATIBILITY_TEST.md b/br/COMPATIBILITY_TEST.md index b5580835baee8..44984ebcd2bfa 100644 --- a/br/COMPATIBILITY_TEST.md +++ b/br/COMPATIBILITY_TEST.md @@ -3,7 +3,7 @@ ## Background We had some incompatibility issues in the past, which made BR cannot restore backed up data in some situations. -So we need a test workflow to check the compatiblity. +So we need a test workflow to check the compatibility. ## Goal diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 317124d0b8d19..e32606207082e 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -394,6 +394,13 @@ func openDuplicateDB(storeDir string) (*pebble.DB, error) { return pebble.Open(dbPath, opts) } +var ( + // RunInTest indicates whether the current process is running in test. + RunInTest bool + // LastAlloc is the last ID allocator. + LastAlloc manual.Allocator +) + // NewLocalBackend creates new connections to tikv. func NewLocalBackend( ctx context.Context, @@ -461,6 +468,11 @@ func NewLocalBackend( } else { writeLimiter = noopStoreWriteLimiter{} } + alloc := manual.Allocator{} + if RunInTest { + alloc.RefCnt = new(atomic.Int64) + LastAlloc = alloc + } local := &local{ engines: sync.Map{}, pdCtl: pdCtl, @@ -486,7 +498,7 @@ func NewLocalBackend( keyAdapter: keyAdapter, errorMgr: errorMgr, importClientFactory: importClientFactory, - bufferPool: membuf.NewPool(membuf.WithAllocator(manual.Allocator{})), + bufferPool: membuf.NewPool(membuf.WithAllocator(alloc)), writeLimiter: writeLimiter, logger: log.FromContext(ctx), encBuilder: NewEncodingBuilder(ctx), diff --git a/br/pkg/lightning/manual/BUILD.bazel b/br/pkg/lightning/manual/BUILD.bazel index 6d1fc18dd2495..d54902a23c066 100644 --- a/br/pkg/lightning/manual/BUILD.bazel +++ b/br/pkg/lightning/manual/BUILD.bazel @@ -10,4 +10,5 @@ go_library( cgo = True, importpath = "github.com/pingcap/tidb/br/pkg/lightning/manual", visibility = ["//visibility:public"], + deps = ["@org_uber_go_atomic//:atomic"], ) diff --git a/br/pkg/lightning/manual/allocator.go b/br/pkg/lightning/manual/allocator.go index 821eb750c5030..18aa8cc9353c4 100644 --- a/br/pkg/lightning/manual/allocator.go +++ b/br/pkg/lightning/manual/allocator.go @@ -14,8 +14,33 @@ package manual -type Allocator struct{} +import ( + "fmt" -func (Allocator) Alloc(n int) []byte { return New(n) } + "go.uber.org/atomic" +) -func (Allocator) Free(b []byte) { Free(b) } +type Allocator struct { + RefCnt *atomic.Int64 +} + +func (a Allocator) Alloc(n int) []byte { + if a.RefCnt != nil { + a.RefCnt.Add(1) + } + return New(n) +} + +func (a Allocator) Free(b []byte) { + if a.RefCnt != nil { + a.RefCnt.Add(-1) + } + Free(b) +} + +func (a Allocator) CheckRefCnt() error { + if a.RefCnt != nil && a.RefCnt.Load() != 0 { + return fmt.Errorf("memory leak detected, refCnt: %d", a.RefCnt.Load()) + } + return nil +} diff --git a/br/pkg/storage/memstore_test.go b/br/pkg/storage/memstore_test.go index a85a2ff467fa1..3ae9a08d168bc 100644 --- a/br/pkg/storage/memstore_test.go +++ b/br/pkg/storage/memstore_test.go @@ -17,7 +17,6 @@ import ( "bytes" "context" "io" - "io/ioutil" "sync" "testing" "time" @@ -70,7 +69,7 @@ func TestMemStoreBasic(t *testing.T) { require.Nil(t, err) r2, err := store.Open(ctx, "/hello.txt") require.Nil(t, err) - fileContent, err = ioutil.ReadAll(r) + fileContent, err = io.ReadAll(r) require.Nil(t, err) require.True(t, bytes.Equal([]byte("hello world 3"), fileContent)) require.Nil(t, r.Close()) @@ -83,7 +82,7 @@ func TestMemStoreBasic(t *testing.T) { _, err = r2.Seek(5, io.SeekStart) require.Nil(t, err) - fileContent, err = ioutil.ReadAll(r2) + fileContent, err = io.ReadAll(r2) require.Nil(t, err) require.True(t, bytes.Equal([]byte(" world 3"), fileContent)) diff --git a/br/pkg/streamhelper/advancer.go b/br/pkg/streamhelper/advancer.go index 407134bc6027a..b29cbd6956ae2 100644 --- a/br/pkg/streamhelper/advancer.go +++ b/br/pkg/streamhelper/advancer.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/tikv/client-go/v2/oracle" + "go.uber.org/multierr" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -285,6 +286,7 @@ func (c *CheckpointAdvancer) onTaskEvent(ctx context.Context, e TaskEvent) error c.task = e.Info c.taskRange = spans.Collapse(len(e.Ranges), func(i int) kv.KeyRange { return e.Ranges[i] }) c.checkpoints = spans.Sorted(spans.NewFullWith(e.Ranges, 0)) + c.lastCheckpoint = e.Info.StartTs log.Info("added event", zap.Stringer("task", e.Info), zap.Stringer("ranges", logutil.StringifyKeys(c.taskRange))) case EventDel: utils.LogBackupTaskCountDec() @@ -292,7 +294,10 @@ func (c *CheckpointAdvancer) onTaskEvent(ctx context.Context, e TaskEvent) error c.taskRange = nil c.checkpoints = nil // This would be synced by `taskMu`, perhaps we'd better rename that to `tickMu`. - c.subscriber.Clear() + // Do the null check because some of test cases won't equip the advancer with subscriber. + if c.subscriber != nil { + c.subscriber.Clear() + } if err := c.env.ClearV3GlobalCheckpointForTask(ctx, e.Name); err != nil { log.Warn("failed to clear global checkpoint", logutil.ShortError(err)) } @@ -303,6 +308,18 @@ func (c *CheckpointAdvancer) onTaskEvent(ctx context.Context, e TaskEvent) error return nil } +func (c *CheckpointAdvancer) setCheckpoint(cp uint64) bool { + if cp < c.lastCheckpoint { + log.Warn("failed to update global checkpoint: stale", zap.Uint64("old", c.lastCheckpoint), zap.Uint64("new", cp)) + return false + } + if cp <= c.lastCheckpoint { + return false + } + c.lastCheckpoint = cp + return true +} + // advanceCheckpointBy advances the checkpoint by a checkpoint getter function. func (c *CheckpointAdvancer) advanceCheckpointBy(ctx context.Context, getCheckpoint func(context.Context) (uint64, error)) error { start := time.Now() @@ -310,24 +327,15 @@ 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)) - } - if cp <= c.lastCheckpoint { - return nil - } - log.Info("uploading checkpoint for task", - zap.Stringer("checkpoint", oracle.GetTimeFromTS(cp)), - zap.Uint64("checkpoint", cp), - zap.String("task", c.task.Name), - zap.Stringer("take", time.Since(start))) - if err := c.env.UploadV3GlobalCheckpointForTask(ctx, c.task.Name, cp); err != nil { - return errors.Annotate(err, "failed to upload global checkpoint") + if c.setCheckpoint(cp) { + log.Info("uploading checkpoint for task", + zap.Stringer("checkpoint", oracle.GetTimeFromTS(cp)), + zap.Uint64("checkpoint", cp), + zap.String("task", c.task.Name), + zap.Stringer("take", time.Since(start))) + metrics.LastCheckpoint.WithLabelValues(c.task.GetName()).Set(float64(c.lastCheckpoint)) } - c.lastCheckpoint = cp - metrics.LastCheckpoint.WithLabelValues(c.task.GetName()).Set(float64(c.lastCheckpoint)) return nil } @@ -375,16 +383,17 @@ func (c *CheckpointAdvancer) subscribeTick(ctx context.Context) error { return c.subscriber.PendingErrors() } -func (c *CheckpointAdvancer) tick(ctx context.Context) error { - c.taskMu.Lock() - defer c.taskMu.Unlock() - if c.task == nil { - log.Debug("No tasks yet, skipping advancing.") - return nil +func (c *CheckpointAdvancer) importantTick(ctx context.Context) error { + c.checkpointsMu.Lock() + c.setCheckpoint(c.checkpoints.MinValue()) + c.checkpointsMu.Unlock() + if err := c.env.UploadV3GlobalCheckpointForTask(ctx, c.task.Name, c.lastCheckpoint); err != nil { + return errors.Annotate(err, "failed to upload global checkpoint") } - cx, cancel := context.WithTimeout(ctx, c.Config().TickTimeout()) - defer cancel() + return nil +} +func (c *CheckpointAdvancer) optionalTick(cx context.Context) error { threshold := c.Config().GetDefaultStartPollThreshold() if err := c.subscribeTick(cx); err != nil { log.Warn("[log backup advancer] Subscriber meet error, would polling the checkpoint.", logutil.ShortError(err)) @@ -397,6 +406,32 @@ func (c *CheckpointAdvancer) tick(ctx context.Context) error { if err != nil { return err } - return nil } + +func (c *CheckpointAdvancer) tick(ctx context.Context) error { + c.taskMu.Lock() + defer c.taskMu.Unlock() + if c.task == nil { + log.Debug("No tasks yet, skipping advancing.") + return nil + } + + var errs error + + cx, cancel := context.WithTimeout(ctx, c.Config().TickTimeout()) + defer cancel() + err := c.optionalTick(cx) + if err != nil { + log.Warn("[log backup advancer] option tick failed.", logutil.ShortError(err)) + errs = multierr.Append(errs, err) + } + + err = c.importantTick(ctx) + if err != nil { + log.Warn("[log backup advancer] important tick failed.", logutil.ShortError(err)) + errs = multierr.Append(errs, err) + } + + return errs +} diff --git a/br/pkg/streamhelper/advancer_cliext.go b/br/pkg/streamhelper/advancer_cliext.go index 611ad3744dfa8..d83e5a6ce1eb7 100644 --- a/br/pkg/streamhelper/advancer_cliext.go +++ b/br/pkg/streamhelper/advancer_cliext.go @@ -5,15 +5,19 @@ package streamhelper import ( "bytes" "context" + "encoding/binary" "fmt" "strings" "github.com/golang/protobuf/proto" "github.com/pingcap/errors" backuppb "github.com/pingcap/kvproto/pkg/brpb" + "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/redact" "github.com/pingcap/tidb/kv" clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" ) type EventType int @@ -181,11 +185,43 @@ func (t AdvancerExt) Begin(ctx context.Context, ch chan<- TaskEvent) error { return nil } +func (t AdvancerExt) getGlobalCheckpointForTask(ctx context.Context, taskName string) (uint64, error) { + key := GlobalCheckpointOf(taskName) + resp, err := t.KV.Get(ctx, key) + if err != nil { + return 0, err + } + + if len(resp.Kvs) == 0 { + return 0, nil + } + + firstKV := resp.Kvs[0] + value := firstKV.Value + if len(value) != 8 { + return 0, errors.Annotatef(berrors.ErrPiTRMalformedMetadata, + "the global checkpoint isn't 64bits (it is %d bytes, value = %s)", + len(value), + redact.Key(value)) + } + + return binary.BigEndian.Uint64(value), nil +} + func (t AdvancerExt) UploadV3GlobalCheckpointForTask(ctx context.Context, taskName string, checkpoint uint64) error { key := GlobalCheckpointOf(taskName) value := string(encodeUint64(checkpoint)) - _, err := t.KV.Put(ctx, key, value) + oldValue, err := t.getGlobalCheckpointForTask(ctx, taskName) + if err != nil { + return err + } + + if checkpoint < oldValue { + log.Warn("[log backup advancer] skipping upload global checkpoint", zap.Uint64("old", oldValue), zap.Uint64("new", checkpoint)) + return nil + } + _, err = t.KV.Put(ctx, key, value) if err != nil { return err } diff --git a/br/pkg/streamhelper/config/advancer_conf.go b/br/pkg/streamhelper/config/advancer_conf.go index 45cdab8a68e46..1440a81f932f9 100644 --- a/br/pkg/streamhelper/config/advancer_conf.go +++ b/br/pkg/streamhelper/config/advancer_conf.go @@ -16,7 +16,7 @@ const ( flagTryAdvanceThreshold = "try-advance-threshold" DefaultConsistencyCheckTick = 5 - DefaultTryAdvanceThreshold = 9 * time.Minute + DefaultTryAdvanceThreshold = 4 * time.Minute DefaultBackOffTime = 5 * time.Second DefaultTickInterval = 12 * time.Second DefaultFullScanTick = 4 @@ -76,11 +76,18 @@ func (conf Config) GetDefaultStartPollThreshold() time.Duration { // GetSubscriberErrorStartPollThreshold returns the threshold of begin polling the checkpoint // when the subscriber meets error. func (conf Config) GetSubscriberErrorStartPollThreshold() time.Duration { - return conf.TryAdvanceThreshold / 5 + // 0.45x of the origin threshold. + // The origin threshold is 0.8x the target RPO, + // and the default flush interval is about 0.5x the target RPO. + // So the relationship between the RPO and the threshold is: + // When subscription is all available, it is 1.7x of the flush interval (which allow us to save in abnormal condition). + // When some of subscriptions are not available, it is 0.75x of the flush interval. + // NOTE: can we make subscription better and give up the poll model? + return conf.TryAdvanceThreshold * 9 / 20 } // TickTimeout returns the max duration for each tick. func (conf Config) TickTimeout() time.Duration { - // If a tick blocks 10x the interval of ticking, we may need to break it and retry. - return 10 * conf.TickDuration + // If a tick blocks longer than the interval of ticking, we may need to break it and retry. + return conf.TickDuration } diff --git a/br/pkg/streamhelper/integration_test.go b/br/pkg/streamhelper/integration_test.go index b3baf433c43f6..f00d15c4b1f30 100644 --- a/br/pkg/streamhelper/integration_test.go +++ b/br/pkg/streamhelper/integration_test.go @@ -352,10 +352,12 @@ func testStreamCheckpoint(t *testing.T, metaCli streamhelper.AdvancerExt) { req.Len(resp.Kvs, 1) return binary.BigEndian.Uint64(resp.Kvs[0].Value) } - metaCli.UploadV3GlobalCheckpointForTask(ctx, task, 5) + req.NoError(metaCli.UploadV3GlobalCheckpointForTask(ctx, task, 5)) req.EqualValues(5, getCheckpoint()) - metaCli.UploadV3GlobalCheckpointForTask(ctx, task, 18) + req.NoError(metaCli.UploadV3GlobalCheckpointForTask(ctx, task, 18)) req.EqualValues(18, getCheckpoint()) - metaCli.ClearV3GlobalCheckpointForTask(ctx, task) + req.NoError(metaCli.UploadV3GlobalCheckpointForTask(ctx, task, 16)) + req.EqualValues(18, getCheckpoint()) + req.NoError(metaCli.ClearV3GlobalCheckpointForTask(ctx, task)) req.EqualValues(0, getCheckpoint()) } diff --git a/cmd/explaintest/r/index_merge.result b/cmd/explaintest/r/index_merge.result index 0233dbdb55f52..6b44c6122987e 100644 --- a/cmd/explaintest/r/index_merge.result +++ b/cmd/explaintest/r/index_merge.result @@ -455,9 +455,9 @@ c1 c2 c3 ///// MEMORY Table explain select count(c1) from (select /*+ use_index_merge(t_alias), stream_agg() */ count(1) c1 from information_schema.statements_summary where sum_latency >= 0 or max_latency >= 0 order by 1) dt; id estRows task access object operator info -StreamAgg_10 1.00 root funcs:count(Column#93)->Column#94 -└─Sort_11 1.00 root Column#93 - └─StreamAgg_14 1.00 root funcs:count(1)->Column#93 +StreamAgg_10 1.00 root funcs:count(Column#96)->Column#97 +└─Sort_11 1.00 root Column#96 + └─StreamAgg_14 1.00 root funcs:count(1)->Column#96 └─MemTableScan_18 10000.00 root table:STATEMENTS_SUMMARY show warnings; Level Code Message diff --git a/ddl/backfilling.go b/ddl/backfilling.go index 92d5ee943c647..7c966591016d9 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -626,8 +626,7 @@ func loadDDLReorgVars(ctx context.Context, sessPool *sessionPool) error { return ddlutil.LoadDDLReorgVars(ctx, sCtx) } -func makeupDecodeColMap(sessCtx sessionctx.Context, t table.Table) (map[int64]decoder.Column, error) { - dbName := model.NewCIStr(sessCtx.GetSessionVars().CurrentDB) +func makeupDecodeColMap(sessCtx sessionctx.Context, dbName model.CIStr, t table.Table) (map[int64]decoder.Column, error) { writableColInfos := make([]*model.ColumnInfo, 0, len(t.WritableCols())) for _, col := range t.WritableCols() { writableColInfos = append(writableColInfos, col.ColumnInfo) @@ -860,7 +859,7 @@ func (dc *ddlCtx) writePhysicalTableRecord(sessPool *sessionPool, t table.Physic startKey, endKey := reorgInfo.StartKey, reorgInfo.EndKey sessCtx := newContext(reorgInfo.d.store) - decodeColMap, err := makeupDecodeColMap(sessCtx, t) + decodeColMap, err := makeupDecodeColMap(sessCtx, reorgInfo.dbInfo.Name, t) if err != nil { return errors.Trace(err) } diff --git a/ddl/column.go b/ddl/column.go index e9c353aacf2f5..d9425ceabac2c 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -807,7 +807,11 @@ func doReorgWorkForModifyColumn(w *worker, d *ddlCtx, t *meta.Meta, job *model.J oldCol, changingCol *model.ColumnInfo, changingIdxs []*model.IndexInfo) (done bool, ver int64, err error) { job.ReorgMeta.ReorgTp = model.ReorgTypeTxn rh := newReorgHandler(t, w.sess, w.concurrentDDL) - reorgInfo, err := getReorgInfo(d.jobContext(job), d, rh, job, tbl, BuildElements(changingCol, changingIdxs), false) + dbInfo, err := t.GetDatabase(job.SchemaID) + if err != nil { + return false, ver, errors.Trace(err) + } + reorgInfo, err := getReorgInfo(d.jobContext(job), d, rh, job, dbInfo, tbl, BuildElements(changingCol, changingIdxs), false) if err != nil || reorgInfo.first { // If we run reorg firstly, we should update the job snapshot version // and then run the reorg next time. diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index cf11036a9935e..da49688ccc608 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -1712,6 +1712,14 @@ func TestCreateExpressionIndex(t *testing.T) { require.NoError(t, checkErr) tk.MustExec("admin check table t") tk.MustQuery("select * from t order by a, b").Check(testkit.Rows("0 9", "0 11", "0 11", "1 7", "2 7", "5 7", "8 8", "10 10", "10 10")) + + // https://github.com/pingcap/tidb/issues/39784 + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(name varchar(20))") + tk.MustExec("insert into t values ('Abc'), ('Bcd'), ('abc')") + tk.MustExec("create index idx on test.t((lower(test.t.name)))") + tk.MustExec("admin check table t") } func TestCreateUniqueExpressionIndex(t *testing.T) { diff --git a/ddl/index.go b/ddl/index.go index 8ecc916e7b3f9..0f70b73b61046 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -870,7 +870,11 @@ 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, w.concurrentDDL) - reorgInfo, err := getReorgInfo(d.jobContext(job), d, rh, job, tbl, elements, mergingTmpIdx) + dbInfo, err := t.GetDatabase(job.SchemaID) + if err != nil { + return false, ver, errors.Trace(err) + } + reorgInfo, err := getReorgInfo(d.jobContext(job), d, rh, job, dbInfo, tbl, elements, mergingTmpIdx) if err != nil || reorgInfo.first { // If we run reorg firstly, we should update the job snapshot version // and then run the reorg next time. diff --git a/ddl/ingest/engine.go b/ddl/ingest/engine.go index 8392674c1eae6..c7ed29a71d017 100644 --- a/ddl/ingest/engine.go +++ b/ddl/ingest/engine.go @@ -82,6 +82,11 @@ func (ei *engineInfo) Clean() { zap.Int64("job ID", ei.jobID), zap.Int64("index ID", ei.indexID)) } ei.openedEngine = nil + err = ei.closeWriters() + if err != nil { + logutil.BgLogger().Error(LitErrCloseWriterErr, zap.Error(err), + zap.Int64("job ID", ei.jobID), zap.Int64("index ID", ei.indexID)) + } // Here the local intermediate files will be removed. err = closedEngine.Cleanup(ei.ctx) if err != nil { @@ -101,8 +106,14 @@ func (ei *engineInfo) ImportAndClean() error { return err1 } ei.openedEngine = nil + err := ei.closeWriters() + if err != nil { + logutil.BgLogger().Error(LitErrCloseWriterErr, zap.Error(err), + zap.Int64("job ID", ei.jobID), zap.Int64("index ID", ei.indexID)) + return err + } - err := ei.diskRoot.UpdateUsageAndQuota() + err = ei.diskRoot.UpdateUsageAndQuota() if err != nil { logutil.BgLogger().Error(LitErrUpdateDiskStats, zap.Error(err), zap.Int64("job ID", ei.jobID), zap.Int64("index ID", ei.indexID)) @@ -181,6 +192,22 @@ func (ei *engineInfo) newWriterContext(workerID int) (*WriterContext, error) { }, nil } +func (ei *engineInfo) closeWriters() error { + var firstErr error + for wid := range ei.writerCache.Keys() { + if w, ok := ei.writerCache.Load(wid); ok { + _, err := w.Close(ei.ctx) + if err != nil { + if firstErr == nil { + firstErr = err + } + } + } + ei.writerCache.Delete(wid) + } + return firstErr +} + // WriteRow Write one row into local writer buffer. func (wCtx *WriterContext) WriteRow(key, idxVal []byte) error { kvs := make([]common.KvPair, 1) diff --git a/ddl/ingest/message.go b/ddl/ingest/message.go index 0828d68796ba4..4996aab49a415 100644 --- a/ddl/ingest/message.go +++ b/ddl/ingest/message.go @@ -54,6 +54,7 @@ const ( LitInfoChgMemSetting string = "[ddl-ingest] change memory setting for ingest" LitInfoInitMemSetting string = "[ddl-ingest] initial memory setting for ingest" LitInfoUnsafeImport string = "[ddl-ingest] do a partial import data into the storage" + LitErrCloseWriterErr string = "[ddl-ingest] close writer error" ) func genBackendAllocMemFailedErr(memRoot MemRoot, jobID int64) error { diff --git a/ddl/partition.go b/ddl/partition.go index cf4bd7aed962f..0a1ea4e6fbe66 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -1743,6 +1743,10 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( if err != nil { return ver, errors.Trace(err) } + dbInfo, err := t.GetDatabase(job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } // If table has global indexes, we need reorg to clean up them. if pt, ok := tbl.(table.PartitionedTable); ok && hasGlobalIndex(tblInfo) { // Build elements for compatible with modify column type. elements will not be used when reorganizing. @@ -1753,7 +1757,7 @@ func (w *worker) onDropTablePartition(d *ddlCtx, t *meta.Meta, job *model.Job) ( } } rh := newReorgHandler(t, w.sess, w.concurrentDDL) - reorgInfo, err := getReorgInfoFromPartitions(d.jobContext(job), d, rh, job, tbl, physicalTableIDs, elements) + reorgInfo, err := getReorgInfoFromPartitions(d.jobContext(job), d, rh, job, dbInfo, tbl, physicalTableIDs, elements) if err != nil || reorgInfo.first { // If we run reorg firstly, we should update the job snapshot version diff --git a/ddl/reorg.go b/ddl/reorg.go index a03cf417177dc..d7671031f64d1 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -386,6 +386,7 @@ type reorgInfo struct { // PhysicalTableID is used to trace the current partition we are handling. // If the table is not partitioned, PhysicalTableID would be TableID. PhysicalTableID int64 + dbInfo *model.DBInfo elements []*meta.Element currElement *meta.Element } @@ -585,7 +586,7 @@ func getValidCurrentVersion(store kv.Storage) (ver kv.Version, err error) { return ver, nil } -func getReorgInfo(ctx *JobContext, d *ddlCtx, rh *reorgHandler, job *model.Job, +func getReorgInfo(ctx *JobContext, d *ddlCtx, rh *reorgHandler, job *model.Job, dbInfo *model.DBInfo, tbl table.Table, elements []*meta.Element, mergingTmpIdx bool) (*reorgInfo, error) { var ( element *meta.Element @@ -685,11 +686,12 @@ func getReorgInfo(ctx *JobContext, d *ddlCtx, rh *reorgHandler, job *model.Job, info.currElement = element info.elements = elements info.mergingTmpIdx = mergingTmpIdx + info.dbInfo = dbInfo return &info, nil } -func getReorgInfoFromPartitions(ctx *JobContext, d *ddlCtx, rh *reorgHandler, job *model.Job, tbl table.Table, partitionIDs []int64, elements []*meta.Element) (*reorgInfo, error) { +func getReorgInfoFromPartitions(ctx *JobContext, d *ddlCtx, rh *reorgHandler, job *model.Job, dbInfo *model.DBInfo, tbl table.Table, partitionIDs []int64, elements []*meta.Element) (*reorgInfo, error) { var ( element *meta.Element start kv.Key @@ -745,6 +747,7 @@ func getReorgInfoFromPartitions(ctx *JobContext, d *ddlCtx, rh *reorgHandler, jo info.PhysicalTableID = pid info.currElement = element info.elements = elements + info.dbInfo = dbInfo return &info, nil } diff --git a/docs/design/2019-11-05-index-advisor.md b/docs/design/2019-11-05-index-advisor.md index 53abab5ba0d5a..5606d9bd9a942 100644 --- a/docs/design/2019-11-05-index-advisor.md +++ b/docs/design/2019-11-05-index-advisor.md @@ -57,7 +57,7 @@ for { Note that executing `Swap and Re-evaluate` algorithm is necessary as the `reduced_cost` sometimes is a joint effect of several indexes and it's hard to tell each index's independent contribution to the final `reduced_cost`. For example, assume there is an extremely slow query in input workload and the desired indexes for this query is `a` and `b`. However, the number of allowed recommended indexes for the whole workload is limited and for some reason, `a` ranks top `n` in the final score list while `b` is not. But there are chances that without `b`, `a` can no more optimize that extremely slow query. ---------------------------------------------- -### A quick exmaple for single-column index recommendation +### A quick example for single-column index recommendation **Workload**: diff --git a/docs/design/2020-01-24-collations.md b/docs/design/2020-01-24-collations.md index af7c9e30b2f26..d610514ed76d5 100644 --- a/docs/design/2020-01-24-collations.md +++ b/docs/design/2020-01-24-collations.md @@ -233,7 +233,7 @@ The following features of the general collation algorithm will be supported: * Tertiary Weight i.e. case * PAD / NOPAD -All of them are supported by `text/collate` package of Go, so it is possible to map Go collations to some of UCA-based collations in MySQL like `utf8mb4_unicode_ci`/`utf8mb4_0900_ai_ci`, if we ignore the differences between UCA versions: current `text/collate` uses UCA version `6.2.0` and it is not changable. However, the collations in MySQL are with different UCA versions marked in the names, for example, `utf8mb4_0900_ai_ci` uses version `9.0`. +All of them are supported by `text/collate` package of Go, so it is possible to map Go collations to some of UCA-based collations in MySQL like `utf8mb4_unicode_ci`/`utf8mb4_0900_ai_ci`, if we ignore the differences between UCA versions: current `text/collate` uses UCA version `6.2.0` and it is not changeable. However, the collations in MySQL are with different UCA versions marked in the names, for example, `utf8mb4_0900_ai_ci` uses version `9.0`. For non-standard UCA implementations in MySQL, i.e. the `utf8mb4_general_ci`. The implementation depends on our choice to the [Compatibility with MySQL](#compatibility-with-mysql) chapter, if a 100% compatibility of `utf8mb4_general_ci` is chosen, we need to implement it by our hands. diff --git a/domain/domain.go b/domain/domain.go index 39a29ae2fe235..82055bffbe8a9 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -2457,10 +2457,6 @@ func (do *Domain) runTTLJobManager(ctx context.Context) { ttlJobManager.Start() do.ttlJobManager = ttlJobManager - // TODO: read the worker count from `do.sysVarCache` and resize the workers - ttlworker.ScanWorkersCount.Store(4) - ttlworker.DeleteWorkerCount.Store(4) - <-do.exit ttlJobManager.Stop() diff --git a/domain/plan_replayer.go b/domain/plan_replayer.go index 66f99b96d0fe6..19b5b9144836c 100644 --- a/domain/plan_replayer.go +++ b/domain/plan_replayer.go @@ -527,6 +527,7 @@ type PlanReplayerDumpTask struct { TblStats map[int64]interface{} // variables used to dump the plan + StartTS uint64 SessionBindings []*bindinfo.BindRecord EncodedPlan string SessionVars *variable.SessionVars diff --git a/domain/plan_replayer_dump.go b/domain/plan_replayer_dump.go index 575d0bc35264e..504fba64b7548 100644 --- a/domain/plan_replayer_dump.go +++ b/domain/plan_replayer_dump.go @@ -41,6 +41,8 @@ import ( ) const ( + // planReplayerSQLMeta indicates sql meta path for plan replayer + planReplayerSQLMeta = "sql_meta.toml" // PlanReplayerConfigFile indicates config file path for plan replayer PlanReplayerConfigFile = "config.toml" // PlanReplayerMetaFile meta file path for plan replayer @@ -55,6 +57,11 @@ const ( PlanReplayerGlobalBindingFile = "global_bindings.sql" ) +const ( + // PlanReplayerSQLMetaStartTS indicates the startTS in plan replayer sql meta + PlanReplayerSQLMetaStartTS = "startTS" +) + type tableNamePair struct { DBName string TableName string @@ -131,6 +138,7 @@ func (tne *tableNameExtractor) handleIsView(t *ast.TableName) (bool, error) { // DumpPlanReplayerInfo will dump the information about sqls. // The files will be organized into the following format: /* + |-sql_meta.toml |-meta.txt |-schema | |-db1.table1.schema.txt @@ -164,7 +172,7 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, sessionVars := task.SessionVars execStmts := task.ExecStmts zw := zip.NewWriter(zf) - records := generateRecords(task) + var records []PlanReplayerStatusRecord defer func() { if err != nil { logutil.BgLogger().Error("dump plan replayer failed", zap.Error(err)) @@ -183,6 +191,12 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, } insertPlanReplayerStatus(ctx, sctx, records) }() + + // Dump SQLMeta + if err = dumpSQLMeta(zw, task); err != nil { + return err + } + // Dump config if err = dumpConfig(zw); err != nil { return err @@ -244,10 +258,11 @@ func DumpPlanReplayerInfo(ctx context.Context, sctx sessionctx.Context, } if len(task.EncodedPlan) > 0 { + records = generateRecords(task) return dumpEncodedPlan(sctx, zw, task.EncodedPlan) } // Dump explain - return dumpExplain(sctx, zw, execStmts, task.Analyze) + return dumpExplain(sctx, zw, task, &records) } func generateRecords(task *PlanReplayerDumpTask) []PlanReplayerStatusRecord { @@ -265,6 +280,19 @@ func generateRecords(task *PlanReplayerDumpTask) []PlanReplayerStatusRecord { return records } +func dumpSQLMeta(zw *zip.Writer, task *PlanReplayerDumpTask) error { + cf, err := zw.Create(planReplayerSQLMeta) + if err != nil { + return errors.AddStack(err) + } + varMap := make(map[string]string) + varMap[PlanReplayerSQLMetaStartTS] = strconv.FormatUint(task.StartTS, 10) + if err := toml.NewEncoder(cf).Encode(varMap); err != nil { + return errors.AddStack(err) + } + return nil +} + func dumpConfig(zw *zip.Writer) error { cf, err := zw.Create(PlanReplayerConfigFile) if err != nil { @@ -488,12 +516,12 @@ func dumpEncodedPlan(ctx sessionctx.Context, zw *zip.Writer, encodedPlan string) return nil } -func dumpExplain(ctx sessionctx.Context, zw *zip.Writer, execStmts []ast.StmtNode, isAnalyze bool) error { - for i, stmtExec := range execStmts { +func dumpExplain(ctx sessionctx.Context, zw *zip.Writer, task *PlanReplayerDumpTask, records *[]PlanReplayerStatusRecord) error { + for i, stmtExec := range task.ExecStmts { sql := stmtExec.Text() var recordSets []sqlexec.RecordSet var err error - if isAnalyze { + if task.Analyze { // Explain analyze recordSets, err = ctx.(sqlexec.SQLExecutor).Execute(context.Background(), fmt.Sprintf("explain analyze %s", sql)) if err != nil { @@ -522,6 +550,10 @@ func dumpExplain(ctx sessionctx.Context, zw *zip.Writer, execStmts []ast.StmtNod return err } } + *records = append(*records, PlanReplayerStatusRecord{ + OriginSQL: sql, + Token: task.FileName, + }) } return nil } diff --git a/dumpling/export/BUILD.bazel b/dumpling/export/BUILD.bazel index 08cc7fe2e664e..cf4d938de6042 100644 --- a/dumpling/export/BUILD.bazel +++ b/dumpling/export/BUILD.bazel @@ -103,6 +103,7 @@ go_test( "//util/filter", "//util/promutil", "//util/table-filter", + "@com_github_coreos_go_semver//semver", "@com_github_data_dog_go_sqlmock//:go-sqlmock", "@com_github_go_sql_driver_mysql//:mysql", "@com_github_pingcap_errors//:errors", diff --git a/dumpling/export/dump.go b/dumpling/export/dump.go index 818f91f937231..d9c363eda39dc 100644 --- a/dumpling/export/dump.go +++ b/dumpling/export/dump.go @@ -16,6 +16,7 @@ import ( "sync/atomic" "time" + "github.com/coreos/go-semver/semver" // import mysql driver "github.com/go-sql-driver/mysql" "github.com/google/uuid" @@ -46,6 +47,10 @@ var openDBFunc = openDB var errEmptyHandleVals = errors.New("empty handleVals for TiDB table") +// After TiDB v6.2.0 we always enable tidb_enable_paging by default. +// see https://docs.pingcap.com/zh/tidb/dev/system-variables#tidb_enable_paging-%E4%BB%8E-v540-%E7%89%88%E6%9C%AC%E5%BC%80%E5%A7%8B%E5%BC%95%E5%85%A5 +var enablePagingVersion = semver.New("6.2.0") + // Dumper is the dump progress structure type Dumper struct { tctx *tcontext.Context @@ -1539,6 +1544,19 @@ func updateServiceSafePoint(tctx *tcontext.Context, pdClient pd.Client, ttl int6 } } +// setDefaultSessionParams is a step to set default params for session params. +func setDefaultSessionParams(si version.ServerInfo, sessionParams map[string]interface{}) { + defaultSessionParams := map[string]interface{}{} + if si.ServerType == version.ServerTypeTiDB && si.HasTiKV && si.ServerVersion.Compare(*enablePagingVersion) >= 0 { + defaultSessionParams["tidb_enable_paging"] = "ON" + } + for k, v := range defaultSessionParams { + if _, ok := sessionParams[k]; !ok { + sessionParams[k] = v + } + } +} + // setSessionParam is an initialization step of Dumper. func setSessionParam(d *Dumper) error { conf, pool := d.conf, d.dbHandle diff --git a/dumpling/export/dump_test.go b/dumpling/export/dump_test.go index c9a40bba28d6f..7d621857f3a85 100644 --- a/dumpling/export/dump_test.go +++ b/dumpling/export/dump_test.go @@ -9,6 +9,7 @@ import ( "time" "github.com/DATA-DOG/go-sqlmock" + "github.com/coreos/go-semver/semver" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/version" tcontext "github.com/pingcap/tidb/dumpling/context" @@ -224,3 +225,67 @@ func TestUnregisterMetrics(t *testing.T) { // should not panic require.Error(t, err) } + +func TestSetDefaultSessionParams(t *testing.T) { + testCases := []struct { + si version.ServerInfo + sessionParams map[string]interface{} + expectedParams map[string]interface{} + }{ + { + si: version.ServerInfo{ + ServerType: version.ServerTypeTiDB, + HasTiKV: true, + ServerVersion: semver.New("6.1.0"), + }, + sessionParams: map[string]interface{}{ + "tidb_snapshot": "2020-01-01 00:00:00", + }, + expectedParams: map[string]interface{}{ + "tidb_snapshot": "2020-01-01 00:00:00", + }, + }, + { + si: version.ServerInfo{ + ServerType: version.ServerTypeTiDB, + HasTiKV: true, + ServerVersion: semver.New("6.2.0"), + }, + sessionParams: map[string]interface{}{ + "tidb_snapshot": "2020-01-01 00:00:00", + }, + expectedParams: map[string]interface{}{ + "tidb_enable_paging": "ON", + "tidb_snapshot": "2020-01-01 00:00:00", + }, + }, + { + si: version.ServerInfo{ + ServerType: version.ServerTypeTiDB, + HasTiKV: true, + ServerVersion: semver.New("6.2.0"), + }, + sessionParams: map[string]interface{}{ + "tidb_enable_paging": "OFF", + "tidb_snapshot": "2020-01-01 00:00:00", + }, + expectedParams: map[string]interface{}{ + "tidb_enable_paging": "OFF", + "tidb_snapshot": "2020-01-01 00:00:00", + }, + }, + { + si: version.ServerInfo{ + ServerType: version.ServerTypeMySQL, + ServerVersion: semver.New("8.0.32"), + }, + sessionParams: map[string]interface{}{}, + expectedParams: map[string]interface{}{}, + }, + } + + for _, testCase := range testCases { + setDefaultSessionParams(testCase.si, testCase.sessionParams) + require.Equal(t, testCase.expectedParams, testCase.sessionParams) + } +} diff --git a/dumpling/export/sql_test.go b/dumpling/export/sql_test.go index 04615637be8f1..5ae4c7278efa4 100644 --- a/dumpling/export/sql_test.go +++ b/dumpling/export/sql_test.go @@ -1308,6 +1308,8 @@ func buildMockNewRows(mock sqlmock.Sqlmock, columns []string, driverValues [][]d } func readRegionCsvDriverValues(t *testing.T) [][]driver.Value { + t.Helper() + csvFilename := "region_results.csv" file, err := os.Open(csvFilename) require.NoError(t, err) diff --git a/dumpling/export/writer_test.go b/dumpling/export/writer_test.go index b5f54f3debcb9..09e43c469e5bf 100644 --- a/dumpling/export/writer_test.go +++ b/dumpling/export/writer_test.go @@ -5,7 +5,6 @@ package export import ( "context" "database/sql/driver" - "io/ioutil" "os" "path" "sync" @@ -31,7 +30,7 @@ func TestWriteDatabaseMeta(t *testing.T) { _, err = os.Stat(p) require.NoError(t, err) - bytes, err := ioutil.ReadFile(p) + bytes, err := os.ReadFile(p) require.NoError(t, err) require.Equal(t, "/*!40101 SET NAMES binary*/;\nCREATE DATABASE `test`;\n", string(bytes)) } @@ -50,7 +49,7 @@ func TestWritePolicyMeta(t *testing.T) { _, err = os.Stat(p) require.NoError(t, err) - bytes, err := ioutil.ReadFile(p) + bytes, err := os.ReadFile(p) require.NoError(t, err) require.Equal(t, "/*!40101 SET NAMES binary*/;\ncreate placement policy `y` followers=2;\n", string(bytes)) } @@ -68,7 +67,7 @@ func TestWriteTableMeta(t *testing.T) { p := path.Join(dir, "test.t-schema.sql") _, err = os.Stat(p) require.NoError(t, err) - bytes, err := ioutil.ReadFile(p) + bytes, err := os.ReadFile(p) require.NoError(t, err) require.Equal(t, "/*!40101 SET NAMES binary*/;\nCREATE TABLE t (a INT);\n", string(bytes)) } @@ -89,14 +88,14 @@ func TestWriteViewMeta(t *testing.T) { p := path.Join(dir, "test.v-schema.sql") _, err = os.Stat(p) require.NoError(t, err) - bytes, err := ioutil.ReadFile(p) + bytes, err := os.ReadFile(p) require.NoError(t, err) require.Equal(t, specCmt+createTableSQL, string(bytes)) p = path.Join(dir, "test.v-schema-view.sql") _, err = os.Stat(p) require.NoError(t, err) - bytes, err = ioutil.ReadFile(p) + bytes, err = os.ReadFile(p) require.NoError(t, err) require.Equal(t, specCmt+createViewSQL, string(bytes)) } @@ -126,7 +125,7 @@ func TestWriteTableData(t *testing.T) { p := path.Join(dir, "test.employee.000000000.sql") _, err = os.Stat(p) require.NoError(t, err) - bytes, err := ioutil.ReadFile(p) + bytes, err := os.ReadFile(p) require.NoError(t, err) expected := "/*!40101 SET NAMES binary*/;\n" + @@ -182,7 +181,7 @@ func TestWriteTableDataWithFileSize(t *testing.T) { p = path.Join(dir, p) _, err := os.Stat(p) require.NoError(t, err) - bytes, err := ioutil.ReadFile(p) + bytes, err := os.ReadFile(p) require.NoError(t, err) require.Equal(t, expected, string(bytes)) } @@ -232,7 +231,7 @@ func TestWriteTableDataWithFileSizeAndRows(t *testing.T) { p = path.Join(dir, p) _, err = os.Stat(p) require.NoError(t, err) - bytes, err := ioutil.ReadFile(p) + bytes, err := os.ReadFile(p) require.NoError(t, err) require.Equal(t, expected, string(bytes)) } @@ -281,7 +280,7 @@ func TestWriteTableDataWithStatementSize(t *testing.T) { p = path.Join(config.OutputDirPath, p) _, err = os.Stat(p) require.NoError(t, err) - bytes, err1 := ioutil.ReadFile(p) + bytes, err1 := os.ReadFile(p) require.NoError(t, err1) require.Equal(t, expected, string(bytes)) } @@ -297,7 +296,7 @@ func TestWriteTableDataWithStatementSize(t *testing.T) { require.NoError(t, err) err = os.RemoveAll(config.OutputDirPath) require.NoError(t, err) - config.OutputDirPath, err = ioutil.TempDir("", "dumpling") + config.OutputDirPath, err = os.MkdirTemp("", "dumpling") writer = createTestWriter(config, t) @@ -322,7 +321,7 @@ func TestWriteTableDataWithStatementSize(t *testing.T) { p = path.Join(config.OutputDirPath, p) _, err = os.Stat(p) require.NoError(t, err) - bytes, err := ioutil.ReadFile(p) + bytes, err := os.ReadFile(p) require.NoError(t, err) require.Equal(t, expected, string(bytes)) } @@ -331,6 +330,8 @@ func TestWriteTableDataWithStatementSize(t *testing.T) { var mu sync.Mutex func createTestWriter(conf *Config, t *testing.T) *Writer { + t.Helper() + mu.Lock() extStore, err := conf.createExternalStorage(context.Background()) mu.Unlock() diff --git a/executor/BUILD.bazel b/executor/BUILD.bazel index 7b1a4d603f993..15787b0caad1b 100644 --- a/executor/BUILD.bazel +++ b/executor/BUILD.bazel @@ -125,6 +125,7 @@ go_library( "//parser/model", "//parser/mysql", "//parser/terror", + "//parser/tidb", "//parser/types", "//planner", "//planner/core", diff --git a/executor/adapter.go b/executor/adapter.go index 5f01aa51e0378..5e12cce1ccc69 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -1538,7 +1538,7 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { if a.retryCount > 0 { slowItems.ExecRetryTime = costTime - sessVars.DurationParse - sessVars.DurationCompile - time.Since(a.retryStartTime) } - if _, ok := a.StmtNode.(*ast.CommitStmt); ok { + if _, ok := a.StmtNode.(*ast.CommitStmt); ok && sessVars.PrevStmt != nil { slowItems.PrevStmt = sessVars.PrevStmt.String() } slowLog := sessVars.SlowLogFormat(slowItems) diff --git a/executor/analyze.go b/executor/analyze.go index 28a0a44066c62..af223b24dd4a8 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -312,7 +312,7 @@ func (e *AnalyzeExec) handleResultsError(ctx context.Context, concurrency int, n } handleGlobalStats(needGlobalStats, globalStatsMap, results) - if err1 := statsHandle.SaveTableStatsToStorage(results, e.ctx.GetSessionVars().EnableAnalyzeSnapshot); err1 != nil { + if err1 := statsHandle.SaveTableStatsToStorage(results, e.ctx.GetSessionVars().EnableAnalyzeSnapshot, handle.StatsMetaHistorySourceAnalyze); err1 != nil { tableID := results.TableID.TableID err = err1 logutil.Logger(ctx).Error("save table stats to storage failed", zap.Error(err), zap.Int64("tableID", tableID)) diff --git a/executor/analyze_global_stats.go b/executor/analyze_global_stats.go index 46e9fdbf41544..6b11e68a3e614 100644 --- a/executor/analyze_global_stats.go +++ b/executor/analyze_global_stats.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" @@ -94,6 +95,7 @@ func (e *AnalyzeExec) handleGlobalStats(ctx context.Context, needGlobalStats boo info.statsVersion, 1, true, + handle.StatsMetaHistorySourceAnalyze, ) if err != nil { logutil.Logger(ctx).Error("save global-level stats to storage failed", zap.String("info", job.JobInfo), diff --git a/executor/analyze_worker.go b/executor/analyze_worker.go index 68c2f6a1f6ec8..18edc514c5d4d 100644 --- a/executor/analyze_worker.go +++ b/executor/analyze_worker.go @@ -59,7 +59,7 @@ func (worker *analyzeSaveStatsWorker) run(ctx context.Context, analyzeSnapshot b worker.errCh <- errors.Trace(ErrQueryInterrupted) return } - err := handle.SaveTableStatsToStorage(worker.sctx, results, analyzeSnapshot) + err := handle.SaveTableStatsToStorage(worker.sctx, results, analyzeSnapshot, handle.StatsMetaHistorySourceAnalyze) if err != nil { logutil.Logger(ctx).Error("save table stats to storage failed", zap.Error(err)) finishJobWithLog(worker.sctx, results.Job, err) diff --git a/executor/compiler.go b/executor/compiler.go index ce8b487e24657..10e624d1920bd 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -157,13 +157,17 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (_ *ExecS } } if c.Ctx.GetSessionVars().EnablePlanReplayerCapture && !c.Ctx.GetSessionVars().InRestrictedSQL { - checkPlanReplayerCaptureTask(c.Ctx, stmtNode) + startTS, err := sessiontxn.GetTxnManager(c.Ctx).GetStmtReadTS() + if err != nil { + return nil, err + } + checkPlanReplayerCaptureTask(c.Ctx, stmtNode, startTS) } return stmt, nil } -func checkPlanReplayerCaptureTask(sctx sessionctx.Context, stmtNode ast.StmtNode) { +func checkPlanReplayerCaptureTask(sctx sessionctx.Context, stmtNode ast.StmtNode, startTS uint64) { dom := domain.GetDomain(sctx) if dom == nil { return @@ -178,14 +182,14 @@ func checkPlanReplayerCaptureTask(sctx sessionctx.Context, stmtNode ast.StmtNode for _, task := range tasks { if task.SQLDigest == sqlDigest.String() { if task.PlanDigest == "*" || task.PlanDigest == planDigest.String() { - sendPlanReplayerDumpTask(sqlDigest.String(), planDigest.String(), sctx, stmtNode) + sendPlanReplayerDumpTask(sqlDigest.String(), planDigest.String(), sctx, stmtNode, startTS) return } } } } -func sendPlanReplayerDumpTask(sqlDigest, planDigest string, sctx sessionctx.Context, stmtNode ast.StmtNode) { +func sendPlanReplayerDumpTask(sqlDigest, planDigest string, sctx sessionctx.Context, stmtNode ast.StmtNode, startTS uint64) { stmtCtx := sctx.GetSessionVars().StmtCtx handle := sctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) dumpTask := &domain.PlanReplayerDumpTask{ @@ -193,6 +197,7 @@ func sendPlanReplayerDumpTask(sqlDigest, planDigest string, sctx sessionctx.Cont SQLDigest: sqlDigest, PlanDigest: planDigest, }, + StartTS: startTS, EncodePlan: GetEncodedPlan, TblStats: stmtCtx.TableStats, SessionBindings: handle.GetAllBindRecord(), diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 5c86cd9566cdd..6f4badaa475ed 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -34,7 +34,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" @@ -1525,14 +1524,12 @@ func TestRenameMultiTables(t *testing.T) { } func TestCreateTableWithTTL(t *testing.T) { - parser.TTLFeatureGate = true - store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("CREATE TABLE t (created_at datetime) TTL = `created_at` + INTERVAL 5 DAY") - tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL 5 DAY TTL_ENABLE = 'ON'")) + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`created_at` + INTERVAL 5 DAY */ /*T![ttl] TTL_ENABLE='ON' */")) tk.MustExec("DROP TABLE t") tk.MustGetErrMsg("CREATE TABLE t (id int) TTL = `id` + INTERVAL 5 DAY", "[ddl:8148]Field 'id' is of a not supported type for TTL config, expect DATETIME, DATE or TIMESTAMP") @@ -1541,23 +1538,21 @@ func TestCreateTableWithTTL(t *testing.T) { // when multiple ttl and ttl_enable configs are submitted, only the last one will be handled tk.MustExec("CREATE TABLE t (created_at datetime) TTL_ENABLE = 'ON' TTL = `created_at` + INTERVAL 1 DAY TTL = `created_at` + INTERVAL 2 DAY TTL = `created_at` + INTERVAL 3 DAY TTL_ENABLE = 'OFF'") - tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL 3 DAY TTL_ENABLE = 'OFF'")) + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`created_at` + INTERVAL 3 DAY */ /*T![ttl] TTL_ENABLE='OFF' */")) tk.MustExec("DROP TABLE t") } func TestAlterTTLInfo(t *testing.T) { - parser.TTLFeatureGate = true - store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("CREATE TABLE t (created_at datetime, updated_at datetime, wrong_type int) TTL = `created_at` + INTERVAL 5 DAY") tk.MustExec("ALTER TABLE t TTL = `updated_at` + INTERVAL 2 YEAR") - tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `updated_at` + INTERVAL 2 YEAR TTL_ENABLE = 'ON'")) + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`updated_at` + INTERVAL 2 YEAR */ /*T![ttl] TTL_ENABLE='ON' */")) tk.MustExec("ALTER TABLE t TTL_ENABLE = 'OFF'") - tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `updated_at` + INTERVAL 2 YEAR TTL_ENABLE = 'OFF'")) + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`updated_at` + INTERVAL 2 YEAR */ /*T![ttl] TTL_ENABLE='OFF' */")) tk.MustGetErrMsg("ALTER TABLE t TTL = `not_exist` + INTERVAL 2 YEAR", "[ddl:1054]Unknown column 'not_exist' in 'TTL config'") @@ -1567,13 +1562,13 @@ func TestAlterTTLInfo(t *testing.T) { tk.MustGetErrMsg("ALTER TABLE t CHANGE updated_at updated_at_new INT", "[ddl:8148]Field 'updated_at_new' is of a not supported type for TTL config, expect DATETIME, DATE or TIMESTAMP") tk.MustExec("ALTER TABLE t RENAME COLUMN `updated_at` TO `updated_at_2`") - tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at_2` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `updated_at_2` + INTERVAL 2 YEAR TTL_ENABLE = 'OFF'")) + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at_2` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`updated_at_2` + INTERVAL 2 YEAR */ /*T![ttl] TTL_ENABLE='OFF' */")) tk.MustExec("ALTER TABLE t CHANGE `updated_at_2` `updated_at_3` date") - tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at_3` date DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `updated_at_3` + INTERVAL 2 YEAR TTL_ENABLE = 'OFF'")) + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at_3` date DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`updated_at_3` + INTERVAL 2 YEAR */ /*T![ttl] TTL_ENABLE='OFF' */")) tk.MustExec("ALTER TABLE t TTL = `updated_at_3` + INTERVAL 3 YEAR") - tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at_3` date DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `updated_at_3` + INTERVAL 3 YEAR TTL_ENABLE = 'OFF'")) + tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at_3` date DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`updated_at_3` + INTERVAL 3 YEAR */ /*T![ttl] TTL_ENABLE='OFF' */")) tk.MustGetErrMsg("ALTER TABLE t TTL_ENABLE = 'OFF' REMOVE TTL", "[ddl:8200]Unsupported multi schema change for alter table ttl") @@ -1584,8 +1579,6 @@ func TestAlterTTLInfo(t *testing.T) { } func TestDisableTTLForTempTable(t *testing.T) { - parser.TTLFeatureGate = true - store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -1594,8 +1587,6 @@ func TestDisableTTLForTempTable(t *testing.T) { } func TestDisableTTLForFKParentTable(t *testing.T) { - parser.TTLFeatureGate = true - store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") diff --git a/executor/executor.go b/executor/executor.go index cb96942e8f776..90622ce52e527 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -2167,6 +2167,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars.ClearStmtVars() vars.PrevFoundInBinding = vars.FoundInBinding vars.FoundInBinding = false + vars.DurationWaitTS = 0 return } diff --git a/executor/executor_test.go b/executor/executor_test.go index 59e70022727d5..fe29cdf823349 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -87,6 +87,7 @@ func checkFileName(s string) bool { "global_bindings.sql", "sql/sql0.sql", "explain/sql0.txt", + "sql_meta.toml", } for _, f := range files { if strings.Compare(f, s) == 0 { diff --git a/executor/historical_stats_test.go b/executor/historical_stats_test.go index 20d11d6e4a88c..809c2c862bf43 100644 --- a/executor/historical_stats_test.go +++ b/executor/historical_stats_test.go @@ -19,8 +19,10 @@ import ( "fmt" "strconv" "testing" + "time" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/testkit" "github.com/stretchr/testify/require" @@ -125,6 +127,7 @@ func TestRecordHistoryStatsMetaAfterAnalyze(t *testing.T) { } tk.MustQuery(fmt.Sprintf("select modify_count, count from mysql.stats_meta_history where table_id = '%d' order by create_time", tableInfo.Meta().ID)).Sort().Check( testkit.Rows("18 18", "21 21", "24 24", "27 27", "30 30")) + tk.MustQuery(fmt.Sprintf("select distinct source from mysql.stats_meta_history where table_id = '%d'", tableInfo.Meta().ID)).Sort().Check(testkit.Rows("flush stats")) // assert delete tk.MustExec("delete from test.t where test.t.a = 1") @@ -178,3 +181,38 @@ func TestGCHistoryStatsAfterDropTable(t *testing.T) { tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", tableInfo.Meta().ID)).Check(testkit.Rows("0")) } + +func TestGCOutdatedHistoryStats(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_enable_historical_stats = 1") + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10))") + tk.MustExec("analyze table test.t") + is := dom.InfoSchema() + tableInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + // dump historical stats + h := dom.StatsHandle() + hsWorker := dom.GetHistoricalStatsWorker() + tblID := hsWorker.GetOneHistoricalStatsTable() + err = hsWorker.DumpHistoricalStats(tblID, h) + require.Nil(t, err) + + // assert the records of history stats table + tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_meta_history where table_id = '%d' order by create_time", + tableInfo.Meta().ID)).Check(testkit.Rows("1")) + tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", + tableInfo.Meta().ID)).Check(testkit.Rows("1")) + + variable.HistoricalStatsDuration.Store(1 * time.Second) + time.Sleep(2 * time.Second) + err = dom.StatsHandle().ClearOutdatedHistoryStats() + require.NoError(t, err) + // assert the records of history stats table + tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_meta_history where table_id = '%d' order by create_time", + tableInfo.Meta().ID)).Check(testkit.Rows("0")) + tk.MustQuery(fmt.Sprintf("select count(*) from mysql.stats_history where table_id = '%d'", + tableInfo.Meta().ID)).Check(testkit.Rows("0")) +} diff --git a/executor/plan_replayer.go b/executor/plan_replayer.go index 490e65ef96dd2..6dcee1efc624c 100644 --- a/executor/plan_replayer.go +++ b/executor/plan_replayer.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/sessiontxn" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" @@ -133,7 +134,12 @@ func (e *PlanReplayerExec) createFile() error { func (e *PlanReplayerDumpInfo) dump(ctx context.Context) (err error) { fileName := e.FileName zf := e.File + startTS, err := sessiontxn.GetTxnManager(e.ctx).GetStmtReadTS() + if err != nil { + return err + } task := &domain.PlanReplayerDumpTask{ + StartTS: startTS, FileName: fileName, Zf: zf, SessionVars: e.ctx.GetSessionVars(), @@ -375,21 +381,23 @@ func createSchemaAndItems(ctx sessionctx.Context, f *zip.File) error { if err != nil { return errors.AddStack(err) } - sqls := strings.Split(buf.String(), ";") - if len(sqls) != 3 { - return errors.New("plan replayer: create schema and tables failed") - } + originText := buf.String() + index1 := strings.Index(originText, ";") + createDatabaseSQL := originText[:index1+1] + index2 := strings.Index(originText[index1+1:], ";") + useDatabaseSQL := originText[index1+1:][:index2+1] + createTableSQL := originText[index1+1:][index2+1:] c := context.Background() // create database if not exists - _, err = ctx.(sqlexec.SQLExecutor).Execute(c, sqls[0]) + _, err = ctx.(sqlexec.SQLExecutor).Execute(c, createDatabaseSQL) logutil.BgLogger().Debug("plan replayer: skip error", zap.Error(err)) // use database - _, err = ctx.(sqlexec.SQLExecutor).Execute(c, sqls[1]) + _, err = ctx.(sqlexec.SQLExecutor).Execute(c, useDatabaseSQL) if err != nil { return err } // create table or view - _, err = ctx.(sqlexec.SQLExecutor).Execute(c, sqls[2]) + _, err = ctx.(sqlexec.SQLExecutor).Execute(c, createTableSQL) if err != nil { return err } diff --git a/executor/prepared.go b/executor/prepared.go index ae742a30d8bb5..a9dd9452e3c99 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -53,8 +53,6 @@ type PrepareExec struct { Fields []*ast.ResultField Stmt interface{} - IsGeneralStmt bool - // If it's generated from executing "prepare stmt from '...'", the process is parse -> plan -> executor // If it's generated from the prepare protocol, the process is session.PrepareStmt -> NewPrepareExec // They both generate a PrepareExec struct, but the second case needs to reset the statement context while the first already do that. @@ -135,19 +133,15 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error { if !isNoResultPlan(p) { e.Fields = colNames2ResultFields(p.Schema(), p.OutputNames(), vars.CurrentDB) } - if e.ID == 0 && !e.IsGeneralStmt { + if e.ID == 0 { e.ID = vars.GetNextPreparedStmtID() } - if e.name != "" && !e.IsGeneralStmt { + if e.name != "" { vars.PreparedStmtNameToID[e.name] = e.ID } e.ParamCount = paramCnt e.Stmt = stmt - if e.IsGeneralStmt { - vars.AddGeneralPlanCacheStmt(e.sqlText, stmt) - return nil - } return vars.AddPreparedStmt(e.ID, stmt) } diff --git a/executor/set_test.go b/executor/set_test.go index 75b6c210146b7..1b2b4186bb4a3 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -20,7 +20,6 @@ import ( "errors" "fmt" "io" - "io/ioutil" "net/http" "strconv" "testing" @@ -767,35 +766,35 @@ func TestSetVar(t *testing.T) { tk.MustGetErrCode("set global init_connect = 'invalidstring'", mysql.ErrWrongTypeForVar) tk.MustExec("set global init_connect = 'select now(); select timestamp()'") - // test variable 'tidb_enable_general_plan_cache' + // test variable 'tidb_enable_non_prepared_plan_cache' // global scope - tk.MustQuery("select @@global.tidb_enable_general_plan_cache").Check(testkit.Rows("0")) // default value - tk.MustExec("set global tidb_enable_general_plan_cache = 1") - tk.MustQuery("select @@global.tidb_enable_general_plan_cache").Check(testkit.Rows("1")) - tk.MustExec("set global tidb_enable_general_plan_cache = 0") - tk.MustQuery("select @@global.tidb_enable_general_plan_cache").Check(testkit.Rows("0")) + tk.MustQuery("select @@global.tidb_enable_non_prepared_plan_cache").Check(testkit.Rows("0")) // default value + tk.MustExec("set global tidb_enable_non_prepared_plan_cache = 1") + tk.MustQuery("select @@global.tidb_enable_non_prepared_plan_cache").Check(testkit.Rows("1")) + tk.MustExec("set global tidb_enable_non_prepared_plan_cache = 0") + tk.MustQuery("select @@global.tidb_enable_non_prepared_plan_cache").Check(testkit.Rows("0")) // session scope - tk.MustQuery("select @@session.tidb_enable_general_plan_cache").Check(testkit.Rows("0")) // default value - tk.MustExec("set session tidb_enable_general_plan_cache = 1") - tk.MustQuery("select @@session.tidb_enable_general_plan_cache").Check(testkit.Rows("1")) - tk.MustExec("set session tidb_enable_general_plan_cache = 0") - tk.MustQuery("select @@session.tidb_enable_general_plan_cache").Check(testkit.Rows("0")) + tk.MustQuery("select @@session.tidb_enable_non_prepared_plan_cache").Check(testkit.Rows("0")) // default value + tk.MustExec("set session tidb_enable_non_prepared_plan_cache = 1") + tk.MustQuery("select @@session.tidb_enable_non_prepared_plan_cache").Check(testkit.Rows("1")) + tk.MustExec("set session tidb_enable_non_prepared_plan_cache = 0") + tk.MustQuery("select @@session.tidb_enable_non_prepared_plan_cache").Check(testkit.Rows("0")) - // test variable 'tidb_general_plan_cache-size' + // test variable 'tidb_non_prepared_plan_cache-size' // global scope - tk.MustQuery("select @@global.tidb_general_plan_cache_size").Check(testkit.Rows("100")) // default value - tk.MustExec("set global tidb_general_plan_cache_size = 200") - tk.MustQuery("select @@global.tidb_general_plan_cache_size").Check(testkit.Rows("200")) - tk.MustExec("set global tidb_general_plan_cache_size = 200000000") // overflow - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_general_plan_cache_size value: '200000000'")) - tk.MustQuery("select @@global.tidb_general_plan_cache_size").Check(testkit.Rows("100000")) + tk.MustQuery("select @@global.tidb_non_prepared_plan_cache_size").Check(testkit.Rows("100")) // default value + tk.MustExec("set global tidb_non_prepared_plan_cache_size = 200") + tk.MustQuery("select @@global.tidb_non_prepared_plan_cache_size").Check(testkit.Rows("200")) + tk.MustExec("set global tidb_non_prepared_plan_cache_size = 200000000") // overflow + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_non_prepared_plan_cache_size value: '200000000'")) + tk.MustQuery("select @@global.tidb_non_prepared_plan_cache_size").Check(testkit.Rows("100000")) // session scope - tk.MustQuery("select @@session.tidb_general_plan_cache_size").Check(testkit.Rows("100")) // default value - tk.MustExec("set session tidb_general_plan_cache_size = 300") - tk.MustQuery("select @@session.tidb_general_plan_cache_size").Check(testkit.Rows("300")) - tk.MustExec("set session tidb_general_plan_cache_size = -1") // underflow - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_general_plan_cache_size value: '-1'")) - tk.MustQuery("select @@session.tidb_general_plan_cache_size").Check(testkit.Rows("1")) + tk.MustQuery("select @@session.tidb_non_prepared_plan_cache_size").Check(testkit.Rows("100")) // default value + tk.MustExec("set session tidb_non_prepared_plan_cache_size = 300") + tk.MustQuery("select @@session.tidb_non_prepared_plan_cache_size").Check(testkit.Rows("300")) + tk.MustExec("set session tidb_non_prepared_plan_cache_size = -1") // underflow + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_non_prepared_plan_cache_size value: '-1'")) + tk.MustQuery("select @@session.tidb_non_prepared_plan_cache_size").Check(testkit.Rows("1")) // test variable 'foreign_key_checks' // global scope @@ -1802,7 +1801,7 @@ func TestSetClusterConfig(t *testing.T) { httpCnt = 0 tk.Session().SetValue(executor.TestSetConfigHTTPHandlerKey, func(req *http.Request) (*http.Response, error) { httpCnt++ - body, err := ioutil.ReadAll(req.Body) + body, err := io.ReadAll(req.Body) require.NoError(t, err) // The `raftstore.` prefix is stripped. require.JSONEq(t, `{"server.snap-max-write-bytes-per-sec":"500MB"}`, string(body)) diff --git a/executor/show.go b/executor/show.go index 28a69f2febd28..2206610cfc517 100644 --- a/executor/show.go +++ b/executor/show.go @@ -43,6 +43,7 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/parser/tidb" field_types "github.com/pingcap/tidb/parser/types" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/plugin" @@ -1227,24 +1228,38 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T ddl.AppendPartitionInfo(tableInfo.Partition, buf, sqlMode) if tableInfo.TTLInfo != nil { - restoreFlags := parserformat.RestoreStringSingleQuotes | parserformat.RestoreNameBackQuotes + restoreFlags := parserformat.RestoreStringSingleQuotes | parserformat.RestoreNameBackQuotes | parserformat.RestoreTiDBSpecialComment restoreCtx := parserformat.NewRestoreCtx(restoreFlags, buf) - columnName := ast.ColumnName{Name: tableInfo.TTLInfo.ColumnName} - timeUnit := ast.TimeUnitExpr{Unit: ast.TimeUnitType(tableInfo.TTLInfo.IntervalTimeUnit)} + restoreCtx.WritePlain(" ") + err = restoreCtx.WriteWithSpecialComments(tidb.FeatureIDTTL, func() error { + columnName := ast.ColumnName{Name: tableInfo.TTLInfo.ColumnName} + timeUnit := ast.TimeUnitExpr{Unit: ast.TimeUnitType(tableInfo.TTLInfo.IntervalTimeUnit)} + restoreCtx.WriteKeyWord("TTL") + restoreCtx.WritePlain("=") + restoreCtx.WriteName(columnName.String()) + restoreCtx.WritePlainf(" + INTERVAL %s ", tableInfo.TTLInfo.IntervalExprStr) + return timeUnit.Restore(restoreCtx) + }) - restoreCtx.WriteKeyWord(" TTL ") - restoreCtx.WritePlain("= ") - restoreCtx.WriteName(columnName.String()) - restoreCtx.WritePlainf(" + INTERVAL %s ", tableInfo.TTLInfo.IntervalExprStr) - err = timeUnit.Restore(restoreCtx) if err != nil { return err } - if tableInfo.TTLInfo.Enable { - fmt.Fprintf(buf, " TTL_ENABLE = 'ON'") - } else { - fmt.Fprintf(buf, " TTL_ENABLE = 'OFF'") + + restoreCtx.WritePlain(" ") + err = restoreCtx.WriteWithSpecialComments(tidb.FeatureIDTTL, func() error { + restoreCtx.WriteKeyWord("TTL_ENABLE") + restoreCtx.WritePlain("=") + if tableInfo.TTLInfo.Enable { + restoreCtx.WriteString("ON") + } else { + restoreCtx.WriteString("OFF") + } + return nil + }) + + if err != nil { + return err } } return nil diff --git a/executor/showtest/BUILD.bazel b/executor/showtest/BUILD.bazel index aa61ad5bacd3b..807e00c8e88ec 100644 --- a/executor/showtest/BUILD.bazel +++ b/executor/showtest/BUILD.bazel @@ -15,7 +15,6 @@ go_test( "//executor", "//infoschema", "//meta/autoid", - "//parser", "//parser/auth", "//parser/model", "//parser/mysql", diff --git a/executor/showtest/show_test.go b/executor/showtest/show_test.go index b56f476f58c1e..0573de30137f6 100644 --- a/executor/showtest/show_test.go +++ b/executor/showtest/show_test.go @@ -23,7 +23,6 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -1516,7 +1515,7 @@ func TestShowBuiltin(t *testing.T) { res := tk.MustQuery("show builtins;") require.NotNil(t, res) rows := res.Rows() - const builtinFuncNum = 283 + const builtinFuncNum = 285 require.Equal(t, builtinFuncNum, len(rows)) require.Equal(t, rows[0][0].(string), "abs") require.Equal(t, rows[builtinFuncNum-1][0].(string), "yearweek") @@ -2022,26 +2021,24 @@ func TestShowLimitReturnRow(t *testing.T) { } func TestShowTTLOption(t *testing.T) { - parser.TTLFeatureGate = true - store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(created_at datetime) ttl = `created_at` + INTERVAL 100 YEAR") - tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL 100 YEAR TTL_ENABLE = 'ON'")) + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`created_at` + INTERVAL 100 YEAR */ /*T![ttl] TTL_ENABLE='ON' */")) tk.MustExec("drop table if exists t") tk.MustExec("create table t(created_at datetime) ttl = `created_at` + INTERVAL 100 YEAR ttl_enable = 'OFF'") - tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL 100 YEAR TTL_ENABLE = 'OFF'")) + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`created_at` + INTERVAL 100 YEAR */ /*T![ttl] TTL_ENABLE='OFF' */")) tk.MustExec("drop table if exists t") tk.MustExec("create table t (created_at datetime) TTL = created_at + INTERVAL 3.14159 HOUR_MINUTE") - tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL 3.14159 HOUR_MINUTE TTL_ENABLE = 'ON'")) + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`created_at` + INTERVAL 3.14159 HOUR_MINUTE */ /*T![ttl] TTL_ENABLE='ON' */")) tk.MustExec("drop table if exists t") tk.MustExec("create table t (created_at datetime) TTL = created_at + INTERVAL \"15:20\" HOUR_MINUTE") - tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin TTL = `created_at` + INTERVAL _utf8mb4'15:20' HOUR_MINUTE TTL_ENABLE = 'ON'")) + tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`created_at` + INTERVAL _utf8mb4'15:20' HOUR_MINUTE */ /*T![ttl] TTL_ENABLE='ON' */")) } func TestShowBindingDigestField(t *testing.T) { diff --git a/executor/slow_query_sql_test.go b/executor/slow_query_sql_test.go index a77e32e3bfb16..f7029454023fa 100644 --- a/executor/slow_query_sql_test.go +++ b/executor/slow_query_sql_test.go @@ -46,8 +46,6 @@ func TestSlowQueryWithoutSlowLog(t *testing.T) { } func TestSlowQuerySensitiveQuery(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) originCfg := config.GetGlobalConfig() newCfg := *originCfg @@ -57,11 +55,15 @@ func TestSlowQuerySensitiveQuery(t *testing.T) { newCfg.Log.SlowQueryFile = f.Name() config.StoreGlobalConfig(&newCfg) defer func() { - tk.MustExec("set tidb_slow_log_threshold=300;") config.StoreGlobalConfig(originCfg) require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) }() require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + defer func() { + tk.MustExec("set tidb_slow_log_threshold=300;") + }() tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustExec("set tidb_slow_log_threshold=0;") @@ -80,8 +82,6 @@ func TestSlowQuerySensitiveQuery(t *testing.T) { } func TestSlowQueryPrepared(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) originCfg := config.GetGlobalConfig() newCfg := *originCfg @@ -91,12 +91,16 @@ func TestSlowQueryPrepared(t *testing.T) { newCfg.Log.SlowQueryFile = f.Name() config.StoreGlobalConfig(&newCfg) defer func() { - tk.MustExec("set tidb_slow_log_threshold=300;") - tk.MustExec("set tidb_redact_log=0;") config.StoreGlobalConfig(originCfg) require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) }() require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + defer func() { + tk.MustExec("set tidb_slow_log_threshold=300;") + tk.MustExec("set tidb_redact_log=0;") + }() tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustExec("set tidb_slow_log_threshold=0;") @@ -116,8 +120,6 @@ func TestSlowQueryPrepared(t *testing.T) { } func TestLogSlowLogIndex(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) f, err := os.CreateTemp("", "tidb-slow-*.log") require.NoError(t, err) require.NoError(t, f.Close()) @@ -127,6 +129,8 @@ func TestLogSlowLogIndex(t *testing.T) { conf.Log.SlowQueryFile = f.Name() }) require.NoError(t, logutil.InitLogger(config.GetGlobalConfig().Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustExec("use test") @@ -140,9 +144,6 @@ func TestLogSlowLogIndex(t *testing.T) { } func TestSlowQuery(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - f, err := os.CreateTemp("", "tidb-slow-*.log") require.NoError(t, err) _, err = f.WriteString(` @@ -197,6 +198,8 @@ SELECT original_sql, bind_sql, default_db, status, create_time, update_time, cha require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) }() require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustQuery("select count(*) from `information_schema`.`slow_query` where time > '2020-10-16 20:08:13' and time < '2020-10-16 21:08:13'").Check(testkit.Rows("1")) @@ -208,10 +211,6 @@ SELECT original_sql, bind_sql, default_db, status, create_time, update_time, cha } func TestIssue37066(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) - originCfg := config.GetGlobalConfig() newCfg := *originCfg f, err := os.CreateTemp("", "tidb-slow-*.log") @@ -224,6 +223,9 @@ func TestIssue37066(t *testing.T) { require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) }() require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustExec("set tidb_slow_log_threshold=0;") defer func() { diff --git a/executor/tiflashtest/BUILD.bazel b/executor/tiflashtest/BUILD.bazel index fb8388109b737..5223fa79cc2d9 100644 --- a/executor/tiflashtest/BUILD.bazel +++ b/executor/tiflashtest/BUILD.bazel @@ -15,7 +15,6 @@ go_test( "//domain", "//executor", "//meta/autoid", - "//parser/auth", "//parser/terror", "//store/mockstore", "//store/mockstore/unistore", diff --git a/executor/tiflashtest/tiflash_test.go b/executor/tiflashtest/tiflash_test.go index baba133b3d5a7..222e1bfdaff0b 100644 --- a/executor/tiflashtest/tiflash_test.go +++ b/executor/tiflashtest/tiflash_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" - "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/unistore" @@ -1259,27 +1258,6 @@ func TestTiflashEmptyDynamicPruneResult(t *testing.T) { tk.MustQuery("select /*+ read_from_storage(tiflash[t1, t2]) */ * from IDT_RP24833 partition(p3, p4) t1 join IDT_RP24833 partition(p2) t2 on t1.col1 = t2.col1 where t1. col1 between -8448770111093677011 and -8448770111093677011 and t2. col1 <= -8448770111093677011;").Check(testkit.Rows()) } -func TestBindingFromHistoryWithTiFlashBindable(t *testing.T) { - store := testkit.CreateMockStore(t, withMockTiFlash(2)) - tk := testkit.NewTestKit(t, store) - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) - tk.MustExec("use test;") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t(a int);") - tk.MustExec("alter table test.t set tiflash replica 1") - tb := external.GetTableByName(t, tk, "test", "t") - err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) - require.NoError(t, err) - tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") - - sql := "select * from t" - tk.MustExec(sql) - rows := tk.MustQuery("explain select * from t").Rows() - fmt.Println(rows) - planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows() - tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with tiflash engine") -} - func TestDisaggregatedTiFlash(t *testing.T) { config.UpdateGlobal(func(conf *config.Config) { conf.DisaggregatedTiFlash = true diff --git a/expression/builtin.go b/expression/builtin.go index f10cf9aa3dfa9..66abac551a3e6 100644 --- a/expression/builtin.go +++ b/expression/builtin.go @@ -873,7 +873,9 @@ var funcs = map[string]functionClass{ ast.JSONMerge: &jsonMergeFunctionClass{baseFunctionClass{ast.JSONMerge, 2, -1}}, ast.JSONObject: &jsonObjectFunctionClass{baseFunctionClass{ast.JSONObject, 0, -1}}, ast.JSONArray: &jsonArrayFunctionClass{baseFunctionClass{ast.JSONArray, 0, -1}}, + ast.JSONMemberOf: &jsonMemberOfFunctionClass{baseFunctionClass{ast.JSONMemberOf, 2, 2}}, ast.JSONContains: &jsonContainsFunctionClass{baseFunctionClass{ast.JSONContains, 2, 3}}, + ast.JSONOverlaps: &jsonOverlapsFunctionClass{baseFunctionClass{ast.JSONOverlaps, 2, 2}}, ast.JSONContainsPath: &jsonContainsPathFunctionClass{baseFunctionClass{ast.JSONContainsPath, 3, -1}}, ast.JSONValid: &jsonValidFunctionClass{baseFunctionClass{ast.JSONValid, 1, 1}}, ast.JSONArrayAppend: &jsonArrayAppendFunctionClass{baseFunctionClass{ast.JSONArrayAppend, 3, -1}}, diff --git a/expression/builtin_json.go b/expression/builtin_json.go index e317fa88e952a..e9f803dcc86df 100644 --- a/expression/builtin_json.go +++ b/expression/builtin_json.go @@ -43,7 +43,9 @@ var ( _ functionClass = &jsonMergeFunctionClass{} _ functionClass = &jsonObjectFunctionClass{} _ functionClass = &jsonArrayFunctionClass{} + _ functionClass = &jsonMemberOfFunctionClass{} _ functionClass = &jsonContainsFunctionClass{} + _ functionClass = &jsonOverlapsFunctionClass{} _ functionClass = &jsonContainsPathFunctionClass{} _ functionClass = &jsonValidFunctionClass{} _ functionClass = &jsonArrayAppendFunctionClass{} @@ -71,7 +73,9 @@ var ( _ builtinFunc = &builtinJSONReplaceSig{} _ builtinFunc = &builtinJSONRemoveSig{} _ builtinFunc = &builtinJSONMergeSig{} + _ builtinFunc = &builtinJSONMemberOfSig{} _ builtinFunc = &builtinJSONContainsSig{} + _ builtinFunc = &builtinJSONOverlapsSig{} _ builtinFunc = &builtinJSONStorageSizeSig{} _ builtinFunc = &builtinJSONDepthSig{} _ builtinFunc = &builtinJSONSearchSig{} @@ -740,6 +744,68 @@ func jsonModify(ctx sessionctx.Context, args []Expression, row chunk.Row, mt typ return res, false, nil } +type jsonMemberOfFunctionClass struct { + baseFunctionClass +} + +type builtinJSONMemberOfSig struct { + baseBuiltinFunc +} + +func (b *builtinJSONMemberOfSig) Clone() builtinFunc { + newSig := &builtinJSONMemberOfSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (c *jsonMemberOfFunctionClass) verifyArgs(args []Expression) error { + if err := c.baseFunctionClass.verifyArgs(args); err != nil { + return err + } + if evalType := args[1].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString { + return types.ErrInvalidJSONData.GenWithStackByArgs(2, "member of") + } + return nil +} + +func (c *jsonMemberOfFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { + if err := c.verifyArgs(args); err != nil { + return nil, err + } + argTps := []types.EvalType{types.ETJson, types.ETJson} + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } + DisableParseJSONFlag4Expr(args[0]) + sig := &builtinJSONMemberOfSig{bf} + return sig, nil +} + +func (b *builtinJSONMemberOfSig) evalInt(row chunk.Row) (res int64, isNull bool, err error) { + target, isNull, err := b.args[0].EvalJSON(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + obj, isNull, err := b.args[1].EvalJSON(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + + if obj.TypeCode != types.JSONTypeCodeArray { + return boolToInt64(types.CompareBinaryJSON(obj, target) == 0), false, nil + } + + elemCount := obj.GetElemCount() + for i := 0; i < elemCount; i++ { + if types.CompareBinaryJSON(obj.ArrayGetElem(i), target) == 0 { + return 1, false, nil + } + } + + return 0, false, nil +} + type jsonContainsFunctionClass struct { baseFunctionClass } @@ -820,6 +886,62 @@ func (b *builtinJSONContainsSig) evalInt(row chunk.Row) (res int64, isNull bool, return 0, false, nil } +type jsonOverlapsFunctionClass struct { + baseFunctionClass +} + +type builtinJSONOverlapsSig struct { + baseBuiltinFunc +} + +func (b *builtinJSONOverlapsSig) Clone() builtinFunc { + newSig := &builtinJSONOverlapsSig{} + newSig.cloneFrom(&b.baseBuiltinFunc) + return newSig +} + +func (c *jsonOverlapsFunctionClass) verifyArgs(args []Expression) error { + if err := c.baseFunctionClass.verifyArgs(args); err != nil { + return err + } + if evalType := args[0].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString { + return types.ErrInvalidJSONData.GenWithStackByArgs(1, "json_overlaps") + } + if evalType := args[1].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString { + return types.ErrInvalidJSONData.GenWithStackByArgs(2, "json_overlaps") + } + return nil +} + +func (c *jsonOverlapsFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { + if err := c.verifyArgs(args); err != nil { + return nil, err + } + + argTps := []types.EvalType{types.ETJson, types.ETJson} + bf, err := newBaseBuiltinFuncWithTp(ctx, c.funcName, args, types.ETInt, argTps...) + if err != nil { + return nil, err + } + sig := &builtinJSONOverlapsSig{bf} + return sig, nil +} + +func (b *builtinJSONOverlapsSig) evalInt(row chunk.Row) (res int64, isNull bool, err error) { + obj, isNull, err := b.args[0].EvalJSON(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + target, isNull, err := b.args[1].EvalJSON(b.ctx, row) + if isNull || err != nil { + return res, isNull, err + } + if types.OverlapsBinaryJSON(obj, target) { + return 1, false, nil + } + return 0, false, nil +} + type jsonValidFunctionClass struct { baseFunctionClass } diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index 3e142860fdd27..23da60f380652 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -386,6 +386,47 @@ func TestJSONRemove(t *testing.T) { } } +func TestJSONMemberOf(t *testing.T) { + ctx := createContext(t) + fc := funcs[ast.JSONMemberOf] + tbl := []struct { + input []interface{} + expected interface{} + err error + }{ + {[]interface{}{`1`, `a:1`}, 1, types.ErrInvalidJSONText}, + + {[]interface{}{1, `[1, 2]`}, 1, nil}, + {[]interface{}{1, `[1]`}, 1, nil}, + {[]interface{}{1, `[0]`}, 0, nil}, + {[]interface{}{1, `[1]`}, 1, nil}, + {[]interface{}{1, `[[1]]`}, 0, nil}, + {[]interface{}{"1", `[1]`}, 0, nil}, + {[]interface{}{"1", `["1"]`}, 1, nil}, + {[]interface{}{`{"a":1}`, `{"a":1}`}, 0, nil}, + {[]interface{}{`{"a":1}`, `[{"a":1}]`}, 0, nil}, + {[]interface{}{`{"a":1}`, `[{"a":1}, 1]`}, 0, nil}, + {[]interface{}{`{"a":1}`, `["{\"a\":1}"]`}, 1, nil}, + {[]interface{}{`{"a":1}`, `["{\"a\":1}", 1]`}, 1, nil}, + } + for _, tt := range tbl { + args := types.MakeDatums(tt.input...) + f, err := fc.getFunction(ctx, datumsToConstants(args)) + require.NoError(t, err, tt.input) + d, err := evalBuiltinFunc(f, chunk.Row{}) + if tt.err == nil { + require.NoError(t, err, tt.input) + if tt.expected == nil { + require.True(t, d.IsNull(), tt.input) + } else { + require.Equal(t, int64(tt.expected.(int)), d.GetInt64(), tt.input) + } + } else { + require.True(t, tt.err.(*terror.Error).Equal(err), tt.input) + } + } +} + func TestJSONContains(t *testing.T) { ctx := createContext(t) fc := funcs[ast.JSONContains] @@ -466,6 +507,71 @@ func TestJSONContains(t *testing.T) { } } +func TestJSONOverlaps(t *testing.T) { + ctx := createContext(t) + fc := funcs[ast.JSONOverlaps] + tbl := []struct { + input []any + expected any + err error + }{ + {[]any{`[1,2,[1,3]]`, `a:1`}, 1, types.ErrInvalidJSONText}, + {[]any{`a:1`, `1`}, 1, types.ErrInvalidJSONText}, + {[]any{nil, `1`}, nil, nil}, + {[]any{`1`, nil}, nil, nil}, + + {[]any{`[1, 2]`, `[2,3]`}, 1, nil}, + {[]any{`[1, 2]`, `[2]`}, 1, nil}, + {[]any{`[1, 2]`, `2`}, 1, nil}, + {[]any{`[{"a":1}]`, `{"a":1}`}, 1, nil}, + {[]any{`[{"a":1}]`, `{"a":1,"b":2}`}, 0, nil}, + {[]any{`[{"a":1}]`, `{"a":2}`}, 0, nil}, + {[]any{`{"a":[1,2]}`, `{"a":[1]}`}, 0, nil}, + {[]any{`{"a":[1,2]}`, `{"a":[2,1]}`}, 0, nil}, + {[]any{`[1,1,1]`, `1`}, 1, nil}, + {[]any{`1`, `1`}, 1, nil}, + {[]any{`0`, `1`}, 0, nil}, + {[]any{`[[1,2], 3]`, `[1,[2,3]]`}, 0, nil}, + {[]any{`[[1,2], 3]`, `[1,3]`}, 1, nil}, + {[]any{`{"a":1,"b":10,"d":10}`, `{"a":5,"e":10,"f":1,"d":20}`}, 0, nil}, + {[]any{`[4,5,"6",7]`, `6`}, 0, nil}, + {[]any{`[4,5,6,7]`, `"6"`}, 0, nil}, + + {[]any{`[2,3]`, `[1, 2]`}, 1, nil}, + {[]any{`[2]`, `[1, 2]`}, 1, nil}, + {[]any{`2`, `[1, 2]`}, 1, nil}, + {[]any{`{"a":1}`, `[{"a":1}]`}, 1, nil}, + {[]any{`{"a":1,"b":2}`, `[{"a":1}]`}, 0, nil}, + {[]any{`{"a":2}`, `[{"a":1}]`}, 0, nil}, + {[]any{`{"a":[1]}`, `{"a":[1,2]}`}, 0, nil}, + {[]any{`{"a":[2,1]}`, `{"a":[1,2]}`}, 0, nil}, + {[]any{`1`, `[1,1,1]`}, 1, nil}, + {[]any{`1`, `1`}, 1, nil}, + {[]any{`1`, `0`}, 0, nil}, + {[]any{`[1,[2,3]]`, `[[1,2], 3]`}, 0, nil}, + {[]any{`[1,3]`, `[[1,2], 3]`}, 1, nil}, + {[]any{`{"a":5,"e":10,"f":1,"d":20}`, `{"a":1,"b":10,"d":10}`}, 0, nil}, + {[]any{`6`, `[4,5,"6",7]`}, 0, nil}, + {[]any{`"6"`, `[4,5,6,7]`}, 0, nil}, + } + for _, tt := range tbl { + args := types.MakeDatums(tt.input...) + f, err := fc.getFunction(ctx, datumsToConstants(args)) + require.NoError(t, err, tt.input) + d, err := evalBuiltinFunc(f, chunk.Row{}) + if tt.err == nil { + require.NoError(t, err, tt.input) + if tt.expected == nil { + require.True(t, d.IsNull(), tt.input) + } else { + require.Equal(t, int64(tt.expected.(int)), d.GetInt64(), tt.input) + } + } else { + require.True(t, tt.err.(*terror.Error).Equal(err), tt.input) + } + } +} + func TestJSONContainsPath(t *testing.T) { ctx := createContext(t) fc := funcs[ast.JSONContainsPath] diff --git a/expression/builtin_json_vec.go b/expression/builtin_json_vec.go index fb24808ff2c73..0610a1f6ea3ca 100644 --- a/expression/builtin_json_vec.go +++ b/expression/builtin_json_vec.go @@ -274,6 +274,59 @@ func (b *builtinJSONArraySig) vecEvalJSON(input *chunk.Chunk, result *chunk.Colu return nil } +func (b *builtinJSONMemberOfSig) vectorized() bool { + return true +} + +func (b *builtinJSONMemberOfSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { + nr := input.NumRows() + + targetCol, err := b.bufAllocator.get() + if err != nil { + return err + } + defer b.bufAllocator.put(targetCol) + + if err := b.args[0].VecEvalJSON(b.ctx, input, targetCol); err != nil { + return err + } + + objCol, err := b.bufAllocator.get() + if err != nil { + return err + } + defer b.bufAllocator.put(objCol) + + if err := b.args[1].VecEvalJSON(b.ctx, input, objCol); err != nil { + return err + } + + result.ResizeInt64(nr, false) + resI64s := result.Int64s() + + result.MergeNulls(targetCol, objCol) + for i := 0; i < nr; i++ { + if result.IsNull(i) { + continue + } + obj := objCol.GetJSON(i) + target := targetCol.GetJSON(i) + if obj.TypeCode != types.JSONTypeCodeArray { + resI64s[i] = boolToInt64(types.CompareBinaryJSON(obj, target) == 0) + } else { + elemCount := obj.GetElemCount() + for j := 0; j < elemCount; j++ { + if types.CompareBinaryJSON(obj.ArrayGetElem(j), target) == 0 { + resI64s[i] = 1 + break + } + } + } + } + + return nil +} + func (b *builtinJSONContainsSig) vectorized() bool { return true } @@ -359,6 +412,51 @@ func (b *builtinJSONContainsSig) vecEvalInt(input *chunk.Chunk, result *chunk.Co return nil } +func (b *builtinJSONOverlapsSig) vectorized() bool { + return true +} + +func (b *builtinJSONOverlapsSig) vecEvalInt(input *chunk.Chunk, result *chunk.Column) error { + nr := input.NumRows() + + objCol, err := b.bufAllocator.get() + if err != nil { + return err + } + defer b.bufAllocator.put(objCol) + + if err := b.args[0].VecEvalJSON(b.ctx, input, objCol); err != nil { + return err + } + + targetCol, err := b.bufAllocator.get() + if err != nil { + return err + } + defer b.bufAllocator.put(targetCol) + + if err := b.args[1].VecEvalJSON(b.ctx, input, targetCol); err != nil { + return err + } + + result.ResizeInt64(nr, false) + resI64s := result.Int64s() + + result.MergeNulls(objCol, targetCol) + for i := 0; i < nr; i++ { + if result.IsNull(i) { + continue + } + if types.OverlapsBinaryJSON(objCol.GetJSON(i), targetCol.GetJSON(i)) { + resI64s[i] = 1 + } else { + resI64s[i] = 0 + } + } + + return nil +} + func (b *builtinJSONQuoteSig) vectorized() bool { return true } diff --git a/expression/function_traits.go b/expression/function_traits.go index aba61d9b2d92a..4d6fa98da6a99 100644 --- a/expression/function_traits.go +++ b/expression/function_traits.go @@ -19,8 +19,8 @@ import ( "github.com/pingcap/tidb/parser/opcode" ) -// GeneralPlanCacheableOp stores function which can be cached to general plan cache. -var GeneralPlanCacheableOp = map[string]struct{}{ +// NonPreparedPlanCacheableOp stores function which can be cached to non-prepared plan cache. +var NonPreparedPlanCacheableOp = map[string]struct{}{ ast.LogicAnd: {}, ast.LogicOr: {}, ast.GE: {}, diff --git a/expression/integration_test.go b/expression/integration_test.go index 55c8f389a5df3..3efa93eff5d13 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -1228,8 +1228,7 @@ func TestDatetimeOverflow(t *testing.T) { } for _, sql := range overflowSQLs { - _, err := tk.Exec(sql) - require.Error(t, err, "[types:1441]Datetime function: datetime field overflow") + tk.MustGetErrMsg(sql, "[types:1441]Datetime function: datetime field overflow") } tk.MustExec("set sql_mode=''") @@ -1716,8 +1715,7 @@ func TestArithmeticBuiltin(t *testing.T) { tk.MustQuery("select v from t;").Check(testkit.Rows("")) tk.MustQuery("select 0.000 % 0.11234500000000000000;").Check(testkit.Rows("0.00000000000000000000")) - _, err = tk.Exec("INSERT INTO t VALUE(12 MOD 0);") - require.True(t, terror.ErrorEqual(err, expression.ErrDivisionByZero)) + tk.MustGetDBError("INSERT INTO t VALUE(12 MOD 0);", expression.ErrDivisionByZero) tk.MustQuery("select sum(1.2e2) * 0.1").Check(testkit.Rows("12")) tk.MustExec("drop table if exists t") @@ -1960,8 +1958,7 @@ func TestCompareBuiltin(t *testing.T) { tk.MustQuery("select * from t").Check(testkit.Rows("1991-05-06 13:59:28")) // insert an nonexistent time tk.MustExec("set time_zone = 'America/Los_Angeles'") - _, err := tk.Exec("insert into t value('2011-03-13 02:00:00')") - require.Error(t, err) + tk.MustExecToErr("insert into t value('2011-03-13 02:00:00')") // reset timezone to a +8 offset tk.MustExec("set time_zone = '+08:00'") tk.MustQuery("select * from t").Check(testkit.Rows("1991-05-06 12:59:28")) @@ -2125,11 +2122,9 @@ func TestAggregationBuiltinGroupConcat(t *testing.T) { result.Check(testkit.Rows("hello,h")) tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning 1260 Some rows were cut by GROUPCONCAT(test.t.a)")) - _, err := tk.Exec("insert into d select group_concat(a) from t") - require.Equal(t, errors.ErrCode(mysql.ErrCutValueGroupConcat), errors.Cause(err).(*terror.Error).Code()) + tk.MustGetErrCode("insert into d select group_concat(a) from t", mysql.ErrCutValueGroupConcat) - _, err = tk.Exec("set sql_mode=''") - require.NoError(t, err) + tk.MustExec("set sql_mode=''") tk.MustExec("insert into d select group_concat(a) from t") tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning 1260 Some rows were cut by GROUPCONCAT(test.t.a)")) tk.MustQuery("select * from d").Check(testkit.Rows("hello,h")) @@ -2374,23 +2369,18 @@ func TestDateBuiltin(t *testing.T) { r = tk.MustQuery("select date'731124', date '011124'") r.Check(testkit.Rows("1973-11-24 2001-11-24")) - _, err = tk.Exec("select date '0000-00-00 00:00:00';") + err = tk.ExecToErr("select date '0000-00-00 00:00:00';") require.Error(t, err) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue.GenWithStackByArgs(types.DateStr, "0000-00-00 00:00:00"))) - _, err = tk.Exec("select date '2017-99-99';") - require.Error(t, err) - require.True(t, terror.ErrorEqual(err, types.ErrWrongValue), "err: %v", err) + tk.MustGetDBError("select date '2017-99-99';", types.ErrWrongValue) + tk.MustGetDBError("select date '2017-2-31';", types.ErrWrongValue) - _, err = tk.Exec("select date '2017-2-31';") - require.Error(t, err) - require.True(t, terror.ErrorEqual(err, types.ErrWrongValue), "err: %v", err) - - _, err = tk.Exec("select date '201712-31';") + err = tk.ExecToErr("select date '201712-31';") require.Error(t, err) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue.GenWithStackByArgs(types.DateStr, "201712-31")), "err: %v", err) - _, err = tk.Exec("select date 'abcdefg';") + err = tk.ExecToErr("select date 'abcdefg';") require.Error(t, err) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue.GenWithStackByArgs(types.DateStr, "abcdefg")), "err: %v", err) } @@ -2402,8 +2392,7 @@ func TestJSONBuiltin(t *testing.T) { tk.MustExec("USE test;") tk.MustExec("DROP TABLE IF EXISTS t;") tk.MustExec("CREATE TABLE `my_collection` ( `doc` json DEFAULT NULL, `_id` varchar(32) GENERATED ALWAYS AS (JSON_UNQUOTE(JSON_EXTRACT(doc,'$._id'))) STORED NOT NULL, PRIMARY KEY (`_id`))") - _, err := tk.Exec("UPDATE `test`.`my_collection` SET doc=JSON_SET(doc) WHERE (JSON_EXTRACT(doc,'$.name') = 'clare');") - require.Error(t, err) + tk.MustExecToErr("UPDATE `test`.`my_collection` SET doc=JSON_SET(doc) WHERE (JSON_EXTRACT(doc,'$.name') = 'clare');") r := tk.MustQuery("select json_valid(null);") r.Check(testkit.Rows("")) @@ -2479,20 +2468,19 @@ func TestTimeLiteral(t *testing.T) { r = tk.MustQuery("select time '20 20:20';") r.Check(testkit.Rows("500:20:00")) - _, err := tk.Exec("select time '2017-01-01 00:00:00';") + err := tk.ExecToErr("select time '2017-01-01 00:00:00';") require.Error(t, err) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue.GenWithStackByArgs(types.TimeStr, "2017-01-01 00:00:00"))) - _, err = tk.Exec("select time '071231235959.999999';") + err = tk.ExecToErr("select time '071231235959.999999';") require.Error(t, err) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue.GenWithStackByArgs(types.TimeStr, "071231235959.999999"))) - _, err = tk.Exec("select time '20171231235959.999999';") + err = tk.ExecToErr("select time '20171231235959.999999';") require.Error(t, err) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue.GenWithStackByArgs(types.TimeStr, "20171231235959.999999"))) - _, err = tk.Exec("select ADDDATE('2008-01-34', -1);") - require.NoError(t, err) + tk.MustExec("select ADDDATE('2008-01-34', -1);") tk.MustQuery("Show warnings;").Check(testkit.RowsWithSep("|", "Warning|1292|Incorrect datetime value: '2008-01-34'")) } @@ -2522,15 +2510,15 @@ func TestTimestampLiteral(t *testing.T) { r = tk.MustQuery("select timestamp '2017@01@0001 00~00~00.333';") r.Check(testkit.Rows("2017-01-01 00:00:00.333")) - _, err := tk.Exec("select timestamp '00:00:00';") + err := tk.ExecToErr("select timestamp '00:00:00';") require.Error(t, err) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, "00:00:00"))) - _, err = tk.Exec("select timestamp '1992-01-03';") + err = tk.ExecToErr("select timestamp '1992-01-03';") require.Error(t, err) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, "1992-01-03"))) - _, err = tk.Exec("select timestamp '20171231235959.999999';") + err = tk.ExecToErr("select timestamp '20171231235959.999999';") require.Error(t, err) require.True(t, terror.ErrorEqual(err, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, "20171231235959.999999"))) } @@ -2721,12 +2709,36 @@ func TestFuncJSON(t *testing.T) { // #16267 tk.MustQuery(`select json_array(922337203685477580) = json_array(922337203685477581);`).Check(testkit.Rows("0")) + tk.MustQuery("select json_overlaps('[[1,2], 3]', '[1, 3]');").Check(testkit.Rows("1")) + tk.MustQuery("select json_overlaps('[{\"a\":1}]', '{\"a\":1}');").Check(testkit.Rows("1")) + tk.MustQuery("select json_overlaps('{\"a\":1}', '[{\"a\":1}]');").Check(testkit.Rows("1")) + tk.MustQuery("select json_overlaps('[1,[2,3]]', '[[1,2], 3]');").Check(testkit.Rows("0")) + tk.MustQuery("select json_overlaps('{\"a\":[1,2]}', '{\"a\":[2,1]}');").Check(testkit.Rows("0")) + tk.MustQuery("select json_overlaps('{\"a\":[1,2]}', '{\"a\":[2,1]}');").Check(testkit.Rows("0")) + // #10461 tk.MustExec("drop table if exists tx1") tk.MustExec("create table tx1(id int key, a double, b double, c double, d double)") tk.MustExec("insert into tx1 values (1, 0.1, 0.2, 0.3, 0.0)") tk.MustQuery("select a+b, c from tx1").Check(testkit.Rows("0.30000000000000004 0.3")) tk.MustQuery("select json_array(a+b) = json_array(c) from tx1").Check(testkit.Rows("0")) + + tk.MustQuery("SELECT '{\"a\":1}' MEMBER OF('{\"a\":1}');").Check(testkit.Rows("0")) + tk.MustQuery("SELECT '{\"a\":1}' MEMBER OF('[{\"a\":1}]');").Check(testkit.Rows("0")) + tk.MustQuery("SELECT 1 MEMBER OF('1');").Check(testkit.Rows("1")) + tk.MustQuery("SELECT '{\"a\":1}' MEMBER OF('{\"a\":1}');").Check(testkit.Rows("0")) + tk.MustQuery("SELECT '[4,5]' MEMBER OF('[[3,4],[4,5]]');").Check(testkit.Rows("0")) + tk.MustQuery("SELECT '[4,5]' MEMBER OF('[[3,4],\"[4,5]\"]');").Check(testkit.Rows("1")) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a enum('a', 'b'), b time, c binary(10))") + tk.MustExec("insert into t values ('a', '11:00:00', 'a')") + tk.MustQuery("select a member of ('\"a\"') from t").Check(testkit.Rows(`1`)) + tk.MustQuery("select b member of (json_array(cast('11:00:00' as time))) from t;").Check(testkit.Rows(`1`)) + tk.MustQuery("select b member of ('\"11:00:00\"') from t").Check(testkit.Rows(`0`)) + tk.MustQuery("select c member of ('\"a\"') from t").Check(testkit.Rows(`0`)) + err = tk.QueryToErr("select 'a' member of ('a')") + require.Error(t, err, "ERROR 3140 (22032): Invalid JSON text: The document root must not be followed by other values.") } func TestColumnInfoModified(t *testing.T) { @@ -3833,7 +3845,7 @@ func TestInvalidEndingStatement(t *testing.T) { errMsgLen := len(parseErrMsg) assertParseErr := func(sql string) { - _, err := tk.Exec(sql) + err := tk.ExecToErr(sql) require.Error(t, err) require.Equal(t, err.Error()[:errMsgLen], parseErrMsg) } @@ -4152,23 +4164,17 @@ func TestNotExistFunc(t *testing.T) { tk := testkit.NewTestKit(t, store) // current db is empty - _, err := tk.Exec("SELECT xxx(1)") - require.Error(t, err, "[planner:1046]No database selected") + tk.MustGetErrMsg("SELECT xxx(1)", "[planner:1046]No database selected") - _, err = tk.Exec("SELECT yyy()") - require.Error(t, err, "[planner:1046]No database selected") + tk.MustGetErrMsg("SELECT yyy()", "[planner:1046]No database selected") // current db is not empty tk.MustExec("use test") - _, err = tk.Exec("SELECT xxx(1)") - require.Error(t, err, "[expression:1305]FUNCTION test.xxx does not exist") - - _, err = tk.Exec("SELECT yyy()") - require.Error(t, err, "[expression:1305]FUNCTION test.yyy does not exist") + tk.MustGetErrMsg("SELECT xxx(1)", "[expression:1305]FUNCTION test.xxx does not exist") + tk.MustGetErrMsg("SELECT yyy()", "[expression:1305]FUNCTION test.yyy does not exist") tk.MustExec("use test") - _, err = tk.Exec("SELECT timestampliteral(rand())") - require.Error(t, err, "[expression:1305]FUNCTION test.timestampliteral does not exist") + tk.MustGetErrMsg("SELECT timestampliteral(rand())", "[expression:1305]FUNCTION test.timestampliteral does not exist") } func TestDecodetoChunkReuse(t *testing.T) { @@ -5951,8 +5957,7 @@ func TestSecurityEnhancedMode(t *testing.T) { // When SEM is enabled these features are restricted to all users // regardless of what privileges they have available. - _, err := tk.Exec("SELECT 1 INTO OUTFILE '/tmp/aaaa'") - require.Error(t, err, "[planner:8132]Feature 'SELECT INTO' is not supported when security enhanced mode is enabled") + tk.MustGetErrMsg("SELECT 1 INTO OUTFILE '/tmp/aaaa'", "[planner:8132]Feature 'SELECT INTO' is not supported when security enhanced mode is enabled") } func TestIssue23925(t *testing.T) { @@ -7145,29 +7150,25 @@ func TestIssue29708(t *testing.T) { tk.MustExec("use test;") tk.MustExec("drop table if exists t1;") tk.MustExec("CREATE TABLE t1 (a text)character set utf8 ;") - _, err := tk.Exec("INSERT INTO t1 VALUES (REPEAT(0125,200000000));") - require.NotNil(t, err) + tk.MustExecToErr("INSERT INTO t1 VALUES (REPEAT(0125,200000000));") tk.MustQuery("select * from t1").Check(nil) // test vectorized build-in function tk.MustExec("insert into t1 (a) values ('a'),('b');") - _, err = tk.Exec("insert into t1 select REPEAT(a,200000000) from t1;") - require.NotNil(t, err) + tk.MustExecToErr("insert into t1 select REPEAT(a,200000000) from t1;") tk.MustQuery("select a from t1 order by a;").Check([][]interface{}{ {"a"}, {"b"}, }) // test cast - _, err = tk.Exec(`insert into t1 values (cast("a" as binary(4294967295)));`) - require.NotNil(t, err) + tk.MustExecToErr(`insert into t1 values (cast("a" as binary(4294967295)));`) tk.MustQuery("select a from t1 order by a;").Check([][]interface{}{ {"a"}, {"b"}, }) - _, err = tk.Exec("INSERT IGNORE INTO t1 VALUES (REPEAT(0125,200000000));") - require.NoError(t, err) + tk.MustExec("INSERT IGNORE INTO t1 VALUES (REPEAT(0125,200000000));") tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1301 Result of repeat() was larger than max_allowed_packet (67108864) - truncated")) tk.MustQuery("select a from t1 order by a;").Check([][]interface{}{ {nil}, diff --git a/infoschema/BUILD.bazel b/infoschema/BUILD.bazel index 53bf35a2d43e3..37fd2bc423dc9 100644 --- a/infoschema/BUILD.bazel +++ b/infoschema/BUILD.bazel @@ -93,8 +93,10 @@ go_test( "//store/helper", "//store/mockstore", "//store/mockstore/mockstorage", + "//store/mockstore/unistore", "//table", "//testkit", + "//testkit/external", "//testkit/testsetup", "//testkit/testutil", "//types", @@ -104,14 +106,17 @@ go_test( "//util/pdapi", "//util/resourcegrouptag", "//util/set", + "//util/stmtsummary", "@com_github_gorilla_mux//:mux", "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", "@com_github_pingcap_fn//:fn", "@com_github_pingcap_kvproto//pkg/deadlock", + "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_tipb//go-tipb", "@com_github_prometheus_prometheus//promql", "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//testutils", "@org_golang_google_grpc//:grpc", "@org_uber_go_goleak//:goleak", ], diff --git a/infoschema/cluster_tables_test.go b/infoschema/cluster_tables_test.go index 5e323b2adc88a..00892c95cfe9c 100644 --- a/infoschema/cluster_tables_test.go +++ b/infoschema/cluster_tables_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/fn" "github.com/pingcap/kvproto/pkg/deadlock" + "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" @@ -42,14 +43,19 @@ import ( "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/helper" + "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mockstorage" + "github.com/pingcap/tidb/store/mockstore/unistore" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/testkit/external" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/resourcegrouptag" "github.com/pingcap/tidb/util/set" + "github.com/pingcap/tidb/util/stmtsummary" "github.com/pingcap/tipb/go-tipb" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/testutils" "google.golang.org/grpc" ) @@ -852,3 +858,173 @@ func TestMDLView(t *testing.T) { wg.Wait() } + +func TestCreateBindingFromHistory(t *testing.T) { + s := new(clusterTablesSuite) + s.store, s.dom = testkit.CreateMockStoreAndDomain(t) + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0", nil) + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + tk := s.newTestKitWithRoot(t) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t1(id int primary key, a int, b int, key(a))") + tk.MustExec("create table t2(id int primary key, a int, b int, key(a))") + + var testCases = []struct { + sqls []string + hint string + }{ + { + sqls: []string{ + "select %s * from t1, t2 where t1.id = t2.id", + "select %s * from test.t1, t2 where t1.id = t2.id", + "select %s * from test.t1, test.t2 where t1.id = t2.id", + "select %s * from t1, test.t2 where t1.id = t2.id", + }, + hint: "/*+ merge_join(t1, t2) */", + }, + { + sqls: []string{ + "select %s * from t1 where a = 1", + "select %s * from test.t1 where a = 1", + }, + hint: "/*+ ignore_index(t, a) */", + }, + } + + for _, testCase := range testCases { + for _, bind := range testCase.sqls { + stmtsummary.StmtSummaryByDigestMap.Clear() + bindSQL := fmt.Sprintf(bind, testCase.hint) + tk.MustExec(bindSQL) + planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", bindSQL)).Rows() + tk.MustExec(fmt.Sprintf("create session binding from history using plan digest '%s'", planDigest[0][0])) + showRes := tk.MustQuery("show bindings").Rows() + require.Equal(t, len(showRes), 1) + require.Equal(t, planDigest[0][0], showRes[0][10]) + for _, sql := range testCase.sqls { + tk.MustExec(fmt.Sprintf(sql, "")) + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + } + } + showRes := tk.MustQuery("show bindings").Rows() + require.Equal(t, len(showRes), 1) + tk.MustExec(fmt.Sprintf("drop binding for sql digest '%s'", showRes[0][9])) + } + + // exception cases + tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", "1"), "can't find any plans for '1'") + tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", ""), "plan digest is empty") + tk.MustExec("create binding for select * from t1, t2 where t1.id = t2.id using select /*+ merge_join(t1, t2) */ * from t1, t2 where t1.id = t2.id") + showRes := tk.MustQuery("show bindings").Rows() + require.Equal(t, showRes[0][10], "") // plan digest should be nil by create for +} + +func TestCreateBindingForPrepareFromHistory(t *testing.T) { + s := new(clusterTablesSuite) + s.store, s.dom = testkit.CreateMockStoreAndDomain(t) + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0", nil) + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + tk := s.newTestKitWithRoot(t) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int primary key, a int, key(a))") + + tk.MustExec("prepare stmt from 'select /*+ ignore_index(t,a) */ * from t where a = ?'") + tk.MustExec("set @a = 1") + tk.MustExec("execute stmt using @a") + planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", "select /*+ ignore_index(t,a) */ * from t where a = ? [arguments: 1]")).Rows() + showRes := tk.MustQuery("show bindings").Rows() + require.Equal(t, len(showRes), 0) + tk.MustExec(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0])) + showRes = tk.MustQuery("show bindings").Rows() + require.Equal(t, len(showRes), 1) + require.Equal(t, planDigest[0][0], showRes[0][10]) + tk.MustExec("execute stmt using @a") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) +} + +func TestErrorCasesCreateBindingFromHistory(t *testing.T) { + s := new(clusterTablesSuite) + s.store, s.dom = testkit.CreateMockStoreAndDomain(t) + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0", nil) + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + tk := s.newTestKitWithRoot(t) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2, t3") + tk.MustExec("create table t1(id int)") + tk.MustExec("create table t2(id int)") + tk.MustExec("create table t3(id int)") + + sql := "select * from t1 where t1.id in (select id from t2)" + tk.MustExec(sql) + planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows() + tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with sub query") + + sql = "select * from t1, t2, t3 where t1.id = t2.id and t2.id = t3.id" + tk.MustExec(sql) + planDigest = tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows() + tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with more than two table join") +} + +// withMockTiFlash sets the mockStore to have N TiFlash stores (naming as tiflash0, tiflash1, ...). +func withMockTiFlash(nodes int) mockstore.MockTiKVStoreOption { + return mockstore.WithMultipleOptions( + mockstore.WithClusterInspector(func(c testutils.Cluster) { + mockCluster := c.(*unistore.Cluster) + _, _, region1 := mockstore.BootstrapWithSingleStore(c) + tiflashIdx := 0 + for tiflashIdx < nodes { + store2 := c.AllocID() + peer2 := c.AllocID() + addr2 := fmt.Sprintf("tiflash%d", tiflashIdx) + mockCluster.AddStore(store2, addr2, &metapb.StoreLabel{Key: "engine", Value: "tiflash"}) + mockCluster.AddPeer(region1, store2, peer2) + tiflashIdx++ + } + }), + mockstore.WithStoreType(mockstore.EmbedUnistore), + ) +} + +func TestBindingFromHistoryWithTiFlashBindable(t *testing.T) { + s := new(clusterTablesSuite) + s.store, s.dom = testkit.CreateMockStoreAndDomain(t) + s.store = testkit.CreateMockStore(t, withMockTiFlash(2)) + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0", nil) + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + tk := s.newTestKitWithRoot(t) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + + tk.MustExec("use test;") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int);") + tk.MustExec("alter table test.t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_isolation_read_engines = 'tiflash'") + + sql := "select * from t" + tk.MustExec(sql) + planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows() + tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with tiflash engine") +} diff --git a/infoschema/tables.go b/infoschema/tables.go index d76d8b8be60b0..2f481d4e004b5 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1323,6 +1323,9 @@ var tableStatementsSummaryCols = []columnInfo{ {name: stmtsummary.PlanDigestStr, tp: mysql.TypeVarchar, size: 64, comment: "Digest of its execution plan"}, {name: stmtsummary.PlanStr, tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Sampled execution plan"}, {name: stmtsummary.BinaryPlan, tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Sampled binary plan"}, + {name: stmtsummary.Charset, tp: mysql.TypeVarchar, size: 64, comment: "Sampled charset"}, + {name: stmtsummary.Collation, tp: mysql.TypeVarchar, size: 64, comment: "Sampled collation"}, + {name: stmtsummary.PlanHint, tp: mysql.TypeVarchar, size: 64, comment: "Sampled plan hint"}, } var tableStorageStatsCols = []columnInfo{ diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index bd6168d035873..ebf3e9b535893 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1661,3 +1661,33 @@ func TestMemoryUsageAndOpsHistory(t *testing.T) { require.Equal(t, row[10], "e3237ec256015a3566757e0c2742507cd30ae04e4cac2fbc14d269eafe7b067b") // SQL_DIGEST require.Equal(t, row[11], "explain analyze select * from t t1 join t t2 join t t3 on t1.a=t2.a and t1.a=t3.a order by t1.a") // SQL_TEXT } + +func TestAddFieldsForBinding(t *testing.T) { + s := new(clusterTablesSuite) + s.store, s.dom = testkit.CreateMockStoreAndDomain(t) + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0", nil) + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + tk := s.newTestKitWithRoot(t) + + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, key(a))") + tk.MustExec("select /*+ ignore_index(t, a)*/ * from t where a = 1") + planDigest := "4e3159169cc63c14b139a4e7d72eae1759875c9a9581f94bb2079aae961189cb" + rows := tk.MustQuery(fmt.Sprintf("select stmt_type, prepared, sample_user, schema_name, query_sample_text, charset, collation, plan_hint, digest_text "+ + "from information_schema.cluster_statements_summary where plan_digest = '%s'", planDigest)).Rows() + + require.Equal(t, rows[0][0], "Select") + require.Equal(t, rows[0][1], "0") + require.Equal(t, rows[0][2], "root") + require.Equal(t, rows[0][3], "test") + require.Equal(t, rows[0][4], "select /*+ ignore_index(t, a)*/ * from t where a = 1") + require.Equal(t, rows[0][5], "utf8mb4") + require.Equal(t, rows[0][6], "utf8mb4_bin") + require.Equal(t, rows[0][7], "use_index(@`sel_1` `test`.`t` ), ignore_index(`t` `a`)") + require.Equal(t, rows[0][8], "select * from `t` where `a` = ?") +} diff --git a/metrics/BUILD.bazel b/metrics/BUILD.bazel index ff37513801ede..4b6b6cf9f2ee1 100644 --- a/metrics/BUILD.bazel +++ b/metrics/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "stats.go", "telemetry.go", "topsql.go", + "ttl.go", ], importpath = "github.com/pingcap/tidb/metrics", visibility = ["//visibility:public"], diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index bd686ffd05c92..7677a718e156f 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -16985,6 +16985,844 @@ ], "title": "SourceSQL", "type": "row" + }, + { + "collapsed": false, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 75 + }, + "id": 274, + "panels": [], + "title": "TTL", + "type": "row" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The query count per second for each type of query in TTL jobs", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 76 + }, + "hiddenSeries": false, + "id": 279, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "delete ok", + "color": "#73BF69" + }, + { + "alias": "select ok", + "color": "#5794F2" + }, + { + "alias": "delete error", + "color": "#F2495C" + }, + { + "alias": "select error", + "color": "#FF7383" + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tidb_server_ttl_query_duration_count{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (sql_type, result)", + "interval": "", + "legendFormat": "{{sql_type}} {{result}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "TTL QPS By Type", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The processed rows per second by TTL jobs", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 76 + }, + "hiddenSeries": false, + "id": 287, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "delete error", + "color": "#F2495C" + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tidb_server_ttl_processed_expired_rows{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (sql_type, result)", + "interval": "", + "legendFormat": "{{sql_type}} {{result}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "TTL Processed Rows Per Second", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The duration of the TTL scan queries", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 84 + }, + "hiddenSeries": false, + "id": 284, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.50, sum(rate(tidb_server_ttl_query_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", sql_type=\"select\", result=\"ok\"}[1m])) by (le))", + "interval": "", + "legendFormat": "50", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.80, sum(rate(tidb_server_ttl_query_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", sql_type=\"select\", result=\"ok\"}[1m])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "80", + "refId": "B" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.90, sum(rate(tidb_server_ttl_query_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", sql_type=\"select\", result=\"ok\"}[1m])) by (le))\n", + "hide": false, + "interval": "", + "legendFormat": "90", + "refId": "C" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_ttl_query_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", sql_type=\"select\", result=\"ok\"}[1m])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "99", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "TTL Scan Query Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The duration of the TTL delete queries", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 84 + }, + "hiddenSeries": false, + "id": 285, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.50, sum(rate(tidb_server_ttl_query_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", sql_type=\"delete\", result=\"ok\"}[1m])) by (le))", + "interval": "", + "legendFormat": "50", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.80, sum(rate(tidb_server_ttl_query_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", sql_type=\"delete\", result=\"ok\"}[1m])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "80", + "refId": "B" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.90, sum(rate(tidb_server_ttl_query_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", sql_type=\"delete\", result=\"ok\"}[1m])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "90", + "refId": "C" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(tidb_server_ttl_query_duration_bucket{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", sql_type=\"delete\", result=\"ok\"}[1m])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "99", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "TTL Delete Query Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time spent on each phase for scan workers", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 92 + }, + "hiddenSeries": false, + "id": 276, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": false, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "idle", + "color": "#73BF69" + }, + { + "alias": "query", + "color": "#FADE2A" + }, + { + "alias": "begin_txn", + "color": "#FFA6B0" + }, + { + "alias": "commit_txn", + "color": "#FF7383" + }, + { + "alias": "wait_retry", + "color": "#FF9830" + }, + { + "alias": "check_ttl", + "color": "#C4162A" + }, + { + "alias": "dispatch", + "color": "#8F3BB8" + }, + { + "alias": "wait_token", + "color": "#8AB8FF" + } + ], + "spaceLength": 10, + "stack": true, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tidb_server_ttl_phase_time{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"scan_worker\"}[1m])) by (phase)", + "interval": "", + "legendFormat": "{{phase}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Scan Worker Time By Phase", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time spent on each phase for delete workers", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 92 + }, + "hiddenSeries": false, + "id": 282, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": false, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "idle", + "color": "#73BF69" + }, + { + "alias": "query", + "color": "#FADE2A" + }, + { + "alias": "begin_txn", + "color": "#FFA6B0" + }, + { + "alias": "commit_txn", + "color": "#FF7383" + }, + { + "alias": "wait_retry", + "color": "#FF9830" + }, + { + "alias": "check_ttl", + "color": "#C4162A" + }, + { + "alias": "dispatch", + "color": "#8F3BB8" + }, + { + "alias": "wait_token", + "color": "#8AB8FF" + } + ], + "spaceLength": 10, + "stack": true, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tidb_server_ttl_phase_time{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=\"delete_worker\"}[1m])) by (phase)\n", + "interval": "", + "legendFormat": "{{phase}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Delete Worker Time By Phase", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The TTL job statuses in each worker", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 100 + }, + "hiddenSeries": false, + "id": 281, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.10", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "running", + "color": "#5794F2" + }, + { + "alias": "cancelling", + "color": "#F2495C" + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(tidb_server_ttl_job_status{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type, instance)", + "interval": "", + "legendFormat": "{{ instance }} {{ type }}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "TTL Job Count By Status", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "refresh": "30s", diff --git a/metrics/metrics.go b/metrics/metrics.go index 93b278bf87d32..2984b66ddb27c 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -210,6 +210,11 @@ func RegisterMetrics() { prometheus.MustRegister(RegionCheckpointSubscriptionEvent) prometheus.MustRegister(RCCheckTSWriteConfilictCounter) + prometheus.MustRegister(TTLQueryDuration) + prometheus.MustRegister(TTLProcessedExpiredRowsCounter) + prometheus.MustRegister(TTLJobStatus) + prometheus.MustRegister(TTLPhaseTime) + tikvmetrics.InitMetrics(TiDB, TiKVClient) tikvmetrics.RegisterMetrics() tikvmetrics.TiKVPanicCounter = PanicCounter // reset tidb metrics for tikv metrics diff --git a/metrics/ttl.go b/metrics/ttl.go new file mode 100644 index 0000000000000..ab7e47e615e28 --- /dev/null +++ b/metrics/ttl.go @@ -0,0 +1,53 @@ +// 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 metrics + +import "github.com/prometheus/client_golang/prometheus" + +// TTL metrics +var ( + TTLQueryDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "ttl_query_duration", + Help: "Bucketed histogram of processing time (s) of handled TTL queries.", + Buckets: prometheus.ExponentialBuckets(0.01, 2, 20), // 10ms ~ 1.45hour + }, []string{LblSQLType, LblResult}) + + TTLProcessedExpiredRowsCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "ttl_processed_expired_rows", + Help: "The count of expired rows processed in TTL jobs", + }, []string{LblSQLType, LblResult}) + + TTLJobStatus = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "ttl_job_status", + Help: "The jobs count in the specified status", + }, []string{LblType}) + + TTLPhaseTime = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "ttl_phase_time", + Help: "The time spent in each phase", + }, []string{LblType, LblPhase}) +) diff --git a/parser/BUILD.bazel b/parser/BUILD.bazel index e6b5bed7c4075..f52b1fc9ac4f3 100644 --- a/parser/BUILD.bazel +++ b/parser/BUILD.bazel @@ -9,7 +9,6 @@ go_library( "lexer.go", "misc.go", "parser.go", - "ttlfeaturegate.go", "yy_parser.go", ], importpath = "github.com/pingcap/tidb/parser", diff --git a/parser/ast/ddl_test.go b/parser/ast/ddl_test.go index fed5c1c759fbf..e6107f34513ec 100644 --- a/parser/ast/ddl_test.go +++ b/parser/ast/ddl_test.go @@ -16,7 +16,6 @@ package ast_test import ( "testing" - "github.com/pingcap/tidb/parser" . "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/format" "github.com/stretchr/testify/require" @@ -843,8 +842,6 @@ func TestFlashBackDatabaseRestore(t *testing.T) { } func TestTableOptionTTLRestore(t *testing.T) { - parser.TTLFeatureGate = true - sourceSQL1 := "create table t (created_at datetime) ttl = created_at + INTERVAL 1 YEAR" sourceSQL2 := "alter table t ttl_enable = 'OFF'" sourceSQL3 := "alter table t remove ttl" diff --git a/parser/ast/functions.go b/parser/ast/functions.go index d33550fc67626..fdedf53b701cf 100644 --- a/parser/ast/functions.go +++ b/parser/ast/functions.go @@ -331,6 +331,7 @@ const ( JSONInsert = "json_insert" JSONReplace = "json_replace" JSONRemove = "json_remove" + JSONOverlaps = "json_overlaps" JSONContains = "json_contains" JSONMemberOf = "json_memberof" JSONContainsPath = "json_contains_path" diff --git a/parser/consistent_test.go b/parser/consistent_test.go index e78b7f31ddddd..1acc1a58bc850 100644 --- a/parser/consistent_test.go +++ b/parser/consistent_test.go @@ -14,7 +14,7 @@ package parser import ( - "io/ioutil" + gio "io" "os" "sort" "strings" @@ -27,7 +27,7 @@ func TestKeywordConsistent(t *testing.T) { parserFilename := "parser.y" parserFile, err := os.Open(parserFilename) requires.NoError(t, err) - data, err := ioutil.ReadAll(parserFile) + data, err := gio.ReadAll(parserFile) requires.NoError(t, err) content := string(data) diff --git a/parser/format/format_test.go b/parser/format/format_test.go index 429c2c27c19d1..bc04033214cdd 100644 --- a/parser/format/format_test.go +++ b/parser/format/format_test.go @@ -15,7 +15,7 @@ package format import ( "bytes" - "io/ioutil" + "io" "strings" "testing" @@ -26,7 +26,7 @@ import ( func checkFormat(t *testing.T, f Formatter, buf *bytes.Buffer, str, expect string) { _, err := f.Format(str, 3) require.NoError(t, err) - b, err := ioutil.ReadAll(buf) + b, err := io.ReadAll(buf) require.NoError(t, err) require.Equal(t, expect, string(b)) } diff --git a/parser/parser.go b/parser/parser.go index 4e1c07a31509e..05592f9fac45e 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -12120,10 +12120,6 @@ yynewstate: } case 46: { - if !TTLFeatureGate { - yylex.AppendError(ErrSyntax) - return 1 - } parser.yyVAL.item = &ast.AlterTableSpec{ Tp: ast.AlterTableRemoveTTL, } @@ -20251,10 +20247,6 @@ yynewstate: } case 2176: { - if !TTLFeatureGate { - yylex.AppendError(ErrSyntax) - return 1 - } parser.yyVAL.item = &ast.TableOption{ Tp: ast.TableOptionTTL, ColumnName: &ast.ColumnName{Name: model.NewCIStr(yyS[yypt-4].ident)}, @@ -20264,10 +20256,6 @@ yynewstate: } case 2177: { - if !TTLFeatureGate { - yylex.AppendError(ErrSyntax) - return 1 - } onOrOff := strings.ToLower(yyS[yypt-0].ident) if onOrOff == "on" { parser.yyVAL.item = &ast.TableOption{Tp: ast.TableOptionTTLEnable, BoolValue: true} diff --git a/parser/parser.y b/parser/parser.y index 6b60e9d329623..979815a74ef7c 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -1793,10 +1793,6 @@ AlterTableSpecSingleOpt: } | "REMOVE" "TTL" { - if !TTLFeatureGate { - yylex.AppendError(ErrSyntax) - return 1 - } $$ = &ast.AlterTableSpec{ Tp: ast.AlterTableRemoveTTL, } @@ -11863,10 +11859,6 @@ TableOption: } | "TTL" EqOpt Identifier '+' "INTERVAL" Literal TimeUnit { - if !TTLFeatureGate { - yylex.AppendError(ErrSyntax) - return 1 - } $$ = &ast.TableOption{ Tp: ast.TableOptionTTL, ColumnName: &ast.ColumnName{Name: model.NewCIStr($3)}, @@ -11876,10 +11868,6 @@ TableOption: } | "TTL_ENABLE" EqOpt stringLit { - if !TTLFeatureGate { - yylex.AppendError(ErrSyntax) - return 1 - } onOrOff := strings.ToLower($3) if onOrOff == "on" { $$ = &ast.TableOption{Tp: ast.TableOptionTTLEnable, BoolValue: true} diff --git a/parser/parser_test.go b/parser/parser_test.go index b6d744dd19dc9..1a29b6388b143 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -7057,8 +7057,6 @@ func TestIntervalPartition(t *testing.T) { } func TestTTLTableOption(t *testing.T) { - parser.TTLFeatureGate = true - table := []testCase{ // create table with various temporal interval {"create table t (created_at datetime) TTL = created_at + INTERVAL 3.1415 YEAR", true, "CREATE TABLE `t` (`created_at` DATETIME) TTL = `created_at` + INTERVAL 3.1415 YEAR"}, @@ -7086,21 +7084,3 @@ func TestTTLTableOption(t *testing.T) { RunTest(t, table, false) } - -func TestTTLFeatureGate(t *testing.T) { - parser.TTLFeatureGate = false - - table := []testCase{ - {"create table t (created_at datetime) TTL = created_at + INTERVAL 3.1415 YEAR", false, ""}, - {"create table t (created_at datetime) TTL_ENABLE = 'OFF'", false, ""}, - {"create table t (created_at datetime) TTL created_at + INTERVAL 1 YEAR TTL_ENABLE 'OFF'", false, ""}, - {"create table t (created_at datetime) /*T![ttl] ttl=created_at + INTERVAL 1 YEAR ttl_enable='ON'*/", false, ""}, - {"alter table t TTL = created_at + INTERVAL 1 MONTH", false, ""}, - {"alter table t TTL_ENABLE = 'ON'", false, ""}, - {"alter table t TTL = created_at + INTERVAL 1 MONTH TTL_ENABLE 'OFF'", false, ""}, - {"alter table t /*T![ttl] ttl=created_at + INTERVAL 1 YEAR ttl_enable='ON'*/", false, ""}, - {"alter table t remove ttl", false, ""}, - } - - RunTest(t, table, false) -} diff --git a/parser/reserved_words_test.go b/parser/reserved_words_test.go index f58359a7d20a3..6896167741284 100644 --- a/parser/reserved_words_test.go +++ b/parser/reserved_words_test.go @@ -28,7 +28,7 @@ import ( // needed to connect to MySQL dbsql "database/sql" - "io/ioutil" + gio "io" "os" "testing" @@ -41,7 +41,7 @@ func TestCompareReservedWordsWithMySQL(t *testing.T) { parserFilename := "parser.y" parserFile, err := os.Open(parserFilename) requires.NoError(t, err) - data, err := ioutil.ReadAll(parserFile) + data, err := gio.ReadAll(parserFile) requires.NoError(t, err) content := string(data) diff --git a/parser/ttlfeaturegate.go b/parser/ttlfeaturegate.go deleted file mode 100644 index d18633f137afe..0000000000000 --- a/parser/ttlfeaturegate.go +++ /dev/null @@ -1,17 +0,0 @@ -// Copyright 2019 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package parser - -// TTLFeatureGate determines whether to enable the ttl related syntax in parser -var TTLFeatureGate = false diff --git a/planner/core/binary_plan_test.go b/planner/core/binary_plan_test.go index 806e750d33fac..95c67990b09be 100644 --- a/planner/core/binary_plan_test.go +++ b/planner/core/binary_plan_test.go @@ -17,7 +17,7 @@ package core_test import ( "encoding/base64" "fmt" - "io/ioutil" + "io" "os" "regexp" "strings" @@ -77,13 +77,6 @@ func simplifyAndCheckBinaryOperator(t *testing.T, pb *tipb.ExplainOperator, with } func TestBinaryPlanInExplainAndSlowLog(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - tk.MustExec("set tidb_cost_model_version=2") - // 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") - // Prepare the slow log originCfg := config.GetGlobalConfig() newCfg := *originCfg @@ -97,6 +90,12 @@ func TestBinaryPlanInExplainAndSlowLog(t *testing.T) { require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) }() require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + // 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") tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustExec("set tidb_slow_log_threshold=0;") defer func() { @@ -149,10 +148,6 @@ func TestBinaryPlanInExplainAndSlowLog(t *testing.T) { } func TestBinaryPlanSwitch(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) - originCfg := config.GetGlobalConfig() newCfg := *originCfg f, err := os.CreateTemp("", "tidb-slow-*.log") @@ -165,6 +160,9 @@ func TestBinaryPlanSwitch(t *testing.T) { require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) }() require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustExec("use test") @@ -219,10 +217,6 @@ func TestBinaryPlanSwitch(t *testing.T) { // TestTooLongBinaryPlan asserts that if the binary plan is larger than 1024*1024 bytes, it should be output to slow query but not to stmt summary. func TestTooLongBinaryPlan(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) - originCfg := config.GetGlobalConfig() newCfg := *originCfg f, err := os.CreateTemp("", "tidb-slow-*.log") @@ -235,6 +229,9 @@ func TestTooLongBinaryPlan(t *testing.T) { require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) }() require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustExec("use test") @@ -336,10 +333,6 @@ func TestLongBinaryPlan(t *testing.T) { } func TestBinaryPlanOfPreparedStmt(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) - originCfg := config.GetGlobalConfig() newCfg := *originCfg f, err := os.CreateTemp("", "tidb-slow-*.log") @@ -352,6 +345,9 @@ func TestBinaryPlanOfPreparedStmt(t *testing.T) { require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) }() require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustExec("use test") @@ -389,10 +385,6 @@ func TestBinaryPlanOfPreparedStmt(t *testing.T) { // TestDecodeBinaryPlan asserts that the result of EXPLAIN ANALYZE FORMAT = 'verbose' is the same as tidb_decode_binary_plan(). func TestDecodeBinaryPlan(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - tk.MustExec("use test") - // Prepare the slow log originCfg := config.GetGlobalConfig() newCfg := *originCfg @@ -406,6 +398,9 @@ func TestDecodeBinaryPlan(t *testing.T) { require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) }() require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustExec("set tidb_slow_log_threshold=0;") defer func() { @@ -499,10 +494,6 @@ func TestInvalidDecodeBinaryPlan(t *testing.T) { } func TestUnnecessaryBinaryPlanInSlowLog(t *testing.T) { - store := testkit.CreateMockStore(t) - tk := testkit.NewTestKit(t, store) - require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) - originCfg := config.GetGlobalConfig() newCfg := *originCfg f, err := os.CreateTemp("", "tidb-slow-*.log") @@ -515,13 +506,16 @@ func TestUnnecessaryBinaryPlanInSlowLog(t *testing.T) { require.NoError(t, os.Remove(newCfg.Log.SlowQueryFile)) }() require.NoError(t, logutil.InitLogger(newCfg.Log.ToLogConfig())) + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", f.Name())) tk.MustExec("use test") tk.MustExec("drop table if exists th") tk.MustExec("set global tidb_slow_log_threshold = 1;") tk.MustExec("create table th (i int, a int,b int, c int, index (a)) partition by hash (a) partitions 100;") - slowLogBytes, err := ioutil.ReadAll(f) + slowLogBytes, err := io.ReadAll(f) require.NoError(t, err) require.NotContains(t, string(slowLogBytes), `tidb_decode_binary_plan('')`) } diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index af4aa836d335a..8448a66cd0bf0 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -57,6 +57,34 @@ func loadTableStats(fileName string, dom *domain.Domain) error { return nil } +func TestExplainCostTrace(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("insert into t values (1)") + + tk.MustExec("set tidb_cost_model_version=2") + tk.MustQuery("explain format='cost_trace' select * from t").Check(testkit.Rows( + `TableReader_5 10000.00 177906.67 ((scan(10000*logrowsize(32)*tikv_scan_factor(40.7))) + (net(10000*rowsize(16)*tidb_kv_net_factor(3.96))))/15.00 root data:TableFullScan_4`, + `└─TableFullScan_4 10000.00 2035000.00 scan(10000*logrowsize(32)*tikv_scan_factor(40.7)) cop[tikv] table:t keep order:false, stats:pseudo`)) + tk.MustQuery("explain analyze format='cost_trace' select * from t").CheckAt([]int{0, 1, 2, 3, 4}, [][]interface{}{ + {"TableReader_5", "10000.00", "177906.67", "((scan(10000*logrowsize(32)*tikv_scan_factor(40.7))) + (net(10000*rowsize(16)*tidb_kv_net_factor(3.96))))/15.00", "1"}, + {"└─TableFullScan_4", "10000.00", "2035000.00", "scan(10000*logrowsize(32)*tikv_scan_factor(40.7))", "1"}, + }) + + tk.MustExec("set tidb_cost_model_version=1") + tk.MustQuery("explain format='cost_trace' select * from t").Check(testkit.Rows( + // cost trace on model ver1 is not supported + `TableReader_5 10000.00 34418.00 N/A root data:TableFullScan_4`, + `└─TableFullScan_4 10000.00 435000.00 N/A cop[tikv] table:t keep order:false, stats:pseudo`, + )) + tk.MustQuery("explain analyze format='cost_trace' select * from t").CheckAt([]int{0, 1, 2, 3, 4}, [][]interface{}{ + {"TableReader_5", "10000.00", "34418.00", "N/A", "1"}, + {"└─TableFullScan_4", "10000.00", "435000.00", "N/A", "1"}, + }) +} + func TestExplainAnalyze(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 73bb25e0d874c..f11f60b95cfe5 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -742,6 +742,12 @@ func (e *Explain) prepareSchema() error { } case format == types.ExplainFormatTrueCardCost: fieldNames = []string{"id", "estRows", "estCost", "costFormula", "actRows", "task", "access object", "execution info", "operator info", "memory", "disk"} + case format == types.ExplainFormatCostTrace: + if e.Analyze || e.RuntimeStatsColl != nil { + fieldNames = []string{"id", "estRows", "estCost", "costFormula", "actRows", "task", "access object", "execution info", "operator info", "memory", "disk"} + } else { + fieldNames = []string{"id", "estRows", "estCost", "costFormula", "task", "access object", "operator info"} + } case (format == types.ExplainFormatROW || format == types.ExplainFormatBrief) && (e.Analyze || e.RuntimeStatsColl != nil): fieldNames = []string{"id", "estRows", "actRows", "task", "access object", "execution info", "operator info", "memory", "disk"} case format == types.ExplainFormatDOT: @@ -813,8 +819,18 @@ func (e *Explain) RenderResult() error { } } + if strings.ToLower(e.Format) == types.ExplainFormatCostTrace { + if pp, ok := e.TargetPlan.(PhysicalPlan); ok { + // trigger getPlanCost again with CostFlagTrace to record all cost formulas + if _, err := getPlanCost(pp, property.RootTaskType, + NewDefaultPlanCostOption().WithCostFlag(CostFlagRecalculate|CostFlagTrace)); err != nil { + return err + } + } + } + switch strings.ToLower(e.Format) { - case types.ExplainFormatROW, types.ExplainFormatBrief, types.ExplainFormatVerbose, types.ExplainFormatTrueCardCost: + case types.ExplainFormatROW, types.ExplainFormatBrief, types.ExplainFormatVerbose, types.ExplainFormatTrueCardCost, types.ExplainFormatCostTrace: if e.Rows == nil || e.Analyze { flat := FlattenPhysicalPlan(e.TargetPlan, true) e.explainFlatPlanInRowFormat(flat) @@ -995,19 +1011,23 @@ func (e *Explain) prepareOperatorInfo(p Plan, taskType, id string) { var row []string if e.Analyze || e.RuntimeStatsColl != nil { row = []string{id, estRows} - if strings.ToLower(e.Format) == types.ExplainFormatVerbose || strings.ToLower(e.Format) == types.ExplainFormatTrueCardCost { + if strings.ToLower(e.Format) == types.ExplainFormatVerbose || strings.ToLower(e.Format) == types.ExplainFormatTrueCardCost || strings.ToLower(e.Format) == types.ExplainFormatCostTrace { row = append(row, estCost) } - if strings.ToLower(e.Format) == types.ExplainFormatTrueCardCost { + if strings.ToLower(e.Format) == types.ExplainFormatTrueCardCost || strings.ToLower(e.Format) == types.ExplainFormatCostTrace { row = append(row, costFormula) } actRows, analyzeInfo, memoryInfo, diskInfo := getRuntimeInfoStr(e.ctx, p, e.RuntimeStatsColl) row = append(row, actRows, taskType, accessObject, analyzeInfo, operatorInfo, memoryInfo, diskInfo) } else { row = []string{id, estRows} - if strings.ToLower(e.Format) == types.ExplainFormatVerbose || strings.ToLower(e.Format) == types.ExplainFormatTrueCardCost { + if strings.ToLower(e.Format) == types.ExplainFormatVerbose || strings.ToLower(e.Format) == types.ExplainFormatTrueCardCost || + strings.ToLower(e.Format) == types.ExplainFormatCostTrace { row = append(row, estCost) } + if strings.ToLower(e.Format) == types.ExplainFormatCostTrace { + row = append(row, costFormula) + } row = append(row, taskType, accessObject, operatorInfo) } e.Rows = append(e.Rows, row) diff --git a/planner/core/plan_cache.go b/planner/core/plan_cache.go index e58e8b6d91708..d9df658aaa46e 100644 --- a/planner/core/plan_cache.go +++ b/planner/core/plan_cache.go @@ -42,7 +42,7 @@ import ( "go.uber.org/zap" ) -func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) error { +func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) error { vars := sctx.GetSessionVars() stmtAst := stmt.PreparedAst vars.StmtCtx.StmtType = stmtAst.StmtType @@ -101,7 +101,7 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral // And update lastUpdateTime to the newest one. expiredTimeStamp4PC := domain.GetDomain(sctx).ExpiredTimeStamp4PC() if stmtAst.UseCache && expiredTimeStamp4PC.Compare(vars.LastUpdateTime4PC) > 0 { - sctx.GetPlanCache(isGeneralPlanCache).DeleteAll() + sctx.GetPlanCache(isNonPrepared).DeleteAll() stmtAst.CachedPlan = nil vars.LastUpdateTime4PC = expiredTimeStamp4PC } @@ -111,11 +111,11 @@ func planCachePreprocess(ctx context.Context, sctx sessionctx.Context, isGeneral // GetPlanFromSessionPlanCache is the entry point of Plan Cache. // It tries to get a valid cached plan from this session's plan cache. // If there is no such a plan, it'll call the optimizer to generate a new one. -// isGeneralPlanCache indicates whether to use the general plan cache or the prepared plan cache. +// isNonPrepared indicates whether to use the non-prepared plan cache or the prepared plan cache. func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, - isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, + isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, params []expression.Expression) (plan Plan, names []*types.FieldName, err error) { - if err := planCachePreprocess(ctx, sctx, isGeneralPlanCache, is, stmt, params); err != nil { + if err := planCachePreprocess(ctx, sctx, isNonPrepared, is, stmt, params); err != nil { return nil, nil, err } @@ -149,19 +149,19 @@ func GetPlanFromSessionPlanCache(ctx context.Context, sctx sessionctx.Context, paramNum, paramTypes := parseParamTypes(sctx, params) if stmtAst.UseCache && stmtAst.CachedPlan != nil && !ignorePlanCache { // for point query plan - if plan, names, ok, err := getPointQueryPlan(stmtAst, sessVars, stmtCtx); ok { + if plan, names, ok, err := getCachedPointPlan(stmtAst, sessVars, stmtCtx); ok { return plan, names, err } } - if stmtAst.UseCache && !ignorePlanCache { // for general plans - if plan, names, ok, err := getGeneralPlan(sctx, isGeneralPlanCache, cacheKey, bindSQL, is, stmt, + if stmtAst.UseCache && !ignorePlanCache { // for non-point plans + if plan, names, ok, err := getCachedPlan(sctx, isNonPrepared, cacheKey, bindSQL, is, stmt, paramTypes); err != nil || ok { return plan, names, err } } - return generateNewPlan(ctx, sctx, isGeneralPlanCache, is, stmt, ignorePlanCache, cacheKey, + return generateNewPlan(ctx, sctx, isNonPrepared, is, stmt, ignorePlanCache, cacheKey, latestSchemaVersion, paramNum, paramTypes, bindSQL) } @@ -185,7 +185,7 @@ func parseParamTypes(sctx sessionctx.Context, params []expression.Expression) (p return } -func getPointQueryPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (Plan, +func getCachedPointPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmtCtx *stmtctx.StatementContext) (Plan, []*types.FieldName, bool, error) { // short path for point-get plans // Rewriting the expression in the select.where condition will convert its @@ -209,13 +209,13 @@ func getPointQueryPlan(stmt *ast.Prepared, sessVars *variable.SessionVars, stmtC return plan, names, true, nil } -func getGeneralPlan(sctx sessionctx.Context, isGeneralPlanCache bool, cacheKey kvcache.Key, bindSQL string, +func getCachedPlan(sctx sessionctx.Context, isNonPrepared bool, cacheKey kvcache.Key, bindSQL string, is infoschema.InfoSchema, stmt *PlanCacheStmt, paramTypes []*types.FieldType) (Plan, []*types.FieldName, bool, error) { sessVars := sctx.GetSessionVars() stmtCtx := sessVars.StmtCtx - candidate, exist := sctx.GetPlanCache(isGeneralPlanCache).Get(cacheKey, paramTypes) + candidate, exist := sctx.GetPlanCache(isNonPrepared).Get(cacheKey, paramTypes) if !exist { return nil, nil, false, nil } @@ -227,7 +227,7 @@ func getGeneralPlan(sctx sessionctx.Context, isGeneralPlanCache bool, cacheKey k if !unionScan && tableHasDirtyContent(sctx, tblInfo) { // TODO we can inject UnionScan into cached plan to avoid invalidating it, though // rebuilding the filters in UnionScan is pretty trivial. - sctx.GetPlanCache(isGeneralPlanCache).Delete(cacheKey) + sctx.GetPlanCache(isNonPrepared).Delete(cacheKey) return nil, nil, false, nil } } @@ -253,7 +253,7 @@ func getGeneralPlan(sctx sessionctx.Context, isGeneralPlanCache bool, cacheKey k // generateNewPlan call the optimizer to generate a new plan for current statement // and try to add it to cache -func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isGeneralPlanCache bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, +func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isNonPrepared bool, is infoschema.InfoSchema, stmt *PlanCacheStmt, ignorePlanCache bool, cacheKey kvcache.Key, latestSchemaVersion int64, paramNum int, paramTypes []*types.FieldType, bindSQL string) (Plan, []*types.FieldName, error) { stmtAst := stmt.PreparedAst @@ -290,7 +290,7 @@ func generateNewPlan(ctx context.Context, sctx sessionctx.Context, isGeneralPlan stmt.NormalizedPlan, stmt.PlanDigest = NormalizePlan(p) stmtCtx.SetPlan(p) stmtCtx.SetPlanDigest(stmt.NormalizedPlan, stmt.PlanDigest) - sctx.GetPlanCache(isGeneralPlanCache).Put(cacheKey, cached, paramTypes) + sctx.GetPlanCache(isNonPrepared).Put(cacheKey, cached, paramTypes) } sessVars.FoundInPlanCache = false return p, names, err diff --git a/planner/core/plan_cache_param.go b/planner/core/plan_cache_param.go index a1e4b5a3f6703..7c79b2a6416a0 100644 --- a/planner/core/plan_cache_param.go +++ b/planner/core/plan_cache_param.go @@ -54,7 +54,7 @@ func (pr *paramReplacer) Enter(in ast.Node) (out ast.Node, skipChildren bool) { switch n := in.(type) { case *driver.ValueExpr: pr.params = append(pr.params, n) - // offset is used as order in general plan cache. + // offset is used as order in non-prepared plan cache. param := ast.NewParamMarkerExpr(len(pr.params) - 1) return param, true } @@ -100,7 +100,7 @@ func (pr *paramRestorer) Enter(in ast.Node) (out ast.Node, skipChildren bool) { pr.err = errors.New("failed to restore ast.Node") return nil, true } - // offset is used as order in general plan cache. + // offset is used as order in non-prepared plan cache. return pr.params[n.Offset], true } if pr.err != nil { diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index 868f35229ce6e..f541c441fd4f5 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -81,7 +81,7 @@ func TestInitLRUWithSystemVar(t *testing.T) { require.NotNil(t, lru) } -func TestGeneralPlanCacheBasically(t *testing.T) { +func TestNonPreparedPlanCacheBasically(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec(`use test`) @@ -101,11 +101,11 @@ func TestGeneralPlanCacheBasically(t *testing.T) { } for _, query := range queries { - tk.MustExec(`set tidb_enable_general_plan_cache=0`) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=0`) resultNormal := tk.MustQuery(query).Sort() tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("0")) - tk.MustExec(`set tidb_enable_general_plan_cache=1`) + tk.MustExec(`set tidb_enable_non_prepared_plan_cache=1`) tk.MustQuery(query) // first process tk.MustQuery(query).Sort().Check(resultNormal.Rows()) // equal to the result without plan-cache tk.MustQuery(`select @@last_plan_from_cache`).Check(testkit.Rows("1")) // this plan is from plan-cache diff --git a/planner/core/plan_cache_utils.go b/planner/core/plan_cache_utils.go index 4229e2b134f06..2b1621857b9ca 100644 --- a/planner/core/plan_cache_utils.go +++ b/planner/core/plan_cache_utils.go @@ -161,21 +161,6 @@ func GeneratePlanCacheStmtWithAST(ctx context.Context, sctx sessionctx.Context, return preparedObj, p, ParamCount, nil } -func getValidPlanFromCache(sctx sessionctx.Context, isGeneralPlanCache bool, key kvcache.Key, paramTypes []*types.FieldType) (*PlanCacheValue, bool) { - cache := sctx.GetPlanCache(isGeneralPlanCache) - val, exist := cache.Get(key, paramTypes) - if !exist { - return nil, exist - } - candidate := val.(*PlanCacheValue) - return candidate, true -} - -func putPlanIntoCache(sctx sessionctx.Context, isGeneralPlanCache bool, key kvcache.Key, plan *PlanCacheValue, paramTypes []*types.FieldType) { - cache := sctx.GetPlanCache(isGeneralPlanCache) - cache.Put(key, plan, paramTypes) -} - // planCacheKey is used to access Plan Cache. We put some variables that do not affect the plan into planCacheKey, such as the sql text. // Put the parameters that may affect the plan in planCacheValue. // However, due to some compatibility reasons, we will temporarily keep some system variable-related values in planCacheKey. diff --git a/planner/core/plan_cacheable_checker.go b/planner/core/plan_cacheable_checker.go index 55b1e45cd0e99..3412ace379271 100644 --- a/planner/core/plan_cacheable_checker.go +++ b/planner/core/plan_cacheable_checker.go @@ -175,16 +175,16 @@ func (checker *cacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) { return in, checker.cacheable } -// GeneralPlanCacheable checks whether the input ast is cacheable for general plan cache with empty session context, which is mainly for testing. -func GeneralPlanCacheable(node ast.Node, is infoschema.InfoSchema) bool { - return GeneralPlanCacheableWithCtx(nil, node, is) +// NonPreparedPlanCacheable checks whether the input ast is cacheable for non-prepared plan cache with empty session context, which is mainly for testing. +func NonPreparedPlanCacheable(node ast.Node, is infoschema.InfoSchema) bool { + return NonPreparedPlanCacheableWithCtx(nil, node, is) } -// GeneralPlanCacheableWithCtx checks whether the input ast is cacheable for general plan cache. +// NonPreparedPlanCacheableWithCtx checks whether the input ast is cacheable for non-prepared plan cache. // Only support: select {field} from {single-table} where {cond} and {cond} ... // {cond}: {col} {op} {val} // {op}: >, <, = -func GeneralPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) bool { +func NonPreparedPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) bool { selectStmt, isSelect := node.(*ast.SelectStmt) if !isSelect { // only support select statement now return false @@ -206,7 +206,7 @@ func GeneralPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is info } tableRefs := from.TableRefs if tableRefs.Right != nil { - // We don't support the join for the general plan cache now. + // We don't support the join for the non-prepared plan cache now. return false } switch x := tableRefs.Left.(type) { @@ -217,7 +217,7 @@ func GeneralPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is info } } - checker := generalPlanCacheableChecker{ + checker := nonPreparedPlanCacheableChecker{ sctx: sctx, cacheable: true, schema: is, @@ -226,19 +226,19 @@ func GeneralPlanCacheableWithCtx(sctx sessionctx.Context, node ast.Node, is info return checker.cacheable } -// generalPlanCacheableChecker checks whether a query's plan can be cached for general plan cache. +// nonPreparedPlanCacheableChecker checks whether a query's plan can be cached for non-prepared plan cache. // NOTE: we can add more rules in the future. -type generalPlanCacheableChecker struct { +type nonPreparedPlanCacheableChecker struct { sctx sessionctx.Context cacheable bool schema infoschema.InfoSchema } // Enter implements Visitor interface. -func (checker *generalPlanCacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren bool) { +func (checker *nonPreparedPlanCacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren bool) { switch node := in.(type) { case *ast.BinaryOperationExpr: - if _, found := expression.GeneralPlanCacheableOp[node.Op.String()]; !found { + if _, found := expression.NonPreparedPlanCacheableOp[node.Op.String()]; !found { checker.cacheable = false return in, true } @@ -265,7 +265,7 @@ func (checker *generalPlanCacheableChecker) Enter(in ast.Node) (out ast.Node, sk } // Leave implements Visitor interface. -func (checker *generalPlanCacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) { +func (checker *nonPreparedPlanCacheableChecker) Leave(in ast.Node) (out ast.Node, ok bool) { return in, checker.cacheable } diff --git a/planner/core/plan_cacheable_checker_test.go b/planner/core/plan_cacheable_checker_test.go index a32294e34c54c..658537bde4188 100644 --- a/planner/core/plan_cacheable_checker_test.go +++ b/planner/core/plan_cacheable_checker_test.go @@ -247,7 +247,7 @@ func TestCacheable(t *testing.T) { require.True(t, core.Cacheable(stmt, is)) } -func TestGeneralPlanCacheable(t *testing.T) { +func TestNonPreparedPlanCacheable(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -297,12 +297,12 @@ func TestGeneralPlanCacheable(t *testing.T) { for _, q := range unsupported { stmt, err := p.ParseOneStmt(q, charset, collation) require.NoError(t, err) - require.False(t, core.GeneralPlanCacheable(stmt, is)) + require.False(t, core.NonPreparedPlanCacheable(stmt, is)) } for _, q := range supported { stmt, err := p.ParseOneStmt(q, charset, collation) require.NoError(t, err) - require.True(t, core.GeneralPlanCacheable(stmt, is)) + require.True(t, core.NonPreparedPlanCacheable(stmt, is)) } } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 057ff536a7102..9201f953bdcdc 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1040,11 +1040,38 @@ func checkHintedSQL(sql, charset, collation, db string) error { return nil } +func fetchRecordFromClusterStmtSummary(sctx sessionctx.Context, planDigest string) ([]chunk.Row, error) { + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnBindInfo) + exec, _ := sctx.(sqlexec.SQLExecutor) + fields := "stmt_type, schema_name, digest_text, sample_user, prepared, query_sample_text, charset, collation, plan_hint, plan_digest" + sql := fmt.Sprintf("select %s from information_schema.cluster_statements_summary where plan_digest = '%s' union distinct ", fields, planDigest) + + fmt.Sprintf("select %s from information_schema.cluster_statements_summary_history where plan_digest = '%s' ", fields, planDigest) + + "order by length(plan_digest) desc" + rs, err := exec.ExecuteInternal(ctx, sql) + if rs == nil { + return nil, errors.New("can't find any records for '" + planDigest + "' in statement summary") + } + if err != nil { + return nil, err + } + + var rows []chunk.Row + defer terror.Call(rs.Close) + if rows, err = sqlexec.DrainRecordSet(ctx, rs, 8); err != nil { + return nil, err + } + return rows, nil +} + func (b *PlanBuilder) buildCreateBindPlanFromPlanDigest(v *ast.CreateBindingStmt) (Plan, error) { if v.PlanDigest == "" { return nil, errors.New("plan digest is empty") } - bindableStmt := stmtsummary.StmtSummaryByDigestMap.GetBindableStmtByPlanDigest(v.PlanDigest) + rows, err := fetchRecordFromClusterStmtSummary(b.ctx, v.PlanDigest) + if err != nil { + return nil, err + } + bindableStmt := stmtsummary.GetBindableStmtFromCluster(rows) if bindableStmt == nil { return nil, errors.New("can't find any plans for '" + v.PlanDigest + "'") } diff --git a/planner/optimize.go b/planner/optimize.go index d5ee997057180..3a6804d5fa319 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -73,23 +73,23 @@ func matchSQLBinding(sctx sessionctx.Context, stmtNode ast.StmtNode) (bindRecord return bindRecord, scope, true } -// getPlanFromGeneralPlanCache tries to get an available cached plan from the General Plan Cache for this stmt. -func getPlanFromGeneralPlanCache(ctx context.Context, sctx sessionctx.Context, stmt ast.StmtNode, is infoschema.InfoSchema) (core.Plan, types.NameSlice, bool, error) { +// getPlanFromNonPreparedPlanCache tries to get an available cached plan from the NonPrepared Plan Cache for this stmt. +func getPlanFromNonPreparedPlanCache(ctx context.Context, sctx sessionctx.Context, stmt ast.StmtNode, is infoschema.InfoSchema) (core.Plan, types.NameSlice, bool, error) { if sctx.GetSessionVars().StmtCtx.InPreparedPlanBuilding || // already in cached plan rebuilding phase - !core.GeneralPlanCacheableWithCtx(sctx, stmt, is) { + !core.NonPreparedPlanCacheableWithCtx(sctx, stmt, is) { return nil, nil, false, nil } paramSQL, params, err := core.ParameterizeAST(sctx, stmt) if err != nil { return nil, nil, false, err } - val := sctx.GetSessionVars().GetGeneralPlanCacheStmt(paramSQL) + val := sctx.GetSessionVars().GetNonPreparedPlanCacheStmt(paramSQL) if val == nil { cachedStmt, _, _, err := core.GeneratePlanCacheStmtWithAST(ctx, sctx, stmt) if err != nil { return nil, nil, false, err } - sctx.GetSessionVars().AddGeneralPlanCacheStmt(paramSQL, cachedStmt) + sctx.GetSessionVars().AddNonPreparedPlanCacheStmt(paramSQL, cachedStmt) val = cachedStmt } cachedStmt := val.(*core.PlanCacheStmt) @@ -176,11 +176,11 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in node = stmtNode } - // try to get Plan from the General Plan Cache - if sctx.GetSessionVars().EnableGeneralPlanCache && + // try to get Plan from the NonPrepared Plan Cache + if sctx.GetSessionVars().EnableNonPreparedPlanCache && isStmtNode && !useBinding { // TODO: support binding - cachedPlan, names, ok, err := getPlanFromGeneralPlanCache(ctx, sctx, stmtNode, is) + cachedPlan, names, ok, err := getPlanFromNonPreparedPlanCache(ctx, sctx, stmtNode, is) if err != nil { return nil, nil, err } @@ -436,7 +436,7 @@ func OptimizeExecStmt(ctx context.Context, sctx sessionctx.Context, if !ok { return nil, nil, errors.Errorf("invalid result plan type, should be Execute") } - plan, names, err := core.GetPlanFromSessionPlanCache(ctx, sctx, execAst.FromGeneralStmt, is, exec.PrepStmt, exec.Params) + plan, names, err := core.GetPlanFromSessionPlanCache(ctx, sctx, false, is, exec.PrepStmt, exec.Params) if err != nil { return nil, nil, err } diff --git a/server/plan_replayer.go b/server/plan_replayer.go index 30a00e4eda112..7bfb680a756ba 100644 --- a/server/plan_replayer.go +++ b/server/plan_replayer.go @@ -16,7 +16,7 @@ package server import ( "fmt" - "io/ioutil" + "io" "net/http" "os" "path/filepath" @@ -83,7 +83,7 @@ func handleDownloadFile(handler downloadFileHandler, w http.ResponseWriter, req writeError(w, err) return } - content, err := ioutil.ReadAll(file) + content, err := io.ReadAll(file) if err != nil { writeError(w, err) return @@ -137,7 +137,7 @@ func handleDownloadFile(handler downloadFileHandler, w http.ResponseWriter, req zap.String("remote-addr", remoteAddr), zap.Int("status-code", resp.StatusCode)) continue } - content, err := ioutil.ReadAll(resp.Body) + content, err := io.ReadAll(resp.Body) if err != nil { writeError(w, err) return diff --git a/server/plan_replayer_test.go b/server/plan_replayer_test.go index 2a00bc0db04de..2f2308efdd1a9 100644 --- a/server/plan_replayer_test.go +++ b/server/plan_replayer_test.go @@ -18,7 +18,6 @@ import ( "bytes" "database/sql" "io" - "io/ioutil" "os" "path/filepath" "testing" @@ -69,7 +68,7 @@ func TestDumpPlanReplayerAPI(t *testing.T) { require.NoError(t, resp0.Body.Close()) }() - body, err := ioutil.ReadAll(resp0.Body) + body, err := io.ReadAll(resp0.Body) require.NoError(t, err) path := "/tmp/plan_replayer.zip" diff --git a/session/bootstrap.go b/session/bootstrap.go index 39859616e4784..6644da210f4c7 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -412,6 +412,7 @@ const ( modify_count bigint(64) NOT NULL, count bigint(64) NOT NULL, version bigint(64) NOT NULL comment 'stats version which corresponding to stats:version in EXPLAIN', + source varchar(40) NOT NULL, create_time datetime(6) NOT NULL, UNIQUE KEY table_version (table_id, version), KEY table_create_time (table_id, create_time) @@ -732,11 +733,13 @@ const ( version107 = 107 // version108 adds the table tidb_ttl_table_status version108 = 108 + // version109 add column source to mysql.stats_meta_history + version109 = 109 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. // please make sure this is the largest version -var currentBootstrapVersion int64 = version108 +var currentBootstrapVersion int64 = version109 // DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it. var internalSQLTimeout = owner.ManagerSessionTTL + 15 @@ -849,6 +852,7 @@ var ( upgradeToVer106, upgradeToVer107, upgradeToVer108, + upgradeToVer109, } ) @@ -2190,6 +2194,13 @@ func upgradeToVer108(s Session, ver int64) { doReentrantDDL(s, CreateTTLTableStatus) } +func upgradeToVer109(s Session, ver int64) { + if ver >= version109 { + return + } + doReentrantDDL(s, "ALTER TABLE mysql.stats_meta_history ADD COLUMN IF NOT EXISTS `source` varchar(40) NOT NULL after `version`;") +} + func writeOOMAction(s Session) { comment := "oom-action is `log` by default in v3.0.x, `cancel` by default in v4.0.11+" mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE VARIABLE_VALUE= %?`, diff --git a/session/bootstrap_upgrade_test.go b/session/bootstrap_upgrade_test.go index 27a5c34a2eb7f..59cdddf218a2d 100644 --- a/session/bootstrap_upgrade_test.go +++ b/session/bootstrap_upgrade_test.go @@ -72,14 +72,15 @@ func TestUpgradeVersion84(t *testing.T) { {"modify_count", "bigint(64)"}, {"count", "bigint(64)"}, {"version", "bigint(64)"}, + {"source", "varchar(40)"}, {"create_time", "datetime(6)"}, } rStatsHistoryTbl, err := tk.Exec(`desc mysql.stats_meta_history`) require.NoError(t, err) req := rStatsHistoryTbl.NewChunk(nil) require.NoError(t, rStatsHistoryTbl.Next(ctx, req)) - require.Equal(t, 5, req.NumRows()) - for i := 0; i < 5; i++ { + require.Equal(t, 6, req.NumRows()) + for i := 0; i < 6; i++ { row := req.GetRow(i) require.Equal(t, statsHistoryTblFields[i].field, strings.ToLower(row.GetString(0))) require.Equal(t, statsHistoryTblFields[i].tp, strings.ToLower(row.GetString(1))) diff --git a/session/session.go b/session/session.go index df01e592f7506..be15798c7f236 100644 --- a/session/session.go +++ b/session/session.go @@ -175,8 +175,6 @@ type Session interface { RollbackTxn(context.Context) // PrepareStmt executes prepare statement in binary protocol. PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*ast.ResultField, err error) - // CacheGeneralStmt parses the sql, generates the corresponding PlanCacheStmt and cache it. - CacheGeneralStmt(sql string) (interface{}, error) // ExecutePreparedStmt executes a prepared statement. // Deprecated: please use ExecuteStmt, this function is left for testing only. // TODO: remove ExecutePreparedStmt. @@ -254,8 +252,8 @@ type session struct { store kv.Storage - preparedPlanCache sessionctx.PlanCache - generalPlanCache sessionctx.PlanCache + preparedPlanCache sessionctx.PlanCache + nonPreparedPlanCache sessionctx.PlanCache sessionVars *variable.SessionVars sessionManager util.SessionManager @@ -458,17 +456,17 @@ func (s *session) SetCollation(coID int) error { return s.sessionVars.SetSystemVarWithoutValidation(variable.CollationConnection, co) } -func (s *session) GetPlanCache(isGeneralPlanCache bool) sessionctx.PlanCache { - if isGeneralPlanCache { // use the general plan cache - if !s.GetSessionVars().EnableGeneralPlanCache { +func (s *session) GetPlanCache(isNonPrepared bool) sessionctx.PlanCache { + if isNonPrepared { // use the non-prepared plan cache + if !s.GetSessionVars().EnableNonPreparedPlanCache { return nil } - if s.generalPlanCache == nil { // lazy construction - s.generalPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().GeneralPlanCacheSize), + if s.nonPreparedPlanCache == nil { // lazy construction + s.nonPreparedPlanCache = plannercore.NewLRUPlanCache(uint(s.GetSessionVars().NonPreparedPlanCacheSize), variable.PreparedPlanCacheMemoryGuardRatio.Load(), plannercore.PreparedPlanCacheMaxMemory.Load(), plannercore.PickPlanFromBucket, s) } - return s.generalPlanCache + return s.nonPreparedPlanCache } // use the prepared plan cache @@ -2443,22 +2441,6 @@ func (s *session) rollbackOnError(ctx context.Context) { } } -// CacheGeneralStmt parses the sql, generates the corresponding PlanCacheStmt and cache it. -// The sql have to be parameterized, e.g. select * from t where a>?. -func (s *session) CacheGeneralStmt(sql string) (interface{}, error) { - if stmt := s.sessionVars.GetGeneralPlanCacheStmt(sql); stmt != nil { - // skip this step if there is already a PlanCacheStmt for this ql - return stmt, nil - } - - prepareExec := executor.NewPrepareExec(s, sql) - prepareExec.IsGeneralStmt = true - if err := prepareExec.Next(context.Background(), nil); err != nil { - return nil, err - } - return prepareExec.Stmt, nil -} - // PrepareStmt is used for executing prepare statement in binary protocol func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*ast.ResultField, err error) { if s.sessionVars.TxnCtx.InfoSchema == nil { @@ -3073,22 +3055,21 @@ func loadCollationParameter(ctx context.Context, se *session) (bool, error) { return false, nil } +type tableBasicInfo struct { + SQL string + id int64 +} + var ( errResultIsEmpty = dbterror.ClassExecutor.NewStd(errno.ErrResultIsEmpty) // DDLJobTables is a list of tables definitions used in concurrent DDL. - DDLJobTables = []struct { - SQL string - id int64 - }{ + DDLJobTables = []tableBasicInfo{ {ddl.JobTableSQL, ddl.JobTableID}, {ddl.ReorgTableSQL, ddl.ReorgTableID}, {ddl.HistoryTableSQL, ddl.HistoryTableID}, } // BackfillTables is a list of tables definitions used in dist reorg DDL. - BackfillTables = []struct { - SQL string - id int64 - }{ + BackfillTables = []tableBasicInfo{ {ddl.BackfillTableSQL, ddl.BackfillTableID}, {ddl.BackfillHistoryTableSQL, ddl.BackfillHistoryTableID}, } @@ -3109,7 +3090,7 @@ func splitAndScatterTable(store kv.Storage, tableIDs []int64) { } } -// InitDDLJobTables is to create tidb_ddl_job, tidb_ddl_reorg and tidb_ddl_history, or tidb_ddl_backfill and tidb_ddl_backfill_history. +// InitDDLJobTables is to create tidb_ddl_job, tidb_ddl_reorg, tidb_ddl_history, tidb_ddl_backfill and tidb_ddl_backfill_history. func InitDDLJobTables(store kv.Storage) error { return kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, true, func(ctx context.Context, txn kv.Transaction) error { t := meta.NewMeta(txn) @@ -3121,41 +3102,56 @@ func InitDDLJobTables(store kv.Storage) error { if err != nil { return err } - tables := append(DDLJobTables, BackfillTables...) if exists { - tblExist, err := t.CheckTableExists(dbID, BackfillTables[0].id) - if err != nil || tblExist { - return errors.Trace(err) - } - tables = BackfillTables + return initBackfillJobTables(store, t, dbID) } - tableIDs := make([]int64, 0, len(tables)) - for _, tbl := range tables { - tableIDs = append(tableIDs, tbl.id) + + if err = createAndSplitTables(store, t, dbID, DDLJobTables); err != nil { + return err } - splitAndScatterTable(store, tableIDs) - p := parser.New() - for _, tbl := range tables { - stmt, err := p.ParseOneStmt(tbl.SQL, "", "") - if err != nil { - return errors.Trace(err) - } - tblInfo, err := ddl.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) - if err != nil { - return errors.Trace(err) - } - tblInfo.State = model.StatePublic - tblInfo.ID = tbl.id - tblInfo.UpdateTS = t.StartTS - err = t.CreateTableOrView(dbID, tblInfo) - if err != nil { - return errors.Trace(err) - } + if err = initBackfillJobTables(store, t, dbID); err != nil { + return err } return t.SetDDLTables() }) } +// initBackfillJobTables is to create tidb_ddl_backfill and tidb_ddl_backfill_history. +func initBackfillJobTables(store kv.Storage, t *meta.Meta, dbID int64) error { + tblExist, err := t.CheckTableExists(dbID, BackfillTables[0].id) + if err != nil || tblExist { + return errors.Trace(err) + } + return createAndSplitTables(store, t, dbID, BackfillTables) +} + +func createAndSplitTables(store kv.Storage, t *meta.Meta, dbID int64, tables []tableBasicInfo) error { + tableIDs := make([]int64, 0, len(tables)) + for _, tbl := range tables { + tableIDs = append(tableIDs, tbl.id) + } + splitAndScatterTable(store, tableIDs) + p := parser.New() + for _, tbl := range tables { + stmt, err := p.ParseOneStmt(tbl.SQL, "", "") + if err != nil { + return errors.Trace(err) + } + tblInfo, err := ddl.BuildTableInfoFromAST(stmt.(*ast.CreateTableStmt)) + if err != nil { + return errors.Trace(err) + } + tblInfo.State = model.StatePublic + tblInfo.ID = tbl.id + tblInfo.UpdateTS = t.StartTS + err = t.CreateTableOrView(dbID, tblInfo) + if err != nil { + return errors.Trace(err) + } + } + return nil +} + // InitMDLTable is to create tidb_mdl_info, which is used for metadata lock. func InitMDLTable(store kv.Storage) error { return kv.RunInNewTxn(kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL), store, true, func(ctx context.Context, txn kv.Transaction) error { diff --git a/sessionctx/context.go b/sessionctx/context.go index f39d3a82a8f38..4cc201206df07 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -52,7 +52,7 @@ type SessionStatesHandler interface { DecodeSessionStates(context.Context, Context, *sessionstates.SessionStates) error } -// PlanCache is an interface for prepare and general plan cache +// PlanCache is an interface for prepare and non-prepared plan cache type PlanCache interface { Get(key kvcache.Key, paramTypes []*types.FieldType) (value kvcache.Value, ok bool) Put(key kvcache.Key, value kvcache.Value, paramTypes []*types.FieldType) @@ -120,8 +120,8 @@ type Context interface { GetStore() kv.Storage // GetPlanCache returns the cache of the physical plan. - // generalPlanCache indicates to return the general plan cache or the prepared plan cache. - GetPlanCache(isGeneralPlanCache bool) PlanCache + // isNonPrepared indicates to return the non-prepared plan cache or the prepared plan cache. + GetPlanCache(isNonPrepared bool) PlanCache // StoreQueryFeedback stores the query feedback. StoreQueryFeedback(feedback interface{}) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 8c977aa832579..ad60ccba2cfa4 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -638,8 +638,8 @@ type SessionVars struct { SysWarningCount int // SysErrorCount is the system variable "error_count", because it is on the hot path, so we extract it from the systems SysErrorCount uint16 - // generalPlanCacheStmts stores PlanCacheStmts for general plan cache. - generalPlanCacheStmts *kvcache.SimpleLRUCache + // nonPreparedPlanCacheStmts stores PlanCacheStmts for non-prepared plan cache. + nonPreparedPlanCacheStmts *kvcache.SimpleLRUCache // PreparedStmts stores prepared statement. PreparedStmts map[uint32]interface{} PreparedStmtNameToID map[string]uint32 @@ -1255,17 +1255,17 @@ type SessionVars struct { // EnablePreparedPlanCache indicates whether to enable prepared plan cache. EnablePreparedPlanCache bool - // GeneralPlanCacheSize controls the size of general plan cache. + // PreparedPlanCacheSize controls the size of prepared plan cache. PreparedPlanCacheSize uint64 // PreparedPlanCacheMonitor indicates whether to enable prepared plan cache monitor. EnablePreparedPlanCacheMemoryMonitor bool - // EnableGeneralPlanCache indicates whether to enable general plan cache. - EnableGeneralPlanCache bool + // EnableNonPreparedPlanCache indicates whether to enable non-prepared plan cache. + EnableNonPreparedPlanCache bool - // GeneralPlanCacheSize controls the size of general plan cache. - GeneralPlanCacheSize uint64 + // NonPreparedPlanCacheSize controls the size of non-prepared plan cache. + NonPreparedPlanCacheSize uint64 // ConstraintCheckInPlacePessimistic controls whether to skip the locking of some keys in pessimistic transactions. // Postpone the conflict check and constraint check to prewrite or later pessimistic locking requests. @@ -2038,20 +2038,20 @@ func (k planCacheStmtKey) Hash() []byte { return []byte(k) } -// AddGeneralPlanCacheStmt adds this PlanCacheStmt into general-plan-cache-stmt cache -func (s *SessionVars) AddGeneralPlanCacheStmt(sql string, stmt interface{}) { - if s.generalPlanCacheStmts == nil { - s.generalPlanCacheStmts = kvcache.NewSimpleLRUCache(uint(s.GeneralPlanCacheSize), 0, 0) +// AddNonPreparedPlanCacheStmt adds this PlanCacheStmt into non-preapred plan-cache stmt cache +func (s *SessionVars) AddNonPreparedPlanCacheStmt(sql string, stmt interface{}) { + if s.nonPreparedPlanCacheStmts == nil { + s.nonPreparedPlanCacheStmts = kvcache.NewSimpleLRUCache(uint(s.NonPreparedPlanCacheSize), 0, 0) } - s.generalPlanCacheStmts.Put(planCacheStmtKey(sql), stmt) + s.nonPreparedPlanCacheStmts.Put(planCacheStmtKey(sql), stmt) } -// GetGeneralPlanCacheStmt gets the PlanCacheStmt. -func (s *SessionVars) GetGeneralPlanCacheStmt(sql string) interface{} { - if s.generalPlanCacheStmts == nil { +// GetNonPreparedPlanCacheStmt gets the PlanCacheStmt. +func (s *SessionVars) GetNonPreparedPlanCacheStmt(sql string) interface{} { + if s.nonPreparedPlanCacheStmts == nil { return nil } - stmt, _ := s.generalPlanCacheStmts.Get(planCacheStmtKey(sql)) + stmt, _ := s.nonPreparedPlanCacheStmts.Get(planCacheStmtKey(sql)) return stmt } diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 732ce4ad606cf..67483e0c4f75a 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -398,21 +398,21 @@ func TestTransactionContextSavepoint(t *testing.T) { require.Equal(t, 0, len(tc.Savepoints)) } -func TestGeneralPlanCacheStmt(t *testing.T) { +func TestNonPreparedPlanCacheStmt(t *testing.T) { sessVars := variable.NewSessionVars(nil) - sessVars.GeneralPlanCacheSize = 100 + sessVars.NonPreparedPlanCacheSize = 100 sql1 := "select * from t where a>?" sql2 := "select * from t where a 3 and id1 < 6") + waitTs3 := sctx.GetSessionVars().DurationWaitTS + tk.MustExec("commit") + require.NotEqual(t, waitTs1, waitTs2) + require.NotEqual(t, waitTs1, waitTs2) + require.NotEqual(t, waitTs2, waitTs3) +} diff --git a/statistics/handle/BUILD.bazel b/statistics/handle/BUILD.bazel index bcafe4260eb30..81dff92c5b143 100644 --- a/statistics/handle/BUILD.bazel +++ b/statistics/handle/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "gc.go", "handle.go", "handle_hist.go", + "historical_stats_handler.go", "lru_cache.go", "statscache.go", "update.go", diff --git a/statistics/handle/ddl.go b/statistics/handle/ddl.go index d5926c84f49c0..1f93447b55c14 100644 --- a/statistics/handle/ddl.go +++ b/statistics/handle/ddl.go @@ -156,7 +156,8 @@ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error { for i := 0; i < newColGlobalStats.Num; i++ { hg, cms, topN := newColGlobalStats.Hg[i], newColGlobalStats.Cms[i], newColGlobalStats.TopN[i] // fms for global stats doesn't need to dump to kv. - err = h.SaveStatsToStorage(tableID, newColGlobalStats.Count, newColGlobalStats.ModifyCount, 0, hg, cms, topN, 2, 1, false) + err = h.SaveStatsToStorage(tableID, newColGlobalStats.Count, newColGlobalStats.ModifyCount, + 0, hg, cms, topN, 2, 1, false, StatsMetaHistorySourceSchemaChange) if err != nil { return err } @@ -186,7 +187,7 @@ func (h *Handle) updateGlobalStats(tblInfo *model.TableInfo) error { for i := 0; i < newIndexGlobalStats.Num; i++ { hg, cms, topN := newIndexGlobalStats.Hg[i], newIndexGlobalStats.Cms[i], newIndexGlobalStats.TopN[i] // fms for global stats doesn't need to dump to kv. - err = h.SaveStatsToStorage(tableID, newIndexGlobalStats.Count, newIndexGlobalStats.ModifyCount, 1, hg, cms, topN, 2, 1, false) + err = h.SaveStatsToStorage(tableID, newIndexGlobalStats.Count, newIndexGlobalStats.ModifyCount, 1, hg, cms, topN, 2, 1, false, StatsMetaHistorySourceSchemaChange) if err != nil { return err } @@ -221,7 +222,7 @@ func (h *Handle) insertTableStats2KV(info *model.TableInfo, physicalID int64) (e statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - h.recordHistoricalStatsMeta(physicalID, statsVer) + h.recordHistoricalStatsMeta(physicalID, statsVer, StatsMetaHistorySourceSchemaChange) } }() h.mu.Lock() @@ -263,7 +264,7 @@ func (h *Handle) insertColStats2KV(physicalID int64, colInfos []*model.ColumnInf statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - h.recordHistoricalStatsMeta(physicalID, statsVer) + h.recordHistoricalStatsMeta(physicalID, statsVer, StatsMetaHistorySourceSchemaChange) } }() h.mu.Lock() diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 546916751c32d..daaf28ead7573 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -19,7 +19,7 @@ import ( "compress/gzip" "encoding/json" "fmt" - "io/ioutil" + "io" "time" "github.com/pingcap/errors" @@ -347,7 +347,7 @@ func (h *Handle) loadStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, // loadStatsFromJSON doesn't support partition table now. // The table level Count and Modify_count would be overridden by the SaveMetaToStorage below, so we don't need // to care about them here. - err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, 0, &col.Histogram, col.CMSketch, col.TopN, int(col.StatsVer), 1, false) + err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, 0, &col.Histogram, col.CMSketch, col.TopN, int(col.StatsVer), 1, false, StatsMetaHistorySourceLoadStats) if err != nil { return errors.Trace(err) } @@ -356,7 +356,7 @@ func (h *Handle) loadStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, // loadStatsFromJSON doesn't support partition table now. // The table level Count and Modify_count would be overridden by the SaveMetaToStorage below, so we don't need // to care about them here. - err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, 1, &idx.Histogram, idx.CMSketch, idx.TopN, int(idx.StatsVer), 1, false) + err = h.SaveStatsToStorage(tbl.PhysicalID, tbl.Count, 0, 1, &idx.Histogram, idx.CMSketch, idx.TopN, int(idx.StatsVer), 1, false, StatsMetaHistorySourceLoadStats) if err != nil { return errors.Trace(err) } @@ -365,7 +365,7 @@ func (h *Handle) loadStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, if err != nil { return errors.Trace(err) } - return h.SaveMetaToStorage(tbl.PhysicalID, tbl.Count, tbl.ModifyCount) + return h.SaveMetaToStorage(tbl.PhysicalID, tbl.Count, tbl.ModifyCount, StatsMetaHistorySourceLoadStats) } // TableStatsFromJSON loads statistic from JSONTable and return the Table of statistic. @@ -501,7 +501,7 @@ func BlocksToJSONTable(blocks [][]byte) (*JSONTable, error) { if err := gzipReader.Close(); err != nil { return nil, err } - jsonStr, err := ioutil.ReadAll(gzipReader) + jsonStr, err := io.ReadAll(gzipReader) if err != nil { return nil, errors.Trace(err) } diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index 857c05749d066..165ea999ae25a 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -261,7 +261,7 @@ func TestDumpCMSketchWithTopN(t *testing.T) { cms, _, _, _ := statistics.NewCMSketchAndTopN(5, 2048, fakeData, 20, 100) stat := h.GetTableStats(tableInfo) - err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, nil, statistics.Version2, 1, false) + err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, nil, statistics.Version2, 1, false, handle.StatsMetaHistorySourceLoadStats) require.NoError(t, err) require.Nil(t, h.Update(is)) diff --git a/statistics/handle/gc.go b/statistics/handle/gc.go index 6ef3ee3f7c12c..ab2159bf22bf0 100644 --- a/statistics/handle/gc.go +++ b/statistics/handle/gc.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/sqlexec" @@ -54,6 +55,11 @@ func (h *Handle) GCStats(is infoschema.InfoSchema, ddlLease time.Duration) error return errors.Trace(err) } } + if err := h.ClearOutdatedHistoryStats(); err != nil { + logutil.BgLogger().Warn("failed to gc outdated historical stats", + zap.Duration("duration", variable.HistoricalStatsDuration.Load()), + zap.Error(err)) + } return h.removeDeletedExtendedStats(gcVer) } @@ -141,6 +147,22 @@ func (h *Handle) gcTableStats(is infoschema.InfoSchema, physicalID int64) error return nil } +// ClearOutdatedHistoryStats clear outdated historical stats +func (h *Handle) ClearOutdatedHistoryStats() error { + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) + h.mu.Lock() + defer h.mu.Unlock() + exec := h.mu.ctx.(sqlexec.SQLExecutor) + sql := "delete from mysql.stats_meta_history where NOW() - create_time >= %?" + _, err := exec.ExecuteInternal(ctx, sql, variable.HistoricalStatsDuration.Load().Seconds()) + if err != nil { + return err + } + sql = "delete from mysql.stats_history where NOW() - create_time >= %? " + _, err = exec.ExecuteInternal(ctx, sql, variable.HistoricalStatsDuration.Load().Seconds()) + return err +} + func (h *Handle) gcHistoryStatsFromKV(physicalID int64) error { h.mu.Lock() defer h.mu.Unlock() diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 4a23f6959655c..fe8a401b72295 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1614,27 +1614,20 @@ func saveBucketsToStorage(ctx context.Context, exec sqlexec.SQLExecutor, sc *stm } // SaveTableStatsToStorage saves the stats of a table to storage. -func (h *Handle) SaveTableStatsToStorage(results *statistics.AnalyzeResults, analyzeSnapshot bool) (err error) { - tableID := results.TableID.GetStatisticsID() - statsVer := uint64(0) - defer func() { - if err == nil && statsVer != 0 { - h.recordHistoricalStatsMeta(tableID, statsVer) - } - }() +func (h *Handle) SaveTableStatsToStorage(results *statistics.AnalyzeResults, analyzeSnapshot bool, source string) (err error) { h.mu.Lock() defer h.mu.Unlock() - return SaveTableStatsToStorage(h.mu.ctx, results, analyzeSnapshot) + return SaveTableStatsToStorage(h.mu.ctx, results, analyzeSnapshot, source) } // SaveTableStatsToStorage saves the stats of a table to storage. -func SaveTableStatsToStorage(sctx sessionctx.Context, results *statistics.AnalyzeResults, analyzeSnapshot bool) (err error) { +func SaveTableStatsToStorage(sctx sessionctx.Context, results *statistics.AnalyzeResults, analyzeSnapshot bool, source string) (err error) { needDumpFMS := results.TableID.IsPartitionTable() tableID := results.TableID.GetStatisticsID() statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - if err1 := recordHistoricalStatsMeta(sctx, tableID, statsVer); err1 != nil { + if err1 := recordHistoricalStatsMeta(sctx, tableID, statsVer, source); err1 != nil { logutil.BgLogger().Error("record historical stats meta failed", zap.Int64("table-id", tableID), zap.Uint64("version", statsVer), @@ -1809,11 +1802,12 @@ func SaveTableStatsToStorage(sctx sessionctx.Context, results *statistics.Analyz // If count is negative, both count and modify count would not be used and not be written to the table. Unless, corresponding // fields in the stats_meta table will be updated. // TODO: refactor to reduce the number of parameters -func (h *Handle) SaveStatsToStorage(tableID int64, count, modifyCount int64, isIndex int, hg *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, statsVersion int, isAnalyzed int64, updateAnalyzeTime bool) (err error) { +func (h *Handle) SaveStatsToStorage(tableID int64, count, modifyCount int64, isIndex int, hg *statistics.Histogram, + cms *statistics.CMSketch, topN *statistics.TopN, statsVersion int, isAnalyzed int64, updateAnalyzeTime bool, source string) (err error) { statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer, source) } }() h.mu.Lock() @@ -1888,11 +1882,11 @@ func (h *Handle) SaveStatsToStorage(tableID int64, count, modifyCount int64, isI } // SaveMetaToStorage will save stats_meta to storage. -func (h *Handle) SaveMetaToStorage(tableID, count, modifyCount int64) (err error) { +func (h *Handle) SaveMetaToStorage(tableID, count, modifyCount int64, source string) (err error) { statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer, source) } }() h.mu.Lock() @@ -2098,7 +2092,7 @@ func (h *Handle) InsertExtendedStats(statsName string, colIDs []int64, tp int, t statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer, StatsMetaHistorySourceExtendedStats) } }() slices.Sort(colIDs) @@ -2169,7 +2163,7 @@ func (h *Handle) MarkExtendedStatsDeleted(statsName string, tableID int64, ifExi statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer, StatsMetaHistorySourceExtendedStats) } }() ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) @@ -2382,7 +2376,7 @@ func (h *Handle) SaveExtendedStatsToStorage(tableID int64, extStats *statistics. statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - h.recordHistoricalStatsMeta(tableID, statsVer) + h.recordHistoricalStatsMeta(tableID, statsVer, StatsMetaHistorySourceExtendedStats) } }() if extStats == nil || len(extStats.Stats) == 0 { @@ -2643,56 +2637,6 @@ func (h *Handle) CheckHistoricalStatsEnable() (enable bool, err error) { return checkHistoricalStatsEnable(h.mu.ctx) } -func recordHistoricalStatsMeta(sctx sessionctx.Context, tableID int64, version uint64) error { - if tableID == 0 || version == 0 { - return errors.Errorf("tableID %d, version %d are invalid", tableID, version) - } - historicalStatsEnabled, err := checkHistoricalStatsEnable(sctx) - if err != nil { - return errors.Errorf("check tidb_enable_historical_stats failed: %v", err) - } - if !historicalStatsEnabled { - return nil - } - ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) - exec := sctx.(sqlexec.SQLExecutor) - rexec := sctx.(sqlexec.RestrictedSQLExecutor) - rows, _, err := rexec.ExecRestrictedSQL(ctx, []sqlexec.OptionFuncAlias{sqlexec.ExecOptionUseCurSession}, "select modify_count, count from mysql.stats_meta where table_id = %? and version = %?", tableID, version) - if err != nil { - return errors.Trace(err) - } - if len(rows) == 0 { - return errors.New("no historical meta stats can be recorded") - } - modifyCount, count := rows[0].GetInt64(0), rows[0].GetInt64(1) - - _, err = exec.ExecuteInternal(ctx, "begin pessimistic") - if err != nil { - return errors.Trace(err) - } - defer func() { - err = finishTransaction(ctx, exec, err) - }() - - const sql = "REPLACE INTO mysql.stats_meta_history(table_id, modify_count, count, version, create_time) VALUES (%?, %?, %?, %?, NOW())" - if _, err := exec.ExecuteInternal(ctx, sql, tableID, modifyCount, count, version); err != nil { - return errors.Trace(err) - } - return nil -} - -func (h *Handle) recordHistoricalStatsMeta(tableID int64, version uint64) { - h.mu.Lock() - defer h.mu.Unlock() - err := recordHistoricalStatsMeta(h.mu.ctx, tableID, version) - if err != nil { - logutil.BgLogger().Error("record historical stats meta failed", - zap.Int64("table-id", tableID), - zap.Uint64("version", version), - zap.Error(err)) - } -} - // InsertAnalyzeJob inserts analyze job into mysql.analyze_jobs and gets job ID for further updating job. func (h *Handle) InsertAnalyzeJob(job *statistics.AnalyzeJob, instance string, procID uint64) error { h.mu.Lock() diff --git a/statistics/handle/historical_stats_handler.go b/statistics/handle/historical_stats_handler.go new file mode 100644 index 0000000000000..c7a683da8b740 --- /dev/null +++ b/statistics/handle/historical_stats_handler.go @@ -0,0 +1,91 @@ +// 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 handle + +import ( + "context" + + "github.com/pingcap/errors" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" + "go.uber.org/zap" +) + +const ( + // StatsMetaHistorySourceAnalyze indicates stats history meta source from analyze + StatsMetaHistorySourceAnalyze = "analyze" + // StatsMetaHistorySourceLoadStats indicates stats history meta source from load stats + StatsMetaHistorySourceLoadStats = "load stats" + // StatsMetaHistorySourceFlushStats indicates stats history meta source from flush stats + StatsMetaHistorySourceFlushStats = "flush stats" + // StatsMetaHistorySourceExtendedStats indicates stats history meta source from extended stats + StatsMetaHistorySourceExtendedStats = "extended stats" + // StatsMetaHistorySourceSchemaChange indicates stats history meta source from schema change + StatsMetaHistorySourceSchemaChange = "schema change" + // StatsMetaHistorySourceFeedBack indicates stats history meta source from feedback + StatsMetaHistorySourceFeedBack = "feedback" +) + +func recordHistoricalStatsMeta(sctx sessionctx.Context, tableID int64, version uint64, source string) error { + if tableID == 0 || version == 0 { + return errors.Errorf("tableID %d, version %d are invalid", tableID, version) + } + historicalStatsEnabled, err := checkHistoricalStatsEnable(sctx) + if err != nil { + return errors.Errorf("check tidb_enable_historical_stats failed: %v", err) + } + if !historicalStatsEnabled { + return nil + } + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats) + exec := sctx.(sqlexec.SQLExecutor) + rexec := sctx.(sqlexec.RestrictedSQLExecutor) + rows, _, err := rexec.ExecRestrictedSQL(ctx, []sqlexec.OptionFuncAlias{sqlexec.ExecOptionUseCurSession}, "select modify_count, count from mysql.stats_meta where table_id = %? and version = %?", tableID, version) + if err != nil { + return errors.Trace(err) + } + if len(rows) == 0 { + return errors.New("no historical meta stats can be recorded") + } + modifyCount, count := rows[0].GetInt64(0), rows[0].GetInt64(1) + + _, err = exec.ExecuteInternal(ctx, "begin pessimistic") + if err != nil { + return errors.Trace(err) + } + defer func() { + err = finishTransaction(ctx, exec, err) + }() + + const sql = "REPLACE INTO mysql.stats_meta_history(table_id, modify_count, count, version, source, create_time) VALUES (%?, %?, %?, %?, %?, NOW())" + if _, err := exec.ExecuteInternal(ctx, sql, tableID, modifyCount, count, version, source); err != nil { + return errors.Trace(err) + } + return nil +} + +func (h *Handle) recordHistoricalStatsMeta(tableID int64, version uint64, source string) { + h.mu.Lock() + defer h.mu.Unlock() + err := recordHistoricalStatsMeta(h.mu.ctx, tableID, version, source) + if err != nil { + logutil.BgLogger().Error("record historical stats meta failed", + zap.Int64("table-id", tableID), + zap.Uint64("version", version), + zap.Error(err)) + } +} diff --git a/statistics/handle/update.go b/statistics/handle/update.go index aaebf78b323d3..d4ffeab2f389d 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -524,7 +524,7 @@ func (h *Handle) dumpTableStatCountToKV(id int64, delta variable.TableDelta) (up statsVer := uint64(0) defer func() { if err == nil && statsVer != 0 { - h.recordHistoricalStatsMeta(id, statsVer) + h.recordHistoricalStatsMeta(id, statsVer, StatsMetaHistorySourceFlushStats) } }() if delta.Count == 0 { @@ -904,7 +904,7 @@ func (h *Handle) deleteOutdatedFeedback(tableID, histID, isIndex int64) error { func (h *Handle) dumpStatsUpdateToKV(tableID, isIndex int64, q *statistics.QueryFeedback, hist *statistics.Histogram, cms *statistics.CMSketch, topN *statistics.TopN, statsVersion int64) error { hist = statistics.UpdateHistogram(hist, q, int(statsVersion)) // feedback for partition is not ready. - err := h.SaveStatsToStorage(tableID, -1, 0, int(isIndex), hist, cms, topN, int(statsVersion), 0, false) + err := h.SaveStatsToStorage(tableID, -1, 0, int(isIndex), hist, cms, topN, int(statsVersion), 0, false, StatsMetaHistorySourceFeedBack) metrics.UpdateStatsCounter.WithLabelValues(metrics.RetLabel(err)).Inc() return errors.Trace(err) } diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index a4cea914c3aa9..390c5ffe8e63a 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -140,7 +140,7 @@ func (c *CopClient) BuildCopIterator(ctx context.Context, req *kv.Request, vars // disable batch copr for follower read req.StoreBatchSize = 0 } - // disable paging for batch copr + // disable batch copr when paging is enabled. if req.Paging.Enable { req.StoreBatchSize = 0 } @@ -315,13 +315,13 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv chanSize = 18 } - tasks := make([]*copTask, 0, len(locs)) - origRangeIdx := 0 - taskID := uint64(0) - var store2Idx map[uint64]int + var builder taskBuilder if req.StoreBatchSize > 0 { - store2Idx = make(map[uint64]int, 16) + builder = newBatchTaskBuilder(bo, req, cache) + } else { + builder = newLegacyTaskBuilder(len(locs)) } + origRangeIdx := 0 for _, loc := range locs { // TiKV will return gRPC error if the message is too large. So we need to limit the length of the ranges slice // to make sure the message can be sent successfully. @@ -357,7 +357,6 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv } } task := &copTask{ - taskID: taskID, region: loc.Location.Region, bucketsVer: loc.getBucketVersion(), ranges: loc.Ranges.Slice(i, nextI), @@ -370,50 +369,138 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *KeyRanges, req *kv requestSource: req.RequestSource, RowCountHint: hint, } - if req.StoreBatchSize > 0 { - batchedTask, err := cache.BuildBatchTask(bo, task, req.ReplicaRead) - if err != nil { - return nil, err - } - if idx, ok := store2Idx[batchedTask.storeID]; !ok || len(tasks[idx].batchTaskList) >= req.StoreBatchSize { - tasks = append(tasks, batchedTask.task) - store2Idx[batchedTask.storeID] = len(tasks) - 1 - } else { - if tasks[idx].batchTaskList == nil { - tasks[idx].batchTaskList = make(map[uint64]*batchedCopTask, req.StoreBatchSize) - // disable paging for batched task. - tasks[idx].paging = false - tasks[idx].pagingSize = 0 - } - if task.RowCountHint > 0 { - tasks[idx].RowCountHint += task.RowCountHint - } - tasks[idx].batchTaskList[taskID] = batchedTask - } - } else { - tasks = append(tasks, task) + if err = builder.handle(task); err != nil { + return nil, err } i = nextI if req.Paging.Enable { pagingSize = paging.GrowPagingSize(pagingSize, req.Paging.MaxPagingSize) } - taskID++ } } if req.Desc { - reverseTasks(tasks) + builder.reverse() } + tasks := builder.build() if elapsed := time.Since(start); elapsed > time.Millisecond*500 { logutil.BgLogger().Warn("buildCopTasks takes too much time", zap.Duration("elapsed", elapsed), zap.Int("range len", rangesLen), zap.Int("task len", len(tasks))) } - metrics.TxnRegionsNumHistogramWithCoprocessor.Observe(float64(len(tasks))) + metrics.TxnRegionsNumHistogramWithCoprocessor.Observe(float64(builder.regionNum())) return tasks, nil } +type taskBuilder interface { + handle(*copTask) error + reverse() + build() []*copTask + regionNum() int +} + +type legacyTaskBuilder struct { + tasks []*copTask +} + +func newLegacyTaskBuilder(hint int) *legacyTaskBuilder { + return &legacyTaskBuilder{ + tasks: make([]*copTask, 0, hint), + } +} + +func (b *legacyTaskBuilder) handle(task *copTask) error { + b.tasks = append(b.tasks, task) + return nil +} + +func (b *legacyTaskBuilder) regionNum() int { + return len(b.tasks) +} + +func (b *legacyTaskBuilder) reverse() { + reverseTasks(b.tasks) +} + +func (b *legacyTaskBuilder) build() []*copTask { + return b.tasks +} + +type batchStoreTaskBuilder struct { + bo *Backoffer + req *kv.Request + cache *RegionCache + taskID uint64 + limit int + store2Idx map[uint64]int + tasks []*copTask +} + +func newBatchTaskBuilder(bo *Backoffer, req *kv.Request, cache *RegionCache) *batchStoreTaskBuilder { + return &batchStoreTaskBuilder{ + bo: bo, + req: req, + cache: cache, + taskID: 0, + limit: req.StoreBatchSize, + store2Idx: make(map[uint64]int, 16), + tasks: make([]*copTask, 0, 16), + } +} + +func (b *batchStoreTaskBuilder) handle(task *copTask) (err error) { + b.taskID++ + task.taskID = b.taskID + handled := false + defer func() { + if !handled && err == nil { + // fallback to non-batch way. It's mainly caused by region miss. + b.tasks = append(b.tasks, task) + } + }() + if b.limit <= 0 { + return nil + } + batchedTask, err := b.cache.BuildBatchTask(b.bo, task, b.req.ReplicaRead) + if err != nil { + return err + } + if batchedTask == nil { + return nil + } + if idx, ok := b.store2Idx[batchedTask.storeID]; !ok || len(b.tasks[idx].batchTaskList) >= b.limit { + b.tasks = append(b.tasks, batchedTask.task) + b.store2Idx[batchedTask.storeID] = len(b.tasks) - 1 + } else { + if b.tasks[idx].batchTaskList == nil { + b.tasks[idx].batchTaskList = make(map[uint64]*batchedCopTask, b.limit) + // disable paging for batched task. + b.tasks[idx].paging = false + b.tasks[idx].pagingSize = 0 + } + if task.RowCountHint > 0 { + b.tasks[idx].RowCountHint += task.RowCountHint + } + b.tasks[idx].batchTaskList[task.taskID] = batchedTask + } + handled = true + return nil +} + +func (b *batchStoreTaskBuilder) regionNum() int { + // we allocate b.taskID for each region task, so the final b.taskID is equal to the related region number. + return int(b.taskID) +} + +func (b *batchStoreTaskBuilder) reverse() { + reverseTasks(b.tasks) +} + +func (b *batchStoreTaskBuilder) build() []*copTask { + return b.tasks +} + func buildTiDBMemCopTasks(ranges *KeyRanges, req *kv.Request) ([]*copTask, error) { servers, err := infosync.GetAllServerInfo(context.Background()) if err != nil { @@ -1138,13 +1225,13 @@ func (worker *copIteratorWorker) handleCopResponse(bo *Backoffer, rpcCtx *tikv.R if err != nil { return remains, err } - return worker.handleBatchRemainsOnErr(bo, remains, resp.pbResp.BatchResponses, task, ch) + return worker.handleBatchRemainsOnErr(bo, remains, resp.pbResp.GetBatchResponses(), task, ch) } if lockErr := resp.pbResp.GetLocked(); lockErr != nil { if err := worker.handleLockErr(bo, lockErr, task); err != nil { return nil, err } - return worker.handleBatchRemainsOnErr(bo, []*copTask{task}, resp.pbResp.BatchResponses, task, ch) + return worker.handleBatchRemainsOnErr(bo, []*copTask{task}, resp.pbResp.GetBatchResponses(), task, ch) } if otherErr := resp.pbResp.GetOtherError(); otherErr != "" { err := errors.Errorf("other error: %s", otherErr) @@ -1250,16 +1337,26 @@ func (worker *copIteratorWorker) handleBatchRemainsOnErr(bo *Backoffer, remains } // handle the batched cop response. +// tasks will be changed, so the input tasks should not be used after calling this function. func (worker *copIteratorWorker) handleBatchCopResponse(bo *Backoffer, batchResps []*coprocessor.StoreBatchTaskResponse, tasks map[uint64]*batchedCopTask, ch chan<- *copResponse) ([]*copTask, error) { if len(tasks) == 0 { return nil, nil } var remainTasks []*copTask + appendRemainTasks := func(tasks ...*copTask) { + if remainTasks == nil { + // allocate size fo remain length + remainTasks = make([]*copTask, 0, len(tasks)) + } + remainTasks = append(remainTasks, tasks...) + } for _, batchResp := range batchResps { - batchedTask, ok := tasks[batchResp.GetTaskId()] + taskID := batchResp.GetTaskId() + batchedTask, ok := tasks[taskID] if !ok { return nil, errors.Errorf("task id %d not found", batchResp.GetTaskId()) } + delete(tasks, taskID) resp := &copResponse{ pbResp: &coprocessor.Response{ Data: batchResp.Data, @@ -1276,7 +1373,7 @@ func (worker *copIteratorWorker) handleBatchCopResponse(bo *Backoffer, batchResp if err != nil { return nil, err } - remainTasks = append(remainTasks, remains...) + appendRemainTasks(remains...) continue } //TODO: handle locks in batch @@ -1284,7 +1381,7 @@ func (worker *copIteratorWorker) handleBatchCopResponse(bo *Backoffer, batchResp if err := worker.handleLockErr(bo, resp.pbResp.GetLocked(), task); err != nil { return nil, err } - remainTasks = append(remainTasks, task) + appendRemainTasks(task) continue } if otherErr := batchResp.GetOtherError(); otherErr != "" { @@ -1312,6 +1409,24 @@ func (worker *copIteratorWorker) handleBatchCopResponse(bo *Backoffer, batchResp // TODO: check OOM worker.sendToRespCh(resp, ch, false) } + for _, t := range tasks { + task := t.task + // when the error is generated by client, response is empty, skip warning for this case. + if len(batchResps) != 0 { + firstRangeStartKey := task.ranges.At(0).StartKey + lastRangeEndKey := task.ranges.At(task.ranges.Len() - 1).EndKey + logutil.Logger(bo.GetCtx()).Error("response of batched task missing", + zap.Uint64("id", task.taskID), + zap.Uint64("txnStartTS", worker.req.StartTs), + zap.Uint64("regionID", task.region.GetID()), + zap.Uint64("bucketsVer", task.bucketsVer), + zap.Int("rangeNums", task.ranges.Len()), + zap.ByteString("firstRangeStartKey", firstRangeStartKey), + zap.ByteString("lastRangeEndKey", lastRangeEndKey), + zap.String("storeAddr", task.storeAddr)) + } + appendRemainTasks(t.task) + } return remainTasks, nil } diff --git a/store/copr/region_cache.go b/store/copr/region_cache.go index a3fd20e036d43..97c3d705c223b 100644 --- a/store/copr/region_cache.go +++ b/store/copr/region_cache.go @@ -18,7 +18,6 @@ import ( "bytes" "strconv" - "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" @@ -210,8 +209,9 @@ func (c *RegionCache) BuildBatchTask(bo *Backoffer, task *copTask, replicaRead k if err != nil { return nil, err } + // fallback to non-batch path if rpcContext == nil { - return nil, errors.Errorf("region %s missing", task.region.String()) + return nil, nil } return &batchedCopTask{ task: task, diff --git a/tests/realtikvtest/addindextest/BUILD.bazel b/tests/realtikvtest/addindextest/BUILD.bazel index 1ca10f9db34f2..a2e9c9906380b 100644 --- a/tests/realtikvtest/addindextest/BUILD.bazel +++ b/tests/realtikvtest/addindextest/BUILD.bazel @@ -33,6 +33,7 @@ go_test( ], embed = [":addindextest"], deps = [ + "//br/pkg/lightning/backend/local", "//config", "//ddl", "//ddl/ingest", diff --git a/tests/realtikvtest/addindextest/integration_test.go b/tests/realtikvtest/addindextest/integration_test.go index 5567113696810..352dc83a1d1a2 100644 --- a/tests/realtikvtest/addindextest/integration_test.go +++ b/tests/realtikvtest/addindextest/integration_test.go @@ -22,6 +22,7 @@ import ( "testing" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/br/pkg/lightning/backend/local" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/ddl/ingest" "github.com/pingcap/tidb/ddl/testutil" @@ -44,6 +45,8 @@ func TestAddIndexIngestMemoryUsage(t *testing.T) { tk.MustExec("use addindexlit;") tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`) + local.RunInTest = true + tk.MustExec("create table t (a int, b int, c int);") var sb strings.Builder sb.WriteString("insert into t values ") @@ -61,6 +64,7 @@ func TestAddIndexIngestMemoryUsage(t *testing.T) { tk.MustExec("alter table t add unique index idx1(b);") tk.MustExec("admin check table t;") require.Equal(t, int64(0), ingest.LitMemRoot.CurrentUsage()) + require.NoError(t, local.LastAlloc.CheckRefCnt()) } func TestAddIndexIngestLimitOneBackend(t *testing.T) { diff --git a/ttl/cache/BUILD.bazel b/ttl/cache/BUILD.bazel index f051716ced40b..8dd9724472dba 100644 --- a/ttl/cache/BUILD.bazel +++ b/ttl/cache/BUILD.bazel @@ -11,6 +11,7 @@ go_library( importpath = "github.com/pingcap/tidb/ttl/cache", visibility = ["//visibility:public"], deps = [ + "//infoschema", "//kv", "//parser/ast", "//parser/model", @@ -46,7 +47,6 @@ go_test( deps = [ "//infoschema", "//kv", - "//parser", "//parser/model", "//server", "//store/helper", diff --git a/ttl/cache/infoschema.go b/ttl/cache/infoschema.go index 5fdf8d2081dc6..87535a4e951b3 100644 --- a/ttl/cache/infoschema.go +++ b/ttl/cache/infoschema.go @@ -17,6 +17,7 @@ package cache import ( "time" + "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/ttl/session" "github.com/pingcap/tidb/util/logutil" @@ -40,7 +41,7 @@ func NewInfoSchemaCache(updateInterval time.Duration) *InfoSchemaCache { // Update updates the info schema cache func (isc *InfoSchemaCache) Update(se session.Session) error { - is := se.SessionInfoSchema() + is := se.GetDomainInfoSchema().(infoschema.InfoSchema) if isc.schemaVer == is.SchemaMetaVersion() { return nil diff --git a/ttl/cache/infoschema_test.go b/ttl/cache/infoschema_test.go index 7e811050b4601..4cec3db563fa7 100644 --- a/ttl/cache/infoschema_test.go +++ b/ttl/cache/infoschema_test.go @@ -18,7 +18,6 @@ import ( "testing" "time" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/server" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/ttl/cache" @@ -27,8 +26,6 @@ import ( ) func TestInfoSchemaCache(t *testing.T) { - parser.TTLFeatureGate = true - store, dom := testkit.CreateMockStoreAndDomain(t) sv := server.CreateMockServer(t, store) sv.SetDomain(dom) diff --git a/ttl/cache/split_test.go b/ttl/cache/split_test.go index d244e852fb904..1d2279eb8d0f9 100644 --- a/ttl/cache/split_test.go +++ b/ttl/cache/split_test.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/tablecodec" @@ -234,6 +233,10 @@ func createTTLTable(t *testing.T, tk *testkit.TestKit, name string, option strin return createTTLTableWithSQL(t, tk, name, fmt.Sprintf("create table test.%s(id %s primary key, t timestamp) TTL = `t` + interval 1 day", name, option)) } +func create2PKTTLTable(t *testing.T, tk *testkit.TestKit, name string, option string) *cache.PhysicalTable { + return createTTLTableWithSQL(t, tk, name, fmt.Sprintf("create table test.%s(id %s, id2 int, t timestamp, primary key(id, id2)) TTL = `t` + interval 1 day", name, option)) +} + func createTTLTableWithSQL(t *testing.T, tk *testkit.TestKit, name string, sql string) *cache.PhysicalTable { tk.MustExec(sql) is, ok := tk.Session().GetDomainInfoSchema().(infoschema.InfoSchema) @@ -264,11 +267,6 @@ func checkRange(t *testing.T, r cache.ScanRange, start, end types.Datum) { } func TestSplitTTLScanRangesWithSignedInt(t *testing.T) { - parser.TTLFeatureGate = true - defer func() { - parser.TTLFeatureGate = false - }() - store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -279,6 +277,7 @@ func TestSplitTTLScanRangesWithSignedInt(t *testing.T) { createTTLTable(t, tk, "t4", "int"), createTTLTable(t, tk, "t5", "bigint"), createTTLTable(t, tk, "t6", ""), // no clustered + create2PKTTLTable(t, tk, "t7", "tinyint"), } tikvStore := newMockTiKVStore(t) @@ -331,11 +330,6 @@ func TestSplitTTLScanRangesWithSignedInt(t *testing.T) { } func TestSplitTTLScanRangesWithUnsignedInt(t *testing.T) { - parser.TTLFeatureGate = true - defer func() { - parser.TTLFeatureGate = false - }() - store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -345,6 +339,7 @@ func TestSplitTTLScanRangesWithUnsignedInt(t *testing.T) { createTTLTable(t, tk, "t3", "mediumint unsigned"), createTTLTable(t, tk, "t4", "int unsigned"), createTTLTable(t, tk, "t5", "bigint unsigned"), + create2PKTTLTable(t, tk, "t6", "tinyint unsigned"), } tikvStore := newMockTiKVStore(t) @@ -400,11 +395,6 @@ func TestSplitTTLScanRangesWithUnsignedInt(t *testing.T) { } func TestSplitTTLScanRangesWithBytes(t *testing.T) { - parser.TTLFeatureGate = true - defer func() { - parser.TTLFeatureGate = false - }() - store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -413,6 +403,7 @@ func TestSplitTTLScanRangesWithBytes(t *testing.T) { createTTLTable(t, tk, "t2", "char(32) CHARACTER SET BINARY"), createTTLTable(t, tk, "t3", "varchar(32) CHARACTER SET BINARY"), createTTLTable(t, tk, "t4", "bit(32)"), + create2PKTTLTable(t, tk, "t5", "binary(32)"), } tikvStore := newMockTiKVStore(t) @@ -454,11 +445,6 @@ func TestSplitTTLScanRangesWithBytes(t *testing.T) { } func TestNoTTLSplitSupportTables(t *testing.T) { - parser.TTLFeatureGate = true - defer func() { - parser.TTLFeatureGate = false - }() - store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -467,6 +453,7 @@ func TestNoTTLSplitSupportTables(t *testing.T) { createTTLTable(t, tk, "t2", "varchar(32) CHARACTER SET UTF8MB4"), createTTLTable(t, tk, "t3", "double"), createTTLTable(t, tk, "t4", "decimal(32, 2)"), + create2PKTTLTable(t, tk, "t5", "char(32) CHARACTER SET UTF8MB4"), } tikvStore := newMockTiKVStore(t) @@ -547,6 +534,14 @@ func TestGetNextBytesHandleDatum(t *testing.T) { key: buildBytesRowKey([]byte{1, 2, 3, 4, 5, 6, 7, 8, 0}), result: []byte{1, 2, 3, 4, 5, 6, 7, 8, 0}, }, + { + key: append(buildBytesRowKey([]byte{1, 2, 3, 4, 5, 6, 7, 8, 0}), 0), + result: []byte{1, 2, 3, 4, 5, 6, 7, 8, 0, 0}, + }, + { + key: append(buildBytesRowKey([]byte{1, 2, 3, 4, 5, 6, 7, 8, 0}), 1), + result: []byte{1, 2, 3, 4, 5, 6, 7, 8, 0, 0}, + }, { key: []byte{}, result: []byte{}, @@ -634,7 +629,7 @@ func TestGetNextBytesHandleDatum(t *testing.T) { bs[len(bs)-10] = 254 return bs }, - result: []byte{1, 2, 3, 4, 5, 6, 7}, + result: []byte{1, 2, 3, 4, 5, 6, 7, 0}, }, { // recordPrefix + bytesFlag + [1, 2, 3, 4, 5, 6, 7, 0, 253, 9, 0, 0, 0, 0, 0, 0, 0, 248] @@ -739,6 +734,18 @@ func TestGetNextIntHandle(t *testing.T) { key: tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(math.MinInt64)), result: math.MinInt64, }, + { + key: append(tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(7)), 0), + result: 8, + }, + { + key: append(tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(math.MaxInt64)), 0), + isNull: true, + }, + { + key: append(tablecodec.EncodeRowKeyWithHandle(tblID, kv.IntHandle(math.MinInt64)), 0), + result: math.MinInt64 + 1, + }, { key: []byte{}, result: math.MinInt64, diff --git a/ttl/cache/table.go b/ttl/cache/table.go index 0cf50d092e437..35aca7f8532b2 100644 --- a/ttl/cache/table.go +++ b/ttl/cache/table.go @@ -166,9 +166,9 @@ func NewPhysicalTable(schema model.CIStr, tbl *model.TableInfo, partition model. }, nil } -// ValidateKey validates a key -func (t *PhysicalTable) ValidateKey(key []types.Datum) error { - if len(t.KeyColumns) != len(key) { +// ValidateKeyPrefix validates a key prefix +func (t *PhysicalTable) ValidateKeyPrefix(key []types.Datum) error { + if len(key) > len(t.KeyColumns) { return errors.Errorf("invalid key length: %d, expected %d", len(key), len(t.KeyColumns)) } return nil @@ -198,7 +198,7 @@ func (t *PhysicalTable) EvalExpireTime(ctx context.Context, se session.Session, // SplitScanRanges split ranges for TTL scan func (t *PhysicalTable) SplitScanRanges(ctx context.Context, store kv.Storage, splitCnt int) ([]ScanRange, error) { - if len(t.KeyColumns) != 1 || splitCnt <= 1 { + if len(t.KeyColumns) < 1 || splitCnt <= 1 { return []ScanRange{newFullRange()}, nil } @@ -431,7 +431,10 @@ func GetNextBytesHandleDatum(key kv.Key, recordPrefix []byte) (d types.Datum) { return d } - if _, v, err := codec.DecodeOne(encodedVal); err == nil { + if remain, v, err := codec.DecodeOne(encodedVal); err == nil { + if len(remain) > 0 { + v.SetBytes(kv.Key(v.GetBytes()).Next()) + } return v } diff --git a/ttl/cache/table_test.go b/ttl/cache/table_test.go index f79d4c3bf4256..ca280d9b36251 100644 --- a/ttl/cache/table_test.go +++ b/ttl/cache/table_test.go @@ -20,7 +20,6 @@ import ( "testing" "time" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/ttl/cache" @@ -29,8 +28,6 @@ import ( ) func TestNewTTLTable(t *testing.T) { - parser.TTLFeatureGate = true - cases := []struct { db string tbl string @@ -166,8 +163,6 @@ func TestNewTTLTable(t *testing.T) { } func TestEvalTTLExpireTime(t *testing.T) { - parser.TTLFeatureGate = true - store, do := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("create table test.t(a int, t datetime) ttl = `t` + interval 1 day") diff --git a/ttl/cache/ttlstatus.go b/ttl/cache/ttlstatus.go index cb1b8ef5942fe..1657105e6c3e7 100644 --- a/ttl/cache/ttlstatus.go +++ b/ttl/cache/ttlstatus.go @@ -40,7 +40,7 @@ const ( JobStatusTimeout = "timeout" ) -const selectFromTTLTableStatus = "SELECT table_id,parent_table_id,table_statistics,last_job_id,last_job_start_time,last_job_finish_time,last_job_ttl_expire,last_job_summary,current_job_id,current_job_owner_id,current_job_owner_addr,current_job_owner_hb_time,current_job_start_time,current_job_ttl_expire,current_job_state,current_job_status,current_job_status_update_time FROM mysql.tidb_ttl_table_status" +const selectFromTTLTableStatus = "SELECT LOW_PRIORITY table_id,parent_table_id,table_statistics,last_job_id,last_job_start_time,last_job_finish_time,last_job_ttl_expire,last_job_summary,current_job_id,current_job_owner_id,current_job_owner_addr,current_job_owner_hb_time,current_job_start_time,current_job_ttl_expire,current_job_state,current_job_status,current_job_status_update_time FROM mysql.tidb_ttl_table_status" // SelectFromTTLTableStatusWithID returns an SQL statement to get the table status from table id func SelectFromTTLTableStatusWithID(tableID int64) string { diff --git a/ttl/metrics/BUILD.bazel b/ttl/metrics/BUILD.bazel new file mode 100644 index 0000000000000..f0666b5c59530 --- /dev/null +++ b/ttl/metrics/BUILD.bazel @@ -0,0 +1,19 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "metrics", + srcs = ["metrics.go"], + importpath = "github.com/pingcap/tidb/ttl/metrics", + visibility = ["//visibility:public"], + deps = [ + "//metrics", + "@com_github_prometheus_client_golang//prometheus", + ], +) + +go_test( + name = "metrics_test", + srcs = ["metrics_test.go"], + embed = [":metrics"], + deps = ["@com_github_stretchr_testify//require"], +) diff --git a/ttl/metrics/metrics.go b/ttl/metrics/metrics.go new file mode 100644 index 0000000000000..8768b0e267388 --- /dev/null +++ b/ttl/metrics/metrics.go @@ -0,0 +1,149 @@ +// 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 metrics + +import ( + "context" + "time" + + "github.com/pingcap/tidb/metrics" + "github.com/prometheus/client_golang/prometheus" +) + +// Phases to trace +var ( + PhaseIdle = "idle" + PhaseBeginTxn = "begin_txn" + PhaseCommitTxn = "commit_txn" + PhaseQuery = "query" + PhaseCheckTTL = "check_ttl" + PhaseWaitRetry = "wait_retry" + PhaseDispatch = "dispatch" + PhaseWaitToken = "wait_token" + PhaseOther = "other" +) + +// TTL metrics +var ( + SelectSuccessDuration = metrics.TTLQueryDuration.With(prometheus.Labels{metrics.LblSQLType: "select", metrics.LblResult: metrics.LblOK}) + SelectErrorDuration = metrics.TTLQueryDuration.With(prometheus.Labels{metrics.LblSQLType: "select", metrics.LblResult: metrics.LblError}) + DeleteSuccessDuration = metrics.TTLQueryDuration.With(prometheus.Labels{metrics.LblSQLType: "delete", metrics.LblResult: metrics.LblOK}) + DeleteErrorDuration = metrics.TTLQueryDuration.With(prometheus.Labels{metrics.LblSQLType: "delete", metrics.LblResult: metrics.LblError}) + + ScannedExpiredRows = metrics.TTLProcessedExpiredRowsCounter.With(prometheus.Labels{metrics.LblSQLType: "select", metrics.LblResult: metrics.LblOK}) + DeleteSuccessExpiredRows = metrics.TTLProcessedExpiredRowsCounter.With(prometheus.Labels{metrics.LblSQLType: "delete", metrics.LblResult: metrics.LblOK}) + DeleteErrorExpiredRows = metrics.TTLProcessedExpiredRowsCounter.With(prometheus.Labels{metrics.LblSQLType: "delete", metrics.LblResult: metrics.LblError}) + + RunningJobsCnt = metrics.TTLJobStatus.With(prometheus.Labels{metrics.LblType: "running"}) + CancellingJobsCnt = metrics.TTLJobStatus.With(prometheus.Labels{metrics.LblType: "cancelling"}) +) + +func initWorkerPhases(workerType string) map[string]prometheus.Counter { + return map[string]prometheus.Counter{ + PhaseIdle: metrics.TTLPhaseTime.WithLabelValues(workerType, PhaseIdle), + PhaseBeginTxn: metrics.TTLPhaseTime.WithLabelValues(workerType, PhaseBeginTxn), + PhaseCommitTxn: metrics.TTLPhaseTime.WithLabelValues(workerType, PhaseCommitTxn), + PhaseQuery: metrics.TTLPhaseTime.WithLabelValues(workerType, PhaseQuery), + PhaseWaitRetry: metrics.TTLPhaseTime.WithLabelValues(workerType, PhaseWaitRetry), + PhaseDispatch: metrics.TTLPhaseTime.WithLabelValues(workerType, PhaseDispatch), + PhaseCheckTTL: metrics.TTLPhaseTime.WithLabelValues(workerType, PhaseCheckTTL), + PhaseWaitToken: metrics.TTLPhaseTime.WithLabelValues(workerType, PhaseWaitToken), + PhaseOther: metrics.TTLPhaseTime.WithLabelValues(workerType, PhaseOther), + } +} + +var scanWorkerPhases = initWorkerPhases("scan_worker") +var deleteWorkerPhases = initWorkerPhases("delete_worker") + +// PhaseTracer is used to tracer the phases duration +type PhaseTracer struct { + getTime func() time.Time + recordDuration func(phase string, duration time.Duration) + + phase string + phaseTime time.Time +} + +// NewScanWorkerPhaseTracer returns a tracer for scan worker +func NewScanWorkerPhaseTracer() *PhaseTracer { + return newPhaseTracer(time.Now, func(status string, duration time.Duration) { + if counter, ok := scanWorkerPhases[status]; ok { + counter.Add(duration.Seconds()) + } + }) +} + +// NewDeleteWorkerPhaseTracer returns a tracer for delete worker +func NewDeleteWorkerPhaseTracer() *PhaseTracer { + return newPhaseTracer(time.Now, func(status string, duration time.Duration) { + if counter, ok := deleteWorkerPhases[status]; ok { + counter.Add(duration.Seconds()) + } + }) +} + +func newPhaseTracer(getTime func() time.Time, recordDuration func(status string, duration time.Duration)) *PhaseTracer { + return &PhaseTracer{ + getTime: getTime, + recordDuration: recordDuration, + phaseTime: getTime(), + } +} + +// Phase returns the current phase +func (t *PhaseTracer) Phase() string { + if t == nil { + return "" + } + return t.phase +} + +// EnterPhase enters into a new phase +func (t *PhaseTracer) EnterPhase(phase string) { + if t == nil { + return + } + + now := t.getTime() + if t.phase != "" { + t.recordDuration(t.phase, now.Sub(t.phaseTime)) + } + + t.phase = phase + t.phaseTime = now +} + +// EndPhase ends the current phase +func (t *PhaseTracer) EndPhase() { + if t == nil { + return + } + t.EnterPhase("") +} + +const ttlPhaseTraceKey = "ttlPhaseTraceKey" + +// CtxWithPhaseTracer create a new context with tracer +func CtxWithPhaseTracer(ctx context.Context, tracer *PhaseTracer) context.Context { + return context.WithValue(ctx, ttlPhaseTraceKey, tracer) +} + +// PhaseTracerFromCtx returns a tracer from a given context +func PhaseTracerFromCtx(ctx context.Context) *PhaseTracer { + if tracer, ok := ctx.Value(ttlPhaseTraceKey).(*PhaseTracer); ok { + return tracer + } + return nil +} diff --git a/ttl/metrics/metrics_test.go b/ttl/metrics/metrics_test.go new file mode 100644 index 0000000000000..68ca303756ce0 --- /dev/null +++ b/ttl/metrics/metrics_test.go @@ -0,0 +1,70 @@ +// 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 metrics + +import ( + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestPhaseTracer(t *testing.T) { + tm := time.Now() + getTime := func() time.Time { + return tm + } + + lastReportStatus := "" + lastReportDuration := time.Duration(0) + resetReport := func() { + lastReportStatus = "" + lastReportDuration = time.Duration(0) + } + + tracer := newPhaseTracer(getTime, func(status string, duration time.Duration) { + require.Equal(t, "", lastReportStatus) + require.Equal(t, int64(0), lastReportDuration.Nanoseconds()) + lastReportStatus = status + lastReportDuration = duration + }) + + resetReport() + tm = tm.Add(time.Second * 2) + tracer.EnterPhase("p1") + require.Equal(t, "", lastReportStatus) + require.Equal(t, int64(0), lastReportDuration.Nanoseconds()) + require.Equal(t, "p1", tracer.Phase()) + + tm = tm.Add(time.Second * 5) + tracer.EnterPhase("p2") + require.Equal(t, "p1", lastReportStatus) + require.Equal(t, time.Second*5, lastReportDuration) + require.Equal(t, "p2", tracer.Phase()) + + resetReport() + tm = tm.Add(time.Second * 10) + tracer.EnterPhase("p2") + require.Equal(t, "p2", lastReportStatus) + require.Equal(t, time.Second*10, lastReportDuration) + require.Equal(t, "p2", tracer.Phase()) + + resetReport() + tm = tm.Add(time.Second * 20) + tracer.EndPhase() + require.Equal(t, "p2", lastReportStatus) + require.Equal(t, time.Second*20, lastReportDuration) + require.Equal(t, "", tracer.Phase()) +} diff --git a/ttl/session/BUILD.bazel b/ttl/session/BUILD.bazel index 6d28bff0730dc..2c9dae3fc426f 100644 --- a/ttl/session/BUILD.bazel +++ b/ttl/session/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//sessionctx", "//sessionctx/variable", "//sessiontxn", + "//ttl/metrics", "//util/chunk", "//util/sqlexec", "@com_github_pingcap_errors//:errors", diff --git a/ttl/session/session.go b/ttl/session/session.go index 927b5f570bc92..d07419651a103 100644 --- a/ttl/session/session.go +++ b/ttl/session/session.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/sessiontxn" + "github.com/pingcap/tidb/ttl/metrics" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" ) @@ -94,15 +95,20 @@ func (s *session) ExecuteSQL(ctx context.Context, sql string, args ...interface{ // RunInTxn executes the specified function in a txn func (s *session) RunInTxn(ctx context.Context, fn func() error) (err error) { + tracer := metrics.PhaseTracerFromCtx(ctx) + defer tracer.EnterPhase(tracer.Phase()) + + tracer.EnterPhase(metrics.PhaseBeginTxn) if _, err = s.ExecuteSQL(ctx, "BEGIN"); err != nil { return err } + tracer.EnterPhase(metrics.PhaseOther) success := false defer func() { if !success { - _, err = s.ExecuteSQL(ctx, "ROLLBACK") - terror.Log(err) + _, rollbackErr := s.ExecuteSQL(ctx, "ROLLBACK") + terror.Log(rollbackErr) } }() @@ -110,9 +116,11 @@ func (s *session) RunInTxn(ctx context.Context, fn func() error) (err error) { return err } + tracer.EnterPhase(metrics.PhaseCommitTxn) if _, err = s.ExecuteSQL(ctx, "COMMIT"); err != nil { return err } + tracer.EnterPhase(metrics.PhaseOther) success = true return err diff --git a/ttl/session/session_test.go b/ttl/session/session_test.go index a30949206223a..ecabbc8683158 100644 --- a/ttl/session/session_test.go +++ b/ttl/session/session_test.go @@ -39,10 +39,11 @@ func TestSessionRunInTxn(t *testing.T) { })) tk2.MustQuery("select * from t order by id asc").Check(testkit.Rows("1 10")) - require.NoError(t, se.RunInTxn(context.TODO(), func() error { + err := se.RunInTxn(context.TODO(), func() error { tk.MustExec("insert into t values (2, 20)") - return errors.New("err") - })) + return errors.New("mockErr") + }) + require.EqualError(t, err, "mockErr") tk2.MustQuery("select * from t order by id asc").Check(testkit.Rows("1 10")) require.NoError(t, se.RunInTxn(context.TODO(), func() error { diff --git a/ttl/sqlbuilder/sql.go b/ttl/sqlbuilder/sql.go index 833327c7af404..c9e4181ccfdda 100644 --- a/ttl/sqlbuilder/sql.go +++ b/ttl/sqlbuilder/sql.go @@ -319,16 +319,12 @@ type ScanQueryGenerator struct { // NewScanQueryGenerator creates a new ScanQueryGenerator func NewScanQueryGenerator(tbl *cache.PhysicalTable, expire time.Time, rangeStart []types.Datum, rangeEnd []types.Datum) (*ScanQueryGenerator, error) { - if len(rangeStart) > 0 { - if err := tbl.ValidateKey(rangeStart); err != nil { - return nil, err - } + if err := tbl.ValidateKeyPrefix(rangeStart); err != nil { + return nil, err } - if len(rangeEnd) > 0 { - if err := tbl.ValidateKey(rangeEnd); err != nil { - return nil, err - } + if err := tbl.ValidateKeyPrefix(rangeEnd); err != nil { + return nil, err } return &ScanQueryGenerator{ @@ -393,11 +389,11 @@ func (g *ScanQueryGenerator) setStack(key []types.Datum) error { return nil } - if err := g.tbl.ValidateKey(key); err != nil { + if err := g.tbl.ValidateKeyPrefix(key); err != nil { return err } - g.stack = g.stack[:cap(g.stack)] + g.stack = g.stack[:len(key)] for i := 0; i < len(key); i++ { g.stack[i] = key[0 : i+1] } @@ -440,7 +436,7 @@ func (g *ScanQueryGenerator) buildSQL() (string, error) { } if len(g.keyRangeEnd) > 0 { - if err := b.WriteCommonCondition(g.tbl.KeyColumns, "<", g.keyRangeEnd); err != nil { + if err := b.WriteCommonCondition(g.tbl.KeyColumns[0:len(g.keyRangeEnd)], "<", g.keyRangeEnd); err != nil { return "", err } } diff --git a/ttl/sqlbuilder/sql_test.go b/ttl/sqlbuilder/sql_test.go index 3bc982a092014..ca7719d59574e 100644 --- a/ttl/sqlbuilder/sql_test.go +++ b/ttl/sqlbuilder/sql_test.go @@ -702,6 +702,54 @@ func TestScanQueryGenerator(t *testing.T) { }, }, }, + { + tbl: t2, + expire: time.UnixMilli(0).In(time.UTC), + rangeStart: d(1), + rangeEnd: d(100), + path: [][]interface{}{ + { + nil, 5, + "SELECT LOW_PRIORITY `a`, `b`, `c` FROM `test2`.`t2` WHERE `a` >= 1 AND `a` < 100 AND `time` < '1970-01-01 00:00:00' ORDER BY `a`, `b`, `c` ASC LIMIT 5", + }, + { + result(d(1, "x", []byte{0x1a}), 5), 5, + "SELECT LOW_PRIORITY `a`, `b`, `c` FROM `test2`.`t2` WHERE `a` = 1 AND `b` = 'x' AND `c` > x'1a' AND `a` < 100 AND `time` < '1970-01-01 00:00:00' ORDER BY `a`, `b`, `c` ASC LIMIT 5", + }, + { + result(d(1, "x", []byte{0x20}), 4), 5, + "SELECT LOW_PRIORITY `a`, `b`, `c` FROM `test2`.`t2` WHERE `a` = 1 AND `b` > 'x' AND `a` < 100 AND `time` < '1970-01-01 00:00:00' ORDER BY `a`, `b`, `c` ASC LIMIT 5", + }, + { + result(d(1, "y", []byte{0x0a}), 4), 5, + "SELECT LOW_PRIORITY `a`, `b`, `c` FROM `test2`.`t2` WHERE `a` > 1 AND `a` < 100 AND `time` < '1970-01-01 00:00:00' ORDER BY `a`, `b`, `c` ASC LIMIT 5", + }, + }, + }, + { + tbl: t2, + expire: time.UnixMilli(0).In(time.UTC), + rangeStart: d(1, "x"), + rangeEnd: d(100, "z"), + path: [][]interface{}{ + { + nil, 5, + "SELECT LOW_PRIORITY `a`, `b`, `c` FROM `test2`.`t2` WHERE `a` = 1 AND `b` >= 'x' AND (`a`, `b`) < (100, 'z') AND `time` < '1970-01-01 00:00:00' ORDER BY `a`, `b`, `c` ASC LIMIT 5", + }, + { + result(d(1, "x", []byte{0x1a}), 5), 5, + "SELECT LOW_PRIORITY `a`, `b`, `c` FROM `test2`.`t2` WHERE `a` = 1 AND `b` = 'x' AND `c` > x'1a' AND (`a`, `b`) < (100, 'z') AND `time` < '1970-01-01 00:00:00' ORDER BY `a`, `b`, `c` ASC LIMIT 5", + }, + { + result(d(1, "x", []byte{0x20}), 4), 5, + "SELECT LOW_PRIORITY `a`, `b`, `c` FROM `test2`.`t2` WHERE `a` = 1 AND `b` > 'x' AND (`a`, `b`) < (100, 'z') AND `time` < '1970-01-01 00:00:00' ORDER BY `a`, `b`, `c` ASC LIMIT 5", + }, + { + result(d(1, "y", []byte{0x0a}), 4), 5, + "SELECT LOW_PRIORITY `a`, `b`, `c` FROM `test2`.`t2` WHERE `a` > 1 AND (`a`, `b`) < (100, 'z') AND `time` < '1970-01-01 00:00:00' ORDER BY `a`, `b`, `c` ASC LIMIT 5", + }, + }, + }, } for i, c := range cases { diff --git a/ttl/ttlworker/BUILD.bazel b/ttl/ttlworker/BUILD.bazel index a06a4af2d27c0..8b7c39270807e 100644 --- a/ttl/ttlworker/BUILD.bazel +++ b/ttl/ttlworker/BUILD.bazel @@ -19,6 +19,7 @@ go_library( "//sessionctx", "//sessionctx/variable", "//ttl/cache", + "//ttl/metrics", "//ttl/session", "//ttl/sqlbuilder", "//types", @@ -30,7 +31,6 @@ go_library( "@com_github_ngaut_pools//:pools", "@com_github_pingcap_errors//:errors", "@org_golang_x_time//rate", - "@org_uber_go_atomic//:atomic", "@org_uber_go_multierr//:multierr", "@org_uber_go_zap//:zap", ], diff --git a/ttl/ttlworker/config.go b/ttl/ttlworker/config.go index e7a8e344c3e16..a92f362241fcf 100644 --- a/ttl/ttlworker/config.go +++ b/ttl/ttlworker/config.go @@ -16,35 +16,14 @@ package ttlworker import ( "time" - - "go.uber.org/atomic" ) -// TODO: the following functions should be put in the variable pkg to avoid cyclic dependency after adding variables for the TTL -// some of them are only used in test - const jobManagerLoopTickerInterval = 10 * time.Second -const updateInfoSchemaCacheInterval = time.Minute -const updateTTLTableStatusCacheInterval = 10 * time.Minute +const updateInfoSchemaCacheInterval = 2 * time.Minute +const updateTTLTableStatusCacheInterval = 2 * time.Minute const ttlInternalSQLTimeout = 30 * time.Second -const ttlJobTimeout = 6 * time.Hour - -// TODO: add this variable to the sysvar -const ttlJobInterval = time.Hour - -// TODO: add these variables to the sysvar -var ttlJobScheduleWindowStartTime, _ = time.Parse(timeFormat, "2006-01-02 00:00:00") -var ttlJobScheduleWindowEndTime, _ = time.Parse(timeFormat, "2006-01-02 23:59:00") - -// TODO: migrate these two count to sysvar - -// ScanWorkersCount defines the count of scan worker -var ScanWorkersCount = atomic.NewUint64(0) - -// DeleteWorkerCount defines the count of delete worker -var DeleteWorkerCount = atomic.NewUint64(0) - const resizeWorkersInterval = 30 * time.Second const splitScanCount = 64 +const ttlJobTimeout = 6 * time.Hour diff --git a/ttl/ttlworker/del.go b/ttl/ttlworker/del.go index eb86b1f3c6cf0..8f66fb7fad246 100644 --- a/ttl/ttlworker/del.go +++ b/ttl/ttlworker/del.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/ttl/cache" + "github.com/pingcap/tidb/ttl/metrics" "github.com/pingcap/tidb/ttl/session" "github.com/pingcap/tidb/ttl/sqlbuilder" "github.com/pingcap/tidb/types" @@ -85,6 +86,10 @@ type ttlDeleteTask struct { } func (t *ttlDeleteTask) doDelete(ctx context.Context, rawSe session.Session) (retryRows [][]types.Datum) { + tracer := metrics.PhaseTracerFromCtx(ctx) + defer tracer.EnterPhase(tracer.Phase()) + tracer.EnterPhase(metrics.PhaseOther) + leftRows := t.rows se := newTableSession(rawSe, t.tbl, t.expire) for len(leftRows) > 0 { @@ -108,13 +113,18 @@ func (t *ttlDeleteTask) doDelete(ctx context.Context, rawSe session.Session) (re ) } + tracer.EnterPhase(metrics.PhaseWaitToken) if err = globalDelRateLimiter.Wait(ctx); err != nil { t.statistics.IncErrorRows(len(delBatch)) return } + tracer.EnterPhase(metrics.PhaseOther) + sqlStart := time.Now() _, needRetry, err := se.ExecuteSQLWithCheck(ctx, sql) + sqlInterval := time.Since(sqlStart) if err != nil { + metrics.DeleteErrorDuration.Observe(sqlInterval.Seconds()) needRetry = needRetry && ctx.Err() == nil logutil.BgLogger().Warn( "delete SQL in TTL failed", @@ -134,6 +144,7 @@ func (t *ttlDeleteTask) doDelete(ctx context.Context, rawSe session.Session) (re continue } + metrics.DeleteSuccessDuration.Observe(sqlInterval.Seconds()) t.statistics.IncSuccessRows(len(delBatch)) } return retryRows @@ -243,12 +254,16 @@ func newDeleteWorker(delCh <-chan *ttlDeleteTask, sessPool sessionPool) *ttlDele } func (w *ttlDeleteWorker) loop() error { + tracer := metrics.NewDeleteWorkerPhaseTracer() + defer tracer.EndPhase() + + tracer.EnterPhase(metrics.PhaseOther) se, err := getSession(w.sessionPool) if err != nil { return err } - ctx := w.baseWorker.ctx + ctx := metrics.CtxWithPhaseTracer(w.baseWorker.ctx, tracer) doRetry := func(task *ttlDeleteTask) [][]types.Datum { return task.doDelete(ctx, se) @@ -258,13 +273,16 @@ func (w *ttlDeleteWorker) loop() error { defer timer.Stop() for w.Status() == workerStatusRunning { + tracer.EnterPhase(metrics.PhaseIdle) select { case <-ctx.Done(): return nil case <-timer.C: + tracer.EnterPhase(metrics.PhaseOther) nextInterval := w.retryBuffer.DoRetry(doRetry) timer.Reset(nextInterval) case task, ok := <-w.delCh: + tracer.EnterPhase(metrics.PhaseOther) if !ok { return nil } diff --git a/ttl/ttlworker/job.go b/ttl/ttlworker/job.go index 0d9d65bf54fe9..5551984cceb08 100644 --- a/ttl/ttlworker/job.go +++ b/ttl/ttlworker/job.go @@ -83,7 +83,11 @@ func (job *ttlJob) changeStatus(ctx context.Context, se session.Session, status } func (job *ttlJob) updateState(ctx context.Context, se session.Session) error { - _, err := se.ExecuteSQL(ctx, updateJobState(job.tbl.ID, job.id, job.statistics.String(), job.ownerID)) + jsonStatistics, err := job.statistics.MarshalJSON() + if err != nil { + return err + } + _, err = se.ExecuteSQL(ctx, updateJobState(job.tbl.ID, job.id, string(jsonStatistics), job.ownerID)) if err != nil { return errors.Trace(err) } diff --git a/ttl/ttlworker/job_manager.go b/ttl/ttlworker/job_manager.go index a141cfc046fb3..cd3c1208f1f37 100644 --- a/ttl/ttlworker/job_manager.go +++ b/ttl/ttlworker/job_manager.go @@ -21,7 +21,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/ttl/cache" + "github.com/pingcap/tidb/ttl/metrics" "github.com/pingcap/tidb/ttl/session" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/timeutil" @@ -117,6 +119,7 @@ func (m *JobManager) jobLoop() error { tableStatusCacheUpdateTicker := time.Tick(m.tableStatusCache.GetInterval()) resizeWorkersTicker := time.Tick(resizeWorkersInterval) for { + m.reportMetrics() now := se.Now() select { @@ -140,18 +143,22 @@ func (m *JobManager) jobLoop() error { } cancel() case <-updateScanTaskStateTicker: - m.updateTaskState() + if m.updateTaskState() { + m.rescheduleJobs(se, now) + } case <-m.notifyStateCh: - m.updateTaskState() + if m.updateTaskState() { + m.rescheduleJobs(se, now) + } case <-jobCheckTicker: m.checkFinishedJob(se, now) m.checkNotOwnJob() case <-resizeWorkersTicker: - err := m.resizeScanWorkers(int(ScanWorkersCount.Load())) + err := m.resizeScanWorkers(int(variable.TTLScanWorkerCount.Load())) if err != nil { logutil.Logger(m.ctx).Warn("fail to resize scan workers", zap.Error(err)) } - err = m.resizeDelWorkers(int(DeleteWorkerCount.Load())) + err = m.resizeDelWorkers(int(variable.TTLDeleteWorkerCount.Load())) if err != nil { logutil.Logger(m.ctx).Warn("fail to resize delete workers", zap.Error(err)) } @@ -161,41 +168,89 @@ func (m *JobManager) jobLoop() error { } } +func (m *JobManager) reportMetrics() { + var runningJobs, cancellingJobs float64 + for _, job := range m.runningJobs { + switch job.status { + case cache.JobStatusRunning: + runningJobs++ + case cache.JobStatusCancelling: + cancellingJobs++ + } + } + metrics.RunningJobsCnt.Set(runningJobs) + metrics.CancellingJobsCnt.Set(cancellingJobs) +} + func (m *JobManager) resizeScanWorkers(count int) error { var err error - m.scanWorkers, err = m.resizeWorkers(m.scanWorkers, count, func() worker { + var canceledWorkers []worker + m.scanWorkers, canceledWorkers, err = m.resizeWorkers(m.scanWorkers, count, func() worker { return newScanWorker(m.delCh, m.notifyStateCh, m.sessPool) }) + for _, w := range canceledWorkers { + s := w.(scanWorker) + + var tableID int64 + var scanErr error + result := s.PollTaskResult() + if result != nil { + tableID = result.task.tbl.ID + scanErr = result.err + } else { + // if the scan worker failed to poll the task, it's possible that the `WaitStopped` has timeout + // we still consider the scan task as finished + curTask := s.CurrentTask() + if curTask == nil { + continue + } + tableID = curTask.tbl.ID + scanErr = errors.New("timeout to cancel scan task") + } + + job := findJobWithTableID(m.runningJobs, tableID) + if job == nil { + logutil.Logger(m.ctx).Warn("task state changed but job not found", zap.Int64("tableID", tableID)) + continue + } + logutil.Logger(m.ctx).Debug("scan task finished", zap.String("jobID", job.id)) + job.finishedScanTaskCounter += 1 + job.scanTaskErr = multierr.Append(job.scanTaskErr, scanErr) + } return err } func (m *JobManager) resizeDelWorkers(count int) error { var err error - m.delWorkers, err = m.resizeWorkers(m.delWorkers, count, func() worker { + m.delWorkers, _, err = m.resizeWorkers(m.delWorkers, count, func() worker { return newDeleteWorker(m.delCh, m.sessPool) }) return err } -func (m *JobManager) resizeWorkers(workers []worker, count int, factory func() worker) ([]worker, error) { +// resizeWorkers scales the worker, and returns the full set of workers as the first return value. If there are workers +// stopped, return the stopped worker in the second return value +func (m *JobManager) resizeWorkers(workers []worker, count int, factory func() worker) ([]worker, []worker, error) { if count < len(workers) { logutil.Logger(m.ctx).Info("shrink ttl worker", zap.Int("originalCount", len(workers)), zap.Int("newCount", count)) for _, w := range workers[count:] { w.Stop() } + var errs error + ctx, cancel := context.WithTimeout(m.ctx, 30*time.Second) for _, w := range workers[count:] { - err := w.WaitStopped(m.ctx, 30*time.Second) + err := w.WaitStopped(ctx, 30*time.Second) if err != nil { logutil.Logger(m.ctx).Warn("fail to stop ttl worker", zap.Error(err)) errs = multierr.Append(errs, err) } } + cancel() // remove the existing workers, and keep the left workers - workers = workers[:count] - return workers, errs + return workers[:count], workers[count:], errs } if count > len(workers) { @@ -206,29 +261,34 @@ func (m *JobManager) resizeWorkers(workers []worker, count int, factory func() w w.Start() workers = append(workers, w) } - return workers, nil + return workers, nil, nil } - return workers, nil + return workers, nil, nil } -func (m *JobManager) updateTaskState() { +// updateTaskState polls the result from scan worker and returns whether there are result polled +func (m *JobManager) updateTaskState() bool { results := m.pollScanWorkerResults() for _, result := range results { job := findJobWithTableID(m.runningJobs, result.task.tbl.ID) - if job != nil { - logutil.Logger(m.ctx).Debug("scan task state changed", zap.String("jobID", job.id)) - - job.finishedScanTaskCounter += 1 - job.scanTaskErr = multierr.Append(job.scanTaskErr, result.err) + if job == nil { + logutil.Logger(m.ctx).Warn("task state changed but job not found", zap.Int64("tableID", result.task.tbl.ID)) + continue } + logutil.Logger(m.ctx).Debug("scan task finished", zap.String("jobID", job.id)) + + job.finishedScanTaskCounter += 1 + job.scanTaskErr = multierr.Append(job.scanTaskErr, result.err) } + + return len(results) > 0 } func (m *JobManager) pollScanWorkerResults() []*ttlScanTaskExecResult { results := make([]*ttlScanTaskExecResult, 0, len(m.scanWorkers)) for _, w := range m.scanWorkers { - worker := w.(*ttlScanWorker) + worker := w.(scanWorker) result := worker.PollTaskResult() if result != nil { results = append(results, result) @@ -271,7 +331,7 @@ func (m *JobManager) checkFinishedJob(se session.Session, now time.Time) { } func (m *JobManager) rescheduleJobs(se session.Session, now time.Time) { - if !timeutil.WithinDayTimePeriod(ttlJobScheduleWindowStartTime, ttlJobScheduleWindowEndTime, now) { + if !timeutil.WithinDayTimePeriod(variable.TTLJobScheduleWindowStartTime.Load(), variable.TTLJobScheduleWindowEndTime.Load(), now) { // Local jobs will also not run, but as the server is still sending heartbeat, // and keep the job in memory, it could start the left task in the next window. return @@ -378,7 +438,18 @@ func (m *JobManager) localJobs() []*ttlJob { // readyForNewJobTables returns all tables which should spawn a TTL job according to cache func (m *JobManager) readyForNewJobTables(now time.Time) []*cache.PhysicalTable { tables := make([]*cache.PhysicalTable, 0, len(m.infoSchemaCache.Tables)) + +tblLoop: for _, table := range m.infoSchemaCache.Tables { + // If this node already has a job for this table, just ignore. + // Actually, the logic should ensure this condition never meet, we still add the check here to keep safety + // (especially when the content of the status table is incorrect) + for _, job := range m.runningJobs { + if job.tbl.ID == table.ID { + continue tblLoop + } + } + status := m.tableStatusCache.Tables[table.ID] ok := m.couldTrySchedule(status, now) if ok { @@ -413,7 +484,7 @@ func (m *JobManager) couldTrySchedule(table *cache.TableStatus, now time.Time) b finishTime := table.LastJobFinishTime - return finishTime.Add(ttlJobInterval).Before(now) + return finishTime.Add(variable.TTLJobRunInterval.Load()).Before(now) } // occupyNewJob tries to occupy a new job in the ttl_table_status table. If it locks successfully, it will create a new diff --git a/ttl/ttlworker/job_manager_test.go b/ttl/ttlworker/job_manager_test.go index 7261eb2edf8f7..87bc19ca08261 100644 --- a/ttl/ttlworker/job_manager_test.go +++ b/ttl/ttlworker/job_manager_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/ttl/cache" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -132,7 +133,7 @@ func newTTLTableStatusRows(status ...*cache.TableStatus) []chunk.Row { return rows } -var updateStatusSQL = "SELECT table_id,parent_table_id,table_statistics,last_job_id,last_job_start_time,last_job_finish_time,last_job_ttl_expire,last_job_summary,current_job_id,current_job_owner_id,current_job_owner_addr,current_job_owner_hb_time,current_job_start_time,current_job_ttl_expire,current_job_state,current_job_status,current_job_status_update_time FROM mysql.tidb_ttl_table_status" +var updateStatusSQL = "SELECT LOW_PRIORITY table_id,parent_table_id,table_statistics,last_job_id,last_job_start_time,last_job_finish_time,last_job_ttl_expire,last_job_summary,current_job_id,current_job_owner_id,current_job_owner_addr,current_job_owner_hb_time,current_job_start_time,current_job_ttl_expire,current_job_state,current_job_status,current_job_status_update_time FROM mysql.tidb_ttl_table_status" func (m *JobManager) SetScanWorkers4Test(workers []worker) { m.scanWorkers = workers @@ -305,7 +306,7 @@ func TestResizeWorkers(t *testing.T) { m.SetScanWorkers4Test([]worker{ scanWorker1, }) - newWorkers, err := m.resizeWorkers(m.scanWorkers, 2, func() worker { + newWorkers, _, err := m.resizeWorkers(m.scanWorkers, 2, func() worker { return scanWorker2 }) assert.NoError(t, err) @@ -327,6 +328,24 @@ func TestResizeWorkers(t *testing.T) { assert.NoError(t, m.resizeScanWorkers(1)) scanWorker2.checkWorkerStatus(workerStatusStopped, false, nil) + + // shrink scan workers after job is run + scanWorker1 = newMockScanWorker(t) + scanWorker1.Start() + scanWorker2 = newMockScanWorker(t) + scanWorker2.Start() + + m = NewJobManager("test-id", newMockSessionPool(t, tbl), nil) + m.SetScanWorkers4Test([]worker{ + scanWorker1, + scanWorker2, + }) + m.runningJobs = append(m.runningJobs, &ttlJob{tbl: tbl}) + + scanWorker2.curTaskResult = &ttlScanTaskExecResult{task: &ttlScanTask{tbl: tbl}} + assert.NoError(t, m.resizeScanWorkers(1)) + scanWorker2.checkWorkerStatus(workerStatusStopped, false, nil) + assert.Equal(t, m.runningJobs[0].finishedScanTaskCounter, 1) } func TestLocalJobs(t *testing.T) { @@ -426,16 +445,18 @@ func TestRescheduleJobsOutOfWindow(t *testing.T) { }, } m.runningJobs = []*ttlJob{newMockTTLJob(tbl, cache.JobStatusWaiting)} - savedttlJobScheduleWindowStartTime := ttlJobScheduleWindowStartTime - savedttlJobScheduleWindowEndTime := ttlJobScheduleWindowEndTime - ttlJobScheduleWindowStartTime, _ = time.Parse(timeFormat, "2022-12-06 12:00:00") - ttlJobScheduleWindowEndTime, _ = time.Parse(timeFormat, "2022-12-06 12:05:00") + savedttlJobScheduleWindowStartTime := variable.TTLJobScheduleWindowStartTime.Load() + savedttlJobScheduleWindowEndTime := variable.TTLJobScheduleWindowEndTime.Load() + ttlJobScheduleWindowStartTime, _ := time.ParseInLocation(variable.FullDayTimeFormat, "12:00 +0000", time.UTC) + variable.TTLJobScheduleWindowStartTime.Store(ttlJobScheduleWindowStartTime) + ttlJobScheduleWindowEndTime, _ := time.ParseInLocation(variable.FullDayTimeFormat, "12:05 +0000", time.UTC) + variable.TTLJobScheduleWindowEndTime.Store(ttlJobScheduleWindowEndTime) defer func() { - ttlJobScheduleWindowStartTime = savedttlJobScheduleWindowStartTime - ttlJobScheduleWindowEndTime = savedttlJobScheduleWindowEndTime + variable.TTLJobScheduleWindowStartTime.Store(savedttlJobScheduleWindowStartTime) + variable.TTLJobScheduleWindowEndTime.Store(savedttlJobScheduleWindowEndTime) }() - now, _ := time.Parse(timeFormat, "2022-12-06 12:06:00") + now, _ := time.ParseInLocation(variable.FullDayTimeFormat, "12:06 +0000", time.UTC) m.rescheduleJobs(se, now) scanWorker1.checkWorkerStatus(workerStatusRunning, true, nil) scanWorker1.checkPollResult(false, "") @@ -443,7 +464,7 @@ func TestRescheduleJobsOutOfWindow(t *testing.T) { scanWorker2.checkPollResult(false, "") // jobs will be scheduled within the time window - now, _ = time.Parse(timeFormat, "2022-12-06 12:02:00") + now, _ = time.ParseInLocation(variable.FullDayTimeFormat, "12:02 +0000", time.UTC) m.rescheduleJobs(se, now) scanWorker1.checkWorkerStatus(workerStatusRunning, false, m.runningJobs[0].tasks[0]) scanWorker1.checkPollResult(false, "") diff --git a/ttl/ttlworker/scan.go b/ttl/ttlworker/scan.go index 538d3aeefc070..242c51fb8b686 100644 --- a/ttl/ttlworker/scan.go +++ b/ttl/ttlworker/scan.go @@ -16,6 +16,7 @@ package ttlworker import ( "context" + "encoding/json" "fmt" "strconv" "sync/atomic" @@ -25,6 +26,7 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/ttl/cache" + "github.com/pingcap/tidb/ttl/metrics" "github.com/pingcap/tidb/ttl/sqlbuilder" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -46,14 +48,17 @@ type ttlStatistics struct { } func (s *ttlStatistics) IncTotalRows(cnt int) { + metrics.ScannedExpiredRows.Add(float64(cnt)) s.TotalRows.Add(uint64(cnt)) } func (s *ttlStatistics) IncSuccessRows(cnt int) { + metrics.DeleteSuccessExpiredRows.Add(float64(cnt)) s.SuccessRows.Add(uint64(cnt)) } func (s *ttlStatistics) IncErrorRows(cnt int) { + metrics.DeleteErrorExpiredRows.Add(float64(cnt)) s.ErrorRows.Add(uint64(cnt)) } @@ -67,6 +72,20 @@ func (s *ttlStatistics) String() string { return fmt.Sprintf("Total Rows: %d, Success Rows: %d, Error Rows: %d", s.TotalRows.Load(), s.SuccessRows.Load(), s.ErrorRows.Load()) } +func (s *ttlStatistics) MarshalJSON() ([]byte, error) { + type jsonStatistics struct { + TotalRows uint64 `json:"total_rows"` + SuccessRows uint64 `json:"success_rows"` + ErrorRows uint64 `json:"error_rows"` + } + + return json.Marshal(jsonStatistics{ + TotalRows: s.TotalRows.Load(), + SuccessRows: s.SuccessRows.Load(), + ErrorRows: s.ErrorRows.Load(), + }) +} + type ttlScanTask struct { ctx context.Context @@ -96,9 +115,11 @@ func (t *ttlScanTask) getDatumRows(rows []chunk.Row) [][]types.Datum { func (t *ttlScanTask) doScan(ctx context.Context, delCh chan<- *ttlDeleteTask, sessPool sessionPool) *ttlScanTaskExecResult { // TODO: merge the ctx and the taskCtx in ttl scan task, to allow both "cancel" and gracefully stop workers // now, the taskCtx is only check at the beginning of every loop - taskCtx := t.ctx + tracer := metrics.PhaseTracerFromCtx(ctx) + defer tracer.EnterPhase(tracer.Phase()) + tracer.EnterPhase(metrics.PhaseOther) rawSess, err := getSession(sessPool) if err != nil { return t.result(err) @@ -150,8 +171,11 @@ func (t *ttlScanTask) doScan(ctx context.Context, delCh chan<- *ttlDeleteTask, s return t.result(nil) } + sqlStart := time.Now() rows, retryable, sqlErr := sess.ExecuteSQLWithCheck(ctx, sql) + selectInterval := time.Since(sqlStart) if sqlErr != nil { + metrics.SelectErrorDuration.Observe(selectInterval.Seconds()) needRetry := retryable && retryTimes < scanTaskExecuteSQLMaxRetry && ctx.Err() == nil logutil.BgLogger().Error("execute query for ttl scan task failed", zap.String("SQL", sql), @@ -165,14 +189,18 @@ func (t *ttlScanTask) doScan(ctx context.Context, delCh chan<- *ttlDeleteTask, s } retrySQL = sql retryTimes++ + + tracer.EnterPhase(metrics.PhaseWaitRetry) select { case <-ctx.Done(): return t.result(ctx.Err()) case <-time.After(scanTaskExecuteSQLRetryInterval): } + tracer.EnterPhase(metrics.PhaseOther) continue } + metrics.SelectSuccessDuration.Observe(selectInterval.Seconds()) retrySQL = "" retryTimes = 0 lastResult = t.getDatumRows(rows) @@ -186,12 +214,15 @@ func (t *ttlScanTask) doScan(ctx context.Context, delCh chan<- *ttlDeleteTask, s rows: lastResult, statistics: t.statistics, } + + tracer.EnterPhase(metrics.PhaseDispatch) select { case <-ctx.Done(): return t.result(ctx.Err()) case delCh <- delTask: t.statistics.IncTotalRows(len(lastResult)) } + tracer.EnterPhase(metrics.PhaseOther) } } @@ -267,17 +298,25 @@ func (w *ttlScanWorker) PollTaskResult() *ttlScanTaskExecResult { func (w *ttlScanWorker) loop() error { ctx := w.baseWorker.ctx + tracer := metrics.NewScanWorkerPhaseTracer() + defer tracer.EndPhase() + + ticker := time.Tick(time.Second * 5) for w.Status() == workerStatusRunning { + tracer.EnterPhase(metrics.PhaseIdle) select { case <-ctx.Done(): return nil + case <-ticker: + // ticker is used to update metrics on time case msg, ok := <-w.baseWorker.ch: + tracer.EnterPhase(metrics.PhaseOther) if !ok { return nil } switch task := msg.(type) { case *ttlScanTask: - w.handleScanTask(task) + w.handleScanTask(tracer, task) default: logutil.BgLogger().Warn("unrecognized message for ttlScanWorker", zap.Any("msg", msg)) } @@ -286,8 +325,9 @@ func (w *ttlScanWorker) loop() error { return nil } -func (w *ttlScanWorker) handleScanTask(task *ttlScanTask) { - result := task.doScan(w.ctx, w.delCh, w.sessionPool) +func (w *ttlScanWorker) handleScanTask(tracer *metrics.PhaseTracer, task *ttlScanTask) { + ctx := metrics.CtxWithPhaseTracer(w.ctx, tracer) + result := task.doScan(ctx, w.delCh, w.sessionPool) if result == nil { result = task.result(nil) } @@ -309,4 +349,6 @@ type scanWorker interface { Idle() bool Schedule(*ttlScanTask) error + PollTaskResult() *ttlScanTaskExecResult + CurrentTask() *ttlScanTask } diff --git a/ttl/ttlworker/scan_test.go b/ttl/ttlworker/scan_test.go index 66582084b18f3..34a25a2539612 100644 --- a/ttl/ttlworker/scan_test.go +++ b/ttl/ttlworker/scan_test.go @@ -403,3 +403,13 @@ func TestScanTaskDoScan(t *testing.T) { task.schemaChangeInRetry = 2 task.runDoScanForTest(1, "table 'test.t1' meta changed, should abort current job: [schema:1146]Table 'test.t1' doesn't exist") } + +func TestTTLStatisticsMarshalJSON(t *testing.T) { + statistics := &ttlStatistics{} + statistics.TotalRows.Store(1) + statistics.ErrorRows.Store(255) + statistics.SuccessRows.Store(128) + j, err := statistics.MarshalJSON() + require.NoError(t, err) + require.Equal(t, `{"total_rows":1,"success_rows":128,"error_rows":255}`, string(j)) +} diff --git a/ttl/ttlworker/session.go b/ttl/ttlworker/session.go index 722ea4ab3aa52..b20f436a61859 100644 --- a/ttl/ttlworker/session.go +++ b/ttl/ttlworker/session.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/ttl/cache" + "github.com/pingcap/tidb/ttl/metrics" "github.com/pingcap/tidb/ttl/session" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" @@ -60,7 +61,8 @@ func getSession(pool sessionPool) (session.Session, error) { pool.Put(resource) }) - if _, err = se.ExecuteSQL(context.Background(), "commit"); err != nil { + // Force rollback the session to guarantee the session is not in any explicit transaction + if _, err = se.ExecuteSQL(context.Background(), "ROLLBACK"); err != nil { se.Close() return nil, err } @@ -83,6 +85,10 @@ type ttlTableSession struct { } func (s *ttlTableSession) ExecuteSQLWithCheck(ctx context.Context, sql string) (rows []chunk.Row, shouldRetry bool, err error) { + tracer := metrics.PhaseTracerFromCtx(ctx) + defer tracer.EnterPhase(tracer.Phase()) + + tracer.EnterPhase(metrics.PhaseOther) if !variable.EnableTTLJob.Load() { return nil, false, errors.New("global TTL job is disabled") } @@ -92,7 +98,10 @@ func (s *ttlTableSession) ExecuteSQLWithCheck(ctx context.Context, sql string) ( } err = s.RunInTxn(ctx, func() error { + tracer.EnterPhase(metrics.PhaseQuery) + defer tracer.EnterPhase(tracer.Phase()) rows, err = s.ExecuteSQL(ctx, sql) + tracer.EnterPhase(metrics.PhaseCheckTTL) // We must check the configuration after ExecuteSQL because of MDL and the meta the current transaction used // can only be determined after executed one query. if validateErr := validateTTLWork(ctx, s.Session, s.tbl, s.expire); validateErr != nil { diff --git a/ttl/ttlworker/session_test.go b/ttl/ttlworker/session_test.go index 8cceaed7ac72b..877fd7996eaa7 100644 --- a/ttl/ttlworker/session_test.go +++ b/ttl/ttlworker/session_test.go @@ -154,6 +154,10 @@ func newMockSession(t *testing.T, tbl ...*cache.PhysicalTable) *mockSession { } } +func (s *mockSession) GetDomainInfoSchema() sessionctx.InfoschemaMetaVersion { + return s.sessionInfoSchema +} + func (s *mockSession) SessionInfoSchema() infoschema.InfoSchema { require.False(s.t, s.closed) return s.sessionInfoSchema diff --git a/ttl/ttlworker/worker.go b/ttl/ttlworker/worker.go index a04110373cdbf..783384862cacf 100644 --- a/ttl/ttlworker/worker.go +++ b/ttl/ttlworker/worker.go @@ -96,6 +96,12 @@ func (w *baseWorker) Error() error { } func (w *baseWorker) WaitStopped(ctx context.Context, timeout time.Duration) error { + // consider the situation when the worker has stopped, but the context has also stopped. We should + // return without error + if w.Status() == workerStatusStopped { + return nil + } + ctx, cancel := context.WithTimeout(ctx, timeout) go func() { w.wg.Wait() diff --git a/types/explain_format.go b/types/explain_format.go index 00bb9ac1e4336..9ef1dd0ccb975 100644 --- a/types/explain_format.go +++ b/types/explain_format.go @@ -35,6 +35,8 @@ var ( ExplainFormatBinary = "binary" // ExplainFormatTiDBJSON warp the default result in JSON format ExplainFormatTiDBJSON = "tidb_json" + // ExplainFormatCostTrace prints the cost and cost formula of each operator. + ExplainFormatCostTrace = "cost_trace" // ExplainFormats stores the valid formats for explain statement, used by validator. ExplainFormats = []string{ @@ -48,5 +50,6 @@ var ( ExplainFormatTrueCardCost, ExplainFormatBinary, ExplainFormatTiDBJSON, + ExplainFormatCostTrace, } ) diff --git a/types/json_binary.go b/types/json_binary.go index eb9a818b6a7f5..6fe01d2b4f28e 100644 --- a/types/json_binary.go +++ b/types/json_binary.go @@ -275,7 +275,8 @@ func (bj BinaryJSON) GetElemCount() int { return int(jsonEndian.Uint32(bj.Value)) } -func (bj BinaryJSON) arrayGetElem(idx int) BinaryJSON { +// ArrayGetElem gets the element of the index `idx`. +func (bj BinaryJSON) ArrayGetElem(idx int) BinaryJSON { return bj.valEntryGet(headerSize + idx*valEntrySize) } @@ -355,7 +356,7 @@ func (bj BinaryJSON) marshalArrayTo(buf []byte) ([]byte, error) { buf = append(buf, ", "...) } var err error - buf, err = bj.arrayGetElem(i).marshalTo(buf) + buf, err = bj.ArrayGetElem(i).marshalTo(buf) if err != nil { return nil, errors.Trace(err) } @@ -557,7 +558,7 @@ func (bj BinaryJSON) HashValue(buf []byte) []byte { elemCount := int(jsonEndian.Uint32(bj.Value)) buf = append(buf, bj.Value[0:dataSizeOff]...) for i := 0; i < elemCount; i++ { - buf = bj.arrayGetElem(i).HashValue(buf) + buf = bj.ArrayGetElem(i).HashValue(buf) } case JSONTypeCodeObject: // this hash value is bidirectional, because you can get the key using the json diff --git a/types/json_binary_functions.go b/types/json_binary_functions.go index 2b02d5a0f65e7..7c4d7bf7f97bc 100644 --- a/types/json_binary_functions.go +++ b/types/json_binary_functions.go @@ -294,7 +294,7 @@ func (bj BinaryJSON) extractTo(buf []BinaryJSON, pathExpr JSONPathExpression, du start, end := currentLeg.arraySelection.getIndexRange(bj) if start >= 0 && start <= end { for i := start; i <= end; i++ { - buf = bj.arrayGetElem(i).extractTo(buf, subPathExpr, dup, one) + buf = bj.ArrayGetElem(i).extractTo(buf, subPathExpr, dup, one) } } } else if currentLeg.typ == jsonPathLegKey && bj.TypeCode == JSONTypeCodeObject { @@ -314,7 +314,7 @@ func (bj BinaryJSON) extractTo(buf []BinaryJSON, pathExpr JSONPathExpression, du if bj.TypeCode == JSONTypeCodeArray { elemCount := bj.GetElemCount() for i := 0; i < elemCount && !jsonFinished(buf, one); i++ { - buf = bj.arrayGetElem(i).extractTo(buf, pathExpr, dup, one) + buf = bj.ArrayGetElem(i).extractTo(buf, pathExpr, dup, one) } } else if bj.TypeCode == JSONTypeCodeObject { elemCount := bj.GetElemCount() @@ -459,12 +459,12 @@ func (bj BinaryJSON) ArrayInsert(pathExpr JSONPathExpression, value BinaryJSON) // Insert into the array newArray := make([]BinaryJSON, 0, count+1) for i := 0; i < idx; i++ { - elem := obj.arrayGetElem(i) + elem := obj.ArrayGetElem(i) newArray = append(newArray, elem) } newArray = append(newArray, value) for i := idx; i < count; i++ { - elem := obj.arrayGetElem(i) + elem := obj.ArrayGetElem(i) newArray = append(newArray, elem) } obj = buildBinaryJSONArray(newArray) @@ -556,7 +556,7 @@ func (bm *binaryModifier) doInsert(path JSONPathExpression, newBj BinaryJSON) { elemCount := parentBj.GetElemCount() elems := make([]BinaryJSON, 0, elemCount+1) for i := 0; i < elemCount; i++ { - elems = append(elems, parentBj.arrayGetElem(i)) + elems = append(elems, parentBj.ArrayGetElem(i)) } elems = append(elems, newBj) bm.modifyValue = buildBinaryJSONArray(elems) @@ -622,7 +622,7 @@ func (bm *binaryModifier) doRemove(path JSONPathExpression) { elems := make([]BinaryJSON, 0, elemCount-1) for i := 0; i < elemCount; i++ { if i != idx { - elems = append(elems, parentBj.arrayGetElem(i)) + elems = append(elems, parentBj.ArrayGetElem(i)) } } bm.modifyValue = buildBinaryJSONArray(elems) @@ -809,8 +809,8 @@ func CompareBinaryJSON(left, right BinaryJSON) int { leftCount := left.GetElemCount() rightCount := right.GetElemCount() for i := 0; i < leftCount && i < rightCount; i++ { - elem1 := left.arrayGetElem(i) - elem2 := right.arrayGetElem(i) + elem1 := left.ArrayGetElem(i) + elem2 := right.ArrayGetElem(i) cmp = CompareBinaryJSON(elem1, elem2) if cmp != 0 { return cmp @@ -993,7 +993,7 @@ func mergeBinaryArray(elems []BinaryJSON) BinaryJSON { } else { childCount := elem.GetElemCount() for j := 0; j < childCount; j++ { - buf = append(buf, elem.arrayGetElem(j)) + buf = append(buf, elem.ArrayGetElem(j)) } } } @@ -1088,7 +1088,7 @@ func ContainsBinaryJSON(obj, target BinaryJSON) bool { if target.TypeCode == JSONTypeCodeArray { elemCount := target.GetElemCount() for i := 0; i < elemCount; i++ { - if !ContainsBinaryJSON(obj, target.arrayGetElem(i)) { + if !ContainsBinaryJSON(obj, target.ArrayGetElem(i)) { return false } } @@ -1096,7 +1096,49 @@ func ContainsBinaryJSON(obj, target BinaryJSON) bool { } elemCount := obj.GetElemCount() for i := 0; i < elemCount; i++ { - if ContainsBinaryJSON(obj.arrayGetElem(i), target) { + if ContainsBinaryJSON(obj.ArrayGetElem(i), target) { + return true + } + } + return false + default: + return CompareBinaryJSON(obj, target) == 0 + } +} + +// OverlapsBinaryJSON is similar with ContainsBinaryJSON, but it checks the `OR` relationship. +func OverlapsBinaryJSON(obj, target BinaryJSON) bool { + if obj.TypeCode != JSONTypeCodeArray && target.TypeCode == JSONTypeCodeArray { + obj, target = target, obj + } + switch obj.TypeCode { + case JSONTypeCodeObject: + if target.TypeCode == JSONTypeCodeObject { + elemCount := target.GetElemCount() + for i := 0; i < elemCount; i++ { + key := target.objectGetKey(i) + val := target.objectGetVal(i) + if exp, exists := obj.objectSearchKey(key); exists && CompareBinaryJSON(exp, val) == 0 { + return true + } + } + } + return false + case JSONTypeCodeArray: + if target.TypeCode == JSONTypeCodeArray { + for i := 0; i < obj.GetElemCount(); i++ { + o := obj.ArrayGetElem(i) + for j := 0; j < target.GetElemCount(); j++ { + if CompareBinaryJSON(o, target.ArrayGetElem(j)) == 0 { + return true + } + } + } + return false + } + elemCount := obj.GetElemCount() + for i := 0; i < elemCount; i++ { + if CompareBinaryJSON(obj.ArrayGetElem(i), target) == 0 { return true } } @@ -1133,7 +1175,7 @@ func (bj BinaryJSON) GetElemDepth() int { elemCount := bj.GetElemCount() maxDepth := 0 for i := 0; i < elemCount; i++ { - obj := bj.arrayGetElem(i) + obj := bj.ArrayGetElem(i) depth := obj.GetElemDepth() if depth > maxDepth { maxDepth = depth @@ -1204,9 +1246,9 @@ func (bj BinaryJSON) extractToCallback(pathExpr JSONPathExpression, callbackFn e switch selection := currentLeg.arraySelection.(type) { case jsonPathArraySelectionAsterisk: for i := 0; i < elemCount; i++ { - // buf = bj.arrayGetElem(i).extractTo(buf, subPathExpr) + // buf = bj.ArrayGetElem(i).extractTo(buf, subPathExpr) path := fullpath.pushBackOneArraySelectionLeg(jsonPathArraySelectionIndex{jsonPathArrayIndexFromStart(i)}) - stop, err = bj.arrayGetElem(i).extractToCallback(subPathExpr, callbackFn, path) + stop, err = bj.ArrayGetElem(i).extractToCallback(subPathExpr, callbackFn, path) if stop || err != nil { return } @@ -1214,9 +1256,9 @@ func (bj BinaryJSON) extractToCallback(pathExpr JSONPathExpression, callbackFn e case jsonPathArraySelectionIndex: idx := selection.index.getIndexFromStart(bj) if idx < elemCount && idx >= 0 { - // buf = bj.arrayGetElem(currentLeg.arraySelection).extractTo(buf, subPathExpr) + // buf = bj.ArrayGetElem(currentLeg.arraySelection).extractTo(buf, subPathExpr) path := fullpath.pushBackOneArraySelectionLeg(currentLeg.arraySelection) - stop, err = bj.arrayGetElem(idx).extractToCallback(subPathExpr, callbackFn, path) + stop, err = bj.ArrayGetElem(idx).extractToCallback(subPathExpr, callbackFn, path) if stop || err != nil { return } @@ -1230,7 +1272,7 @@ func (bj BinaryJSON) extractToCallback(pathExpr JSONPathExpression, callbackFn e if start <= end && start >= 0 { for i := start; i <= end; i++ { path := fullpath.pushBackOneArraySelectionLeg(jsonPathArraySelectionIndex{jsonPathArrayIndexFromStart(i)}) - stop, err = bj.arrayGetElem(i).extractToCallback(subPathExpr, callbackFn, path) + stop, err = bj.ArrayGetElem(i).extractToCallback(subPathExpr, callbackFn, path) if stop || err != nil { return } @@ -1269,9 +1311,9 @@ func (bj BinaryJSON) extractToCallback(pathExpr JSONPathExpression, callbackFn e if bj.TypeCode == JSONTypeCodeArray { elemCount := bj.GetElemCount() for i := 0; i < elemCount; i++ { - // buf = bj.arrayGetElem(i).extractTo(buf, pathExpr) + // buf = bj.ArrayGetElem(i).extractTo(buf, pathExpr) path := fullpath.pushBackOneArraySelectionLeg(jsonPathArraySelectionIndex{jsonPathArrayIndexFromStart(i)}) - stop, err = bj.arrayGetElem(i).extractToCallback(pathExpr, callbackFn, path) + stop, err = bj.ArrayGetElem(i).extractToCallback(pathExpr, callbackFn, path) if stop || err != nil { return } @@ -1315,7 +1357,7 @@ func (bj BinaryJSON) Walk(walkFn BinaryJSONWalkFunc, pathExprList ...JSONPathExp elemCount := bj.GetElemCount() for i := 0; i < elemCount; i++ { path := fullpath.pushBackOneArraySelectionLeg(jsonPathArraySelectionIndex{jsonPathArrayIndexFromStart(i)}) - stop, err = doWalk(path, bj.arrayGetElem(i)) + stop, err = doWalk(path, bj.ArrayGetElem(i)) if stop || err != nil { return } diff --git a/util/cpuprofile/cpuprofile_test.go b/util/cpuprofile/cpuprofile_test.go index 2d400264ede10..b20428dcf21fd 100644 --- a/util/cpuprofile/cpuprofile_test.go +++ b/util/cpuprofile/cpuprofile_test.go @@ -18,7 +18,6 @@ import ( "bytes" "context" "io" - "io/ioutil" "net" "net/http" "runtime/pprof" @@ -237,7 +236,7 @@ func TestProfileHTTPHandler(t *testing.T) { resp, err = http.Get("http://" + address + "/debug/pprof/profile?seconds=100000") require.NoError(t, err) require.Equal(t, 400, resp.StatusCode) - body, err := ioutil.ReadAll(resp.Body) + body, err := io.ReadAll(resp.Body) require.NoError(t, err) require.Equal(t, "profile duration exceeds server's WriteTimeout\n", string(body)) require.NoError(t, resp.Body.Close()) diff --git a/util/stmtsummary/BUILD.bazel b/util/stmtsummary/BUILD.bazel index 7c85e57d52aee..3c8c295a5d659 100644 --- a/util/stmtsummary/BUILD.bazel +++ b/util/stmtsummary/BUILD.bazel @@ -15,6 +15,7 @@ go_library( "//parser/mysql", "//sessionctx/stmtctx", "//types", + "//util/chunk", "//util/execdetails", "//util/hack", "//util/kvcache", diff --git a/util/stmtsummary/reader.go b/util/stmtsummary/reader.go index eca48ae4e82eb..e5e9a2db39705 100644 --- a/util/stmtsummary/reader.go +++ b/util/stmtsummary/reader.go @@ -310,6 +310,9 @@ const ( PlanDigestStr = "PLAN_DIGEST" PlanStr = "PLAN" BinaryPlan = "BINARY_PLAN" + Charset = "CHARSET" + Collation = "COLLATION" + PlanHint = "PLAN_HINT" ) type columnValueFactory func(reader *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) interface{} @@ -620,4 +623,13 @@ var columnValueFactoryMap = map[string]columnValueFactory{ BinaryPlan: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { return ssElement.sampleBinaryPlan }, + Charset: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + return ssElement.charset + }, + Collation: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + return ssElement.collation + }, + PlanHint: func(_ *stmtSummaryReader, ssElement *stmtSummaryByDigestElement, _ *stmtSummaryByDigest) interface{} { + return ssElement.planHint + }, } diff --git a/util/stmtsummary/statement_summary.go b/util/stmtsummary/statement_summary.go index 09dd9a57fa3a6..0ac60f70a1e4a 100644 --- a/util/stmtsummary/statement_summary.go +++ b/util/stmtsummary/statement_summary.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/kvcache" @@ -500,28 +501,24 @@ func (ssMap *stmtSummaryByDigestMap) maxSQLLength() int { return int(ssMap.optMaxSQLLength.Load()) } -func getBindableStmtByPlanDigest(ssbd *stmtSummaryByDigest, planDigest string) *BindableStmt { - ssbd.Lock() - defer ssbd.Unlock() - if ssbd.initialized && ssbd.planDigest == planDigest && ssbd.history.Len() > 0 && - (ssbd.stmtType == "Select" || ssbd.stmtType == "Delete" || ssbd.stmtType == "Update" || ssbd.stmtType == "Insert" || ssbd.stmtType == "Replace") { - ssElement := ssbd.history.Back().Value.(*stmtSummaryByDigestElement) - ssElement.Lock() - defer ssElement.Unlock() - // Empty auth users means that it is an internal queries. - if len(ssElement.authUsers) > 0 { +// GetBindableStmtFromCluster gets users' select/update/delete SQL. +func GetBindableStmtFromCluster(rows []chunk.Row) *BindableStmt { + for _, row := range rows { + user := row.GetString(3) + stmtType := row.GetString(0) + if user != "" && (stmtType == "Select" || stmtType == "Delete" || stmtType == "Update" || stmtType == "Insert" || stmtType == "Replace") { + // Empty auth users means that it is an internal queries. stmt := &BindableStmt{ - Schema: ssbd.schemaName, - Query: ssElement.sampleSQL, - PlanHint: ssElement.planHint, - Charset: ssElement.charset, - Collation: ssElement.collation, - Users: ssElement.authUsers, + Schema: row.GetString(1), //schemaName + Query: row.GetString(5), //sampleSQL + PlanHint: row.GetString(8), //planHint + Charset: row.GetString(6), //charset + Collation: row.GetString(7), //collation } // If it is SQL command prepare / execute, the ssElement.sampleSQL is `execute ...`, we should get the original select query. // If it is binary protocol prepare / execute, ssbd.normalizedSQL should be same as ssElement.sampleSQL. - if ssElement.prepared { - stmt.Query = ssbd.normalizedSQL + if row.GetInt64(4) == 1 { + stmt.Query = row.GetString(2) //normalizedSQL } return stmt } @@ -529,20 +526,6 @@ func getBindableStmtByPlanDigest(ssbd *stmtSummaryByDigest, planDigest string) * return nil } -// GetBindableStmtByPlanDigest gets users' select/update/delete SQL by plan digest. -func (ssMap *stmtSummaryByDigestMap) GetBindableStmtByPlanDigest(planDigest string) *BindableStmt { - ssMap.Lock() - values := ssMap.summaryMap.Values() - ssMap.Unlock() - - for _, value := range values { - if stmt := getBindableStmtByPlanDigest(value.(*stmtSummaryByDigest), planDigest); stmt != nil { - return stmt - } - } - return nil -} - // newStmtSummaryByDigest creates a stmtSummaryByDigest from StmtExecInfo. func (ssbd *stmtSummaryByDigest) init(sei *StmtExecInfo, _ int64, _ int64, _ int) { // Use "," to separate table names to support FIND_IN_SET.